From 72cb711000be4c1c3db2d7f982f9201ce24ba258 Mon Sep 17 00:00:00 2001 From: tchepavel Date: Sun, 8 May 2022 15:12:15 +0300 Subject: [PATCH 001/627] Copy Rabbit to become NATS --- .gitmodules | 3 + contrib/CMakeLists.txt | 1 + contrib/nats-io | 1 + src/CMakeLists.txt | 4 + src/Storages/NATS/Buffer_fwd.h | 14 + src/Storages/NATS/NATSConnection.cpp | 121 ++ src/Storages/NATS/NATSConnection.h | 68 + src/Storages/NATS/NATSHandler.cpp | 67 + src/Storages/NATS/NATSHandler.h | 66 + src/Storages/NATS/NATSSettings.cpp | 39 + src/Storages/NATS/NATSSettings.h | 45 + src/Storages/NATS/NATSSink.cpp | 56 + src/Storages/NATS/NATSSink.h | 32 + src/Storages/NATS/NATSSource.cpp | 189 +++ src/Storages/NATS/NATSSource.h | 61 + .../NATS/ReadBufferFromNATSConsumer.cpp | 188 +++ .../NATS/ReadBufferFromNATSConsumer.h | 104 ++ src/Storages/NATS/StorageNATS.cpp | 1186 +++++++++++++++++ src/Storages/NATS/StorageNATS.h | 207 +++ src/Storages/NATS/UVLoop.h | 44 + .../NATS/WriteBufferToNATSProducer.cpp | 317 +++++ src/Storages/NATS/WriteBufferToNATSProducer.h | 121 ++ 22 files changed, 2934 insertions(+) create mode 160000 contrib/nats-io create mode 100644 src/Storages/NATS/Buffer_fwd.h create mode 100644 src/Storages/NATS/NATSConnection.cpp create mode 100644 src/Storages/NATS/NATSConnection.h create mode 100644 src/Storages/NATS/NATSHandler.cpp create mode 100644 src/Storages/NATS/NATSHandler.h create mode 100644 src/Storages/NATS/NATSSettings.cpp create mode 100644 src/Storages/NATS/NATSSettings.h create mode 100644 src/Storages/NATS/NATSSink.cpp create mode 100644 src/Storages/NATS/NATSSink.h create mode 100644 src/Storages/NATS/NATSSource.cpp create mode 100644 src/Storages/NATS/NATSSource.h create mode 100644 src/Storages/NATS/ReadBufferFromNATSConsumer.cpp create mode 100644 src/Storages/NATS/ReadBufferFromNATSConsumer.h create mode 100644 src/Storages/NATS/StorageNATS.cpp create mode 100644 src/Storages/NATS/StorageNATS.h create mode 100644 src/Storages/NATS/UVLoop.h create mode 100644 src/Storages/NATS/WriteBufferToNATSProducer.cpp create mode 100644 src/Storages/NATS/WriteBufferToNATSProducer.h diff --git a/.gitmodules b/.gitmodules index 6c9e66f9cbc..027e3f72929 100644 --- a/.gitmodules +++ b/.gitmodules @@ -262,3 +262,6 @@ [submodule "contrib/minizip-ng"] path = contrib/minizip-ng url = https://github.com/zlib-ng/minizip-ng +[submodule "contrib/nats-io"] + path = contrib/nats-io + url = https://github.com/nats-io/nats.c.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 1f03c0fd341..9041441029e 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -131,6 +131,7 @@ add_contrib (krb5-cmake krb5) add_contrib (cyrus-sasl-cmake cyrus-sasl) # for krb5 add_contrib (libgsasl-cmake libgsasl) # requires krb5 add_contrib (librdkafka-cmake librdkafka) # requires: libgsasl +add_contrib (nats-io-cmake nats-io) add_contrib (libhdfs3-cmake libhdfs3) # requires: protobuf, krb5 add_contrib (hive-metastore-cmake hive-metastore) # requires: thrift/avro/arrow/libhdfs3 add_contrib (cppkafka-cmake cppkafka) diff --git a/contrib/nats-io b/contrib/nats-io new file mode 160000 index 00000000000..d1f59f7bcf8 --- /dev/null +++ b/contrib/nats-io @@ -0,0 +1 @@ +Subproject commit d1f59f7bcf8465526f7e6d9c99982cbd6b209547 diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 20db948abd0..4e364629f24 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -107,6 +107,8 @@ if (TARGET ch_contrib::rdkafka) add_headers_and_sources(dbms Storages/Kafka) endif() +add_headers_and_sources(dbms Storages/NATS) + if (TARGET ch_contrib::amqp_cpp) add_headers_and_sources(dbms Storages/RabbitMQ) endif() @@ -380,6 +382,8 @@ if (TARGET ch_contrib::rdkafka) dbms_target_link_libraries(PRIVATE ch_contrib::rdkafka ch_contrib::cppkafka) endif() +dbms_target_link_libraries(PRIVATE ch_contrib::nats-io) + if (TARGET ch_contrib::sasl2) dbms_target_link_libraries(PRIVATE ch_contrib::sasl2) endif() diff --git a/src/Storages/NATS/Buffer_fwd.h b/src/Storages/NATS/Buffer_fwd.h new file mode 100644 index 00000000000..caf7b8cfdbe --- /dev/null +++ b/src/Storages/NATS/Buffer_fwd.h @@ -0,0 +1,14 @@ +#pragma once + +#include + +namespace DB +{ + +class ReadBufferFromNATSConsumer; +using ConsumerBufferPtr = std::shared_ptr; + +class WriteBufferToNATSProducer; +using ProducerBufferPtr = std::shared_ptr; + +} diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp new file mode 100644 index 00000000000..7b0abba7535 --- /dev/null +++ b/src/Storages/NATS/NATSConnection.cpp @@ -0,0 +1,121 @@ +#include "NATSConnection.h" + +#include +#include + + +namespace DB +{ + +static const auto CONNECT_SLEEP = 200; +static const auto RETRIES_MAX = 20; + + +NATSConnection::NATSConnection(const NATSConfiguration & configuration_, Poco::Logger * log_) + : configuration(configuration_) + , log(log_) + , event_handler(loop.getLoop(), log) +{ +} + +String NATSConnection::connectionInfoForLog() const +{ + return configuration.host + ':' + toString(configuration.port); +} + +bool NATSConnection::isConnected() +{ + std::lock_guard lock(mutex); + return isConnectedImpl(); +} + +bool NATSConnection::connect() +{ + std::lock_guard lock(mutex); + connectImpl(); + return isConnectedImpl(); +} + +bool NATSConnection::reconnect() +{ + std::lock_guard lock(mutex); + if (isConnectedImpl()) + return true; + + disconnectImpl(); + + /// This will force immediate closure if not yet closed + if (!connection->closed()) + connection->close(true); + + LOG_DEBUG(log, "Trying to restore connection to {}", connectionInfoForLog()); + connectImpl(); + + return isConnectedImpl(); +} + +SubscriptionPtr NATSConnection::createSubscription(const std::string& subject) +{ + std::lock_guard lock(mutex); + natsSubscription * ns; + natsConnection_SubscribeSync(&ns, connection, subject.c_str()); + return SubscriptionPtr(ns, &natsSubscription_Destroy); +} + +void NATSConnection::disconnect() +{ + std::lock_guard lock(mutex); + disconnectImpl(); +} + +bool NATSConnection::closed() +{ + std::lock_guard lock(mutex); + return natsConnection_IsClosed(connection); +} + +bool NATSConnection::isConnectedImpl() const +{ + return event_handler.connectionRunning() && !natsConnection_IsClosed(connection); +} + +void NATSConnection::connectImpl() +{ + if (configuration.connection_string.empty()) + { + LOG_DEBUG(log, "Connecting to: {}:{} (user: {})", configuration.host, configuration.port, configuration.username); + AMQP::Login login(configuration.username, configuration.password); + AMQP::Address address(configuration.host, configuration.port, login, configuration.vhost, configuration.secure); + connection = std::make_unique(&event_handler, address); + } + else + { + AMQP::Address address(configuration.connection_string); + connection = std::make_unique(&event_handler, address); + } + + auto cnt_retries = 0; + while (true) + { + event_handler.iterateLoop(); + + if (connection->ready() || cnt_retries++ == RETRIES_MAX) + break; + + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + } +} + +void NATSConnection::disconnectImpl() +{ + natsConnection_Close(connection); + + /** Connection is not closed immediately (firstly, all pending operations are completed, and then + * an AMQP closing-handshake is performed). But cannot open a new connection until previous one is properly closed + */ + size_t cnt_retries = 0; + while (!closed() && cnt_retries++ != RETRIES_MAX) + event_handler.iterateLoop(); +} + +} diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h new file mode 100644 index 00000000000..2c999e873aa --- /dev/null +++ b/src/Storages/NATS/NATSConnection.h @@ -0,0 +1,68 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +struct NATSConfiguration +{ + String url; + String host; + UInt16 port; + String username; + String password; + String vhost; + + bool secure; + String connection_string; +}; + +using SubscriptionPtr = std::unique_ptr; + +class NATSConnection +{ + +public: + NATSConnection(const NATSConfiguration & configuration_, Poco::Logger * log_); + + bool isConnected(); + + bool connect(); + + bool reconnect(); + + void disconnect(); + + bool closed(); + + SubscriptionPtr createSubscription(const std::string& subject); + + /// NATSHandler is thread safe. Any public methods can be called concurrently. + NATSHandler & getHandler() { return event_handler; } + + String connectionInfoForLog() const; + +private: + bool isConnectedImpl() const; + + void connectImpl(); + + void disconnectImpl(); + + NATSConfiguration configuration; + Poco::Logger * log; + + UVLoop loop; + NATSHandler event_handler; + + natsConnection * connection; + natsStatus status; + std::mutex mutex; +}; + +using NATSConnectionPtr = std::unique_ptr; + +} diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp new file mode 100644 index 00000000000..7fb8ff38c47 --- /dev/null +++ b/src/Storages/NATS/NATSHandler.cpp @@ -0,0 +1,67 @@ +#include +#include +#include + +namespace DB +{ + +/* The object of this class is shared between concurrent consumers (who share the same connection == share the same + * event loop and handler). + */ +NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : + loop(loop_), + log(log_), + connection_running(false), + loop_running(false), + loop_state(Loop::STOP) +{ + natsLibuv_Init(); + natsLibuv_SetThreadLocalLoop(loop); + natsOptions_Create(&opts); + status = natsOptions_SetEventLoop(opts, static_cast(loop), + natsLibuv_Attach, + natsLibuv_Read, + natsLibuv_Write, + natsLibuv_Detach); +} + +void NATSHandler::startLoop() +{ + std::lock_guard lock(startup_mutex); + + LOG_DEBUG(log, "Background loop started"); + loop_running.store(true); + + while (loop_state.load() == Loop::RUN) + uv_run(loop, UV_RUN_NOWAIT); + + LOG_DEBUG(log, "Background loop ended"); + loop_running.store(false); +} + +void NATSHandler::iterateLoop() +{ + std::unique_lock lock(startup_mutex, std::defer_lock); + if (lock.try_lock()) + uv_run(loop, UV_RUN_NOWAIT); +} + +/// Do not need synchronization as in iterateLoop(), because this method is used only for +/// initial NATS setup - at this point there is no background loop thread. +void NATSHandler::startBlockingLoop() +{ + LOG_DEBUG(log, "Started blocking loop."); + uv_run(loop, UV_RUN_DEFAULT); +} + +void NATSHandler::stopLoop() +{ + LOG_DEBUG(log, "Implicit loop stop."); + uv_stop(loop); +} + +NATSHandler::~NATSHandler() { + natsOptions_Destroy(opts); +} + +} diff --git a/src/Storages/NATS/NATSHandler.h b/src/Storages/NATS/NATSHandler.h new file mode 100644 index 00000000000..12ea3454b9d --- /dev/null +++ b/src/Storages/NATS/NATSHandler.h @@ -0,0 +1,66 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace Loop +{ + static const UInt8 RUN = 1; + static const UInt8 STOP = 2; +} + +class NATSHandler +{ + +public: + NATSHandler(uv_loop_t * loop_, Poco::Logger * log_); + + ~NATSHandler(); + + /// Loop for background thread worker. + void startLoop(); + + /// Loop to wait for small tasks in a non-blocking mode. + /// Adds synchronization with main background loop. + void iterateLoop(); + + /// Loop to wait for small tasks in a blocking mode. + /// No synchronization is done with the main loop thread. + void startBlockingLoop(); + + void stopLoop(); + + bool connectionRunning() const { return connection_running.load(); } + bool loopRunning() const { return loop_running.load(); } + + void updateLoopState(UInt8 state) { loop_state.store(state); } + UInt8 getLoopState() { return loop_state.load(); } + + natsStatus getStatus() { return status; } + natsOptions * getOptions() { return opts; } + +private: + uv_loop_t * loop; + natsOptions * opts = nullptr; + natsStatus status = NATS_OK; + Poco::Logger * log; + + std::atomic connection_running, loop_running; + std::atomic loop_state; + std::mutex startup_mutex; +}; + +using NATSHandlerPtr = std::shared_ptr; + +} diff --git a/src/Storages/NATS/NATSSettings.cpp b/src/Storages/NATS/NATSSettings.cpp new file mode 100644 index 00000000000..11659b105b2 --- /dev/null +++ b/src/Storages/NATS/NATSSettings.cpp @@ -0,0 +1,39 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) + +void NATSSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} +} diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h new file mode 100644 index 00000000000..5f4051e06d1 --- /dev/null +++ b/src/Storages/NATS/NATSSettings.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include + +namespace DB +{ + class ASTStorage; + + +#define RABBITMQ_RELATED_SETTINGS(M) \ + M(String, nats_host_port, "", "A host-port to connect to NATS server.", 0) \ + M(String, nats_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \ + M(String, nats_format, "", "The message format.", 0) \ + M(String, nats_exchange_type, "default", "The exchange type.", 0) \ + M(String, nats_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \ + M(Char, nats_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ + M(String, nats_schema, "", "Schema identifier (used by schema-based formats) for NATS engine", 0) \ + M(UInt64, nats_num_consumers, 1, "The number of consumer channels per table.", 0) \ + M(UInt64, nats_num_queues, 1, "The number of queues per consumer.", 0) \ + M(String, nats_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ + M(Bool, nats_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \ + M(Bool, nats_secure, false, "Use SSL connection", 0) \ + M(String, nats_address, "", "Address for connection", 0) \ + M(UInt64, nats_skip_broken_messages, 0, "Skip at least this number of broken messages from NATS per block", 0) \ + M(UInt64, nats_max_block_size, 0, "Number of row collected before flushing data from NATS.", 0) \ + M(Milliseconds, nats_flush_interval_ms, 0, "Timeout for flushing data from NATS.", 0) \ + M(String, nats_vhost, "/", "NATS vhost.", 0) \ + M(String, nats_queue_settings_list, "", "A list of nats queue settings", 0) \ + M(Bool, nats_queue_consume, false, "Use user-defined queues and do not make any NATS setup: declaring exchanges, queues, bindings", 0) \ + M(String, nats_username, "", "NATS username", 0) \ + M(String, nats_password, "", "NATS password", 0) \ + M(Bool, nats_commit_on_select, false, "Commit messages when select query is made", 0) \ + +#define LIST_OF_RABBITMQ_SETTINGS(M) \ + RABBITMQ_RELATED_SETTINGS(M) \ + FORMAT_FACTORY_SETTINGS(M) + +DECLARE_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) + +struct NATSSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; +} diff --git a/src/Storages/NATS/NATSSink.cpp b/src/Storages/NATS/NATSSink.cpp new file mode 100644 index 00000000000..db2620d9ed2 --- /dev/null +++ b/src/Storages/NATS/NATSSink.cpp @@ -0,0 +1,56 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NATSSink::NATSSink( + StorageNATS & storage_, + const StorageMetadataPtr & metadata_snapshot_, + ContextPtr context_) + : SinkToStorage(metadata_snapshot_->getSampleBlockNonMaterialized()) + , storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , context(context_) +{ + storage.unbindExchange(); +} + + +void NATSSink::onStart() +{ + buffer = storage.createWriteBuffer(); + buffer->activateWriting(); + + auto format_settings = getFormatSettings(context); + format_settings.protobuf.allow_multiple_rows_without_delimiter = true; + + format = FormatFactory::instance().getOutputFormat(storage.getFormatName(), *buffer, getHeader(), context, + [this](const Columns & /* columns */, size_t /* rows */) + { + buffer->countRow(); + }, + format_settings); +} + + +void NATSSink::consume(Chunk chunk) +{ + format->write(getHeader().cloneWithColumns(chunk.detachColumns())); +} + + +void NATSSink::onFinish() +{ + format->finalize(); + + if (buffer) + buffer->updateMaxWait(); +} + +} diff --git a/src/Storages/NATS/NATSSink.h b/src/Storages/NATS/NATSSink.h new file mode 100644 index 00000000000..2f887dd3a4f --- /dev/null +++ b/src/Storages/NATS/NATSSink.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class IOutputFormat; +using IOutputFormatPtr = std::shared_ptr; + +class NATSSink : public SinkToStorage +{ + +public: + explicit NATSSink(StorageNATS & storage_, const StorageMetadataPtr & metadata_snapshot_, ContextPtr context_); + + void onStart() override; + void consume(Chunk chunk) override; + void onFinish() override; + + String getName() const override { return "NATSSink"; } + +private: + StorageNATS & storage; + StorageMetadataPtr metadata_snapshot; + ContextPtr context; + ProducerBufferPtr buffer; + IOutputFormatPtr format; +}; +} diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp new file mode 100644 index 00000000000..046b8792ced --- /dev/null +++ b/src/Storages/NATS/NATSSource.cpp @@ -0,0 +1,189 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +static std::pair getHeaders(const StorageSnapshotPtr & storage_snapshot) +{ + auto non_virtual_header = storage_snapshot->metadata->getSampleBlockNonMaterialized(); + auto virtual_header = storage_snapshot->getSampleBlockForColumns( + {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered", "_message_id", "_timestamp"}); + + return {non_virtual_header, virtual_header}; +} + +static Block getSampleBlock(const Block & non_virtual_header, const Block & virtual_header) +{ + auto header = non_virtual_header; + for (const auto & column : virtual_header) + header.insert(column); + + return header; +} + +NATSSource::NATSSource( + StorageNATS & storage_, + const StorageSnapshotPtr & storage_snapshot_, + ContextPtr context_, + const Names & columns, + size_t max_block_size_, + bool ack_in_suffix_) + : NATSSource( + storage_, + storage_snapshot_, + getHeaders(storage_snapshot_), + context_, + columns, + max_block_size_, + ack_in_suffix_) +{ +} + +NATSSource::NATSSource( + StorageNATS & storage_, + const StorageSnapshotPtr & storage_snapshot_, + std::pair headers, + ContextPtr context_, + const Names & columns, + size_t max_block_size_, + bool ack_in_suffix_) + : SourceWithProgress(getSampleBlock(headers.first, headers.second)) + , storage(storage_) + , storage_snapshot(storage_snapshot_) + , context(context_) + , column_names(columns) + , max_block_size(max_block_size_) + , ack_in_suffix(ack_in_suffix_) + , non_virtual_header(std::move(headers.first)) + , virtual_header(std::move(headers.second)) +{ + storage.incrementReader(); +} + + +NATSSource::~NATSSource() +{ + storage.decrementReader(); + + if (!buffer) + return; + + storage.pushReadBuffer(buffer); +} + + +bool NATSSource::needChannelUpdate() +{ + if (!buffer) + return false; + + return buffer->needChannelUpdate(); +} + + +void NATSSource::updateChannel() +{ + if (!buffer) + return; + + buffer->updateAckTracker(); + + if (storage.updateChannel(buffer->getChannel())) + buffer->setupChannel(); +} + +Chunk NATSSource::generate() +{ + auto chunk = generateImpl(); + if (!chunk && ack_in_suffix) + sendAck(); + + return chunk; +} + +Chunk NATSSource::generateImpl() +{ + if (!buffer) + { + auto timeout = std::chrono::milliseconds(context->getSettingsRef().rabbitmq_max_wait_ms.totalMilliseconds()); + buffer = storage.popReadBuffer(timeout); + } + + if (!buffer || is_finished) + return {}; + + is_finished = true; + + MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + auto input_format = FormatFactory::instance().getInputFormat( + storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + + StreamingFormatExecutor executor(non_virtual_header, input_format); + + size_t total_rows = 0; + + while (true) + { + if (buffer->eof()) + break; + + auto new_rows = executor.execute(); + + if (new_rows) + { + auto exchange_name = storage.getExchange(); + auto channel_id = buffer->getChannelID(); + auto delivery_tag = buffer->getDeliveryTag(); + auto redelivered = buffer->getRedelivered(); + auto message_id = buffer->getMessageID(); + auto timestamp = buffer->getTimestamp(); + + buffer->updateAckTracker({delivery_tag, channel_id}); + + for (size_t i = 0; i < new_rows; ++i) + { + virtual_columns[0]->insert(exchange_name); + virtual_columns[1]->insert(channel_id); + virtual_columns[2]->insert(delivery_tag); + virtual_columns[3]->insert(redelivered); + virtual_columns[4]->insert(message_id); + virtual_columns[5]->insert(timestamp); + } + + total_rows = total_rows + new_rows; + } + + buffer->allowNext(); + + if (total_rows >= max_block_size || buffer->queueEmpty() || buffer->isConsumerStopped() || !checkTimeLimit()) + break; + } + + if (total_rows == 0) + return {}; + + auto result_columns = executor.getResultColumns(); + for (auto & column : virtual_columns) + result_columns.push_back(std::move(column)); + + return Chunk(std::move(result_columns), total_rows); +} + + +bool NATSSource::sendAck() +{ + if (!buffer) + return false; + + if (!buffer->ackMessages()) + return false; + + return true; +} + +} diff --git a/src/Storages/NATS/NATSSource.h b/src/Storages/NATS/NATSSource.h new file mode 100644 index 00000000000..ce48e5cf382 --- /dev/null +++ b/src/Storages/NATS/NATSSource.h @@ -0,0 +1,61 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class NATSSource : public SourceWithProgress +{ + +public: + NATSSource( + StorageNATS & storage_, + const StorageSnapshotPtr & storage_snapshot_, + ContextPtr context_, + const Names & columns, + size_t max_block_size_, + bool ack_in_suffix = false); + + ~NATSSource() override; + + String getName() const override { return storage.getName(); } + ConsumerBufferPtr getBuffer() { return buffer; } + + Chunk generate() override; + + bool queueEmpty() const { return !buffer || buffer->queueEmpty(); } + bool needChannelUpdate(); + void updateChannel(); + bool sendAck(); + +private: + StorageNATS & storage; + StorageSnapshotPtr storage_snapshot; + ContextPtr context; + Names column_names; + const size_t max_block_size; + bool ack_in_suffix; + + bool is_finished = false; + const Block non_virtual_header; + const Block virtual_header; + + ConsumerBufferPtr buffer; + + NATSSource( + StorageNATS & storage_, + const StorageSnapshotPtr & storage_snapshot_, + std::pair headers, + ContextPtr context_, + const Names & columns, + size_t max_block_size_, + bool ack_in_suffix); + + Chunk generateImpl(); +}; + +} diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp new file mode 100644 index 00000000000..ad966401cc3 --- /dev/null +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -0,0 +1,188 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Poco/Timer.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( + NATSHandler & event_handler_, + std::vector & queues_, + size_t channel_id_base_, + const String & channel_base_, + Poco::Logger * log_, + char row_delimiter_, + uint32_t queue_size_, + const std::atomic & stopped_) + : ReadBuffer(nullptr, 0) + , event_handler(event_handler_) + , queues(queues_) + , channel_base(channel_base_) + , channel_id_base(channel_id_base_) + , log(log_) + , row_delimiter(row_delimiter_) + , stopped(stopped_) + , received(queue_size_) +{ +} + + +ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() +{ + BufferBase::set(nullptr, 0, 0); +} + + +void ReadBufferFromNATSConsumer::subscribe() +{ + for (const auto & queue_name : queues) + { + consumer_channel->consume(queue_name) + .onSuccess([&](const std::string & /* consumer_tag */) + { + LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name); + + if (++subscribed == queues.size()) + wait_subscription.store(false); + }) + .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) + { + if (message.bodySize()) + { + String message_received = std::string(message.body(), message.body() + message.bodySize()); + if (row_delimiter != '\0') + message_received += row_delimiter; + + if (!received.push({message_received, message.hasMessageID() ? message.messageID() : "", + message.hasTimestamp() ? message.timestamp() : 0, + redelivered, AckTracker(delivery_tag, channel_id)})) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); + } + }) + .onError([&](const char * message) + { + /* End up here either if channel ends up in an error state (then there will be resubscription) or consume call error, which + * arises from queue settings mismatch or queue level error, which should not happen as no one else is supposed to touch them + */ + LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message); + wait_subscription.store(false); + }); + } +} + + +bool ReadBufferFromNATSConsumer::ackMessages() +{ + AckTracker record_info = last_inserted_record_info; + + /* Do not send ack to server if message's channel is not the same as current running channel because delivery tags are scoped per + * channel, so if channel fails, all previous delivery tags become invalid + */ + if (record_info.channel_id == channel_id && record_info.delivery_tag && record_info.delivery_tag > prev_tag) + { + /// Commit all received messages with delivery tags from last committed to last inserted + if (!consumer_channel->ack(record_info.delivery_tag, AMQP::multiple)) + { + LOG_ERROR(log, "Failed to commit messages with delivery tags from last committed to {} on channel {}", + record_info.delivery_tag, channel_id); + return false; + } + + prev_tag = record_info.delivery_tag; + LOG_TRACE(log, "Consumer committed messages with deliveryTags up to {} on channel {}", record_info.delivery_tag, channel_id); + } + + return true; +} + + +void ReadBufferFromNATSConsumer::updateAckTracker(AckTracker record_info) +{ + if (record_info.delivery_tag && channel_error.load()) + return; + + if (!record_info.delivery_tag) + prev_tag = 0; + + last_inserted_record_info = record_info; +} + + +void ReadBufferFromNATSConsumer::setupChannel() +{ + if (!consumer_channel) + return; + + wait_subscription.store(true); + + consumer_channel->onReady([&]() + { + /* First number indicates current consumer buffer; second number indicates serial number of created channel for current buffer, + * i.e. if channel fails - another one is created and its serial number is incremented; channel_base is to guarantee that + * channel_id is unique for each table + */ + channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base; + LOG_TRACE(log, "Channel {} is created", channel_id); + + subscribed = 0; + subscribe(); + channel_error.store(false); + }); + + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Channel {} error: {}", channel_id, message); + + channel_error.store(true); + wait_subscription.store(false); + }); +} + + +bool ReadBufferFromNATSConsumer::needChannelUpdate() +{ + if (wait_subscription) + return false; + + return channel_error || !consumer_channel || !consumer_channel->usable(); +} + + +void ReadBufferFromNATSConsumer::iterateEventLoop() +{ + event_handler.iterateLoop(); +} + + +bool ReadBufferFromNATSConsumer::nextImpl() +{ + if (stopped || !allowed) + return false; + + if (received.tryPop(current)) + { + auto * new_position = const_cast(current.message.data()); + BufferBase::set(new_position, current.message.size(), 0); + allowed = false; + + return true; + } + + return false; +} + +} diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h new file mode 100644 index 00000000000..1206ee4e62d --- /dev/null +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -0,0 +1,104 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace Poco +{ + class Logger; +} + +namespace DB +{ + +class ReadBufferFromNATSConsumer : public ReadBuffer +{ + +public: + ReadBufferFromNATSConsumer( + NATSHandler & event_handler_, + std::vector & queues_, + size_t channel_id_base_, + const String & channel_base_, + Poco::Logger * log_, + char row_delimiter_, + uint32_t queue_size_, + const std::atomic & stopped_); + + ~ReadBufferFromNATSConsumer() override; + + struct AckTracker + { + UInt64 delivery_tag; + String channel_id; + + AckTracker() = default; + AckTracker(UInt64 tag, String id) : delivery_tag(tag), channel_id(id) {} + }; + + struct MessageData + { + String message; + String message_id; + uint64_t timestamp = 0; + bool redelivered = false; + AckTracker track{}; + }; + + ChannelPtr & getChannel() { return consumer_channel; } + void setupChannel(); + bool needChannelUpdate(); + void closeChannel() + { + if (consumer_channel) + consumer_channel->close(); + } + + void updateQueues(std::vector & queues_) { queues = queues_; } + size_t queuesCount() { return queues.size(); } + + bool isConsumerStopped() { return stopped; } + bool ackMessages(); + void updateAckTracker(AckTracker record = AckTracker()); + + bool queueEmpty() { return received.empty(); } + void allowNext() { allowed = true; } // Allow to read next message. + + auto getChannelID() const { return current.track.channel_id; } + auto getDeliveryTag() const { return current.track.delivery_tag; } + auto getRedelivered() const { return current.redelivered; } + auto getMessageID() const { return current.message_id; } + auto getTimestamp() const { return current.timestamp; } + +private: + bool nextImpl() override; + + void subscribe(); + void iterateEventLoop(); + + ChannelPtr consumer_channel; + NATSHandler & event_handler; /// Used concurrently, but is thread safe. + std::vector queues; + const String channel_base; + const size_t channel_id_base; + Poco::Logger * log; + char row_delimiter; + bool allowed = true; + const std::atomic & stopped; + + String channel_id; + std::atomic channel_error = true, wait_subscription = false; + ConcurrentBoundedQueue received; + MessageData current; + size_t subscribed = 0; + + AckTracker last_inserted_record_info; + UInt64 prev_tag = 0, channel_id_counter = 0; +}; + +} diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp new file mode 100644 index 00000000000..41ab2d1fc9b --- /dev/null +++ b/src/Storages/NATS/StorageNATS.cpp @@ -0,0 +1,1186 @@ +#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 +{ + +static const uint32_t QUEUE_SIZE = 100000; +static const auto MAX_FAILED_READ_ATTEMPTS = 10; +static const auto RESCHEDULE_MS = 500; +static const auto BACKOFF_TRESHOLD = 32000; +static const auto MAX_THREAD_WORK_DURATION_MS = 60000; + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int CANNOT_CONNECT_RABBITMQ; + extern const int CANNOT_BIND_RABBITMQ_EXCHANGE; + extern const int CANNOT_DECLARE_RABBITMQ_EXCHANGE; + extern const int CANNOT_REMOVE_RABBITMQ_EXCHANGE; + extern const int CANNOT_CREATE_RABBITMQ_QUEUE_BINDING; + extern const int QUERY_NOT_ALLOWED; +} + +namespace ExchangeType +{ + /// Note that default here means default by implementation and not by nats settings + static const String DEFAULT = "default"; + static const String FANOUT = "fanout"; + static const String DIRECT = "direct"; + static const String TOPIC = "topic"; + static const String HASH = "consistent_hash"; + static const String HEADERS = "headers"; +} + + +StorageNATS::StorageNATS( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr nats_settings_, + bool is_attach_) + : IStorage(table_id_) + , WithContext(context_->getGlobalContext()) + , nats_settings(std::move(nats_settings_)) + , exchange_name(getContext()->getMacros()->expand(nats_settings->nats_exchange_name)) + , format_name(getContext()->getMacros()->expand(nats_settings->nats_format)) + , exchange_type(defineExchangeType(getContext()->getMacros()->expand(nats_settings->nats_exchange_type))) + , routing_keys(parseSettings(getContext()->getMacros()->expand(nats_settings->nats_routing_key_list))) + , row_delimiter(nats_settings->nats_row_delimiter.value) + , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) + , num_consumers(nats_settings->nats_num_consumers.value) + , num_queues(nats_settings->nats_num_queues.value) + , queue_base(getContext()->getMacros()->expand(nats_settings->nats_queue_base)) + , queue_settings_list(parseSettings(getContext()->getMacros()->expand(nats_settings->nats_queue_settings_list))) + , persistent(nats_settings->nats_persistent.value) + , use_user_setup(nats_settings->nats_queue_consume.value) + , hash_exchange(num_consumers > 1 || num_queues > 1) + , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) + , semaphore(0, num_consumers) + , unique_strbase(getRandomName()) + , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) + , milliseconds_to_wait(RESCHEDULE_MS) + , is_attach(is_attach_) +{ + auto parsed_address = parseAddress(getContext()->getMacros()->expand(nats_settings->nats_host_port), 5672); + context_->getRemoteHostFilter().checkHostAndPort(parsed_address.first, toString(parsed_address.second)); + + auto nats_username = nats_settings->nats_username.value; + auto nats_password = nats_settings->nats_password.value; + configuration = + { + .host = parsed_address.first, + .port = parsed_address.second, + .username = nats_username.empty() ? getContext()->getConfigRef().getString("nats.username") : nats_username, + .password = nats_password.empty() ? getContext()->getConfigRef().getString("nats.password") : nats_password, + .vhost = getContext()->getConfigRef().getString("nats.vhost", getContext()->getMacros()->expand(nats_settings->nats_vhost)), + .secure = nats_settings->nats_secure.value, + .connection_string = getContext()->getMacros()->expand(nats_settings->nats_address) + }; + + if (configuration.secure) + SSL_library_init(); + + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); + + nats_context = addSettings(getContext()); + nats_context->makeQueryContext(); + + if (queue_base.empty()) + { + /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to + * be table-based and not just a random string, because local exchanges should be declared the same for same tables + */ + sharding_exchange = getTableBasedName(exchange_name, table_id_); + + /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better + * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every + * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base + * for the names of later declared queues + */ + queue_base = getTableBasedName("", table_id_); + } + else + { + /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and + * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need to share + * sharding exchange and bridge exchange + */ + sharding_exchange = exchange_name + "_" + queue_base; + } + + bridge_exchange = sharding_exchange + "_bridge"; + + try + { + connection = std::make_unique(configuration, log); + if (connection->connect()) + initNATS(); + else if (!is_attach) + throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to {}", connection->connectionInfoForLog()); + } + catch (...) + { + tryLogCurrentException(log); + if (!is_attach) + throw; + } + + /// One looping task for all consumers as they share the same connection == the same handler == the same event loop + looping_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSLoopingTask", [this]{ loopingFunc(); }); + looping_task->deactivate(); + + streaming_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSStreamingTask", [this]{ streamingToViewsFunc(); }); + streaming_task->deactivate(); + + connection_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSConnectionTask", [this]{ connectionFunc(); }); + connection_task->deactivate(); +} + + +Names StorageNATS::parseSettings(String settings_list) +{ + Names result; + if (settings_list.empty()) + return result; + boost::split(result, settings_list, [](char c){ return c == ','; }); + for (String & key : result) + boost::trim(key); + + return result; +} + + +AMQP::ExchangeType StorageNATS::defineExchangeType(String exchange_type_) +{ + AMQP::ExchangeType type; + if (exchange_type_ != ExchangeType::DEFAULT) + { + if (exchange_type_ == ExchangeType::FANOUT) type = AMQP::ExchangeType::fanout; + else if (exchange_type_ == ExchangeType::DIRECT) type = AMQP::ExchangeType::direct; + else if (exchange_type_ == ExchangeType::TOPIC) type = AMQP::ExchangeType::topic; + else if (exchange_type_ == ExchangeType::HASH) type = AMQP::ExchangeType::consistent_hash; + else if (exchange_type_ == ExchangeType::HEADERS) type = AMQP::ExchangeType::headers; + else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); + } + else + { + type = AMQP::ExchangeType::fanout; + } + + return type; +} + + +String StorageNATS::getTableBasedName(String name, const StorageID & table_id) +{ + if (name.empty()) + return fmt::format("{}_{}", table_id.database_name, table_id.table_name); + else + return fmt::format("{}_{}_{}", name, table_id.database_name, table_id.table_name); +} + + +ContextMutablePtr StorageNATS::addSettings(ContextPtr local_context) const +{ + auto modified_context = Context::createCopy(local_context); + modified_context->setSetting("input_format_skip_unknown_fields", true); + modified_context->setSetting("input_format_allow_errors_ratio", 0.); + modified_context->setSetting("input_format_allow_errors_num", nats_settings->nats_skip_broken_messages.value); + + if (!schema_name.empty()) + modified_context->setSetting("format_schema", schema_name); + + for (const auto & setting : *nats_settings) + { + const auto & setting_name = setting.getName(); + + /// check for non-nats-related settings + if (!setting_name.starts_with("nats_")) + modified_context->setSetting(setting_name, setting.getValue()); + } + + return modified_context; +} + + +void StorageNATS::loopingFunc() +{ + connection->getHandler().startLoop(); +} + + +void StorageNATS::stopLoop() +{ + connection->getHandler().updateLoopState(Loop::STOP); +} + +void StorageNATS::stopLoopIfNoReaders() +{ + /// Stop the loop if no select was started. + /// There can be a case that selects are finished + /// but not all sources decremented the counter, then + /// it is ok that the loop is not stopped, because + /// there is a background task (streaming_task), which + /// also checks whether there is an idle loop. + std::lock_guard lock(loop_mutex); + if (readers_count) + return; + connection->getHandler().updateLoopState(Loop::STOP); +} + +void StorageNATS::startLoop() +{ + assert(rabbit_is_ready); + connection->getHandler().updateLoopState(Loop::RUN); + looping_task->activateAndSchedule(); +} + + +void StorageNATS::incrementReader() +{ + ++readers_count; +} + + +void StorageNATS::decrementReader() +{ + --readers_count; +} + + +void StorageNATS::connectionFunc() +{ + if (rabbit_is_ready) + return; + + if (connection->reconnect()) + initNATS(); + else + connection_task->scheduleAfter(RESCHEDULE_MS); +} + + +/* Need to deactivate this way because otherwise might get a deadlock when first deactivate streaming task in shutdown and then + * inside streaming task try to deactivate any other task + */ +void StorageNATS::deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop) +{ + if (stop_loop) + stopLoop(); + + std::unique_lock lock(task_mutex, std::defer_lock); + if (lock.try_lock()) + { + task->deactivate(); + lock.unlock(); + } + else if (wait) /// Wait only if deactivating from shutdown + { + lock.lock(); + task->deactivate(); + } +} + + +size_t StorageNATS::getMaxBlockSize() const +{ + return nats_settings->nats_max_block_size.changed + ? nats_settings->nats_max_block_size.value + : (getContext()->getSettingsRef().max_insert_block_size.value / num_consumers); +} + + +void StorageNATS::initNATS() +{ + if (shutdown_called || rabbit_is_ready) + return; + + if (use_user_setup) + { + queues.emplace_back(queue_base); + rabbit_is_ready = true; + return; + } + + try + { + auto rabbit_channel = connection->createChannel(); + + /// Main exchange -> Bridge exchange -> ( Sharding exchange ) -> Queues -> Consumers + + initExchange(*rabbit_channel); + bindExchange(*rabbit_channel); + + for (const auto i : collections::range(0, num_queues)) + bindQueue(i + 1, *rabbit_channel); + + LOG_TRACE(log, "NATS setup completed"); + rabbit_is_ready = true; + rabbit_channel->close(); + } + catch (...) + { + tryLogCurrentException(log); + if (!is_attach) + throw; + } +} + + +void StorageNATS::initExchange(AMQP::TcpChannel & rabbit_channel) +{ + /// Exchange hierarchy: + /// 1. Main exchange (defined with table settings - nats_exchange_name, nats_exchange_type). + /// 2. Bridge exchange (fanout). Used to easily disconnect main exchange and to simplify queue bindings. + /// 3. Sharding (or hash) exchange. Used in case of multiple queues. + /// 4. Consumer exchange. Just an alias for bridge_exchange or sharding exchange to know to what exchange + /// queues will be bound. + + /// All exchanges are declared with options: + /// 1. `durable` (survive NATS server restart) + /// 2. `autodelete` (auto delete in case of queue bindings are dropped). + + rabbit_channel.declareExchange(exchange_name, exchange_type, AMQP::durable) + .onError([&](const char * message) + { + /// This error can be a result of attempt to declare exchange if it was already declared but + /// 1) with different exchange type. + /// 2) with different exchange settings. + throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " + + std::string(message), ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE); + }); + + rabbit_channel.declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable | AMQP::autodelete) + .onError([&](const char * message) + { + /// This error is not supposed to happen as this exchange name is always unique to type and its settings. + throw Exception( + ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, "Unable to declare bridge exchange ({}). Reason: {}", bridge_exchange, std::string(message)); + }); + + if (!hash_exchange) + { + consumer_exchange = bridge_exchange; + return; + } + + AMQP::Table binding_arguments; + + /// Default routing key property in case of hash exchange is a routing key, which is required to be an integer. + /// Support for arbitrary exchange type (i.e. arbitrary pattern of routing keys) requires to eliminate this dependency. + /// This settings changes hash property to message_id. + binding_arguments["hash-property"] = "message_id"; + + /// Declare hash exchange for sharding. + rabbit_channel.declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable | AMQP::autodelete, binding_arguments) + .onError([&](const char * message) + { + /// This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared + /// to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter + /// is bad. + throw Exception( + ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, + "Unable to declare sharding exchange ({}). Reason: {}", sharding_exchange, std::string(message)); + }); + + rabbit_channel.bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) + .onError([&](const char * message) + { + throw Exception( + ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, + "Unable to bind bridge exchange ({}) to sharding exchange ({}). Reason: {}", + bridge_exchange, + sharding_exchange, + std::string(message)); + }); + + consumer_exchange = sharding_exchange; +} + + +void StorageNATS::bindExchange(AMQP::TcpChannel & rabbit_channel) +{ + size_t bound_keys = 0; + + if (exchange_type == AMQP::ExchangeType::headers) + { + AMQP::Table bind_headers; + for (const auto & header : routing_keys) + { + std::vector matching; + boost::split(matching, header, [](char c){ return c == '='; }); + bind_headers[matching[0]] = matching[1]; + } + + rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) + .onSuccess([&]() { connection->getHandler().stopLoop(); }) + .onError([&](const char * message) + { + throw Exception( + ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, + "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", + exchange_name, bridge_exchange, std::string(message)); + }); + } + else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) + { + rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0]) + .onSuccess([&]() { connection->getHandler().stopLoop(); }) + .onError([&](const char * message) + { + throw Exception( + ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, + "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", + exchange_name, bridge_exchange, std::string(message)); + }); + } + else + { + for (const auto & routing_key : routing_keys) + { + rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_key) + .onSuccess([&]() + { + ++bound_keys; + if (bound_keys == routing_keys.size()) + connection->getHandler().stopLoop(); + }) + .onError([&](const char * message) + { + throw Exception( + ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, + "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", + exchange_name, bridge_exchange, std::string(message)); + }); + } + } + + connection->getHandler().startBlockingLoop(); +} + + +void StorageNATS::bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel) +{ + auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) + { + queues.emplace_back(queue_name); + LOG_DEBUG(log, "Queue {} is declared", queue_name); + + if (msgcount) + LOG_INFO(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); + + /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are + * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for + * fanout exchange it can be arbitrary + */ + rabbit_channel.bindQueue(consumer_exchange, queue_name, std::to_string(queue_id)) + .onSuccess([&] { connection->getHandler().stopLoop(); }) + .onError([&](const char * message) + { + throw Exception( + ErrorCodes::CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, + "Failed to create queue binding for exchange {}. Reason: {}", exchange_name, std::string(message)); + }); + }; + + auto error_callback([&](const char * message) + { + /* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a + * given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different + * max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously + * declared queues via any of the various cli tools. + */ + throw Exception("Failed to declare queue. Probably queue settings are conflicting: max_block_size, deadletter_exchange. Attempt \ + specifying differently those settings or use a different queue_base or manually delete previously declared queues, \ + which were declared with the same names. ERROR reason: " + + std::string(message), ErrorCodes::BAD_ARGUMENTS); + }); + + AMQP::Table queue_settings; + + std::unordered_set integer_settings = {"x-max-length", "x-max-length-bytes", "x-message-ttl", "x-expires", "x-priority", "x-max-priority"}; + std::unordered_set string_settings = {"x-overflow", "x-dead-letter-exchange", "x-queue-type"}; + + /// Check user-defined settings. + if (!queue_settings_list.empty()) + { + for (const auto & setting : queue_settings_list) + { + Strings setting_values; + splitInto<'='>(setting_values, setting); + if (setting_values.size() != 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid settings string: {}", setting); + + String key = setting_values[0], value = setting_values[1]; + + if (integer_settings.contains(key)) + queue_settings[key] = parse(value); + else if (string_settings.find(key) != string_settings.end()) + queue_settings[key] = value; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported queue setting: {}", value); + } + } + + /// Impose default settings if there are no user-defined settings. + if (!queue_settings.contains("x-max-length")) + { + queue_settings["x-max-length"] = queue_size; + } + if (!queue_settings.contains("x-overflow")) + { + queue_settings["x-overflow"] = "reject-publish"; + } + + /// If queue_base - a single name, then it can be used as one specific queue, from which to read. + /// Otherwise it is used as a generator (unique for current table) of queue names, because it allows to + /// maximize performance - via setting `nats_num_queues`. + const String queue_name = !hash_exchange ? queue_base : std::to_string(queue_id) + "_" + queue_base; + + /// AMQP::autodelete setting is not allowed, because in case of server restart there will be no consumers + /// and deleting queues should not take place. + rabbit_channel.declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); + connection->getHandler().startBlockingLoop(); +} + + +bool StorageNATS::updateChannel(ChannelPtr & channel) +{ + try + { + channel = connection->createChannel(); + return true; + } + catch (...) + { + tryLogCurrentException(log); + return false; + } +} + + +void StorageNATS::prepareChannelForBuffer(ConsumerBufferPtr buffer) +{ + if (!buffer) + return; + + if (buffer->queuesCount() != queues.size()) + buffer->updateQueues(queues); + + buffer->updateAckTracker(); + + if (updateChannel(buffer->getChannel())) + buffer->setupChannel(); +} + + +void StorageNATS::unbindExchange() +{ + /* This is needed because with NATS (without special adjustments) can't, for example, properly make mv if there was insert query + * on the same table before, and in another direction it will make redundant copies, but most likely nobody will do that. + * As publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in + * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as + * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible + * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local + * queues, then messages will go both ways and in one of them they will remain not consumed. So need to disconnect local exchange + * bindings to remove redunadant message copies, but after that mv cannot work unless those bindings are recreated. Recreating them is + * not difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. + */ + if (!exchange_removed.exchange(true)) + { + try + { + streaming_task->deactivate(); + + stopLoop(); + looping_task->deactivate(); + + auto rabbit_channel = connection->createChannel(); + rabbit_channel->removeExchange(bridge_exchange) + .onSuccess([&]() + { + connection->getHandler().stopLoop(); + }) + .onError([&](const char * message) + { + throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_REMOVE_RABBITMQ_EXCHANGE); + }); + + connection->getHandler().startBlockingLoop(); + rabbit_channel->close(); + } + catch (...) + { + exchange_removed = false; + throw; + } + } +} + + +Pipe StorageNATS::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /* query_info */, + ContextPtr local_context, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + unsigned /* num_streams */) +{ + if (!rabbit_is_ready) + throw Exception("NATS setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_RABBITMQ); + + if (num_created_consumers == 0) + return {}; + + if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + + if (mv_attached) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageNATS with attached materialized views"); + + std::lock_guard lock(loop_mutex); + + auto sample_block = storage_snapshot->getSampleBlockForColumns(column_names); + auto modified_context = addSettings(local_context); + + if (!connection->isConnected()) + { + if (connection->getHandler().loopRunning()) + deactivateTask(looping_task, false, true); + if (!connection->reconnect()) + throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "No connection to {}", connection->connectionInfoForLog()); + } + + initializeBuffers(); + + Pipes pipes; + pipes.reserve(num_created_consumers); + + for (size_t i = 0; i < num_created_consumers; ++i) + { + auto rabbit_source = std::make_shared( + *this, storage_snapshot, modified_context, column_names, 1, nats_settings->nats_commit_on_select); + + auto converting_dag = ActionsDAG::makeConvertingActions( + rabbit_source->getPort().getHeader().getColumnsWithTypeAndName(), + sample_block.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting = std::make_shared(std::move(converting_dag)); + auto converting_transform = std::make_shared(rabbit_source->getPort().getHeader(), std::move(converting)); + + pipes.emplace_back(std::move(rabbit_source)); + pipes.back().addTransform(std::move(converting_transform)); + } + + if (!connection->getHandler().loopRunning() && connection->isConnected()) + startLoop(); + + LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); + auto united_pipe = Pipe::unitePipes(std::move(pipes)); + united_pipe.addInterpreterContext(modified_context); + return united_pipe; +} + + +SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +{ + return std::make_shared(*this, metadata_snapshot, local_context); +} + + +void StorageNATS::startup() +{ + if (!rabbit_is_ready) + { + if (connection->isConnected()) + { + try + { + initNATS(); + } + catch (...) + { + if (!is_attach) + throw; + tryLogCurrentException(log); + } + } + else + { + connection_task->activateAndSchedule(); + } + } + + for (size_t i = 0; i < num_consumers; ++i) + { + try + { + auto buffer = createReadBuffer(); + pushReadBuffer(std::move(buffer)); + ++num_created_consumers; + } + catch (...) + { + if (!is_attach) + throw; + tryLogCurrentException(log); + } + } + + streaming_task->activateAndSchedule(); +} + + +void StorageNATS::shutdown() +{ + shutdown_called = true; + + /// In case it has not yet been able to setup connection; + deactivateTask(connection_task, true, false); + + /// The order of deactivating tasks is important: wait for streamingToViews() func to finish and + /// then wait for background event loop to finish. + deactivateTask(streaming_task, true, false); + deactivateTask(looping_task, true, true); + + /// Just a paranoid try catch, it is not actually needed. + try + { + if (drop_table) + { + for (auto & buffer : buffers) + buffer->closeChannel(); + + cleanupNATS(); + } + + /// It is important to close connection here - before removing consumer buffers, because + /// it will finish and clean callbacks, which might use those buffers data. + connection->disconnect(); + + for (size_t i = 0; i < num_created_consumers; ++i) + popReadBuffer(); + } + catch (...) + { + tryLogCurrentException(log); + } +} + + +/// The only thing publishers are supposed to be aware of is _exchanges_ and queues are a responsibility of a consumer. +/// Therefore, if a table is dropped, a clean up is needed. +void StorageNATS::cleanupNATS() const +{ + if (use_user_setup) + return; + + connection->heartbeat(); + if (!connection->isConnected()) + { + String queue_names; + for (const auto & queue : queues) + { + if (!queue_names.empty()) + queue_names += ", "; + queue_names += queue; + } + LOG_WARNING(log, + "NATS clean up not done, because there is no connection in table's shutdown." + "There are {} queues ({}), which might need to be deleted manually. Exchanges will be auto-deleted", + queues.size(), queue_names); + return; + } + + auto rabbit_channel = connection->createChannel(); + for (const auto & queue : queues) + { + /// AMQP::ifunused is needed, because it is possible to share queues between multiple tables and dropping + /// on of them should not affect others. + /// AMQP::ifempty is not used on purpose. + + rabbit_channel->removeQueue(queue, AMQP::ifunused) + .onSuccess([&](uint32_t num_messages) + { + LOG_TRACE(log, "Successfully deleted queue {}, messages contained {}", queue, num_messages); + connection->getHandler().stopLoop(); + }) + .onError([&](const char * message) + { + LOG_ERROR(log, "Failed to delete queue {}. Error message: {}", queue, message); + connection->getHandler().stopLoop(); + }); + } + connection->getHandler().startBlockingLoop(); + rabbit_channel->close(); + + /// Also there is no need to cleanup exchanges as they were created with AMQP::autodelete option. Once queues + /// are removed, exchanges will also be cleaned. +} + + +void StorageNATS::pushReadBuffer(ConsumerBufferPtr buffer) +{ + std::lock_guard lock(buffers_mutex); + buffers.push_back(buffer); + semaphore.set(); +} + + +ConsumerBufferPtr StorageNATS::popReadBuffer() +{ + return popReadBuffer(std::chrono::milliseconds::zero()); +} + + +ConsumerBufferPtr StorageNATS::popReadBuffer(std::chrono::milliseconds timeout) +{ + // Wait for the first free buffer + if (timeout == std::chrono::milliseconds::zero()) + semaphore.wait(); + else + { + if (!semaphore.tryWait(timeout.count())) + return nullptr; + } + + // Take the first available buffer from the list + std::lock_guard lock(buffers_mutex); + auto buffer = buffers.back(); + buffers.pop_back(); + + return buffer; +} + + +ConsumerBufferPtr StorageNATS::createReadBuffer() +{ + return std::make_shared( + connection->getHandler(), queues, ++consumer_id, + unique_strbase, log, row_delimiter, queue_size, shutdown_called); +} + + +ProducerBufferPtr StorageNATS::createWriteBuffer() +{ + return std::make_shared( + configuration, getContext(), routing_keys, exchange_name, exchange_type, + producer_id.fetch_add(1), persistent, shutdown_called, log, + row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); +} + + +bool StorageNATS::checkDependencies(const StorageID & table_id) +{ + // Check if all dependencies are attached + auto dependencies = DatabaseCatalog::instance().getDependencies(table_id); + if (dependencies.empty()) + return true; + + // Check the dependencies are ready? + for (const auto & db_tab : dependencies) + { + auto table = DatabaseCatalog::instance().tryGetTable(db_tab, getContext()); + if (!table) + return false; + + // If it materialized view, check it's target table + auto * materialized_view = dynamic_cast(table.get()); + if (materialized_view && !materialized_view->tryGetTargetTable()) + return false; + + // Check all its dependencies + if (!checkDependencies(db_tab)) + return false; + } + + return true; +} + + +void StorageNATS::initializeBuffers() +{ + assert(rabbit_is_ready); + if (!initialized) + { + for (const auto & buffer : buffers) + prepareChannelForBuffer(buffer); + initialized = true; + } +} + + +void StorageNATS::streamingToViewsFunc() +{ + if (rabbit_is_ready) + { + try + { + auto table_id = getStorageID(); + + // Check if at least one direct dependency is attached + size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + bool rabbit_connected = connection->isConnected() || connection->reconnect(); + + if (dependencies_count && rabbit_connected) + { + initializeBuffers(); + auto start_time = std::chrono::steady_clock::now(); + + mv_attached.store(true); + + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!shutdown_called && num_created_consumers > 0) + { + if (!checkDependencies(table_id)) + break; + + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + + if (streamToViews()) + { + /// Reschedule with backoff. + if (milliseconds_to_wait < BACKOFF_TRESHOLD) + milliseconds_to_wait *= 2; + stopLoopIfNoReaders(); + break; + } + else + { + milliseconds_to_wait = RESCHEDULE_MS; + } + + auto end_time = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + if (duration.count() > MAX_THREAD_WORK_DURATION_MS) + { + stopLoopIfNoReaders(); + LOG_TRACE(log, "Reschedule streaming. Thread work duration limit exceeded."); + break; + } + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + mv_attached.store(false); + + /// If there is no running select, stop the loop which was + /// activated by previous select. + if (connection->getHandler().loopRunning()) + stopLoopIfNoReaders(); + + if (!shutdown_called) + streaming_task->scheduleAfter(milliseconds_to_wait); +} + + +bool StorageNATS::streamToViews() +{ + auto table_id = getStorageID(); + auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); + if (!table) + throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); + + // Create an INSERT query for streaming data + auto insert = std::make_shared(); + insert->table_id = table_id; + + // Only insert into dependent views and expect that input blocks contain virtual columns + InterpreterInsertQuery interpreter(insert, nats_context, false, true, true); + auto block_io = interpreter.execute(); + + auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); + auto column_names = block_io.pipeline.getHeader().getNames(); + auto sample_block = storage_snapshot->getSampleBlockForColumns(column_names); + + auto block_size = getMaxBlockSize(); + + // Create a stream for each consumer and join them in a union stream + std::vector> sources; + Pipes pipes; + sources.reserve(num_created_consumers); + pipes.reserve(num_created_consumers); + + for (size_t i = 0; i < num_created_consumers; ++i) + { + auto source = std::make_shared( + *this, storage_snapshot, nats_context, column_names, block_size, false); + sources.emplace_back(source); + pipes.emplace_back(source); + + // Limit read batch to maximum block size to allow DDL + StreamLocalLimits limits; + + limits.speed_limits.max_execution_time = nats_settings->nats_flush_interval_ms.changed + ? nats_settings->nats_flush_interval_ms + : getContext()->getSettingsRef().stream_flush_interval_ms; + + limits.timeout_overflow_mode = OverflowMode::BREAK; + + source->setLimits(limits); + } + + block_io.pipeline.complete(Pipe::unitePipes(std::move(pipes))); + + if (!connection->getHandler().loopRunning()) + startLoop(); + + { + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); + } + + /* Note: sending ack() with loop running in another thread will lead to a lot of data races inside the library, but only in case + * error occurs or connection is lost while ack is being sent + */ + deactivateTask(looping_task, false, true); + size_t queue_empty = 0; + + if (!connection->isConnected()) + { + if (shutdown_called) + return true; + + if (connection->reconnect()) + { + LOG_DEBUG(log, "Connection restored, updating channels"); + for (auto & source : sources) + source->updateChannel(); + } + else + { + LOG_TRACE(log, "Reschedule streaming. Unable to restore connection."); + return true; + } + } + else + { + /// Commit + for (auto & source : sources) + { + if (source->queueEmpty()) + ++queue_empty; + + if (source->needChannelUpdate()) + { + auto buffer = source->getBuffer(); + prepareChannelForBuffer(buffer); + } + + /* false is returned by the sendAck function in only two cases: + * 1) if connection failed. In this case all channels will be closed and will be unable to send ack. Also ack is made based on + * delivery tags, which are unique to channels, so if channels fail, those delivery tags will become invalid and there is + * no way to send specific ack from a different channel. Actually once the server realises that it has messages in a queue + * waiting for confirm from a channel which suddenly closed, it will immediately make those messages accessible to other + * consumers. So in this case duplicates are inevitable. + * 2) size of the sent frame (libraries's internal request interface) exceeds max frame - internal library error. This is more + * common for message frames, but not likely to happen to ack frame I suppose. So I do not believe it is likely to happen. + * Also in this case if channel didn't get closed - it is ok if failed to send ack, because the next attempt to send ack on + * the same channel will also commit all previously not-committed messages. Anyway I do not think that for ack frame this + * will ever happen. + */ + if (!source->sendAck()) + { + /// Iterate loop to activate error callbacks if they happened + connection->getHandler().iterateLoop(); + if (!connection->isConnected()) + break; + } + + connection->getHandler().iterateLoop(); + } + } + + if ((queue_empty == num_created_consumers) && (++read_attempts == MAX_FAILED_READ_ATTEMPTS)) + { + connection->heartbeat(); + read_attempts = 0; + LOG_TRACE(log, "Reschedule streaming. Queues are empty."); + return true; + } + else + { + startLoop(); + } + + /// Do not reschedule, do not stop event loop. + return false; +} + + +void registerStorageNATS(StorageFactory & factory) +{ + auto creator_fn = [](const StorageFactory::Arguments & args) + { + auto nats_settings = std::make_unique(); + bool with_named_collection = getExternalDataSourceConfiguration(args.engine_args, *nats_settings, args.getLocalContext()); + if (!with_named_collection && !args.storage_def->settings) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "NATS engine must have settings"); + + nats_settings->loadFromQuery(*args.storage_def); + + if (!nats_settings->nats_host_port.changed + && !nats_settings->nats_address.changed) + throw Exception("You must specify either `nats_host_port` or `nats_address` settings", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!nats_settings->nats_format.changed) + throw Exception("You must specify `nats_format` setting", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return StorageNATS::create(args.table_id, args.getContext(), args.columns, std::move(nats_settings), args.attach); + }; + + factory.registerStorage("NATS", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); +} + + +NamesAndTypesList StorageNATS::getVirtuals() const +{ + return NamesAndTypesList{ + {"_exchange_name", std::make_shared()}, + {"_channel_id", std::make_shared()}, + {"_delivery_tag", std::make_shared()}, + {"_redelivered", std::make_shared()}, + {"_message_id", std::make_shared()}, + {"_timestamp", std::make_shared()} + }; +} + +} diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h new file mode 100644 index 00000000000..e5b397194b7 --- /dev/null +++ b/src/Storages/NATS/StorageNATS.h @@ -0,0 +1,207 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class StorageNATS final: public shared_ptr_helper, public IStorage, WithContext +{ + friend struct shared_ptr_helper; + +public: + std::string getName() const override { return "NATS"; } + + bool noPushingToViews() const override { return true; } + + void startup() override; + void shutdown() override; + + /// This is a bad way to let storage know in shutdown() that table is going to be dropped. There are some actions which need + /// to be done only when table is dropped (not when detached). Also connection must be closed only in shutdown, but those + /// actions require an open connection. Therefore there needs to be a way inside shutdown() method to know whether it is called + /// because of drop query. And drop() method is not suitable at all, because it will not only require to reopen connection, but also + /// it can be called considerable time after table is dropped (for example, in case of Atomic database), which is not appropriate for the case. + void checkTableCanBeDropped() const override { drop_table = true; } + + /// Always return virtual columns in addition to required columns + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context) override; + + void pushReadBuffer(ConsumerBufferPtr buf); + ConsumerBufferPtr popReadBuffer(); + ConsumerBufferPtr popReadBuffer(std::chrono::milliseconds timeout); + + ProducerBufferPtr createWriteBuffer(); + + const String & getFormatName() const { return format_name; } + NamesAndTypesList getVirtuals() const override; + + String getExchange() const { return exchange_name; } + void unbindExchange(); + + bool updateChannel(ChannelPtr & channel); + void updateQueues(std::vector & queues_) { queues_ = queues; } + void prepareChannelForBuffer(ConsumerBufferPtr buffer); + + void incrementReader(); + void decrementReader(); + +protected: + StorageNATS( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr nats_settings_, + bool is_attach_); + +private: + ContextMutablePtr nats_context; + std::unique_ptr nats_settings; + + const String exchange_name; + const String format_name; + AMQP::ExchangeType exchange_type; + Names routing_keys; + char row_delimiter; + const String schema_name; + size_t num_consumers; + size_t num_queues; + String queue_base; + Names queue_settings_list; + + /// For insert query. Mark messages as durable. + const bool persistent; + + /// A table setting. It is possible not to perform any NATS setup, which is supposed to be consumer-side setup: + /// declaring exchanges, queues, bindings. Instead everything needed from NATS table is to connect to a specific queue. + /// This solution disables all optimizations and is not really optimal, but allows user to fully control all NATS setup. + bool use_user_setup; + + bool hash_exchange; + Poco::Logger * log; + + NATSConnectionPtr connection; /// Connection for all consumers + NATSConfiguration configuration; + + size_t num_created_consumers = 0; + Poco::Semaphore semaphore; + std::mutex buffers_mutex; + std::vector buffers; /// available buffers for NATS consumers + + String unique_strbase; /// to make unique consumer channel id + + /// maximum number of messages in NATS queue (x-max-length). Also used + /// to setup size of inner buffer for received messages + uint32_t queue_size; + + String sharding_exchange, bridge_exchange, consumer_exchange; + size_t consumer_id = 0; /// counter for consumer buffer, needed for channel id + + std::vector queues; + + std::once_flag flag; /// remove exchange only once + std::mutex task_mutex; + BackgroundSchedulePool::TaskHolder streaming_task; + BackgroundSchedulePool::TaskHolder looping_task; + BackgroundSchedulePool::TaskHolder connection_task; + + uint64_t milliseconds_to_wait; + + /** + * ╰( ͡° ͜ʖ ͡° )つ──☆* Evil atomics: + */ + /// Needed for tell MV or producer background tasks + /// that they must finish as soon as possible. + std::atomic shutdown_called{false}; + /// Counter for producer buffers, needed for channel id. + /// Needed to generate unique producer buffer identifiers. + std::atomic producer_id = 1; + /// Has connection background task completed successfully? + /// It is started only once -- in constructor. + std::atomic rabbit_is_ready = false; + /// Allow to remove exchange only once. + std::atomic exchange_removed = false; + /// For select query we must be aware of the end of streaming + /// to be able to turn off the loop. + std::atomic readers_count = 0; + std::atomic mv_attached = false; + + /// In select query we start event loop, but do not stop it + /// after that select is finished. Then in a thread, which + /// checks for MV we also check if we have select readers. + /// If not - we turn off the loop. The checks are done under + /// mutex to avoid having a turned off loop when select was + /// started. + std::mutex loop_mutex; + + size_t read_attempts = 0; + mutable bool drop_table = false; + bool is_attach; + + ConsumerBufferPtr createReadBuffer(); + void initializeBuffers(); + bool initialized = false; + + /// Functions working in the background + void streamingToViewsFunc(); + void loopingFunc(); + void connectionFunc(); + + void startLoop(); + void stopLoop(); + void stopLoopIfNoReaders(); + + static Names parseSettings(String settings_list); + static AMQP::ExchangeType defineExchangeType(String exchange_type_); + static String getTableBasedName(String name, const StorageID & table_id); + + ContextMutablePtr addSettings(ContextPtr context) const; + size_t getMaxBlockSize() const; + void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); + + void initNATS(); + void cleanupNATS() const; + + void initExchange(AMQP::TcpChannel & rabbit_channel); + void bindExchange(AMQP::TcpChannel & rabbit_channel); + void bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel); + + bool streamToViews(); + bool checkDependencies(const StorageID & table_id); + + static String getRandomName() + { + std::uniform_int_distribution distribution('a', 'z'); + String random_str(32, ' '); + for (auto & c : random_str) + c = distribution(thread_local_rng); + return random_str; + } +}; + +} diff --git a/src/Storages/NATS/UVLoop.h b/src/Storages/NATS/UVLoop.h new file mode 100644 index 00000000000..4de67cbc206 --- /dev/null +++ b/src/Storages/NATS/UVLoop.h @@ -0,0 +1,44 @@ +#pragma once + +#include + +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYSTEM_ERROR; +} + +/// RAII wrapper around uv event loop +class UVLoop : public boost::noncopyable +{ +public: + UVLoop(): loop_ptr(new uv_loop_t()) + { + int res = uv_loop_init(loop_ptr.get()); + + if (res != 0) + throw Exception("UVLoop could not initialize", ErrorCodes::SYSTEM_ERROR); + } + + ~UVLoop() + { + if (loop_ptr) + uv_loop_close(loop_ptr.get()); + } + + inline uv_loop_t * getLoop() { return loop_ptr.get(); } + + inline const uv_loop_t * getLoop() const { return loop_ptr.get(); } + +private: + std::unique_ptr loop_ptr; +}; + +} diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp new file mode 100644 index 00000000000..2f4ea7bee53 --- /dev/null +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -0,0 +1,317 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +static const auto BATCH = 1000; +static const auto RETURNED_LIMIT = 50000; + +namespace ErrorCodes +{ + extern const int CANNOT_CONNECT_RABBITMQ; + extern const int LOGICAL_ERROR; +} + +WriteBufferToNATSProducer::WriteBufferToNATSProducer( + const NATSConfiguration & configuration_, + ContextPtr global_context, + const Names & routing_keys_, + const String & exchange_name_, + const AMQP::ExchangeType exchange_type_, + const size_t channel_id_base_, + const bool persistent_, + std::atomic & shutdown_called_, + Poco::Logger * log_, + std::optional delimiter, + size_t rows_per_message, + size_t chunk_size_) + : WriteBuffer(nullptr, 0) + , connection(configuration_, log_) + , routing_keys(routing_keys_) + , exchange_name(exchange_name_) + , exchange_type(exchange_type_) + , channel_id_base(std::to_string(channel_id_base_)) + , persistent(persistent_) + , shutdown_called(shutdown_called_) + , payloads(BATCH) + , returned(RETURNED_LIMIT) + , log(log_) + , delim(delimiter) + , max_rows(rows_per_message) + , chunk_size(chunk_size_) +{ + if (connection.connect()) + setupChannel(); + else + throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to NATS {}", connection.connectionInfoForLog()); + + writing_task = global_context->getSchedulePool().createTask("NATSWritingTask", [this]{ writingFunc(); }); + writing_task->deactivate(); + + if (exchange_type == AMQP::ExchangeType::headers) + { + for (const auto & header : routing_keys) + { + std::vector matching; + boost::split(matching, header, [](char c){ return c == '='; }); + key_arguments[matching[0]] = matching[1]; + } + } + + reinitializeChunks(); +} + + +WriteBufferToNATSProducer::~WriteBufferToNATSProducer() +{ + writing_task->deactivate(); + connection.disconnect(); + assert(rows == 0); +} + + +void WriteBufferToNATSProducer::countRow() +{ + if (++rows % max_rows == 0) + { + const std::string & last_chunk = chunks.back(); + size_t last_chunk_size = offset(); + + if (last_chunk_size && delim && last_chunk[last_chunk_size - 1] == delim) + --last_chunk_size; + + std::string payload; + payload.reserve((chunks.size() - 1) * chunk_size + last_chunk_size); + + for (auto i = chunks.begin(), end = --chunks.end(); i != end; ++i) + payload.append(*i); + + payload.append(last_chunk, 0, last_chunk_size); + + reinitializeChunks(); + + ++payload_counter; + if (!payloads.push(std::make_pair(payload_counter, payload))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); + } +} + + +void WriteBufferToNATSProducer::setupChannel() +{ + producer_channel = connection.createChannel(); + + producer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Producer's channel {} error: {}", channel_id, message); + + /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) + producer_channel->close(); + + /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once + * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid + */ + for (const auto & record : delivery_record) + if (!returned.push(record.second)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); + + LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", + channel_id, delivery_record.size(), payloads.size()); + + /// Delivery tags are scoped per channel. + delivery_record.clear(); + delivery_tag = 0; + producer_ready = false; + }); + + producer_channel->onReady([&]() + { + channel_id = channel_id_base + "_" + std::to_string(channel_id_counter++); + LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); + + /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, + * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times + * slower than the second, so default is second and the first is turned on in table setting. + * + * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html + */ + producer_channel->confirmSelect() + .onAck([&](uint64_t acked_delivery_tag, bool multiple) + { + removeRecord(acked_delivery_tag, multiple, false); + }) + .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) + { + removeRecord(nacked_delivery_tag, multiple, true); + }); + producer_ready = true; + }); +} + + +void WriteBufferToNATSProducer::removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish) +{ + auto record_iter = delivery_record.find(received_delivery_tag); + assert(record_iter != delivery_record.end()); + + if (multiple) + { + /// If multiple is true, then all delivery tags up to and including current are confirmed (with ack or nack). + ++record_iter; + + if (republish) + for (auto record = delivery_record.begin(); record != record_iter; ++record) + if (!returned.push(record->second)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); + + /// Delete the records even in case when republished because new delivery tags will be assigned by the server. + delivery_record.erase(delivery_record.begin(), record_iter); + } + else + { + if (republish) + if (!returned.push(record_iter->second)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); + + delivery_record.erase(record_iter); + } +} + + +void WriteBufferToNATSProducer::publish(ConcurrentBoundedQueue> & messages, bool republishing) +{ + std::pair payload; + + /* It is important to make sure that delivery_record.size() is never bigger than returned.size(), i.e. number if unacknowledged + * messages cannot exceed returned.size(), because they all might end up there + */ + while (!messages.empty() && producer_channel->usable() && delivery_record.size() < RETURNED_LIMIT) + { + bool pop_result = messages.pop(payload); + + if (!pop_result) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not pop payload"); + + AMQP::Envelope envelope(payload.second.data(), payload.second.size()); + + /// if headers exchange is used, routing keys are added here via headers, if not - it is just empty + AMQP::Table message_settings = key_arguments; + + /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the + * server, then it means that publisher will never know whether those messages were delivered or not, and therefore those records + * that received no ack/nack before connection loss will be republished (see onError() callback), so there might be duplicates. To + * let consumer know that received message might be a possible duplicate - a "republished" field is added to message metadata + */ + message_settings["republished"] = std::to_string(republishing); + envelope.setHeaders(message_settings); + + /* Adding here a messageID property to message metadata. Since NATS does not guarantee exactly-once delivery, then on the + * consumer side "republished" field of message metadata can be checked and, if it set to 1, consumer might also check "messageID" + * property. This way detection of duplicates is guaranteed + */ + envelope.setMessageID(std::to_string(payload.first)); + + /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse + if (persistent) + envelope.setDeliveryMode(2); + + if (exchange_type == AMQP::ExchangeType::consistent_hash) + { + producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope); + } + else if (exchange_type == AMQP::ExchangeType::headers) + { + producer_channel->publish(exchange_name, "", envelope); + } + else + { + producer_channel->publish(exchange_name, routing_keys[0], envelope); + } + + /// This is needed for "publisher confirms", which guarantees at-least-once delivery + ++delivery_tag; + delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); + + /// Need to break at some point to let event loop run, because no publishing actually happens before looping + if (delivery_tag % BATCH == 0) + break; + } + + iterateEventLoop(); +} + + +void WriteBufferToNATSProducer::writingFunc() +{ + while ((!payloads.empty() || wait_all) && !shutdown_called.load()) + { + /// If onReady callback is not received, producer->usable() will anyway return true, + /// but must publish only after onReady callback. + if (producer_ready) + { + /* Publish main paylods only when there are no returned messages. This way it is ensured that returned messages are republished + * as fast as possible and no new publishes are made before returned messages are handled + */ + if (!returned.empty() && producer_channel->usable()) + publish(returned, true); + else if (!payloads.empty() && producer_channel->usable()) + publish(payloads, false); + } + + iterateEventLoop(); + + if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) + wait_all = false; + else if (!producer_channel->usable()) + { + if (connection.reconnect()) + setupChannel(); + } + } + + LOG_DEBUG(log, "Producer on channel {} completed", channel_id); +} + + +void WriteBufferToNATSProducer::nextImpl() +{ + addChunk(); +} + +void WriteBufferToNATSProducer::addChunk() +{ + chunks.push_back(std::string()); + chunks.back().resize(chunk_size); + set(chunks.back().data(), chunk_size); +} + +void WriteBufferToNATSProducer::reinitializeChunks() +{ + rows = 0; + chunks.clear(); + /// We cannot leave the buffer in the undefined state (i.e. without any + /// underlying buffer), since in this case the WriteBuffeR::next() will + /// not call our nextImpl() (due to available() == 0) + addChunk(); +} + + +void WriteBufferToNATSProducer::iterateEventLoop() +{ + connection.getHandler().iterateLoop(); +} + +} diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h new file mode 100644 index 00000000000..d0d80a6cf9d --- /dev/null +++ b/src/Storages/NATS/WriteBufferToNATSProducer.h @@ -0,0 +1,121 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class WriteBufferToNATSProducer : public WriteBuffer +{ +public: + WriteBufferToNATSProducer( + const NATSConfiguration & configuration_, + ContextPtr global_context, + const Names & routing_keys_, + const String & exchange_name_, + const AMQP::ExchangeType exchange_type_, + const size_t channel_id_base_, + const bool persistent_, + std::atomic & shutdown_called_, + Poco::Logger * log_, + std::optional delimiter, + size_t rows_per_message, + size_t chunk_size_ + ); + + ~WriteBufferToNATSProducer() override; + + void countRow(); + void activateWriting() { writing_task->activateAndSchedule(); } + void updateMaxWait() { wait_num.store(payload_counter); } + +private: + void nextImpl() override; + void addChunk(); + void reinitializeChunks(); + + void iterateEventLoop(); + void writingFunc(); + void setupChannel(); + void removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish); + void publish(ConcurrentBoundedQueue> & message, bool republishing); + + NATSConnection connection; + + const Names routing_keys; + const String exchange_name; + AMQP::ExchangeType exchange_type; + const String channel_id_base; /// Serial number of current producer buffer + const bool persistent; + + /* false: when shutdown is called; needed because table might be dropped before all acks are received + * true: in all other cases + */ + std::atomic & shutdown_called; + + AMQP::Table key_arguments; + BackgroundSchedulePool::TaskHolder writing_task; + + std::unique_ptr producer_channel; + bool producer_ready = false; + + /// Channel errors lead to channel closure, need to count number of recreated channels to update channel id + UInt64 channel_id_counter = 0; + + /// channel id which contains id of current producer buffer and serial number of recreated channel in this buffer + String channel_id; + + /* payloads.queue: + * - payloads are pushed to queue in countRow and popped by another thread in writingFunc, each payload gets into queue only once + * returned.queue: + * - payloads are pushed to queue: + * 1) inside channel->onError() callback if channel becomes unusable and the record of pending acknowledgements from server + * is non-empty. + * 2) inside removeRecord() if received nack() - negative acknowledgement from the server that message failed to be written + * to disk or it was unable to reach the queue. + * - payloads are popped from the queue once republished + */ + ConcurrentBoundedQueue> payloads, returned; + + /* Counter of current delivery on a current channel. Delivery tags are scoped per channel. The server attaches a delivery tag for each + * published message - a serial number of delivery on current channel. Delivery tag is a way of server to notify publisher if it was + * able or unable to process delivery, i.e. it sends back a response with a corresponding delivery tag. + */ + UInt64 delivery_tag = 0; + + /* false: message delivery successfully ended: publisher received confirm from server that all published + * 1) persistent messages were written to disk + * 2) non-persistent messages reached the queue + * true: continue to process deliveries and returned messages + */ + bool wait_all = true; + + /* false: until writeSuffix is called + * true: means payloads.queue will not grow anymore + */ + std::atomic wait_num = 0; + + /// Needed to fill messageID property + UInt64 payload_counter = 0; + + /// Record of pending acknowledgements from the server; its size never exceeds size of returned.queue + std::map> delivery_record; + + Poco::Logger * log; + const std::optional delim; + const size_t max_rows; + const size_t chunk_size; + size_t rows = 0; + std::list chunks; +}; + +} From ecb5a80495344342fb40f436939e2c168f32308e Mon Sep 17 00:00:00 2001 From: tchepavel Date: Mon, 9 May 2022 23:16:11 +0300 Subject: [PATCH 002/627] MV works; rand() is harmful --- base/harmful/harmful.c | 2 +- .../ExternalDataSourceConfiguration.cpp | 5 + src/Storages/NATS/Buffer_fwd.h | 4 +- src/Storages/NATS/NATSConnection.cpp | 71 +- src/Storages/NATS/NATSConnection.h | 14 +- src/Storages/NATS/NATSHandler.cpp | 13 +- src/Storages/NATS/NATSHandler.h | 4 +- src/Storages/NATS/NATSSettings.cpp | 2 +- src/Storages/NATS/NATSSettings.h | 15 +- src/Storages/NATS/NATSSink.cpp | 4 +- src/Storages/NATS/NATSSource.cpp | 84 +- src/Storages/NATS/NATSSource.h | 8 +- .../NATS/ReadBufferFromNATSConsumer.cpp | 212 ++-- .../NATS/ReadBufferFromNATSConsumer.h | 68 +- src/Storages/NATS/StorageNATS.cpp | 1073 ++++++++--------- src/Storages/NATS/StorageNATS.h | 49 +- .../NATS/WriteBufferToNATSProducer.cpp | 94 +- src/Storages/NATS/WriteBufferToNATSProducer.h | 4 +- src/Storages/registerStorages.cpp | 4 + 19 files changed, 838 insertions(+), 892 deletions(-) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index 5a27cae0383..adc3f7158f6 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -206,7 +206,7 @@ TRAP(lgammal) TRAP(nftw) TRAP(nl_langinfo) TRAP(putc_unlocked) -TRAP(rand) +//TRAP(rand) Used by nats-io at startup /** In the current POSIX.1 specification (POSIX.1-2008), readdir() is not required to be thread-safe. However, in modern * implementations (including the glibc implementation), concurrent calls to readdir() that specify different directory streams * are thread-safe. In cases where multiple threads must read from the same directory stream, using readdir() with external diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index abd20e6e5fd..1cb123120f2 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -19,6 +19,8 @@ #include #endif +#include + #include namespace DB @@ -483,6 +485,9 @@ template bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); #endif +template +bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); + template std::optional getExternalDataSourceConfiguration( const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection, const BaseSettings & storage_settings); diff --git a/src/Storages/NATS/Buffer_fwd.h b/src/Storages/NATS/Buffer_fwd.h index caf7b8cfdbe..3eb52314a79 100644 --- a/src/Storages/NATS/Buffer_fwd.h +++ b/src/Storages/NATS/Buffer_fwd.h @@ -6,9 +6,9 @@ namespace DB { class ReadBufferFromNATSConsumer; -using ConsumerBufferPtr = std::shared_ptr; +using ConsumerBufferPtr = std::shared_ptr; class WriteBufferToNATSProducer; -using ProducerBufferPtr = std::shared_ptr; +using ProducerBufferPtr = std::shared_ptr; } diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index 7b0abba7535..cef8a2eb8c1 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -7,36 +7,36 @@ namespace DB { -static const auto CONNECT_SLEEP = 200; +//static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; -NATSConnection::NATSConnection(const NATSConfiguration & configuration_, Poco::Logger * log_) +NATSConnectionManager::NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_) : configuration(configuration_) , log(log_) , event_handler(loop.getLoop(), log) { } -String NATSConnection::connectionInfoForLog() const +String NATSConnectionManager::connectionInfoForLog() const { return configuration.host + ':' + toString(configuration.port); } -bool NATSConnection::isConnected() +bool NATSConnectionManager::isConnected() { std::lock_guard lock(mutex); return isConnectedImpl(); } -bool NATSConnection::connect() +bool NATSConnectionManager::connect() { std::lock_guard lock(mutex); connectImpl(); return isConnectedImpl(); } -bool NATSConnection::reconnect() +bool NATSConnectionManager::reconnect() { std::lock_guard lock(mutex); if (isConnectedImpl()) @@ -44,69 +44,70 @@ bool NATSConnection::reconnect() disconnectImpl(); - /// This will force immediate closure if not yet closed - if (!connection->closed()) - connection->close(true); - LOG_DEBUG(log, "Trying to restore connection to {}", connectionInfoForLog()); connectImpl(); return isConnectedImpl(); } -SubscriptionPtr NATSConnection::createSubscription(const std::string& subject) +SubscriptionPtr NATSConnectionManager::createSubscription(const std::string& subject, natsMsgHandler handler, ReadBufferFromNATSConsumer * consumer) { std::lock_guard lock(mutex); natsSubscription * ns; - natsConnection_SubscribeSync(&ns, connection, subject.c_str()); + status = natsConnection_Subscribe(&ns, connection, subject.c_str(), handler, static_cast(consumer)); + if (status == NATS_OK) + status = natsSubscription_SetPendingLimits(ns, -1, -1); + if (status == NATS_OK) + LOG_DEBUG(log, "Subscribed to subject {}", subject); return SubscriptionPtr(ns, &natsSubscription_Destroy); } -void NATSConnection::disconnect() +void NATSConnectionManager::disconnect() { std::lock_guard lock(mutex); disconnectImpl(); } -bool NATSConnection::closed() +bool NATSConnectionManager::closed() { std::lock_guard lock(mutex); return natsConnection_IsClosed(connection); } -bool NATSConnection::isConnectedImpl() const +bool NATSConnectionManager::isConnectedImpl() const { - return event_handler.connectionRunning() && !natsConnection_IsClosed(connection); + return event_handler.connectionRunning() && !natsConnection_IsClosed(connection) && status == natsStatus::NATS_OK; } -void NATSConnection::connectImpl() +void NATSConnectionManager::connectImpl() { + natsOptions * options = event_handler.getOptions(); + natsOptions_SetUserInfo(options, configuration.username.c_str(), configuration.password.c_str()); + if (configuration.secure) { + natsOptions_SetSecure(options, true); + natsOptions_SkipServerVerification(options, true); + } + std::string address; if (configuration.connection_string.empty()) { - LOG_DEBUG(log, "Connecting to: {}:{} (user: {})", configuration.host, configuration.port, configuration.username); - AMQP::Login login(configuration.username, configuration.password); - AMQP::Address address(configuration.host, configuration.port, login, configuration.vhost, configuration.secure); - connection = std::make_unique(&event_handler, address); + address = configuration.host + ":" + std::to_string(configuration.port); } else { - AMQP::Address address(configuration.connection_string); - connection = std::make_unique(&event_handler, address); + address = configuration.connection_string; } - - auto cnt_retries = 0; - while (true) + natsOptions_SetURL(options, address.c_str()); + status = natsConnection_Connect(&connection, options); + if (status != NATS_OK) { - event_handler.iterateLoop(); - - if (connection->ready() || cnt_retries++ == RETRIES_MAX) - break; - - std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + LOG_DEBUG(log, "Failed to connect to NATS on address: {}", address); + return; } + + event_handler.changeConnectionStatus(true); } -void NATSConnection::disconnectImpl() +void NATSConnectionManager::disconnectImpl() { natsConnection_Close(connection); @@ -114,8 +115,10 @@ void NATSConnection::disconnectImpl() * an AMQP closing-handshake is performed). But cannot open a new connection until previous one is properly closed */ size_t cnt_retries = 0; - while (!closed() && cnt_retries++ != RETRIES_MAX) + while (!natsConnection_IsClosed(connection) && cnt_retries++ != RETRIES_MAX) event_handler.iterateLoop(); + + event_handler.changeConnectionStatus(false); } } diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h index 2c999e873aa..ab97012061c 100644 --- a/src/Storages/NATS/NATSConnection.h +++ b/src/Storages/NATS/NATSConnection.h @@ -2,14 +2,13 @@ #include #include - +#include namespace DB { struct NATSConfiguration { - String url; String host; UInt16 port; String username; @@ -20,13 +19,12 @@ struct NATSConfiguration String connection_string; }; -using SubscriptionPtr = std::unique_ptr; - -class NATSConnection +class NATSConnectionManager { public: - NATSConnection(const NATSConfiguration & configuration_, Poco::Logger * log_); + NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_); + ~NATSConnectionManager() { natsConnection_Destroy(connection); } bool isConnected(); @@ -38,7 +36,7 @@ public: bool closed(); - SubscriptionPtr createSubscription(const std::string& subject); + SubscriptionPtr createSubscription(const std::string& subject, natsMsgHandler handler, ReadBufferFromNATSConsumer * consumer); /// NATSHandler is thread safe. Any public methods can be called concurrently. NATSHandler & getHandler() { return event_handler; } @@ -63,6 +61,6 @@ private: std::mutex mutex; }; -using NATSConnectionPtr = std::unique_ptr; +using NATSConnectionManagerPtr = std::shared_ptr; } diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index 7fb8ff38c47..cb155522a6a 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -18,7 +19,7 @@ NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : natsLibuv_Init(); natsLibuv_SetThreadLocalLoop(loop); natsOptions_Create(&opts); - status = natsOptions_SetEventLoop(opts, static_cast(loop), + status = natsOptions_SetEventLoop(opts, static_cast(loop), natsLibuv_Attach, natsLibuv_Read, natsLibuv_Write, @@ -28,22 +29,32 @@ NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : void NATSHandler::startLoop() { std::lock_guard lock(startup_mutex); + natsLibuv_SetThreadLocalLoop(loop); LOG_DEBUG(log, "Background loop started"); loop_running.store(true); while (loop_state.load() == Loop::RUN) + { uv_run(loop, UV_RUN_NOWAIT); + } LOG_DEBUG(log, "Background loop ended"); loop_running.store(false); } +void NATSHandler::changeConnectionStatus(bool is_running) { + connection_running.store(is_running); +} + void NATSHandler::iterateLoop() { std::unique_lock lock(startup_mutex, std::defer_lock); if (lock.try_lock()) + { + natsLibuv_SetThreadLocalLoop(loop); uv_run(loop, UV_RUN_NOWAIT); + } } /// Do not need synchronization as in iterateLoop(), because this method is used only for diff --git a/src/Storages/NATS/NATSHandler.h b/src/Storages/NATS/NATSHandler.h index 12ea3454b9d..2854bf9a069 100644 --- a/src/Storages/NATS/NATSHandler.h +++ b/src/Storages/NATS/NATSHandler.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include @@ -20,6 +19,8 @@ namespace Loop static const UInt8 STOP = 2; } +using SubscriptionPtr = std::unique_ptr; + class NATSHandler { @@ -41,6 +42,7 @@ public: void stopLoop(); + void changeConnectionStatus(bool is_running); bool connectionRunning() const { return connection_running.load(); } bool loopRunning() const { return loop_running.load(); } diff --git a/src/Storages/NATS/NATSSettings.cpp b/src/Storages/NATS/NATSSettings.cpp index 11659b105b2..ec1149acee4 100644 --- a/src/Storages/NATS/NATSSettings.cpp +++ b/src/Storages/NATS/NATSSettings.cpp @@ -12,7 +12,7 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_NATS_SETTINGS) void NATSSettings::loadFromQuery(ASTStorage & storage_def) { diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h index 5f4051e06d1..c30869e8ea3 100644 --- a/src/Storages/NATS/NATSSettings.h +++ b/src/Storages/NATS/NATSSettings.h @@ -8,17 +8,13 @@ namespace DB class ASTStorage; -#define RABBITMQ_RELATED_SETTINGS(M) \ +#define NATS_RELATED_SETTINGS(M) \ M(String, nats_host_port, "", "A host-port to connect to NATS server.", 0) \ - M(String, nats_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \ + M(String, nats_subjects, "", "List of subject for NATS table to subscribe/publsh to.", 0) \ M(String, nats_format, "", "The message format.", 0) \ - M(String, nats_exchange_type, "default", "The exchange type.", 0) \ - M(String, nats_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \ M(Char, nats_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ M(String, nats_schema, "", "Schema identifier (used by schema-based formats) for NATS engine", 0) \ M(UInt64, nats_num_consumers, 1, "The number of consumer channels per table.", 0) \ - M(UInt64, nats_num_queues, 1, "The number of queues per consumer.", 0) \ - M(String, nats_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ M(Bool, nats_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \ M(Bool, nats_secure, false, "Use SSL connection", 0) \ M(String, nats_address, "", "Address for connection", 0) \ @@ -26,17 +22,16 @@ namespace DB M(UInt64, nats_max_block_size, 0, "Number of row collected before flushing data from NATS.", 0) \ M(Milliseconds, nats_flush_interval_ms, 0, "Timeout for flushing data from NATS.", 0) \ M(String, nats_vhost, "/", "NATS vhost.", 0) \ - M(String, nats_queue_settings_list, "", "A list of nats queue settings", 0) \ M(Bool, nats_queue_consume, false, "Use user-defined queues and do not make any NATS setup: declaring exchanges, queues, bindings", 0) \ M(String, nats_username, "", "NATS username", 0) \ M(String, nats_password, "", "NATS password", 0) \ M(Bool, nats_commit_on_select, false, "Commit messages when select query is made", 0) \ -#define LIST_OF_RABBITMQ_SETTINGS(M) \ - RABBITMQ_RELATED_SETTINGS(M) \ +#define LIST_OF_NATS_SETTINGS(M) \ + NATS_RELATED_SETTINGS(M) \ FORMAT_FACTORY_SETTINGS(M) -DECLARE_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) +DECLARE_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_NATS_SETTINGS) struct NATSSettings : public BaseSettings { diff --git a/src/Storages/NATS/NATSSink.cpp b/src/Storages/NATS/NATSSink.cpp index db2620d9ed2..00fe93a9e68 100644 --- a/src/Storages/NATS/NATSSink.cpp +++ b/src/Storages/NATS/NATSSink.cpp @@ -18,13 +18,13 @@ NATSSink::NATSSink( , metadata_snapshot(metadata_snapshot_) , context(context_) { - storage.unbindExchange(); +// storage.unbindExchange(); } void NATSSink::onStart() { - buffer = storage.createWriteBuffer(); +// buffer = storage.createWriteBuffer(); buffer->activateWriting(); auto format_settings = getFormatSettings(context); diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index 046b8792ced..d757f7b9a54 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -12,7 +12,7 @@ static std::pair getHeaders(const StorageSnapshotPtr & storage_sna { auto non_virtual_header = storage_snapshot->metadata->getSampleBlockNonMaterialized(); auto virtual_header = storage_snapshot->getSampleBlockForColumns( - {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered", "_message_id", "_timestamp"}); + {"_subject", "_timestamp"}); return {non_virtual_header, virtual_header}; } @@ -51,14 +51,14 @@ NATSSource::NATSSource( ContextPtr context_, const Names & columns, size_t max_block_size_, - bool ack_in_suffix_) + bool /*ack_in_suffix_*/) : SourceWithProgress(getSampleBlock(headers.first, headers.second)) , storage(storage_) , storage_snapshot(storage_snapshot_) , context(context_) , column_names(columns) , max_block_size(max_block_size_) - , ack_in_suffix(ack_in_suffix_) +// , ack_in_suffix(ack_in_suffix_) , non_virtual_header(std::move(headers.first)) , virtual_header(std::move(headers.second)) { @@ -77,31 +77,31 @@ NATSSource::~NATSSource() } -bool NATSSource::needChannelUpdate() -{ - if (!buffer) - return false; - - return buffer->needChannelUpdate(); -} - - -void NATSSource::updateChannel() -{ - if (!buffer) - return; - - buffer->updateAckTracker(); - - if (storage.updateChannel(buffer->getChannel())) - buffer->setupChannel(); -} +//bool NATSSource::needChannelUpdate() +//{ +// if (!buffer) +// return false; +// +// return buffer->needChannelUpdate(); +//} +// +// +//void NATSSource::updateChannel() +//{ +// if (!buffer) +// return; +// +// buffer->updateAckTracker(); +// +// if (storage.updateChannel(buffer->getChannel())) +// buffer->setupChannel(); +//} Chunk NATSSource::generate() { auto chunk = generateImpl(); - if (!chunk && ack_in_suffix) - sendAck(); +// if (!chunk && ack_in_suffix) +// sendAck(); return chunk; } @@ -136,23 +136,13 @@ Chunk NATSSource::generateImpl() if (new_rows) { - auto exchange_name = storage.getExchange(); - auto channel_id = buffer->getChannelID(); - auto delivery_tag = buffer->getDeliveryTag(); - auto redelivered = buffer->getRedelivered(); - auto message_id = buffer->getMessageID(); + auto subject = buffer->getSubject(); auto timestamp = buffer->getTimestamp(); - buffer->updateAckTracker({delivery_tag, channel_id}); - for (size_t i = 0; i < new_rows; ++i) { - virtual_columns[0]->insert(exchange_name); - virtual_columns[1]->insert(channel_id); - virtual_columns[2]->insert(delivery_tag); - virtual_columns[3]->insert(redelivered); - virtual_columns[4]->insert(message_id); - virtual_columns[5]->insert(timestamp); + virtual_columns[0]->insert(subject); + virtual_columns[1]->insert(timestamp); } total_rows = total_rows + new_rows; @@ -175,15 +165,15 @@ Chunk NATSSource::generateImpl() } -bool NATSSource::sendAck() -{ - if (!buffer) - return false; - - if (!buffer->ackMessages()) - return false; - - return true; -} +//bool NATSSource::sendAck() +//{ +// if (!buffer) +// return false; +// +// if (!buffer->ackMessages()) +// return false; +// +// return true; +//} } diff --git a/src/Storages/NATS/NATSSource.h b/src/Storages/NATS/NATSSource.h index ce48e5cf382..263e6824603 100644 --- a/src/Storages/NATS/NATSSource.h +++ b/src/Storages/NATS/NATSSource.h @@ -28,9 +28,9 @@ public: Chunk generate() override; bool queueEmpty() const { return !buffer || buffer->queueEmpty(); } - bool needChannelUpdate(); - void updateChannel(); - bool sendAck(); +// bool needChannelUpdate(); +// void updateChannel(); +// bool sendAck(); private: StorageNATS & storage; @@ -38,7 +38,7 @@ private: ContextPtr context; Names column_names; const size_t max_block_size; - bool ack_in_suffix; +// bool ack_in_suffix; bool is_finished = false; const Block non_virtual_header; diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index ad966401cc3..5edd48d97c8 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -5,8 +5,8 @@ #include #include #include -#include #include +#include #include #include "Poco/Timer.h" #include @@ -21,8 +21,8 @@ namespace ErrorCodes ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( NATSHandler & event_handler_, - std::vector & queues_, - size_t channel_id_base_, + std::shared_ptr connection_, + std::vector & subjects_, const String & channel_base_, Poco::Logger * log_, char row_delimiter_, @@ -30,136 +30,105 @@ ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , event_handler(event_handler_) - , queues(queues_) + , connection(connection_) + , subjects(subjects_) , channel_base(channel_base_) - , channel_id_base(channel_id_base_) , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) , received(queue_size_) { + subscribe(); + LOG_DEBUG(log, "Started NATS consumer"); } ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() { + for (const auto& subscription : subscriptions) { + natsSubscription_Unsubscribe(subscription.get()); + } + BufferBase::set(nullptr, 0, 0); } void ReadBufferFromNATSConsumer::subscribe() { - for (const auto & queue_name : queues) - { - consumer_channel->consume(queue_name) - .onSuccess([&](const std::string & /* consumer_tag */) - { - LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name); - - if (++subscribed == queues.size()) - wait_subscription.store(false); - }) - .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) - { - if (message.bodySize()) - { - String message_received = std::string(message.body(), message.body() + message.bodySize()); - if (row_delimiter != '\0') - message_received += row_delimiter; - - if (!received.push({message_received, message.hasMessageID() ? message.messageID() : "", - message.hasTimestamp() ? message.timestamp() : 0, - redelivered, AckTracker(delivery_tag, channel_id)})) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); - } - }) - .onError([&](const char * message) - { - /* End up here either if channel ends up in an error state (then there will be resubscription) or consume call error, which - * arises from queue settings mismatch or queue level error, which should not happen as no one else is supposed to touch them - */ - LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message); - wait_subscription.store(false); - }); + for (const auto & subject : subjects) { + subscriptions.emplace_back(connection->createSubscription(subject, onMsg, this)); } +// for (const auto & queue_name : subjects) +// { +// consumer_channel->consume(queue_name) +// .onSuccess([&](const std::string & /* consumer_tag */) +// { +// LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name); +// +// if (++subscribed == subjects.size()) +// wait_subscription.store(false); +// }) +// .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) +// { +// if (message.bodySize()) +// { +// String message_received = std::string(message.body(), message.body() + message.bodySize()); +// if (row_delimiter != '\0') +// message_received += row_delimiter; +// +// if (!received.push({message_received, message.hasMessageID() ? message.messageID() : "", +// message.hasTimestamp() ? message.timestamp() : 0, +// redelivered, AckTracker(delivery_tag, channel_id)})) +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); +// } +// }) +// .onError([&](const char * message) +// { +// /* End up here either if channel ends up in an error state (then there will be resubscription) or consume call error, which +// * arises from queue settings mismatch or queue level error, which should not happen as no one else is supposed to touch them +// */ +// LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message); +// wait_subscription.store(false); +// }); +// } } -bool ReadBufferFromNATSConsumer::ackMessages() -{ - AckTracker record_info = last_inserted_record_info; - - /* Do not send ack to server if message's channel is not the same as current running channel because delivery tags are scoped per - * channel, so if channel fails, all previous delivery tags become invalid - */ - if (record_info.channel_id == channel_id && record_info.delivery_tag && record_info.delivery_tag > prev_tag) - { - /// Commit all received messages with delivery tags from last committed to last inserted - if (!consumer_channel->ack(record_info.delivery_tag, AMQP::multiple)) - { - LOG_ERROR(log, "Failed to commit messages with delivery tags from last committed to {} on channel {}", - record_info.delivery_tag, channel_id); - return false; - } - - prev_tag = record_info.delivery_tag; - LOG_TRACE(log, "Consumer committed messages with deliveryTags up to {} on channel {}", record_info.delivery_tag, channel_id); - } - - return true; -} +//bool ReadBufferFromNATSConsumer::ackMessages() +//{ +// AckTracker record_info = last_inserted_record_info; +// +// /* Do not send ack to server if message's channel is not the same as current running channel because delivery tags are scoped per +// * channel, so if channel fails, all previous delivery tags become invalid +// */ +// if (record_info.channel_id == channel_id && record_info.delivery_tag && record_info.delivery_tag > prev_tag) +// { +// /// Commit all received messages with delivery tags from last committed to last inserted +// if (!consumer_channel->ack(record_info.delivery_tag, AMQP::multiple)) +// { +// LOG_ERROR(log, "Failed to commit messages with delivery tags from last committed to {} on channel {}", +// record_info.delivery_tag, channel_id); +// return false; +// } +// +// prev_tag = record_info.delivery_tag; +// LOG_TRACE(log, "Consumer committed messages with deliveryTags up to {} on channel {}", record_info.delivery_tag, channel_id); +// } +// +// return true; +//} -void ReadBufferFromNATSConsumer::updateAckTracker(AckTracker record_info) -{ - if (record_info.delivery_tag && channel_error.load()) - return; - - if (!record_info.delivery_tag) - prev_tag = 0; - - last_inserted_record_info = record_info; -} - - -void ReadBufferFromNATSConsumer::setupChannel() -{ - if (!consumer_channel) - return; - - wait_subscription.store(true); - - consumer_channel->onReady([&]() - { - /* First number indicates current consumer buffer; second number indicates serial number of created channel for current buffer, - * i.e. if channel fails - another one is created and its serial number is incremented; channel_base is to guarantee that - * channel_id is unique for each table - */ - channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base; - LOG_TRACE(log, "Channel {} is created", channel_id); - - subscribed = 0; - subscribe(); - channel_error.store(false); - }); - - consumer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Channel {} error: {}", channel_id, message); - - channel_error.store(true); - wait_subscription.store(false); - }); -} - - -bool ReadBufferFromNATSConsumer::needChannelUpdate() -{ - if (wait_subscription) - return false; - - return channel_error || !consumer_channel || !consumer_channel->usable(); -} +//void ReadBufferFromNATSConsumer::updateAckTracker(AckTracker record_info) +//{ +// if (record_info.delivery_tag && channel_error.load()) +// return; +// +// if (!record_info.delivery_tag) +// prev_tag = 0; +// +// last_inserted_record_info = record_info; +//} void ReadBufferFromNATSConsumer::iterateEventLoop() @@ -185,4 +154,27 @@ bool ReadBufferFromNATSConsumer::nextImpl() return false; } +void ReadBufferFromNATSConsumer::onMsg(natsConnection *, natsSubscription *, natsMsg * msg, void * consumer) +{ + auto * buffer = static_cast(consumer); + const int msg_length = natsMsg_GetDataLength(msg); + LOG_DEBUG(buffer->log, "I'm getting something {} {}", msg_length, natsMsg_GetData(msg)); + + if (msg_length) + { + String message_received = std::string(natsMsg_GetData(msg), msg_length); + if (buffer->row_delimiter != '\0') + message_received += buffer->row_delimiter; + + if (!buffer->received.push({ + .message = message_received, + .subject = natsMsg_GetSubject(msg), + .timestamp = natsMsg_GetTime(msg) + })) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); + } + + natsMsg_Destroy(msg); +} + } diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h index 1206ee4e62d..2c1a9704c03 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include namespace Poco @@ -22,8 +22,8 @@ class ReadBufferFromNATSConsumer : public ReadBuffer public: ReadBufferFromNATSConsumer( NATSHandler & event_handler_, - std::vector & queues_, - size_t channel_id_base_, + std::shared_ptr connection_, + std::vector & subjects_, const String & channel_base_, Poco::Logger * log_, char row_delimiter_, @@ -32,73 +32,69 @@ public: ~ReadBufferFromNATSConsumer() override; - struct AckTracker - { - UInt64 delivery_tag; - String channel_id; - - AckTracker() = default; - AckTracker(UInt64 tag, String id) : delivery_tag(tag), channel_id(id) {} - }; +// struct AckTracker +// { +// UInt64 delivery_tag; +// String channel_id; +// +// AckTracker() = default; +// AckTracker(UInt64 tag, String id) : delivery_tag(tag), channel_id(id) {} +// }; struct MessageData { String message; - String message_id; - uint64_t timestamp = 0; - bool redelivered = false; - AckTracker track{}; + String subject; + int64_t timestamp; +// AckTracker track{}; }; - ChannelPtr & getChannel() { return consumer_channel; } - void setupChannel(); - bool needChannelUpdate(); + std::vector & getChannel() { return subscriptions; } void closeChannel() { - if (consumer_channel) - consumer_channel->close(); + for (const auto & subscription : subscriptions) + natsSubscription_Unsubscribe(subscription.get()); } - void updateQueues(std::vector & queues_) { queues = queues_; } - size_t queuesCount() { return queues.size(); } + void updateSubjects(std::vector & subjects_) { subjects = subjects_; } + size_t subjectsCount() { return subjects.size(); } bool isConsumerStopped() { return stopped; } - bool ackMessages(); - void updateAckTracker(AckTracker record = AckTracker()); +// bool ackMessages(); +// void updateAckTracker(AckTracker record = AckTracker()); bool queueEmpty() { return received.empty(); } void allowNext() { allowed = true; } // Allow to read next message. - auto getChannelID() const { return current.track.channel_id; } - auto getDeliveryTag() const { return current.track.delivery_tag; } - auto getRedelivered() const { return current.redelivered; } - auto getMessageID() const { return current.message_id; } + auto getSubject() const { return current.subject; } auto getTimestamp() const { return current.timestamp; } + void iterateEventLoop(); private: bool nextImpl() override; void subscribe(); - void iterateEventLoop(); + static void onMsg(natsConnection *nc, natsSubscription *sub, natsMsg * msg, void * closure); - ChannelPtr consumer_channel; NATSHandler & event_handler; /// Used concurrently, but is thread safe. - std::vector queues; + std::shared_ptr connection; + std::vector subscriptions; + std::vector subjects; const String channel_base; - const size_t channel_id_base; +// const size_t channel_id_base; Poco::Logger * log; char row_delimiter; bool allowed = true; const std::atomic & stopped; String channel_id; - std::atomic channel_error = true, wait_subscription = false; +// std::atomic channel_error = true, wait_subscription = false; ConcurrentBoundedQueue received; MessageData current; - size_t subscribed = 0; +// size_t subscribed = 0; - AckTracker last_inserted_record_info; - UInt64 prev_tag = 0, channel_id_counter = 0; +// AckTracker last_inserted_record_info; +// UInt64 prev_tag = 0, channel_id_counter = 0; }; } diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 41ab2d1fc9b..2cfe9e8be8d 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include @@ -26,6 +25,18 @@ #include #include +//static void +//onMsg(natsConnection *, natsSubscription *, natsMsg *msg, void *) +//{ +// printf("\n\n\nReceived msg: %s - %.*s\n\n\n", +// natsMsg_GetSubject(msg), +// natsMsg_GetDataLength(msg), +// natsMsg_GetData(msg)); +// fflush(stdout); +// +// natsMsg_Destroy(msg); +//} + namespace DB { @@ -48,17 +59,6 @@ namespace ErrorCodes extern const int QUERY_NOT_ALLOWED; } -namespace ExchangeType -{ - /// Note that default here means default by implementation and not by nats settings - static const String DEFAULT = "default"; - static const String FANOUT = "fanout"; - static const String DIRECT = "direct"; - static const String TOPIC = "topic"; - static const String HASH = "consistent_hash"; - static const String HEADERS = "headers"; -} - StorageNATS::StorageNATS( const StorageID & table_id_, @@ -69,19 +69,13 @@ StorageNATS::StorageNATS( : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , nats_settings(std::move(nats_settings_)) - , exchange_name(getContext()->getMacros()->expand(nats_settings->nats_exchange_name)) + , subjects(parseSubjects(getContext()->getMacros()->expand(nats_settings->nats_subjects))) , format_name(getContext()->getMacros()->expand(nats_settings->nats_format)) - , exchange_type(defineExchangeType(getContext()->getMacros()->expand(nats_settings->nats_exchange_type))) - , routing_keys(parseSettings(getContext()->getMacros()->expand(nats_settings->nats_routing_key_list))) , row_delimiter(nats_settings->nats_row_delimiter.value) , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) , num_consumers(nats_settings->nats_num_consumers.value) - , num_queues(nats_settings->nats_num_queues.value) - , queue_base(getContext()->getMacros()->expand(nats_settings->nats_queue_base)) - , queue_settings_list(parseSettings(getContext()->getMacros()->expand(nats_settings->nats_queue_settings_list))) , persistent(nats_settings->nats_persistent.value) - , use_user_setup(nats_settings->nats_queue_consume.value) - , hash_exchange(num_consumers > 1 || num_queues > 1) +// , use_user_setup(nats_settings->nats_queue_consume.value) , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) , semaphore(0, num_consumers) , unique_strbase(getRandomName()) @@ -114,39 +108,56 @@ StorageNATS::StorageNATS( nats_context = addSettings(getContext()); nats_context->makeQueryContext(); - - if (queue_base.empty()) - { - /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to - * be table-based and not just a random string, because local exchanges should be declared the same for same tables - */ - sharding_exchange = getTableBasedName(exchange_name, table_id_); - - /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better - * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every - * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base - * for the names of later declared queues - */ - queue_base = getTableBasedName("", table_id_); - } - else - { - /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and - * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need to share - * sharding exchange and bridge exchange - */ - sharding_exchange = exchange_name + "_" + queue_base; - } - - bridge_exchange = sharding_exchange + "_bridge"; +// +// if (queue_base.empty()) +// { +// /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to +// * be table-based and not just a random string, because local exchanges should be declared the same for same tables +// */ +// sharding_exchange = getTableBasedName(exchange_name, table_id_); +// +// /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better +// * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every +// * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base +// * for the names of later declared queues +// */ +// queue_base = getTableBasedName("", table_id_); +// } +// else +// { +// /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and +// * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need to share +// * sharding exchange and bridge exchange +// */ +// sharding_exchange = exchange_name + "_" + queue_base; +// } +// +// bridge_exchange = sharding_exchange + "_bridge"; try { - connection = std::make_unique(configuration, log); - if (connection->connect()) - initNATS(); - else if (!is_attach) + connection = std::make_shared(configuration, log); + if (!connection->connect() && !is_attach) throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to {}", connection->connectionInfoForLog()); +// if (connection->connect()) +// initNATS(); +// else if (!is_attach) +// throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to {}", connection->connectionInfoForLog()); +// auto sub = connection->createSubscription("foo", onMsg, nullptr); +// int64_t n; +// connection->getHandler().startBlockingLoop(); +// while (true) { +// natsSubscription_GetDelivered(sub.get(), &n); +// printf("Read n : %ld\n", n); +// fflush(stdout); +// std::this_thread::sleep_for(std::chrono::milliseconds(500)); +// } +// auto t2 = std::thread([this](){ +// connection->getHandler().updateLoopState(Loop::RUN); +// LOG_DEBUG(log, "Storteng lup"); +// connection->getHandler().startLoop(); +// }); +// t2.join(); } catch (...) { @@ -162,17 +173,17 @@ StorageNATS::StorageNATS( streaming_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSStreamingTask", [this]{ streamingToViewsFunc(); }); streaming_task->deactivate(); - connection_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSConnectionTask", [this]{ connectionFunc(); }); + connection_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSConnectionManagerTask", [this]{ connectionFunc(); }); connection_task->deactivate(); } -Names StorageNATS::parseSettings(String settings_list) +Names StorageNATS::parseSubjects(String subjects_list) { Names result; - if (settings_list.empty()) + if (subjects_list.empty()) return result; - boost::split(result, settings_list, [](char c){ return c == ','; }); + boost::split(result, subjects_list, [](char c){ return c == ','; }); for (String & key : result) boost::trim(key); @@ -180,27 +191,6 @@ Names StorageNATS::parseSettings(String settings_list) } -AMQP::ExchangeType StorageNATS::defineExchangeType(String exchange_type_) -{ - AMQP::ExchangeType type; - if (exchange_type_ != ExchangeType::DEFAULT) - { - if (exchange_type_ == ExchangeType::FANOUT) type = AMQP::ExchangeType::fanout; - else if (exchange_type_ == ExchangeType::DIRECT) type = AMQP::ExchangeType::direct; - else if (exchange_type_ == ExchangeType::TOPIC) type = AMQP::ExchangeType::topic; - else if (exchange_type_ == ExchangeType::HASH) type = AMQP::ExchangeType::consistent_hash; - else if (exchange_type_ == ExchangeType::HEADERS) type = AMQP::ExchangeType::headers; - else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); - } - else - { - type = AMQP::ExchangeType::fanout; - } - - return type; -} - - String StorageNATS::getTableBasedName(String name, const StorageID & table_id) { if (name.empty()) @@ -260,7 +250,7 @@ void StorageNATS::stopLoopIfNoReaders() void StorageNATS::startLoop() { - assert(rabbit_is_ready); +// assert(nats_is_ready); connection->getHandler().updateLoopState(Loop::RUN); looping_task->activateAndSchedule(); } @@ -280,13 +270,15 @@ void StorageNATS::decrementReader() void StorageNATS::connectionFunc() { - if (rabbit_is_ready) - return; +// if (nats_is_ready) +// return; - if (connection->reconnect()) - initNATS(); - else + if (!connection->reconnect()) connection_task->scheduleAfter(RESCHEDULE_MS); +// if (connection->reconnect()) +// initNATS(); +// else +// connection_task->scheduleAfter(RESCHEDULE_MS); } @@ -320,332 +312,332 @@ size_t StorageNATS::getMaxBlockSize() const } -void StorageNATS::initNATS() -{ - if (shutdown_called || rabbit_is_ready) - return; - - if (use_user_setup) - { - queues.emplace_back(queue_base); - rabbit_is_ready = true; - return; - } - - try - { - auto rabbit_channel = connection->createChannel(); - - /// Main exchange -> Bridge exchange -> ( Sharding exchange ) -> Queues -> Consumers - - initExchange(*rabbit_channel); - bindExchange(*rabbit_channel); - - for (const auto i : collections::range(0, num_queues)) - bindQueue(i + 1, *rabbit_channel); - - LOG_TRACE(log, "NATS setup completed"); - rabbit_is_ready = true; - rabbit_channel->close(); - } - catch (...) - { - tryLogCurrentException(log); - if (!is_attach) - throw; - } -} - - -void StorageNATS::initExchange(AMQP::TcpChannel & rabbit_channel) -{ - /// Exchange hierarchy: - /// 1. Main exchange (defined with table settings - nats_exchange_name, nats_exchange_type). - /// 2. Bridge exchange (fanout). Used to easily disconnect main exchange and to simplify queue bindings. - /// 3. Sharding (or hash) exchange. Used in case of multiple queues. - /// 4. Consumer exchange. Just an alias for bridge_exchange or sharding exchange to know to what exchange - /// queues will be bound. - - /// All exchanges are declared with options: - /// 1. `durable` (survive NATS server restart) - /// 2. `autodelete` (auto delete in case of queue bindings are dropped). - - rabbit_channel.declareExchange(exchange_name, exchange_type, AMQP::durable) - .onError([&](const char * message) - { - /// This error can be a result of attempt to declare exchange if it was already declared but - /// 1) with different exchange type. - /// 2) with different exchange settings. - throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " - + std::string(message), ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE); - }); - - rabbit_channel.declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable | AMQP::autodelete) - .onError([&](const char * message) - { - /// This error is not supposed to happen as this exchange name is always unique to type and its settings. - throw Exception( - ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, "Unable to declare bridge exchange ({}). Reason: {}", bridge_exchange, std::string(message)); - }); - - if (!hash_exchange) - { - consumer_exchange = bridge_exchange; - return; - } - - AMQP::Table binding_arguments; - - /// Default routing key property in case of hash exchange is a routing key, which is required to be an integer. - /// Support for arbitrary exchange type (i.e. arbitrary pattern of routing keys) requires to eliminate this dependency. - /// This settings changes hash property to message_id. - binding_arguments["hash-property"] = "message_id"; - - /// Declare hash exchange for sharding. - rabbit_channel.declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable | AMQP::autodelete, binding_arguments) - .onError([&](const char * message) - { - /// This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared - /// to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter - /// is bad. - throw Exception( - ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, - "Unable to declare sharding exchange ({}). Reason: {}", sharding_exchange, std::string(message)); - }); - - rabbit_channel.bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) - .onError([&](const char * message) - { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind bridge exchange ({}) to sharding exchange ({}). Reason: {}", - bridge_exchange, - sharding_exchange, - std::string(message)); - }); - - consumer_exchange = sharding_exchange; -} - - -void StorageNATS::bindExchange(AMQP::TcpChannel & rabbit_channel) -{ - size_t bound_keys = 0; - - if (exchange_type == AMQP::ExchangeType::headers) - { - AMQP::Table bind_headers; - for (const auto & header : routing_keys) - { - std::vector matching; - boost::split(matching, header, [](char c){ return c == '='; }); - bind_headers[matching[0]] = matching[1]; - } - - rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) - .onSuccess([&]() { connection->getHandler().stopLoop(); }) - .onError([&](const char * message) - { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", - exchange_name, bridge_exchange, std::string(message)); - }); - } - else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) - { - rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0]) - .onSuccess([&]() { connection->getHandler().stopLoop(); }) - .onError([&](const char * message) - { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", - exchange_name, bridge_exchange, std::string(message)); - }); - } - else - { - for (const auto & routing_key : routing_keys) - { - rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_key) - .onSuccess([&]() - { - ++bound_keys; - if (bound_keys == routing_keys.size()) - connection->getHandler().stopLoop(); - }) - .onError([&](const char * message) - { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", - exchange_name, bridge_exchange, std::string(message)); - }); - } - } - - connection->getHandler().startBlockingLoop(); -} - - -void StorageNATS::bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel) -{ - auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) - { - queues.emplace_back(queue_name); - LOG_DEBUG(log, "Queue {} is declared", queue_name); - - if (msgcount) - LOG_INFO(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); - - /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are - * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for - * fanout exchange it can be arbitrary - */ - rabbit_channel.bindQueue(consumer_exchange, queue_name, std::to_string(queue_id)) - .onSuccess([&] { connection->getHandler().stopLoop(); }) - .onError([&](const char * message) - { - throw Exception( - ErrorCodes::CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, - "Failed to create queue binding for exchange {}. Reason: {}", exchange_name, std::string(message)); - }); - }; - - auto error_callback([&](const char * message) - { - /* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a - * given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different - * max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously - * declared queues via any of the various cli tools. - */ - throw Exception("Failed to declare queue. Probably queue settings are conflicting: max_block_size, deadletter_exchange. Attempt \ - specifying differently those settings or use a different queue_base or manually delete previously declared queues, \ - which were declared with the same names. ERROR reason: " - + std::string(message), ErrorCodes::BAD_ARGUMENTS); - }); - - AMQP::Table queue_settings; - - std::unordered_set integer_settings = {"x-max-length", "x-max-length-bytes", "x-message-ttl", "x-expires", "x-priority", "x-max-priority"}; - std::unordered_set string_settings = {"x-overflow", "x-dead-letter-exchange", "x-queue-type"}; - - /// Check user-defined settings. - if (!queue_settings_list.empty()) - { - for (const auto & setting : queue_settings_list) - { - Strings setting_values; - splitInto<'='>(setting_values, setting); - if (setting_values.size() != 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid settings string: {}", setting); - - String key = setting_values[0], value = setting_values[1]; - - if (integer_settings.contains(key)) - queue_settings[key] = parse(value); - else if (string_settings.find(key) != string_settings.end()) - queue_settings[key] = value; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported queue setting: {}", value); - } - } - - /// Impose default settings if there are no user-defined settings. - if (!queue_settings.contains("x-max-length")) - { - queue_settings["x-max-length"] = queue_size; - } - if (!queue_settings.contains("x-overflow")) - { - queue_settings["x-overflow"] = "reject-publish"; - } - - /// If queue_base - a single name, then it can be used as one specific queue, from which to read. - /// Otherwise it is used as a generator (unique for current table) of queue names, because it allows to - /// maximize performance - via setting `nats_num_queues`. - const String queue_name = !hash_exchange ? queue_base : std::to_string(queue_id) + "_" + queue_base; - - /// AMQP::autodelete setting is not allowed, because in case of server restart there will be no consumers - /// and deleting queues should not take place. - rabbit_channel.declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); - connection->getHandler().startBlockingLoop(); -} - - -bool StorageNATS::updateChannel(ChannelPtr & channel) -{ - try - { - channel = connection->createChannel(); - return true; - } - catch (...) - { - tryLogCurrentException(log); - return false; - } -} - - -void StorageNATS::prepareChannelForBuffer(ConsumerBufferPtr buffer) -{ - if (!buffer) - return; - - if (buffer->queuesCount() != queues.size()) - buffer->updateQueues(queues); - - buffer->updateAckTracker(); - - if (updateChannel(buffer->getChannel())) - buffer->setupChannel(); -} - - -void StorageNATS::unbindExchange() -{ - /* This is needed because with NATS (without special adjustments) can't, for example, properly make mv if there was insert query - * on the same table before, and in another direction it will make redundant copies, but most likely nobody will do that. - * As publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in - * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as - * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible - * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local - * queues, then messages will go both ways and in one of them they will remain not consumed. So need to disconnect local exchange - * bindings to remove redunadant message copies, but after that mv cannot work unless those bindings are recreated. Recreating them is - * not difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. - */ - if (!exchange_removed.exchange(true)) - { - try - { - streaming_task->deactivate(); - - stopLoop(); - looping_task->deactivate(); - - auto rabbit_channel = connection->createChannel(); - rabbit_channel->removeExchange(bridge_exchange) - .onSuccess([&]() - { - connection->getHandler().stopLoop(); - }) - .onError([&](const char * message) - { - throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_REMOVE_RABBITMQ_EXCHANGE); - }); - - connection->getHandler().startBlockingLoop(); - rabbit_channel->close(); - } - catch (...) - { - exchange_removed = false; - throw; - } - } -} +//void StorageNATS::initNATS() +//{ +// if (shutdown_called || nats_is_ready) +// return; +// +// if (use_user_setup) +// { +// queues.emplace_back(queue_base); +// nats_is_ready = true; +// return; +// } +// +// try +// { +// auto nats_channel = connection->createChannel(); +// +// /// Main exchange -> Bridge exchange -> ( Sharding exchange ) -> Queues -> Consumers +// +// initExchange(*nats_channel); +// bindExchange(*nats_channel); +// +// for (const auto i : collections::range(0, num_queues)) +// bindQueue(i + 1, *nats_channel); +// +// LOG_TRACE(log, "NATS setup completed"); +// nats_is_ready = true; +// nats_channel->close(); +// } +// catch (...) +// { +// tryLogCurrentException(log); +// if (!is_attach) +// throw; +// } +//} +// +// +//void StorageNATS::initExchange(AMQP::TcpChannel & nats_channel) +//{ +// /// Exchange hierarchy: +// /// 1. Main exchange (defined with table settings - nats_exchange_name, nats_exchange_type). +// /// 2. Bridge exchange (fanout). Used to easily disconnect main exchange and to simplify queue bindings. +// /// 3. Sharding (or hash) exchange. Used in case of multiple queues. +// /// 4. Consumer exchange. Just an alias for bridge_exchange or sharding exchange to know to what exchange +// /// queues will be bound. +// +// /// All exchanges are declared with options: +// /// 1. `durable` (survive NATS server restart) +// /// 2. `autodelete` (auto delete in case of queue bindings are dropped). +// +// nats_channel.declareExchange(exchange_name, exchange_type, AMQP::durable) +// .onError([&](const char * message) +// { +// /// This error can be a result of attempt to declare exchange if it was already declared but +// /// 1) with different exchange type. +// /// 2) with different exchange settings. +// throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " +// + std::string(message), ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE); +// }); +// +// nats_channel.declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable | AMQP::autodelete) +// .onError([&](const char * message) +// { +// /// This error is not supposed to happen as this exchange name is always unique to type and its settings. +// throw Exception( +// ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, "Unable to declare bridge exchange ({}). Reason: {}", bridge_exchange, std::string(message)); +// }); +// +// if (!hash_exchange) +// { +// consumer_exchange = bridge_exchange; +// return; +// } +// +// AMQP::Table binding_arguments; +// +// /// Default routing key property in case of hash exchange is a routing key, which is required to be an integer. +// /// Support for arbitrary exchange type (i.e. arbitrary pattern of routing keys) requires to eliminate this dependency. +// /// This settings changes hash property to message_id. +// binding_arguments["hash-property"] = "message_id"; +// +// /// Declare hash exchange for sharding. +// nats_channel.declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable | AMQP::autodelete, binding_arguments) +// .onError([&](const char * message) +// { +// /// This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared +// /// to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter +// /// is bad. +// throw Exception( +// ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, +// "Unable to declare sharding exchange ({}). Reason: {}", sharding_exchange, std::string(message)); +// }); +// +// nats_channel.bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) +// .onError([&](const char * message) +// { +// throw Exception( +// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, +// "Unable to bind bridge exchange ({}) to sharding exchange ({}). Reason: {}", +// bridge_exchange, +// sharding_exchange, +// std::string(message)); +// }); +// +// consumer_exchange = sharding_exchange; +//} +// +// +//void StorageNATS::bindExchange(AMQP::TcpChannel & nats_channel) +//{ +// size_t bound_keys = 0; +// +// if (exchange_type == AMQP::ExchangeType::headers) +// { +// AMQP::Table bind_headers; +// for (const auto & header : routing_keys) +// { +// std::vector matching; +// boost::split(matching, header, [](char c){ return c == '='; }); +// bind_headers[matching[0]] = matching[1]; +// } +// +// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) +// .onSuccess([&]() { connection->getHandler().stopLoop(); }) +// .onError([&](const char * message) +// { +// throw Exception( +// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, +// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", +// exchange_name, bridge_exchange, std::string(message)); +// }); +// } +// else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) +// { +// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0]) +// .onSuccess([&]() { connection->getHandler().stopLoop(); }) +// .onError([&](const char * message) +// { +// throw Exception( +// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, +// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", +// exchange_name, bridge_exchange, std::string(message)); +// }); +// } +// else +// { +// for (const auto & routing_key : routing_keys) +// { +// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_key) +// .onSuccess([&]() +// { +// ++bound_keys; +// if (bound_keys == routing_keys.size()) +// connection->getHandler().stopLoop(); +// }) +// .onError([&](const char * message) +// { +// throw Exception( +// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, +// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", +// exchange_name, bridge_exchange, std::string(message)); +// }); +// } +// } +// +// connection->getHandler().startBlockingLoop(); +//} +// +// +//void StorageNATS::bindQueue(size_t queue_id, AMQP::TcpChannel & nats_channel) +//{ +// auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) +// { +// queues.emplace_back(queue_name); +// LOG_DEBUG(log, "Queue {} is declared", queue_name); +// +// if (msgcount) +// LOG_INFO(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); +// +// /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are +// * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for +// * fanout exchange it can be arbitrary +// */ +// nats_channel.bindQueue(consumer_exchange, queue_name, std::to_string(queue_id)) +// .onSuccess([&] { connection->getHandler().stopLoop(); }) +// .onError([&](const char * message) +// { +// throw Exception( +// ErrorCodes::CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, +// "Failed to create queue binding for exchange {}. Reason: {}", exchange_name, std::string(message)); +// }); +// }; +// +// auto error_callback([&](const char * message) +// { +// /* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a +// * given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different +// * max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously +// * declared queues via any of the various cli tools. +// */ +// throw Exception("Failed to declare queue. Probably queue settings are conflicting: max_block_size, deadletter_exchange. Attempt \ +// specifying differently those settings or use a different queue_base or manually delete previously declared queues, \ +// which were declared with the same names. ERROR reason: " +// + std::string(message), ErrorCodes::BAD_ARGUMENTS); +// }); +// +// AMQP::Table queue_settings; +// +// std::unordered_set integer_settings = {"x-max-length", "x-max-length-bytes", "x-message-ttl", "x-expires", "x-priority", "x-max-priority"}; +// std::unordered_set string_settings = {"x-overflow", "x-dead-letter-exchange", "x-queue-type"}; +// +// /// Check user-defined settings. +// if (!queue_settings_list.empty()) +// { +// for (const auto & setting : queue_settings_list) +// { +// Strings setting_values; +// splitInto<'='>(setting_values, setting); +// if (setting_values.size() != 2) +// throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid settings string: {}", setting); +// +// String key = setting_values[0], value = setting_values[1]; +// +// if (integer_settings.contains(key)) +// queue_settings[key] = parse(value); +// else if (string_settings.find(key) != string_settings.end()) +// queue_settings[key] = value; +// else +// throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported queue setting: {}", value); +// } +// } +// +// /// Impose default settings if there are no user-defined settings. +// if (!queue_settings.contains("x-max-length")) +// { +// queue_settings["x-max-length"] = queue_size; +// } +// if (!queue_settings.contains("x-overflow")) +// { +// queue_settings["x-overflow"] = "reject-publish"; +// } +// +// /// If queue_base - a single name, then it can be used as one specific queue, from which to read. +// /// Otherwise it is used as a generator (unique for current table) of queue names, because it allows to +// /// maximize performance - via setting `nats_num_queues`. +// const String queue_name = !hash_exchange ? queue_base : std::to_string(queue_id) + "_" + queue_base; +// +// /// AMQP::autodelete setting is not allowed, because in case of server restart there will be no consumers +// /// and deleting queues should not take place. +// nats_channel.declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); +// connection->getHandler().startBlockingLoop(); +//} +// +// +//bool StorageNATS::updateChannel(ChannelPtr & channel) +//{ +// try +// { +// channel = connection->createChannel(); +// return true; +// } +// catch (...) +// { +// tryLogCurrentException(log); +// return false; +// } +//} +// +// +//void StorageNATS::prepareChannelForBuffer(ConsumerBufferPtr buffer) +//{ +// if (!buffer) +// return; +// +// if (buffer->queuesCount() != queues.size()) +// buffer->updateQueues(queues); +// +// buffer->updateAckTracker(); +// +// if (updateChannel(buffer->getChannel())) +// buffer->setupChannel(); +//} +// +// +//void StorageNATS::unbindExchange() +//{ +// /* This is needed because with NATS (without special adjustments) can't, for example, properly make mv if there was insert query +// * on the same table before, and in another direction it will make redundant copies, but most likely nobody will do that. +// * As publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in +// * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as +// * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible +// * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local +// * queues, then messages will go both ways and in one of them they will remain not consumed. So need to disconnect local exchange +// * bindings to remove redunadant message copies, but after that mv cannot work unless those bindings are recreated. Recreating them is +// * not difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. +// */ +// if (!exchange_removed.exchange(true)) +// { +// try +// { +// streaming_task->deactivate(); +// +// stopLoop(); +// looping_task->deactivate(); +// +// auto nats_channel = connection->createChannel(); +// nats_channel->removeExchange(bridge_exchange) +// .onSuccess([&]() +// { +// connection->getHandler().stopLoop(); +// }) +// .onError([&](const char * message) +// { +// throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_REMOVE_RABBITMQ_EXCHANGE); +// }); +// +// connection->getHandler().startBlockingLoop(); +// nats_channel->close(); +// } +// catch (...) +// { +// exchange_removed = false; +// throw; +// } +// } +//} Pipe StorageNATS::read( @@ -657,8 +649,8 @@ Pipe StorageNATS::read( size_t /* max_block_size */, unsigned /* num_streams */) { - if (!rabbit_is_ready) - throw Exception("NATS setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_RABBITMQ); +// if (!nats_is_ready) +// throw Exception("NATS setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_RABBITMQ); if (num_created_consumers == 0) return {}; @@ -682,25 +674,25 @@ Pipe StorageNATS::read( throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "No connection to {}", connection->connectionInfoForLog()); } - initializeBuffers(); +// initializeBuffers(); Pipes pipes; pipes.reserve(num_created_consumers); for (size_t i = 0; i < num_created_consumers; ++i) { - auto rabbit_source = std::make_shared( + auto nats_source = std::make_shared( *this, storage_snapshot, modified_context, column_names, 1, nats_settings->nats_commit_on_select); auto converting_dag = ActionsDAG::makeConvertingActions( - rabbit_source->getPort().getHeader().getColumnsWithTypeAndName(), + nats_source->getPort().getHeader().getColumnsWithTypeAndName(), sample_block.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); auto converting = std::make_shared(std::move(converting_dag)); - auto converting_transform = std::make_shared(rabbit_source->getPort().getHeader(), std::move(converting)); + auto converting_transform = std::make_shared(nats_source->getPort().getHeader(), std::move(converting)); - pipes.emplace_back(std::move(rabbit_source)); + pipes.emplace_back(std::move(nats_source)); pipes.back().addTransform(std::move(converting_transform)); } @@ -722,13 +714,14 @@ SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & m void StorageNATS::startup() { - if (!rabbit_is_ready) + if (!nats_is_ready) { if (connection->isConnected()) { try { - initNATS(); +// initNATS(); + LOG_DEBUG(log, "Fake init lul"); } catch (...) { @@ -783,11 +776,13 @@ void StorageNATS::shutdown() for (auto & buffer : buffers) buffer->closeChannel(); - cleanupNATS(); +// cleanupNATS(); } /// It is important to close connection here - before removing consumer buffers, because /// it will finish and clean callbacks, which might use those buffers data. + if (connection->getHandler().loopRunning()) + stopLoop(); connection->disconnect(); for (size_t i = 0; i < num_created_consumers; ++i) @@ -802,53 +797,53 @@ void StorageNATS::shutdown() /// The only thing publishers are supposed to be aware of is _exchanges_ and queues are a responsibility of a consumer. /// Therefore, if a table is dropped, a clean up is needed. -void StorageNATS::cleanupNATS() const -{ - if (use_user_setup) - return; - - connection->heartbeat(); - if (!connection->isConnected()) - { - String queue_names; - for (const auto & queue : queues) - { - if (!queue_names.empty()) - queue_names += ", "; - queue_names += queue; - } - LOG_WARNING(log, - "NATS clean up not done, because there is no connection in table's shutdown." - "There are {} queues ({}), which might need to be deleted manually. Exchanges will be auto-deleted", - queues.size(), queue_names); - return; - } - - auto rabbit_channel = connection->createChannel(); - for (const auto & queue : queues) - { - /// AMQP::ifunused is needed, because it is possible to share queues between multiple tables and dropping - /// on of them should not affect others. - /// AMQP::ifempty is not used on purpose. - - rabbit_channel->removeQueue(queue, AMQP::ifunused) - .onSuccess([&](uint32_t num_messages) - { - LOG_TRACE(log, "Successfully deleted queue {}, messages contained {}", queue, num_messages); - connection->getHandler().stopLoop(); - }) - .onError([&](const char * message) - { - LOG_ERROR(log, "Failed to delete queue {}. Error message: {}", queue, message); - connection->getHandler().stopLoop(); - }); - } - connection->getHandler().startBlockingLoop(); - rabbit_channel->close(); - - /// Also there is no need to cleanup exchanges as they were created with AMQP::autodelete option. Once queues - /// are removed, exchanges will also be cleaned. -} +//void StorageNATS::cleanupNATS() const +//{ +// if (use_user_setup) +// return; +// +// connection->heartbeat(); +// if (!connection->isConnected()) +// { +// String queue_names; +// for (const auto & queue : queues) +// { +// if (!queue_names.empty()) +// queue_names += ", "; +// queue_names += queue; +// } +// LOG_WARNING(log, +// "NATS clean up not done, because there is no connection in table's shutdown." +// "There are {} queues ({}), which might need to be deleted manually. Exchanges will be auto-deleted", +// queues.size(), queue_names); +// return; +// } +// +// auto nats_channel = connection->createChannel(); +// for (const auto & queue : queues) +// { +// /// AMQP::ifunused is needed, because it is possible to share queues between multiple tables and dropping +// /// on of them should not affect others. +// /// AMQP::ifempty is not used on purpose. +// +// nats_channel->removeQueue(queue, AMQP::ifunused) +// .onSuccess([&](uint32_t num_messages) +// { +// LOG_TRACE(log, "Successfully deleted queue {}, messages contained {}", queue, num_messages); +// connection->getHandler().stopLoop(); +// }) +// .onError([&](const char * message) +// { +// LOG_ERROR(log, "Failed to delete queue {}. Error message: {}", queue, message); +// connection->getHandler().stopLoop(); +// }); +// } +// connection->getHandler().startBlockingLoop(); +// nats_channel->close(); +// +// /// Also there is no need to cleanup exchanges as they were created with AMQP::autodelete option. Once queues +// /// are removed, exchanges will also be cleaned. +//} void StorageNATS::pushReadBuffer(ConsumerBufferPtr buffer) @@ -888,18 +883,18 @@ ConsumerBufferPtr StorageNATS::popReadBuffer(std::chrono::milliseconds timeout) ConsumerBufferPtr StorageNATS::createReadBuffer() { return std::make_shared( - connection->getHandler(), queues, ++consumer_id, + connection->getHandler(), connection, subjects, unique_strbase, log, row_delimiter, queue_size, shutdown_called); } -ProducerBufferPtr StorageNATS::createWriteBuffer() -{ - return std::make_shared( - configuration, getContext(), routing_keys, exchange_name, exchange_type, - producer_id.fetch_add(1), persistent, shutdown_called, log, - row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); -} +//ProducerBufferPtr StorageNATS::createWriteBuffer() +//{ +// return std::make_shared( +// configuration, getContext(), routing_keys, exchange_name, exchange_type, +// producer_id.fetch_add(1), persistent, shutdown_called, log, +// row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); +//} bool StorageNATS::checkDependencies(const StorageID & table_id) @@ -930,73 +925,69 @@ bool StorageNATS::checkDependencies(const StorageID & table_id) } -void StorageNATS::initializeBuffers() -{ - assert(rabbit_is_ready); - if (!initialized) - { - for (const auto & buffer : buffers) - prepareChannelForBuffer(buffer); - initialized = true; - } -} +//void StorageNATS::initializeBuffers() +//{ +// assert(nats_is_ready); +// if (!initialized) +// { +// for (const auto & buffer : buffers) +// prepareChannelForBuffer(buffer); +// initialized = true; +// } +//} void StorageNATS::streamingToViewsFunc() { - if (rabbit_is_ready) + try { - try + auto table_id = getStorageID(); + + // Check if at least one direct dependency is attached + size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + bool nats_connected = connection->isConnected() || connection->reconnect(); + + if (dependencies_count && nats_connected) { - auto table_id = getStorageID(); + auto start_time = std::chrono::steady_clock::now(); - // Check if at least one direct dependency is attached - size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); - bool rabbit_connected = connection->isConnected() || connection->reconnect(); + mv_attached.store(true); - if (dependencies_count && rabbit_connected) + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!shutdown_called && num_created_consumers > 0) { - initializeBuffers(); - auto start_time = std::chrono::steady_clock::now(); + if (!checkDependencies(table_id)) + break; - mv_attached.store(true); + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - // Keep streaming as long as there are attached views and streaming is not cancelled - while (!shutdown_called && num_created_consumers > 0) + if (streamToViews()) { - if (!checkDependencies(table_id)) - break; + /// Reschedule with backoff. + if (milliseconds_to_wait < BACKOFF_TRESHOLD) + milliseconds_to_wait *= 2; + stopLoopIfNoReaders(); + break; + } + else + { + milliseconds_to_wait = RESCHEDULE_MS; + } - LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - - if (streamToViews()) - { - /// Reschedule with backoff. - if (milliseconds_to_wait < BACKOFF_TRESHOLD) - milliseconds_to_wait *= 2; - stopLoopIfNoReaders(); - break; - } - else - { - milliseconds_to_wait = RESCHEDULE_MS; - } - - auto end_time = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - if (duration.count() > MAX_THREAD_WORK_DURATION_MS) - { - stopLoopIfNoReaders(); - LOG_TRACE(log, "Reschedule streaming. Thread work duration limit exceeded."); - break; - } + auto end_time = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + if (duration.count() > MAX_THREAD_WORK_DURATION_MS) + { + stopLoopIfNoReaders(); + LOG_TRACE(log, "Reschedule streaming. Thread work duration limit exceeded."); + break; } } } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } mv_attached.store(false); @@ -1080,9 +1071,7 @@ bool StorageNATS::streamToViews() if (connection->reconnect()) { - LOG_DEBUG(log, "Connection restored, updating channels"); - for (auto & source : sources) - source->updateChannel(); + LOG_DEBUG(log, "Connection restored"); } else { @@ -1098,39 +1087,13 @@ bool StorageNATS::streamToViews() if (source->queueEmpty()) ++queue_empty; - if (source->needChannelUpdate()) - { - auto buffer = source->getBuffer(); - prepareChannelForBuffer(buffer); - } - - /* false is returned by the sendAck function in only two cases: - * 1) if connection failed. In this case all channels will be closed and will be unable to send ack. Also ack is made based on - * delivery tags, which are unique to channels, so if channels fail, those delivery tags will become invalid and there is - * no way to send specific ack from a different channel. Actually once the server realises that it has messages in a queue - * waiting for confirm from a channel which suddenly closed, it will immediately make those messages accessible to other - * consumers. So in this case duplicates are inevitable. - * 2) size of the sent frame (libraries's internal request interface) exceeds max frame - internal library error. This is more - * common for message frames, but not likely to happen to ack frame I suppose. So I do not believe it is likely to happen. - * Also in this case if channel didn't get closed - it is ok if failed to send ack, because the next attempt to send ack on - * the same channel will also commit all previously not-committed messages. Anyway I do not think that for ack frame this - * will ever happen. - */ - if (!source->sendAck()) - { - /// Iterate loop to activate error callbacks if they happened - connection->getHandler().iterateLoop(); - if (!connection->isConnected()) - break; - } - connection->getHandler().iterateLoop(); } } if ((queue_empty == num_created_consumers) && (++read_attempts == MAX_FAILED_READ_ATTEMPTS)) { - connection->heartbeat(); +// connection->heartbeat(); read_attempts = 0; LOG_TRACE(log, "Reschedule streaming. Queues are empty."); return true; @@ -1174,11 +1137,7 @@ void registerStorageNATS(StorageFactory & factory) NamesAndTypesList StorageNATS::getVirtuals() const { return NamesAndTypesList{ - {"_exchange_name", std::make_shared()}, - {"_channel_id", std::make_shared()}, - {"_delivery_tag", std::make_shared()}, - {"_redelivered", std::make_shared()}, - {"_message_id", std::make_shared()}, + {"_subject", std::make_shared()}, {"_timestamp", std::make_shared()} }; } diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index e5b397194b7..2630df33ae4 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -10,7 +10,6 @@ #include #include #include -#include #include #include @@ -61,12 +60,12 @@ public: const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; - String getExchange() const { return exchange_name; } - void unbindExchange(); - - bool updateChannel(ChannelPtr & channel); - void updateQueues(std::vector & queues_) { queues_ = queues; } - void prepareChannelForBuffer(ConsumerBufferPtr buffer); +// String getExchange() const { return exchange_name; } +// void unbindExchange(); +// +// bool updateChannel(ChannelPtr & channel); + void updateSubjects(std::vector & subjects_) { subjects_ = subjects; } +// void prepareChannelForBuffer(ConsumerBufferPtr buffer); void incrementReader(); void decrementReader(); @@ -82,30 +81,25 @@ protected: private: ContextMutablePtr nats_context; std::unique_ptr nats_settings; + std::vector subjects; - const String exchange_name; const String format_name; - AMQP::ExchangeType exchange_type; - Names routing_keys; char row_delimiter; const String schema_name; size_t num_consumers; - size_t num_queues; - String queue_base; - Names queue_settings_list; /// For insert query. Mark messages as durable. const bool persistent; - /// A table setting. It is possible not to perform any NATS setup, which is supposed to be consumer-side setup: - /// declaring exchanges, queues, bindings. Instead everything needed from NATS table is to connect to a specific queue. - /// This solution disables all optimizations and is not really optimal, but allows user to fully control all NATS setup. - bool use_user_setup; +// /// A table setting. It is possible not to perform any NATS setup, which is supposed to be consumer-side setup: +// /// declaring exchanges, queues, bindings. Instead everything needed from NATS table is to connect to a specific queue. +// /// This solution disables all optimizations and is not really optimal, but allows user to fully control all NATS setup. +// bool use_user_setup; - bool hash_exchange; +// bool hash_exchange; Poco::Logger * log; - NATSConnectionPtr connection; /// Connection for all consumers + NATSConnectionManagerPtr connection; /// Connection for all consumers NATSConfiguration configuration; size_t num_created_consumers = 0; @@ -119,11 +113,8 @@ private: /// to setup size of inner buffer for received messages uint32_t queue_size; - String sharding_exchange, bridge_exchange, consumer_exchange; size_t consumer_id = 0; /// counter for consumer buffer, needed for channel id - std::vector queues; - std::once_flag flag; /// remove exchange only once std::mutex task_mutex; BackgroundSchedulePool::TaskHolder streaming_task; @@ -143,7 +134,7 @@ private: std::atomic producer_id = 1; /// Has connection background task completed successfully? /// It is started only once -- in constructor. - std::atomic rabbit_is_ready = false; + std::atomic nats_is_ready = false; /// Allow to remove exchange only once. std::atomic exchange_removed = false; /// For select query we must be aware of the end of streaming @@ -176,7 +167,7 @@ private: void stopLoop(); void stopLoopIfNoReaders(); - static Names parseSettings(String settings_list); + static Names parseSubjects(String subjects_list); static AMQP::ExchangeType defineExchangeType(String exchange_type_); static String getTableBasedName(String name, const StorageID & table_id); @@ -184,12 +175,12 @@ private: size_t getMaxBlockSize() const; void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); - void initNATS(); - void cleanupNATS() const; +// void initNATS(); +// void cleanupNATS() const; - void initExchange(AMQP::TcpChannel & rabbit_channel); - void bindExchange(AMQP::TcpChannel & rabbit_channel); - void bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel); +// void initExchange(AMQP::TcpChannel & nats_channel); +// void bindExchange(AMQP::TcpChannel & nats_channel); +// void bindQueue(size_t queue_id, AMQP::TcpChannel & nats_channel); bool streamToViews(); bool checkDependencies(const StorageID & table_id); diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index 2f4ea7bee53..175d9458423 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -112,53 +112,53 @@ void WriteBufferToNATSProducer::countRow() void WriteBufferToNATSProducer::setupChannel() { - producer_channel = connection.createChannel(); - - producer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Producer's channel {} error: {}", channel_id, message); - - /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) - producer_channel->close(); - - /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once - * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid - */ - for (const auto & record : delivery_record) - if (!returned.push(record.second)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); - - LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", - channel_id, delivery_record.size(), payloads.size()); - - /// Delivery tags are scoped per channel. - delivery_record.clear(); - delivery_tag = 0; - producer_ready = false; - }); - - producer_channel->onReady([&]() - { - channel_id = channel_id_base + "_" + std::to_string(channel_id_counter++); - LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); - - /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, - * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times - * slower than the second, so default is second and the first is turned on in table setting. - * - * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html - */ - producer_channel->confirmSelect() - .onAck([&](uint64_t acked_delivery_tag, bool multiple) - { - removeRecord(acked_delivery_tag, multiple, false); - }) - .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) - { - removeRecord(nacked_delivery_tag, multiple, true); - }); - producer_ready = true; - }); +// producer_channel = connection.createChannel(); +// +// producer_channel->onError([&](const char * message) +// { +// LOG_ERROR(log, "Producer's channel {} error: {}", channel_id, message); +// +// /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) +// producer_channel->close(); +// +// /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once +// * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid +// */ +// for (const auto & record : delivery_record) +// if (!returned.push(record.second)) +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); +// +// LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", +// channel_id, delivery_record.size(), payloads.size()); +// +// /// Delivery tags are scoped per channel. +// delivery_record.clear(); +// delivery_tag = 0; +// producer_ready = false; +// }); +// +// producer_channel->onReady([&]() +// { +// channel_id = channel_id_base + "_" + std::to_string(channel_id_counter++); +// LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); +// +// /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, +// * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times +// * slower than the second, so default is second and the first is turned on in table setting. +// * +// * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html +// */ +// producer_channel->confirmSelect() +// .onAck([&](uint64_t acked_delivery_tag, bool multiple) +// { +// removeRecord(acked_delivery_tag, multiple, false); +// }) +// .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) +// { +// removeRecord(nacked_delivery_tag, multiple, true); +// }); +// producer_ready = true; +// }); } diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h index d0d80a6cf9d..f2e9e04e227 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.h +++ b/src/Storages/NATS/WriteBufferToNATSProducer.h @@ -49,7 +49,7 @@ private: void removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish); void publish(ConcurrentBoundedQueue> & message, bool republishing); - NATSConnection connection; + NATSConnectionManager connection; const Names routing_keys; const String exchange_name; @@ -69,7 +69,7 @@ private: bool producer_ready = false; /// Channel errors lead to channel closure, need to count number of recreated channels to update channel id - UInt64 channel_id_counter = 0; +// UInt64 channel_id_counter = 0; /// channel id which contains id of current producer buffer and serial number of recreated channel in this buffer String channel_id; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index f567bf6eefc..1e777cbf946 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -51,6 +51,8 @@ void registerStorageMySQL(StorageFactory & factory); void registerStorageMongoDB(StorageFactory & factory); +void registerStorageNATS(StorageFactory & factory); + #if USE_RDKAFKA void registerStorageKafka(StorageFactory & factory); #endif @@ -128,6 +130,8 @@ void registerStorages() registerStorageMongoDB(factory); + registerStorageNATS(factory); + #if USE_RDKAFKA registerStorageKafka(factory); #endif From 5a78d408231dae59de821f1123bc293cf06f6a05 Mon Sep 17 00:00:00 2001 From: tchepavel Date: Tue, 10 May 2022 16:00:07 +0300 Subject: [PATCH 003/627] Change submodule to patched --- .gitmodules | 2 +- contrib/nats-io | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index 027e3f72929..342280be977 100644 --- a/.gitmodules +++ b/.gitmodules @@ -264,4 +264,4 @@ url = https://github.com/zlib-ng/minizip-ng [submodule "contrib/nats-io"] path = contrib/nats-io - url = https://github.com/nats-io/nats.c.git + url = https://github.com/tchepavel/nats.c.git diff --git a/contrib/nats-io b/contrib/nats-io index d1f59f7bcf8..1eefa3aad67 160000 --- a/contrib/nats-io +++ b/contrib/nats-io @@ -1 +1 @@ -Subproject commit d1f59f7bcf8465526f7e6d9c99982cbd6b209547 +Subproject commit 1eefa3aad671d71a5c18815af1f9f4bdbaee7ec2 From 04f256ff554f083a03f93823ce79d78a13ce797e Mon Sep 17 00:00:00 2001 From: tchepavel Date: Tue, 10 May 2022 18:39:57 +0300 Subject: [PATCH 004/627] Select, MV, Insert working --- base/harmful/harmful.c | 2 +- contrib/nats-io | 2 +- src/Common/ErrorCodes.cpp | 1 + src/Storages/NATS/NATSConnection.cpp | 28 +- src/Storages/NATS/NATSConnection.h | 9 +- src/Storages/NATS/NATSHandler.cpp | 2 +- src/Storages/NATS/NATSSettings.h | 5 +- src/Storages/NATS/NATSSink.cpp | 5 +- .../NATS/ReadBufferFromNATSConsumer.cpp | 6 +- src/Storages/NATS/StorageNATS.cpp | 457 +----------------- src/Storages/NATS/StorageNATS.h | 19 +- .../NATS/WriteBufferToNATSProducer.cpp | 206 ++------ src/Storages/NATS/WriteBufferToNATSProducer.h | 49 +- 13 files changed, 102 insertions(+), 689 deletions(-) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index adc3f7158f6..5a27cae0383 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -206,7 +206,7 @@ TRAP(lgammal) TRAP(nftw) TRAP(nl_langinfo) TRAP(putc_unlocked) -//TRAP(rand) Used by nats-io at startup +TRAP(rand) /** In the current POSIX.1 specification (POSIX.1-2008), readdir() is not required to be thread-safe. However, in modern * implementations (including the glibc implementation), concurrent calls to readdir() that specify different directory streams * are thread-safe. In cases where multiple threads must read from the same directory stream, using readdir() with external diff --git a/contrib/nats-io b/contrib/nats-io index 1eefa3aad67..6b2227f3675 160000 --- a/contrib/nats-io +++ b/contrib/nats-io @@ -1 +1 @@ -Subproject commit 1eefa3aad671d71a5c18815af1f9f4bdbaee7ec2 +Subproject commit 6b2227f36757da090321e2d317569d2bd42c4cc1 diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index eb84e24b713..2644fa3be95 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -624,6 +624,7 @@ M(653, CANNOT_PARSE_BACKUP_SETTINGS) \ M(654, WRONG_BACKUP_SETTINGS) \ M(655, FAILED_TO_RESTORE_METADATA_ON_OTHER_NODE) \ + M(656, CANNOT_CONNECT_NATS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index cef8a2eb8c1..c76583dae5f 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -1,6 +1,6 @@ #include "NATSConnection.h" -#include +#include #include @@ -58,8 +58,14 @@ SubscriptionPtr NATSConnectionManager::createSubscription(const std::string& sub if (status == NATS_OK) status = natsSubscription_SetPendingLimits(ns, -1, -1); if (status == NATS_OK) + { LOG_DEBUG(log, "Subscribed to subject {}", subject); - return SubscriptionPtr(ns, &natsSubscription_Destroy); + return SubscriptionPtr(ns, &natsSubscription_Destroy); + } + else + { + return SubscriptionPtr(nullptr, &natsSubscription_Destroy); + } } void NATSConnectionManager::disconnect() @@ -96,6 +102,10 @@ void NATSConnectionManager::connectImpl() { address = configuration.connection_string; } + natsOptions_SetMaxReconnect(options, configuration.max_reconnect); + natsOptions_SetReconnectWait(options, configuration.reconnect_wait); +// natsOptions_SetDisconnectedCB(options, disconnectedCallback, this); +// natsOptions_SetReconnectedCB(options, reconnectedCallback, this); natsOptions_SetURL(options, address.c_str()); status = natsConnection_Connect(&connection, options); if (status != NATS_OK) @@ -121,4 +131,18 @@ void NATSConnectionManager::disconnectImpl() event_handler.changeConnectionStatus(false); } +void NATSConnectionManager::reconnectedCallback(natsConnection * nc, void * manager) +{ + char buffer[64]; + + buffer[0] = '\0'; + natsConnection_GetConnectedUrl(nc, buffer, sizeof(buffer)); + LOG_DEBUG(static_cast(manager)->log, "Got reconnected to NATS server: {}.", buffer); +} + +void NATSConnectionManager::disconnectedCallback(natsConnection *, void * manager) +{ + LOG_DEBUG(static_cast(manager)->log, "Got disconnected from NATS server."); +} + } diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h index ab97012061c..813115454e7 100644 --- a/src/Storages/NATS/NATSConnection.h +++ b/src/Storages/NATS/NATSConnection.h @@ -13,7 +13,9 @@ struct NATSConfiguration UInt16 port; String username; String password; - String vhost; + + int max_reconnect; + int reconnect_wait; bool secure; String connection_string; @@ -26,6 +28,8 @@ public: NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_); ~NATSConnectionManager() { natsConnection_Destroy(connection); } + natsConnection * getConnection() { return connection; } + bool isConnected(); bool connect(); @@ -50,6 +54,9 @@ private: void disconnectImpl(); + static void disconnectedCallback(natsConnection * nc, void * storage); + static void reconnectedCallback(natsConnection * nc, void * storage); + NATSConfiguration configuration; Poco::Logger * log; diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index cb155522a6a..cdaa93ef93c 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h index c30869e8ea3..2cb5365a13b 100644 --- a/src/Storages/NATS/NATSSettings.h +++ b/src/Storages/NATS/NATSSettings.h @@ -17,14 +17,15 @@ namespace DB M(UInt64, nats_num_consumers, 1, "The number of consumer channels per table.", 0) \ M(Bool, nats_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \ M(Bool, nats_secure, false, "Use SSL connection", 0) \ + M(UInt64, nats_max_reconnect, 5, "Maximum amount of reconnection attempts.", 0) \ + M(UInt64, nats_reconnect_wait, 2000, "Amount to sleep between each reconnect attempt.", 0) \ + M(String, nats_password, "", "NATS password", 0) \ M(String, nats_address, "", "Address for connection", 0) \ M(UInt64, nats_skip_broken_messages, 0, "Skip at least this number of broken messages from NATS per block", 0) \ M(UInt64, nats_max_block_size, 0, "Number of row collected before flushing data from NATS.", 0) \ M(Milliseconds, nats_flush_interval_ms, 0, "Timeout for flushing data from NATS.", 0) \ - M(String, nats_vhost, "/", "NATS vhost.", 0) \ M(Bool, nats_queue_consume, false, "Use user-defined queues and do not make any NATS setup: declaring exchanges, queues, bindings", 0) \ M(String, nats_username, "", "NATS username", 0) \ - M(String, nats_password, "", "NATS password", 0) \ M(Bool, nats_commit_on_select, false, "Commit messages when select query is made", 0) \ #define LIST_OF_NATS_SETTINGS(M) \ diff --git a/src/Storages/NATS/NATSSink.cpp b/src/Storages/NATS/NATSSink.cpp index 00fe93a9e68..8be95099261 100644 --- a/src/Storages/NATS/NATSSink.cpp +++ b/src/Storages/NATS/NATSSink.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB @@ -18,13 +18,12 @@ NATSSink::NATSSink( , metadata_snapshot(metadata_snapshot_) , context(context_) { -// storage.unbindExchange(); } void NATSSink::onStart() { -// buffer = storage.createWriteBuffer(); + buffer = storage.createWriteBuffer(); buffer->activateWriting(); auto format_settings = getFormatSettings(context); diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index 5edd48d97c8..1a0d72cc559 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include "Poco/Timer.h" #include @@ -56,7 +56,9 @@ ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() void ReadBufferFromNATSConsumer::subscribe() { for (const auto & subject : subjects) { - subscriptions.emplace_back(connection->createSubscription(subject, onMsg, this)); + SubscriptionPtr subscription = connection->createSubscription(subject, onMsg, this); + if (subscription.get()) + subscriptions.emplace_back(std::move(subscription)); } // for (const auto & queue_name : subjects) // { diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 2cfe9e8be8d..afcbcd28a6d 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -23,19 +23,7 @@ #include #include #include -#include - -//static void -//onMsg(natsConnection *, natsSubscription *, natsMsg *msg, void *) -//{ -// printf("\n\n\nReceived msg: %s - %.*s\n\n\n", -// natsMsg_GetSubject(msg), -// natsMsg_GetDataLength(msg), -// natsMsg_GetData(msg)); -// fflush(stdout); -// -// natsMsg_Destroy(msg); -//} +#include namespace DB { @@ -52,10 +40,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int CANNOT_CONNECT_RABBITMQ; - extern const int CANNOT_BIND_RABBITMQ_EXCHANGE; - extern const int CANNOT_DECLARE_RABBITMQ_EXCHANGE; - extern const int CANNOT_REMOVE_RABBITMQ_EXCHANGE; - extern const int CANNOT_CREATE_RABBITMQ_QUEUE_BINDING; extern const int QUERY_NOT_ALLOWED; } @@ -86,6 +70,7 @@ StorageNATS::StorageNATS( auto parsed_address = parseAddress(getContext()->getMacros()->expand(nats_settings->nats_host_port), 5672); context_->getRemoteHostFilter().checkHostAndPort(parsed_address.first, toString(parsed_address.second)); + LOG_DEBUG(log, "Creds: u-{} p-{}", nats_settings->nats_username.value, nats_settings->nats_password.value); auto nats_username = nats_settings->nats_username.value; auto nats_password = nats_settings->nats_password.value; configuration = @@ -94,7 +79,8 @@ StorageNATS::StorageNATS( .port = parsed_address.second, .username = nats_username.empty() ? getContext()->getConfigRef().getString("nats.username") : nats_username, .password = nats_password.empty() ? getContext()->getConfigRef().getString("nats.password") : nats_password, - .vhost = getContext()->getConfigRef().getString("nats.vhost", getContext()->getMacros()->expand(nats_settings->nats_vhost)), + .max_reconnect = static_cast(nats_settings->nats_max_reconnect.value), + .reconnect_wait = static_cast(nats_settings->nats_reconnect_wait.value), .secure = nats_settings->nats_secure.value, .connection_string = getContext()->getMacros()->expand(nats_settings->nats_address) }; @@ -312,334 +298,6 @@ size_t StorageNATS::getMaxBlockSize() const } -//void StorageNATS::initNATS() -//{ -// if (shutdown_called || nats_is_ready) -// return; -// -// if (use_user_setup) -// { -// queues.emplace_back(queue_base); -// nats_is_ready = true; -// return; -// } -// -// try -// { -// auto nats_channel = connection->createChannel(); -// -// /// Main exchange -> Bridge exchange -> ( Sharding exchange ) -> Queues -> Consumers -// -// initExchange(*nats_channel); -// bindExchange(*nats_channel); -// -// for (const auto i : collections::range(0, num_queues)) -// bindQueue(i + 1, *nats_channel); -// -// LOG_TRACE(log, "NATS setup completed"); -// nats_is_ready = true; -// nats_channel->close(); -// } -// catch (...) -// { -// tryLogCurrentException(log); -// if (!is_attach) -// throw; -// } -//} -// -// -//void StorageNATS::initExchange(AMQP::TcpChannel & nats_channel) -//{ -// /// Exchange hierarchy: -// /// 1. Main exchange (defined with table settings - nats_exchange_name, nats_exchange_type). -// /// 2. Bridge exchange (fanout). Used to easily disconnect main exchange and to simplify queue bindings. -// /// 3. Sharding (or hash) exchange. Used in case of multiple queues. -// /// 4. Consumer exchange. Just an alias for bridge_exchange or sharding exchange to know to what exchange -// /// queues will be bound. -// -// /// All exchanges are declared with options: -// /// 1. `durable` (survive NATS server restart) -// /// 2. `autodelete` (auto delete in case of queue bindings are dropped). -// -// nats_channel.declareExchange(exchange_name, exchange_type, AMQP::durable) -// .onError([&](const char * message) -// { -// /// This error can be a result of attempt to declare exchange if it was already declared but -// /// 1) with different exchange type. -// /// 2) with different exchange settings. -// throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " -// + std::string(message), ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE); -// }); -// -// nats_channel.declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable | AMQP::autodelete) -// .onError([&](const char * message) -// { -// /// This error is not supposed to happen as this exchange name is always unique to type and its settings. -// throw Exception( -// ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, "Unable to declare bridge exchange ({}). Reason: {}", bridge_exchange, std::string(message)); -// }); -// -// if (!hash_exchange) -// { -// consumer_exchange = bridge_exchange; -// return; -// } -// -// AMQP::Table binding_arguments; -// -// /// Default routing key property in case of hash exchange is a routing key, which is required to be an integer. -// /// Support for arbitrary exchange type (i.e. arbitrary pattern of routing keys) requires to eliminate this dependency. -// /// This settings changes hash property to message_id. -// binding_arguments["hash-property"] = "message_id"; -// -// /// Declare hash exchange for sharding. -// nats_channel.declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable | AMQP::autodelete, binding_arguments) -// .onError([&](const char * message) -// { -// /// This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared -// /// to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter -// /// is bad. -// throw Exception( -// ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, -// "Unable to declare sharding exchange ({}). Reason: {}", sharding_exchange, std::string(message)); -// }); -// -// nats_channel.bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) -// .onError([&](const char * message) -// { -// throw Exception( -// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, -// "Unable to bind bridge exchange ({}) to sharding exchange ({}). Reason: {}", -// bridge_exchange, -// sharding_exchange, -// std::string(message)); -// }); -// -// consumer_exchange = sharding_exchange; -//} -// -// -//void StorageNATS::bindExchange(AMQP::TcpChannel & nats_channel) -//{ -// size_t bound_keys = 0; -// -// if (exchange_type == AMQP::ExchangeType::headers) -// { -// AMQP::Table bind_headers; -// for (const auto & header : routing_keys) -// { -// std::vector matching; -// boost::split(matching, header, [](char c){ return c == '='; }); -// bind_headers[matching[0]] = matching[1]; -// } -// -// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) -// .onSuccess([&]() { connection->getHandler().stopLoop(); }) -// .onError([&](const char * message) -// { -// throw Exception( -// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, -// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", -// exchange_name, bridge_exchange, std::string(message)); -// }); -// } -// else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) -// { -// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0]) -// .onSuccess([&]() { connection->getHandler().stopLoop(); }) -// .onError([&](const char * message) -// { -// throw Exception( -// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, -// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", -// exchange_name, bridge_exchange, std::string(message)); -// }); -// } -// else -// { -// for (const auto & routing_key : routing_keys) -// { -// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_key) -// .onSuccess([&]() -// { -// ++bound_keys; -// if (bound_keys == routing_keys.size()) -// connection->getHandler().stopLoop(); -// }) -// .onError([&](const char * message) -// { -// throw Exception( -// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, -// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", -// exchange_name, bridge_exchange, std::string(message)); -// }); -// } -// } -// -// connection->getHandler().startBlockingLoop(); -//} -// -// -//void StorageNATS::bindQueue(size_t queue_id, AMQP::TcpChannel & nats_channel) -//{ -// auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) -// { -// queues.emplace_back(queue_name); -// LOG_DEBUG(log, "Queue {} is declared", queue_name); -// -// if (msgcount) -// LOG_INFO(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); -// -// /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are -// * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for -// * fanout exchange it can be arbitrary -// */ -// nats_channel.bindQueue(consumer_exchange, queue_name, std::to_string(queue_id)) -// .onSuccess([&] { connection->getHandler().stopLoop(); }) -// .onError([&](const char * message) -// { -// throw Exception( -// ErrorCodes::CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, -// "Failed to create queue binding for exchange {}. Reason: {}", exchange_name, std::string(message)); -// }); -// }; -// -// auto error_callback([&](const char * message) -// { -// /* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a -// * given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different -// * max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously -// * declared queues via any of the various cli tools. -// */ -// throw Exception("Failed to declare queue. Probably queue settings are conflicting: max_block_size, deadletter_exchange. Attempt \ -// specifying differently those settings or use a different queue_base or manually delete previously declared queues, \ -// which were declared with the same names. ERROR reason: " -// + std::string(message), ErrorCodes::BAD_ARGUMENTS); -// }); -// -// AMQP::Table queue_settings; -// -// std::unordered_set integer_settings = {"x-max-length", "x-max-length-bytes", "x-message-ttl", "x-expires", "x-priority", "x-max-priority"}; -// std::unordered_set string_settings = {"x-overflow", "x-dead-letter-exchange", "x-queue-type"}; -// -// /// Check user-defined settings. -// if (!queue_settings_list.empty()) -// { -// for (const auto & setting : queue_settings_list) -// { -// Strings setting_values; -// splitInto<'='>(setting_values, setting); -// if (setting_values.size() != 2) -// throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid settings string: {}", setting); -// -// String key = setting_values[0], value = setting_values[1]; -// -// if (integer_settings.contains(key)) -// queue_settings[key] = parse(value); -// else if (string_settings.find(key) != string_settings.end()) -// queue_settings[key] = value; -// else -// throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported queue setting: {}", value); -// } -// } -// -// /// Impose default settings if there are no user-defined settings. -// if (!queue_settings.contains("x-max-length")) -// { -// queue_settings["x-max-length"] = queue_size; -// } -// if (!queue_settings.contains("x-overflow")) -// { -// queue_settings["x-overflow"] = "reject-publish"; -// } -// -// /// If queue_base - a single name, then it can be used as one specific queue, from which to read. -// /// Otherwise it is used as a generator (unique for current table) of queue names, because it allows to -// /// maximize performance - via setting `nats_num_queues`. -// const String queue_name = !hash_exchange ? queue_base : std::to_string(queue_id) + "_" + queue_base; -// -// /// AMQP::autodelete setting is not allowed, because in case of server restart there will be no consumers -// /// and deleting queues should not take place. -// nats_channel.declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); -// connection->getHandler().startBlockingLoop(); -//} -// -// -//bool StorageNATS::updateChannel(ChannelPtr & channel) -//{ -// try -// { -// channel = connection->createChannel(); -// return true; -// } -// catch (...) -// { -// tryLogCurrentException(log); -// return false; -// } -//} -// -// -//void StorageNATS::prepareChannelForBuffer(ConsumerBufferPtr buffer) -//{ -// if (!buffer) -// return; -// -// if (buffer->queuesCount() != queues.size()) -// buffer->updateQueues(queues); -// -// buffer->updateAckTracker(); -// -// if (updateChannel(buffer->getChannel())) -// buffer->setupChannel(); -//} -// -// -//void StorageNATS::unbindExchange() -//{ -// /* This is needed because with NATS (without special adjustments) can't, for example, properly make mv if there was insert query -// * on the same table before, and in another direction it will make redundant copies, but most likely nobody will do that. -// * As publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in -// * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as -// * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible -// * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local -// * queues, then messages will go both ways and in one of them they will remain not consumed. So need to disconnect local exchange -// * bindings to remove redunadant message copies, but after that mv cannot work unless those bindings are recreated. Recreating them is -// * not difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. -// */ -// if (!exchange_removed.exchange(true)) -// { -// try -// { -// streaming_task->deactivate(); -// -// stopLoop(); -// looping_task->deactivate(); -// -// auto nats_channel = connection->createChannel(); -// nats_channel->removeExchange(bridge_exchange) -// .onSuccess([&]() -// { -// connection->getHandler().stopLoop(); -// }) -// .onError([&](const char * message) -// { -// throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_REMOVE_RABBITMQ_EXCHANGE); -// }); -// -// connection->getHandler().startBlockingLoop(); -// nats_channel->close(); -// } -// catch (...) -// { -// exchange_removed = false; -// throw; -// } -// } -//} - - Pipe StorageNATS::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -649,9 +307,6 @@ Pipe StorageNATS::read( size_t /* max_block_size */, unsigned /* num_streams */) { -// if (!nats_is_ready) -// throw Exception("NATS setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_RABBITMQ); - if (num_created_consumers == 0) return {}; @@ -674,8 +329,6 @@ Pipe StorageNATS::read( throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "No connection to {}", connection->connectionInfoForLog()); } -// initializeBuffers(); - Pipes pipes; pipes.reserve(num_created_consumers); @@ -714,26 +367,9 @@ SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & m void StorageNATS::startup() { - if (!nats_is_ready) + if (!connection->isConnected()) { - if (connection->isConnected()) - { - try - { -// initNATS(); - LOG_DEBUG(log, "Fake init lul"); - } - catch (...) - { - if (!is_attach) - throw; - tryLogCurrentException(log); - } - } - else - { - connection_task->activateAndSchedule(); - } + connection_task->activateAndSchedule(); } for (size_t i = 0; i < num_consumers; ++i) @@ -794,58 +430,6 @@ void StorageNATS::shutdown() } } - -/// The only thing publishers are supposed to be aware of is _exchanges_ and queues are a responsibility of a consumer. -/// Therefore, if a table is dropped, a clean up is needed. -//void StorageNATS::cleanupNATS() const -//{ -// if (use_user_setup) -// return; -// -// connection->heartbeat(); -// if (!connection->isConnected()) -// { -// String queue_names; -// for (const auto & queue : queues) -// { -// if (!queue_names.empty()) -// queue_names += ", "; -// queue_names += queue; -// } -// LOG_WARNING(log, -// "NATS clean up not done, because there is no connection in table's shutdown." -// "There are {} queues ({}), which might need to be deleted manually. Exchanges will be auto-deleted", -// queues.size(), queue_names); -// return; -// } -// -// auto nats_channel = connection->createChannel(); -// for (const auto & queue : queues) -// { -// /// AMQP::ifunused is needed, because it is possible to share queues between multiple tables and dropping -// /// on of them should not affect others. -// /// AMQP::ifempty is not used on purpose. -// -// nats_channel->removeQueue(queue, AMQP::ifunused) -// .onSuccess([&](uint32_t num_messages) -// { -// LOG_TRACE(log, "Successfully deleted queue {}, messages contained {}", queue, num_messages); -// connection->getHandler().stopLoop(); -// }) -// .onError([&](const char * message) -// { -// LOG_ERROR(log, "Failed to delete queue {}. Error message: {}", queue, message); -// connection->getHandler().stopLoop(); -// }); -// } -// connection->getHandler().startBlockingLoop(); -// nats_channel->close(); -// -// /// Also there is no need to cleanup exchanges as they were created with AMQP::autodelete option. Once queues -// /// are removed, exchanges will also be cleaned. -//} - - void StorageNATS::pushReadBuffer(ConsumerBufferPtr buffer) { std::lock_guard lock(buffers_mutex); @@ -888,13 +472,12 @@ ConsumerBufferPtr StorageNATS::createReadBuffer() } -//ProducerBufferPtr StorageNATS::createWriteBuffer() -//{ -// return std::make_shared( -// configuration, getContext(), routing_keys, exchange_name, exchange_type, -// producer_id.fetch_add(1), persistent, shutdown_called, log, -// row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); -//} +ProducerBufferPtr StorageNATS::createWriteBuffer() +{ + return std::make_shared( + configuration, getContext(), subjects[0], shutdown_called, log, + row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); +} bool StorageNATS::checkDependencies(const StorageID & table_id) @@ -925,18 +508,6 @@ bool StorageNATS::checkDependencies(const StorageID & table_id) } -//void StorageNATS::initializeBuffers() -//{ -// assert(nats_is_ready); -// if (!initialized) -// { -// for (const auto & buffer : buffers) -// prepareChannelForBuffer(buffer); -// initialized = true; -// } -//} - - void StorageNATS::streamingToViewsFunc() { try @@ -1081,7 +652,6 @@ bool StorageNATS::streamToViews() } else { - /// Commit for (auto & source : sources) { if (source->queueEmpty()) @@ -1093,7 +663,6 @@ bool StorageNATS::streamToViews() if ((queue_empty == num_created_consumers) && (++read_attempts == MAX_FAILED_READ_ATTEMPTS)) { -// connection->heartbeat(); read_attempts = 0; LOG_TRACE(log, "Reschedule streaming. Queues are empty."); return true; @@ -1127,7 +696,7 @@ void registerStorageNATS(StorageFactory & factory) if (!nats_settings->nats_format.changed) throw Exception("You must specify `nats_format` setting", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageNATS::create(args.table_id, args.getContext(), args.columns, std::move(nats_settings), args.attach); + return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(nats_settings), args.attach); }; factory.registerStorage("NATS", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 2630df33ae4..7e9fee198a4 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -17,11 +16,17 @@ namespace DB { -class StorageNATS final: public shared_ptr_helper, public IStorage, WithContext +class StorageNATS final: public IStorage, WithContext { - friend struct shared_ptr_helper; public: + StorageNATS( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr nats_settings_, + bool is_attach_); + std::string getName() const override { return "NATS"; } bool noPushingToViews() const override { return true; } @@ -70,14 +75,6 @@ public: void incrementReader(); void decrementReader(); -protected: - StorageNATS( - const StorageID & table_id_, - ContextPtr context_, - const ColumnsDescription & columns_, - std::unique_ptr nats_settings_, - bool is_attach_); - private: ContextMutablePtr nats_context; std::unique_ptr nats_settings; diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index 175d9458423..44095200d7e 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -17,22 +17,17 @@ namespace DB { static const auto BATCH = 1000; -static const auto RETURNED_LIMIT = 50000; namespace ErrorCodes { - extern const int CANNOT_CONNECT_RABBITMQ; + extern const int CANNOT_CONNECT_NATS; extern const int LOGICAL_ERROR; } WriteBufferToNATSProducer::WriteBufferToNATSProducer( const NATSConfiguration & configuration_, ContextPtr global_context, - const Names & routing_keys_, - const String & exchange_name_, - const AMQP::ExchangeType exchange_type_, - const size_t channel_id_base_, - const bool persistent_, + const String & subject_, std::atomic & shutdown_called_, Poco::Logger * log_, std::optional delimiter, @@ -40,37 +35,20 @@ WriteBufferToNATSProducer::WriteBufferToNATSProducer( size_t chunk_size_) : WriteBuffer(nullptr, 0) , connection(configuration_, log_) - , routing_keys(routing_keys_) - , exchange_name(exchange_name_) - , exchange_type(exchange_type_) - , channel_id_base(std::to_string(channel_id_base_)) - , persistent(persistent_) + , subject(subject_) , shutdown_called(shutdown_called_) , payloads(BATCH) - , returned(RETURNED_LIMIT) , log(log_) , delim(delimiter) , max_rows(rows_per_message) , chunk_size(chunk_size_) { - if (connection.connect()) - setupChannel(); - else - throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to NATS {}", connection.connectionInfoForLog()); + if (!connection.connect()) + throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "Cannot connect to NATS {}", connection.connectionInfoForLog()); writing_task = global_context->getSchedulePool().createTask("NATSWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); - if (exchange_type == AMQP::ExchangeType::headers) - { - for (const auto & header : routing_keys) - { - std::vector matching; - boost::split(matching, header, [](char c){ return c == '='; }); - key_arguments[matching[0]] = matching[1]; - } - } - reinitializeChunks(); } @@ -104,152 +82,42 @@ void WriteBufferToNATSProducer::countRow() reinitializeChunks(); ++payload_counter; - if (!payloads.push(std::make_pair(payload_counter, payload))) + if (!payloads.push(payload)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); + LOG_DEBUG(log, "Pushed payload to queue {} {}", payload, payloads.size()); } } - -void WriteBufferToNATSProducer::setupChannel() +void WriteBufferToNATSProducer::publish() { -// producer_channel = connection.createChannel(); -// -// producer_channel->onError([&](const char * message) -// { -// LOG_ERROR(log, "Producer's channel {} error: {}", channel_id, message); -// -// /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) -// producer_channel->close(); -// -// /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once -// * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid -// */ -// for (const auto & record : delivery_record) -// if (!returned.push(record.second)) -// throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); -// -// LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", -// channel_id, delivery_record.size(), payloads.size()); -// -// /// Delivery tags are scoped per channel. -// delivery_record.clear(); -// delivery_tag = 0; -// producer_ready = false; -// }); -// -// producer_channel->onReady([&]() -// { -// channel_id = channel_id_base + "_" + std::to_string(channel_id_counter++); -// LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); -// -// /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, -// * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times -// * slower than the second, so default is second and the first is turned on in table setting. -// * -// * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html -// */ -// producer_channel->confirmSelect() -// .onAck([&](uint64_t acked_delivery_tag, bool multiple) -// { -// removeRecord(acked_delivery_tag, multiple, false); -// }) -// .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) -// { -// removeRecord(nacked_delivery_tag, multiple, true); -// }); -// producer_ready = true; -// }); -} + String payload; - -void WriteBufferToNATSProducer::removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish) -{ - auto record_iter = delivery_record.find(received_delivery_tag); - assert(record_iter != delivery_record.end()); - - if (multiple) + natsStatus status{NATS_OK}; + while (!payloads.empty()) { - /// If multiple is true, then all delivery tags up to and including current are confirmed (with ack or nack). - ++record_iter; - - if (republish) - for (auto record = delivery_record.begin(); record != record_iter; ++record) - if (!returned.push(record->second)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); - - /// Delete the records even in case when republished because new delivery tags will be assigned by the server. - delivery_record.erase(delivery_record.begin(), record_iter); - } - else - { - if (republish) - if (!returned.push(record_iter->second)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); - - delivery_record.erase(record_iter); - } -} - - -void WriteBufferToNATSProducer::publish(ConcurrentBoundedQueue> & messages, bool republishing) -{ - std::pair payload; - - /* It is important to make sure that delivery_record.size() is never bigger than returned.size(), i.e. number if unacknowledged - * messages cannot exceed returned.size(), because they all might end up there - */ - while (!messages.empty() && producer_channel->usable() && delivery_record.size() < RETURNED_LIMIT) - { - bool pop_result = messages.pop(payload); + bool pop_result = payloads.pop(payload); if (!pop_result) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not pop payload"); - AMQP::Envelope envelope(payload.second.data(), payload.second.size()); - - /// if headers exchange is used, routing keys are added here via headers, if not - it is just empty - AMQP::Table message_settings = key_arguments; - - /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the - * server, then it means that publisher will never know whether those messages were delivered or not, and therefore those records - * that received no ack/nack before connection loss will be republished (see onError() callback), so there might be duplicates. To - * let consumer know that received message might be a possible duplicate - a "republished" field is added to message metadata - */ - message_settings["republished"] = std::to_string(republishing); - envelope.setHeaders(message_settings); - - /* Adding here a messageID property to message metadata. Since NATS does not guarantee exactly-once delivery, then on the - * consumer side "republished" field of message metadata can be checked and, if it set to 1, consumer might also check "messageID" - * property. This way detection of duplicates is guaranteed - */ - envelope.setMessageID(std::to_string(payload.first)); - - /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse - if (persistent) - envelope.setDeliveryMode(2); - - if (exchange_type == AMQP::ExchangeType::consistent_hash) + if (status == NATS_OK) { - producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope); - } - else if (exchange_type == AMQP::ExchangeType::headers) - { - producer_channel->publish(exchange_name, "", envelope); + status = natsConnection_PublishString(connection.getConnection(), subject.c_str(), payload.c_str()); } else { - producer_channel->publish(exchange_name, routing_keys[0], envelope); + LOG_DEBUG(log, "Something went wrong during publishing to NATS subject {}.", subject); } - - /// This is needed for "publisher confirms", which guarantees at-least-once delivery - ++delivery_tag; - delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); - - /// Need to break at some point to let event loop run, because no publishing actually happens before looping - if (delivery_tag % BATCH == 0) - break; } + if (status == NATS_OK) + { + status = natsConnection_Flush(connection.getConnection()); + } + else + { + LOG_DEBUG(log, "Something went wrong during publishing to NATS subject {}.", subject); + } iterateEventLoop(); } @@ -258,31 +126,15 @@ void WriteBufferToNATSProducer::writingFunc() { while ((!payloads.empty() || wait_all) && !shutdown_called.load()) { - /// If onReady callback is not received, producer->usable() will anyway return true, - /// but must publish only after onReady callback. - if (producer_ready) - { - /* Publish main paylods only when there are no returned messages. This way it is ensured that returned messages are republished - * as fast as possible and no new publishes are made before returned messages are handled - */ - if (!returned.empty() && producer_channel->usable()) - publish(returned, true); - else if (!payloads.empty() && producer_channel->usable()) - publish(payloads, false); - } + publish(); + + if (wait_num.load() && payloads.empty()) + wait_all = false; iterateEventLoop(); - - if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) - wait_all = false; - else if (!producer_channel->usable()) - { - if (connection.reconnect()) - setupChannel(); - } } - LOG_DEBUG(log, "Producer on channel {} completed", channel_id); + LOG_DEBUG(log, "Producer on subject {} completed", subject); } diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h index f2e9e04e227..779e35519c7 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.h +++ b/src/Storages/NATS/WriteBufferToNATSProducer.h @@ -20,11 +20,7 @@ public: WriteBufferToNATSProducer( const NATSConfiguration & configuration_, ContextPtr global_context, - const Names & routing_keys_, - const String & exchange_name_, - const AMQP::ExchangeType exchange_type_, - const size_t channel_id_base_, - const bool persistent_, + const String & subject_, std::atomic & shutdown_called_, Poco::Logger * log_, std::optional delimiter, @@ -45,52 +41,22 @@ private: void iterateEventLoop(); void writingFunc(); - void setupChannel(); - void removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish); - void publish(ConcurrentBoundedQueue> & message, bool republishing); + void publish(); NATSConnectionManager connection; + const String subject; - const Names routing_keys; - const String exchange_name; - AMQP::ExchangeType exchange_type; - const String channel_id_base; /// Serial number of current producer buffer - const bool persistent; - - /* false: when shutdown is called; needed because table might be dropped before all acks are received + /* false: when shutdown is called * true: in all other cases */ std::atomic & shutdown_called; - AMQP::Table key_arguments; BackgroundSchedulePool::TaskHolder writing_task; - std::unique_ptr producer_channel; - bool producer_ready = false; - - /// Channel errors lead to channel closure, need to count number of recreated channels to update channel id -// UInt64 channel_id_counter = 0; - - /// channel id which contains id of current producer buffer and serial number of recreated channel in this buffer - String channel_id; - /* payloads.queue: * - payloads are pushed to queue in countRow and popped by another thread in writingFunc, each payload gets into queue only once - * returned.queue: - * - payloads are pushed to queue: - * 1) inside channel->onError() callback if channel becomes unusable and the record of pending acknowledgements from server - * is non-empty. - * 2) inside removeRecord() if received nack() - negative acknowledgement from the server that message failed to be written - * to disk or it was unable to reach the queue. - * - payloads are popped from the queue once republished */ - ConcurrentBoundedQueue> payloads, returned; - - /* Counter of current delivery on a current channel. Delivery tags are scoped per channel. The server attaches a delivery tag for each - * published message - a serial number of delivery on current channel. Delivery tag is a way of server to notify publisher if it was - * able or unable to process delivery, i.e. it sends back a response with a corresponding delivery tag. - */ - UInt64 delivery_tag = 0; + ConcurrentBoundedQueue payloads; /* false: message delivery successfully ended: publisher received confirm from server that all published * 1) persistent messages were written to disk @@ -103,13 +69,8 @@ private: * true: means payloads.queue will not grow anymore */ std::atomic wait_num = 0; - - /// Needed to fill messageID property UInt64 payload_counter = 0; - /// Record of pending acknowledgements from the server; its size never exceeds size of returned.queue - std::map> delivery_record; - Poco::Logger * log; const std::optional delim; const size_t max_rows; From 944a729fc531f5bce1569c414fcc5546914d26bc Mon Sep 17 00:00:00 2001 From: tchepavel Date: Sun, 8 May 2022 15:12:15 +0300 Subject: [PATCH 005/627] Copy Rabbit to become NATS --- .gitmodules | 3 + contrib/CMakeLists.txt | 1 + contrib/nats-io | 1 + src/CMakeLists.txt | 4 + src/Storages/NATS/Buffer_fwd.h | 14 + src/Storages/NATS/NATSConnection.cpp | 121 ++ src/Storages/NATS/NATSConnection.h | 68 + src/Storages/NATS/NATSHandler.cpp | 67 + src/Storages/NATS/NATSHandler.h | 66 + src/Storages/NATS/NATSSettings.cpp | 39 + src/Storages/NATS/NATSSettings.h | 45 + src/Storages/NATS/NATSSink.cpp | 56 + src/Storages/NATS/NATSSink.h | 32 + src/Storages/NATS/NATSSource.cpp | 189 +++ src/Storages/NATS/NATSSource.h | 61 + .../NATS/ReadBufferFromNATSConsumer.cpp | 188 +++ .../NATS/ReadBufferFromNATSConsumer.h | 104 ++ src/Storages/NATS/StorageNATS.cpp | 1186 +++++++++++++++++ src/Storages/NATS/StorageNATS.h | 207 +++ src/Storages/NATS/UVLoop.h | 44 + .../NATS/WriteBufferToNATSProducer.cpp | 317 +++++ src/Storages/NATS/WriteBufferToNATSProducer.h | 121 ++ 22 files changed, 2934 insertions(+) create mode 160000 contrib/nats-io create mode 100644 src/Storages/NATS/Buffer_fwd.h create mode 100644 src/Storages/NATS/NATSConnection.cpp create mode 100644 src/Storages/NATS/NATSConnection.h create mode 100644 src/Storages/NATS/NATSHandler.cpp create mode 100644 src/Storages/NATS/NATSHandler.h create mode 100644 src/Storages/NATS/NATSSettings.cpp create mode 100644 src/Storages/NATS/NATSSettings.h create mode 100644 src/Storages/NATS/NATSSink.cpp create mode 100644 src/Storages/NATS/NATSSink.h create mode 100644 src/Storages/NATS/NATSSource.cpp create mode 100644 src/Storages/NATS/NATSSource.h create mode 100644 src/Storages/NATS/ReadBufferFromNATSConsumer.cpp create mode 100644 src/Storages/NATS/ReadBufferFromNATSConsumer.h create mode 100644 src/Storages/NATS/StorageNATS.cpp create mode 100644 src/Storages/NATS/StorageNATS.h create mode 100644 src/Storages/NATS/UVLoop.h create mode 100644 src/Storages/NATS/WriteBufferToNATSProducer.cpp create mode 100644 src/Storages/NATS/WriteBufferToNATSProducer.h diff --git a/.gitmodules b/.gitmodules index 5fd9e9721f6..412fefea899 100644 --- a/.gitmodules +++ b/.gitmodules @@ -265,3 +265,6 @@ [submodule "contrib/wyhash"] path = contrib/wyhash url = https://github.com/wangyi-fudan/wyhash.git +[submodule "contrib/nats-io"] + path = contrib/nats-io + url = https://github.com/nats-io/nats.c.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 94003f3b3ee..b6fe9da167d 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -132,6 +132,7 @@ add_contrib (krb5-cmake krb5) add_contrib (cyrus-sasl-cmake cyrus-sasl) # for krb5 add_contrib (libgsasl-cmake libgsasl) # requires krb5 add_contrib (librdkafka-cmake librdkafka) # requires: libgsasl +add_contrib (nats-io-cmake nats-io) add_contrib (libhdfs3-cmake libhdfs3) # requires: protobuf, krb5 add_contrib (hive-metastore-cmake hive-metastore) # requires: thrift/avro/arrow/libhdfs3 add_contrib (cppkafka-cmake cppkafka) diff --git a/contrib/nats-io b/contrib/nats-io new file mode 160000 index 00000000000..d1f59f7bcf8 --- /dev/null +++ b/contrib/nats-io @@ -0,0 +1 @@ +Subproject commit d1f59f7bcf8465526f7e6d9c99982cbd6b209547 diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 20db948abd0..4e364629f24 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -107,6 +107,8 @@ if (TARGET ch_contrib::rdkafka) add_headers_and_sources(dbms Storages/Kafka) endif() +add_headers_and_sources(dbms Storages/NATS) + if (TARGET ch_contrib::amqp_cpp) add_headers_and_sources(dbms Storages/RabbitMQ) endif() @@ -380,6 +382,8 @@ if (TARGET ch_contrib::rdkafka) dbms_target_link_libraries(PRIVATE ch_contrib::rdkafka ch_contrib::cppkafka) endif() +dbms_target_link_libraries(PRIVATE ch_contrib::nats-io) + if (TARGET ch_contrib::sasl2) dbms_target_link_libraries(PRIVATE ch_contrib::sasl2) endif() diff --git a/src/Storages/NATS/Buffer_fwd.h b/src/Storages/NATS/Buffer_fwd.h new file mode 100644 index 00000000000..caf7b8cfdbe --- /dev/null +++ b/src/Storages/NATS/Buffer_fwd.h @@ -0,0 +1,14 @@ +#pragma once + +#include + +namespace DB +{ + +class ReadBufferFromNATSConsumer; +using ConsumerBufferPtr = std::shared_ptr; + +class WriteBufferToNATSProducer; +using ProducerBufferPtr = std::shared_ptr; + +} diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp new file mode 100644 index 00000000000..7b0abba7535 --- /dev/null +++ b/src/Storages/NATS/NATSConnection.cpp @@ -0,0 +1,121 @@ +#include "NATSConnection.h" + +#include +#include + + +namespace DB +{ + +static const auto CONNECT_SLEEP = 200; +static const auto RETRIES_MAX = 20; + + +NATSConnection::NATSConnection(const NATSConfiguration & configuration_, Poco::Logger * log_) + : configuration(configuration_) + , log(log_) + , event_handler(loop.getLoop(), log) +{ +} + +String NATSConnection::connectionInfoForLog() const +{ + return configuration.host + ':' + toString(configuration.port); +} + +bool NATSConnection::isConnected() +{ + std::lock_guard lock(mutex); + return isConnectedImpl(); +} + +bool NATSConnection::connect() +{ + std::lock_guard lock(mutex); + connectImpl(); + return isConnectedImpl(); +} + +bool NATSConnection::reconnect() +{ + std::lock_guard lock(mutex); + if (isConnectedImpl()) + return true; + + disconnectImpl(); + + /// This will force immediate closure if not yet closed + if (!connection->closed()) + connection->close(true); + + LOG_DEBUG(log, "Trying to restore connection to {}", connectionInfoForLog()); + connectImpl(); + + return isConnectedImpl(); +} + +SubscriptionPtr NATSConnection::createSubscription(const std::string& subject) +{ + std::lock_guard lock(mutex); + natsSubscription * ns; + natsConnection_SubscribeSync(&ns, connection, subject.c_str()); + return SubscriptionPtr(ns, &natsSubscription_Destroy); +} + +void NATSConnection::disconnect() +{ + std::lock_guard lock(mutex); + disconnectImpl(); +} + +bool NATSConnection::closed() +{ + std::lock_guard lock(mutex); + return natsConnection_IsClosed(connection); +} + +bool NATSConnection::isConnectedImpl() const +{ + return event_handler.connectionRunning() && !natsConnection_IsClosed(connection); +} + +void NATSConnection::connectImpl() +{ + if (configuration.connection_string.empty()) + { + LOG_DEBUG(log, "Connecting to: {}:{} (user: {})", configuration.host, configuration.port, configuration.username); + AMQP::Login login(configuration.username, configuration.password); + AMQP::Address address(configuration.host, configuration.port, login, configuration.vhost, configuration.secure); + connection = std::make_unique(&event_handler, address); + } + else + { + AMQP::Address address(configuration.connection_string); + connection = std::make_unique(&event_handler, address); + } + + auto cnt_retries = 0; + while (true) + { + event_handler.iterateLoop(); + + if (connection->ready() || cnt_retries++ == RETRIES_MAX) + break; + + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + } +} + +void NATSConnection::disconnectImpl() +{ + natsConnection_Close(connection); + + /** Connection is not closed immediately (firstly, all pending operations are completed, and then + * an AMQP closing-handshake is performed). But cannot open a new connection until previous one is properly closed + */ + size_t cnt_retries = 0; + while (!closed() && cnt_retries++ != RETRIES_MAX) + event_handler.iterateLoop(); +} + +} diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h new file mode 100644 index 00000000000..2c999e873aa --- /dev/null +++ b/src/Storages/NATS/NATSConnection.h @@ -0,0 +1,68 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +struct NATSConfiguration +{ + String url; + String host; + UInt16 port; + String username; + String password; + String vhost; + + bool secure; + String connection_string; +}; + +using SubscriptionPtr = std::unique_ptr; + +class NATSConnection +{ + +public: + NATSConnection(const NATSConfiguration & configuration_, Poco::Logger * log_); + + bool isConnected(); + + bool connect(); + + bool reconnect(); + + void disconnect(); + + bool closed(); + + SubscriptionPtr createSubscription(const std::string& subject); + + /// NATSHandler is thread safe. Any public methods can be called concurrently. + NATSHandler & getHandler() { return event_handler; } + + String connectionInfoForLog() const; + +private: + bool isConnectedImpl() const; + + void connectImpl(); + + void disconnectImpl(); + + NATSConfiguration configuration; + Poco::Logger * log; + + UVLoop loop; + NATSHandler event_handler; + + natsConnection * connection; + natsStatus status; + std::mutex mutex; +}; + +using NATSConnectionPtr = std::unique_ptr; + +} diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp new file mode 100644 index 00000000000..7fb8ff38c47 --- /dev/null +++ b/src/Storages/NATS/NATSHandler.cpp @@ -0,0 +1,67 @@ +#include +#include +#include + +namespace DB +{ + +/* The object of this class is shared between concurrent consumers (who share the same connection == share the same + * event loop and handler). + */ +NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : + loop(loop_), + log(log_), + connection_running(false), + loop_running(false), + loop_state(Loop::STOP) +{ + natsLibuv_Init(); + natsLibuv_SetThreadLocalLoop(loop); + natsOptions_Create(&opts); + status = natsOptions_SetEventLoop(opts, static_cast(loop), + natsLibuv_Attach, + natsLibuv_Read, + natsLibuv_Write, + natsLibuv_Detach); +} + +void NATSHandler::startLoop() +{ + std::lock_guard lock(startup_mutex); + + LOG_DEBUG(log, "Background loop started"); + loop_running.store(true); + + while (loop_state.load() == Loop::RUN) + uv_run(loop, UV_RUN_NOWAIT); + + LOG_DEBUG(log, "Background loop ended"); + loop_running.store(false); +} + +void NATSHandler::iterateLoop() +{ + std::unique_lock lock(startup_mutex, std::defer_lock); + if (lock.try_lock()) + uv_run(loop, UV_RUN_NOWAIT); +} + +/// Do not need synchronization as in iterateLoop(), because this method is used only for +/// initial NATS setup - at this point there is no background loop thread. +void NATSHandler::startBlockingLoop() +{ + LOG_DEBUG(log, "Started blocking loop."); + uv_run(loop, UV_RUN_DEFAULT); +} + +void NATSHandler::stopLoop() +{ + LOG_DEBUG(log, "Implicit loop stop."); + uv_stop(loop); +} + +NATSHandler::~NATSHandler() { + natsOptions_Destroy(opts); +} + +} diff --git a/src/Storages/NATS/NATSHandler.h b/src/Storages/NATS/NATSHandler.h new file mode 100644 index 00000000000..12ea3454b9d --- /dev/null +++ b/src/Storages/NATS/NATSHandler.h @@ -0,0 +1,66 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace Loop +{ + static const UInt8 RUN = 1; + static const UInt8 STOP = 2; +} + +class NATSHandler +{ + +public: + NATSHandler(uv_loop_t * loop_, Poco::Logger * log_); + + ~NATSHandler(); + + /// Loop for background thread worker. + void startLoop(); + + /// Loop to wait for small tasks in a non-blocking mode. + /// Adds synchronization with main background loop. + void iterateLoop(); + + /// Loop to wait for small tasks in a blocking mode. + /// No synchronization is done with the main loop thread. + void startBlockingLoop(); + + void stopLoop(); + + bool connectionRunning() const { return connection_running.load(); } + bool loopRunning() const { return loop_running.load(); } + + void updateLoopState(UInt8 state) { loop_state.store(state); } + UInt8 getLoopState() { return loop_state.load(); } + + natsStatus getStatus() { return status; } + natsOptions * getOptions() { return opts; } + +private: + uv_loop_t * loop; + natsOptions * opts = nullptr; + natsStatus status = NATS_OK; + Poco::Logger * log; + + std::atomic connection_running, loop_running; + std::atomic loop_state; + std::mutex startup_mutex; +}; + +using NATSHandlerPtr = std::shared_ptr; + +} diff --git a/src/Storages/NATS/NATSSettings.cpp b/src/Storages/NATS/NATSSettings.cpp new file mode 100644 index 00000000000..11659b105b2 --- /dev/null +++ b/src/Storages/NATS/NATSSettings.cpp @@ -0,0 +1,39 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) + +void NATSSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} +} diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h new file mode 100644 index 00000000000..5f4051e06d1 --- /dev/null +++ b/src/Storages/NATS/NATSSettings.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include + +namespace DB +{ + class ASTStorage; + + +#define RABBITMQ_RELATED_SETTINGS(M) \ + M(String, nats_host_port, "", "A host-port to connect to NATS server.", 0) \ + M(String, nats_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \ + M(String, nats_format, "", "The message format.", 0) \ + M(String, nats_exchange_type, "default", "The exchange type.", 0) \ + M(String, nats_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \ + M(Char, nats_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ + M(String, nats_schema, "", "Schema identifier (used by schema-based formats) for NATS engine", 0) \ + M(UInt64, nats_num_consumers, 1, "The number of consumer channels per table.", 0) \ + M(UInt64, nats_num_queues, 1, "The number of queues per consumer.", 0) \ + M(String, nats_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ + M(Bool, nats_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \ + M(Bool, nats_secure, false, "Use SSL connection", 0) \ + M(String, nats_address, "", "Address for connection", 0) \ + M(UInt64, nats_skip_broken_messages, 0, "Skip at least this number of broken messages from NATS per block", 0) \ + M(UInt64, nats_max_block_size, 0, "Number of row collected before flushing data from NATS.", 0) \ + M(Milliseconds, nats_flush_interval_ms, 0, "Timeout for flushing data from NATS.", 0) \ + M(String, nats_vhost, "/", "NATS vhost.", 0) \ + M(String, nats_queue_settings_list, "", "A list of nats queue settings", 0) \ + M(Bool, nats_queue_consume, false, "Use user-defined queues and do not make any NATS setup: declaring exchanges, queues, bindings", 0) \ + M(String, nats_username, "", "NATS username", 0) \ + M(String, nats_password, "", "NATS password", 0) \ + M(Bool, nats_commit_on_select, false, "Commit messages when select query is made", 0) \ + +#define LIST_OF_RABBITMQ_SETTINGS(M) \ + RABBITMQ_RELATED_SETTINGS(M) \ + FORMAT_FACTORY_SETTINGS(M) + +DECLARE_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) + +struct NATSSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; +} diff --git a/src/Storages/NATS/NATSSink.cpp b/src/Storages/NATS/NATSSink.cpp new file mode 100644 index 00000000000..db2620d9ed2 --- /dev/null +++ b/src/Storages/NATS/NATSSink.cpp @@ -0,0 +1,56 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NATSSink::NATSSink( + StorageNATS & storage_, + const StorageMetadataPtr & metadata_snapshot_, + ContextPtr context_) + : SinkToStorage(metadata_snapshot_->getSampleBlockNonMaterialized()) + , storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , context(context_) +{ + storage.unbindExchange(); +} + + +void NATSSink::onStart() +{ + buffer = storage.createWriteBuffer(); + buffer->activateWriting(); + + auto format_settings = getFormatSettings(context); + format_settings.protobuf.allow_multiple_rows_without_delimiter = true; + + format = FormatFactory::instance().getOutputFormat(storage.getFormatName(), *buffer, getHeader(), context, + [this](const Columns & /* columns */, size_t /* rows */) + { + buffer->countRow(); + }, + format_settings); +} + + +void NATSSink::consume(Chunk chunk) +{ + format->write(getHeader().cloneWithColumns(chunk.detachColumns())); +} + + +void NATSSink::onFinish() +{ + format->finalize(); + + if (buffer) + buffer->updateMaxWait(); +} + +} diff --git a/src/Storages/NATS/NATSSink.h b/src/Storages/NATS/NATSSink.h new file mode 100644 index 00000000000..2f887dd3a4f --- /dev/null +++ b/src/Storages/NATS/NATSSink.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class IOutputFormat; +using IOutputFormatPtr = std::shared_ptr; + +class NATSSink : public SinkToStorage +{ + +public: + explicit NATSSink(StorageNATS & storage_, const StorageMetadataPtr & metadata_snapshot_, ContextPtr context_); + + void onStart() override; + void consume(Chunk chunk) override; + void onFinish() override; + + String getName() const override { return "NATSSink"; } + +private: + StorageNATS & storage; + StorageMetadataPtr metadata_snapshot; + ContextPtr context; + ProducerBufferPtr buffer; + IOutputFormatPtr format; +}; +} diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp new file mode 100644 index 00000000000..046b8792ced --- /dev/null +++ b/src/Storages/NATS/NATSSource.cpp @@ -0,0 +1,189 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +static std::pair getHeaders(const StorageSnapshotPtr & storage_snapshot) +{ + auto non_virtual_header = storage_snapshot->metadata->getSampleBlockNonMaterialized(); + auto virtual_header = storage_snapshot->getSampleBlockForColumns( + {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered", "_message_id", "_timestamp"}); + + return {non_virtual_header, virtual_header}; +} + +static Block getSampleBlock(const Block & non_virtual_header, const Block & virtual_header) +{ + auto header = non_virtual_header; + for (const auto & column : virtual_header) + header.insert(column); + + return header; +} + +NATSSource::NATSSource( + StorageNATS & storage_, + const StorageSnapshotPtr & storage_snapshot_, + ContextPtr context_, + const Names & columns, + size_t max_block_size_, + bool ack_in_suffix_) + : NATSSource( + storage_, + storage_snapshot_, + getHeaders(storage_snapshot_), + context_, + columns, + max_block_size_, + ack_in_suffix_) +{ +} + +NATSSource::NATSSource( + StorageNATS & storage_, + const StorageSnapshotPtr & storage_snapshot_, + std::pair headers, + ContextPtr context_, + const Names & columns, + size_t max_block_size_, + bool ack_in_suffix_) + : SourceWithProgress(getSampleBlock(headers.first, headers.second)) + , storage(storage_) + , storage_snapshot(storage_snapshot_) + , context(context_) + , column_names(columns) + , max_block_size(max_block_size_) + , ack_in_suffix(ack_in_suffix_) + , non_virtual_header(std::move(headers.first)) + , virtual_header(std::move(headers.second)) +{ + storage.incrementReader(); +} + + +NATSSource::~NATSSource() +{ + storage.decrementReader(); + + if (!buffer) + return; + + storage.pushReadBuffer(buffer); +} + + +bool NATSSource::needChannelUpdate() +{ + if (!buffer) + return false; + + return buffer->needChannelUpdate(); +} + + +void NATSSource::updateChannel() +{ + if (!buffer) + return; + + buffer->updateAckTracker(); + + if (storage.updateChannel(buffer->getChannel())) + buffer->setupChannel(); +} + +Chunk NATSSource::generate() +{ + auto chunk = generateImpl(); + if (!chunk && ack_in_suffix) + sendAck(); + + return chunk; +} + +Chunk NATSSource::generateImpl() +{ + if (!buffer) + { + auto timeout = std::chrono::milliseconds(context->getSettingsRef().rabbitmq_max_wait_ms.totalMilliseconds()); + buffer = storage.popReadBuffer(timeout); + } + + if (!buffer || is_finished) + return {}; + + is_finished = true; + + MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + auto input_format = FormatFactory::instance().getInputFormat( + storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + + StreamingFormatExecutor executor(non_virtual_header, input_format); + + size_t total_rows = 0; + + while (true) + { + if (buffer->eof()) + break; + + auto new_rows = executor.execute(); + + if (new_rows) + { + auto exchange_name = storage.getExchange(); + auto channel_id = buffer->getChannelID(); + auto delivery_tag = buffer->getDeliveryTag(); + auto redelivered = buffer->getRedelivered(); + auto message_id = buffer->getMessageID(); + auto timestamp = buffer->getTimestamp(); + + buffer->updateAckTracker({delivery_tag, channel_id}); + + for (size_t i = 0; i < new_rows; ++i) + { + virtual_columns[0]->insert(exchange_name); + virtual_columns[1]->insert(channel_id); + virtual_columns[2]->insert(delivery_tag); + virtual_columns[3]->insert(redelivered); + virtual_columns[4]->insert(message_id); + virtual_columns[5]->insert(timestamp); + } + + total_rows = total_rows + new_rows; + } + + buffer->allowNext(); + + if (total_rows >= max_block_size || buffer->queueEmpty() || buffer->isConsumerStopped() || !checkTimeLimit()) + break; + } + + if (total_rows == 0) + return {}; + + auto result_columns = executor.getResultColumns(); + for (auto & column : virtual_columns) + result_columns.push_back(std::move(column)); + + return Chunk(std::move(result_columns), total_rows); +} + + +bool NATSSource::sendAck() +{ + if (!buffer) + return false; + + if (!buffer->ackMessages()) + return false; + + return true; +} + +} diff --git a/src/Storages/NATS/NATSSource.h b/src/Storages/NATS/NATSSource.h new file mode 100644 index 00000000000..ce48e5cf382 --- /dev/null +++ b/src/Storages/NATS/NATSSource.h @@ -0,0 +1,61 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class NATSSource : public SourceWithProgress +{ + +public: + NATSSource( + StorageNATS & storage_, + const StorageSnapshotPtr & storage_snapshot_, + ContextPtr context_, + const Names & columns, + size_t max_block_size_, + bool ack_in_suffix = false); + + ~NATSSource() override; + + String getName() const override { return storage.getName(); } + ConsumerBufferPtr getBuffer() { return buffer; } + + Chunk generate() override; + + bool queueEmpty() const { return !buffer || buffer->queueEmpty(); } + bool needChannelUpdate(); + void updateChannel(); + bool sendAck(); + +private: + StorageNATS & storage; + StorageSnapshotPtr storage_snapshot; + ContextPtr context; + Names column_names; + const size_t max_block_size; + bool ack_in_suffix; + + bool is_finished = false; + const Block non_virtual_header; + const Block virtual_header; + + ConsumerBufferPtr buffer; + + NATSSource( + StorageNATS & storage_, + const StorageSnapshotPtr & storage_snapshot_, + std::pair headers, + ContextPtr context_, + const Names & columns, + size_t max_block_size_, + bool ack_in_suffix); + + Chunk generateImpl(); +}; + +} diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp new file mode 100644 index 00000000000..ad966401cc3 --- /dev/null +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -0,0 +1,188 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Poco/Timer.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( + NATSHandler & event_handler_, + std::vector & queues_, + size_t channel_id_base_, + const String & channel_base_, + Poco::Logger * log_, + char row_delimiter_, + uint32_t queue_size_, + const std::atomic & stopped_) + : ReadBuffer(nullptr, 0) + , event_handler(event_handler_) + , queues(queues_) + , channel_base(channel_base_) + , channel_id_base(channel_id_base_) + , log(log_) + , row_delimiter(row_delimiter_) + , stopped(stopped_) + , received(queue_size_) +{ +} + + +ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() +{ + BufferBase::set(nullptr, 0, 0); +} + + +void ReadBufferFromNATSConsumer::subscribe() +{ + for (const auto & queue_name : queues) + { + consumer_channel->consume(queue_name) + .onSuccess([&](const std::string & /* consumer_tag */) + { + LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name); + + if (++subscribed == queues.size()) + wait_subscription.store(false); + }) + .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) + { + if (message.bodySize()) + { + String message_received = std::string(message.body(), message.body() + message.bodySize()); + if (row_delimiter != '\0') + message_received += row_delimiter; + + if (!received.push({message_received, message.hasMessageID() ? message.messageID() : "", + message.hasTimestamp() ? message.timestamp() : 0, + redelivered, AckTracker(delivery_tag, channel_id)})) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); + } + }) + .onError([&](const char * message) + { + /* End up here either if channel ends up in an error state (then there will be resubscription) or consume call error, which + * arises from queue settings mismatch or queue level error, which should not happen as no one else is supposed to touch them + */ + LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message); + wait_subscription.store(false); + }); + } +} + + +bool ReadBufferFromNATSConsumer::ackMessages() +{ + AckTracker record_info = last_inserted_record_info; + + /* Do not send ack to server if message's channel is not the same as current running channel because delivery tags are scoped per + * channel, so if channel fails, all previous delivery tags become invalid + */ + if (record_info.channel_id == channel_id && record_info.delivery_tag && record_info.delivery_tag > prev_tag) + { + /// Commit all received messages with delivery tags from last committed to last inserted + if (!consumer_channel->ack(record_info.delivery_tag, AMQP::multiple)) + { + LOG_ERROR(log, "Failed to commit messages with delivery tags from last committed to {} on channel {}", + record_info.delivery_tag, channel_id); + return false; + } + + prev_tag = record_info.delivery_tag; + LOG_TRACE(log, "Consumer committed messages with deliveryTags up to {} on channel {}", record_info.delivery_tag, channel_id); + } + + return true; +} + + +void ReadBufferFromNATSConsumer::updateAckTracker(AckTracker record_info) +{ + if (record_info.delivery_tag && channel_error.load()) + return; + + if (!record_info.delivery_tag) + prev_tag = 0; + + last_inserted_record_info = record_info; +} + + +void ReadBufferFromNATSConsumer::setupChannel() +{ + if (!consumer_channel) + return; + + wait_subscription.store(true); + + consumer_channel->onReady([&]() + { + /* First number indicates current consumer buffer; second number indicates serial number of created channel for current buffer, + * i.e. if channel fails - another one is created and its serial number is incremented; channel_base is to guarantee that + * channel_id is unique for each table + */ + channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base; + LOG_TRACE(log, "Channel {} is created", channel_id); + + subscribed = 0; + subscribe(); + channel_error.store(false); + }); + + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Channel {} error: {}", channel_id, message); + + channel_error.store(true); + wait_subscription.store(false); + }); +} + + +bool ReadBufferFromNATSConsumer::needChannelUpdate() +{ + if (wait_subscription) + return false; + + return channel_error || !consumer_channel || !consumer_channel->usable(); +} + + +void ReadBufferFromNATSConsumer::iterateEventLoop() +{ + event_handler.iterateLoop(); +} + + +bool ReadBufferFromNATSConsumer::nextImpl() +{ + if (stopped || !allowed) + return false; + + if (received.tryPop(current)) + { + auto * new_position = const_cast(current.message.data()); + BufferBase::set(new_position, current.message.size(), 0); + allowed = false; + + return true; + } + + return false; +} + +} diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h new file mode 100644 index 00000000000..1206ee4e62d --- /dev/null +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -0,0 +1,104 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace Poco +{ + class Logger; +} + +namespace DB +{ + +class ReadBufferFromNATSConsumer : public ReadBuffer +{ + +public: + ReadBufferFromNATSConsumer( + NATSHandler & event_handler_, + std::vector & queues_, + size_t channel_id_base_, + const String & channel_base_, + Poco::Logger * log_, + char row_delimiter_, + uint32_t queue_size_, + const std::atomic & stopped_); + + ~ReadBufferFromNATSConsumer() override; + + struct AckTracker + { + UInt64 delivery_tag; + String channel_id; + + AckTracker() = default; + AckTracker(UInt64 tag, String id) : delivery_tag(tag), channel_id(id) {} + }; + + struct MessageData + { + String message; + String message_id; + uint64_t timestamp = 0; + bool redelivered = false; + AckTracker track{}; + }; + + ChannelPtr & getChannel() { return consumer_channel; } + void setupChannel(); + bool needChannelUpdate(); + void closeChannel() + { + if (consumer_channel) + consumer_channel->close(); + } + + void updateQueues(std::vector & queues_) { queues = queues_; } + size_t queuesCount() { return queues.size(); } + + bool isConsumerStopped() { return stopped; } + bool ackMessages(); + void updateAckTracker(AckTracker record = AckTracker()); + + bool queueEmpty() { return received.empty(); } + void allowNext() { allowed = true; } // Allow to read next message. + + auto getChannelID() const { return current.track.channel_id; } + auto getDeliveryTag() const { return current.track.delivery_tag; } + auto getRedelivered() const { return current.redelivered; } + auto getMessageID() const { return current.message_id; } + auto getTimestamp() const { return current.timestamp; } + +private: + bool nextImpl() override; + + void subscribe(); + void iterateEventLoop(); + + ChannelPtr consumer_channel; + NATSHandler & event_handler; /// Used concurrently, but is thread safe. + std::vector queues; + const String channel_base; + const size_t channel_id_base; + Poco::Logger * log; + char row_delimiter; + bool allowed = true; + const std::atomic & stopped; + + String channel_id; + std::atomic channel_error = true, wait_subscription = false; + ConcurrentBoundedQueue received; + MessageData current; + size_t subscribed = 0; + + AckTracker last_inserted_record_info; + UInt64 prev_tag = 0, channel_id_counter = 0; +}; + +} diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp new file mode 100644 index 00000000000..41ab2d1fc9b --- /dev/null +++ b/src/Storages/NATS/StorageNATS.cpp @@ -0,0 +1,1186 @@ +#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 +{ + +static const uint32_t QUEUE_SIZE = 100000; +static const auto MAX_FAILED_READ_ATTEMPTS = 10; +static const auto RESCHEDULE_MS = 500; +static const auto BACKOFF_TRESHOLD = 32000; +static const auto MAX_THREAD_WORK_DURATION_MS = 60000; + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int CANNOT_CONNECT_RABBITMQ; + extern const int CANNOT_BIND_RABBITMQ_EXCHANGE; + extern const int CANNOT_DECLARE_RABBITMQ_EXCHANGE; + extern const int CANNOT_REMOVE_RABBITMQ_EXCHANGE; + extern const int CANNOT_CREATE_RABBITMQ_QUEUE_BINDING; + extern const int QUERY_NOT_ALLOWED; +} + +namespace ExchangeType +{ + /// Note that default here means default by implementation and not by nats settings + static const String DEFAULT = "default"; + static const String FANOUT = "fanout"; + static const String DIRECT = "direct"; + static const String TOPIC = "topic"; + static const String HASH = "consistent_hash"; + static const String HEADERS = "headers"; +} + + +StorageNATS::StorageNATS( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr nats_settings_, + bool is_attach_) + : IStorage(table_id_) + , WithContext(context_->getGlobalContext()) + , nats_settings(std::move(nats_settings_)) + , exchange_name(getContext()->getMacros()->expand(nats_settings->nats_exchange_name)) + , format_name(getContext()->getMacros()->expand(nats_settings->nats_format)) + , exchange_type(defineExchangeType(getContext()->getMacros()->expand(nats_settings->nats_exchange_type))) + , routing_keys(parseSettings(getContext()->getMacros()->expand(nats_settings->nats_routing_key_list))) + , row_delimiter(nats_settings->nats_row_delimiter.value) + , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) + , num_consumers(nats_settings->nats_num_consumers.value) + , num_queues(nats_settings->nats_num_queues.value) + , queue_base(getContext()->getMacros()->expand(nats_settings->nats_queue_base)) + , queue_settings_list(parseSettings(getContext()->getMacros()->expand(nats_settings->nats_queue_settings_list))) + , persistent(nats_settings->nats_persistent.value) + , use_user_setup(nats_settings->nats_queue_consume.value) + , hash_exchange(num_consumers > 1 || num_queues > 1) + , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) + , semaphore(0, num_consumers) + , unique_strbase(getRandomName()) + , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) + , milliseconds_to_wait(RESCHEDULE_MS) + , is_attach(is_attach_) +{ + auto parsed_address = parseAddress(getContext()->getMacros()->expand(nats_settings->nats_host_port), 5672); + context_->getRemoteHostFilter().checkHostAndPort(parsed_address.first, toString(parsed_address.second)); + + auto nats_username = nats_settings->nats_username.value; + auto nats_password = nats_settings->nats_password.value; + configuration = + { + .host = parsed_address.first, + .port = parsed_address.second, + .username = nats_username.empty() ? getContext()->getConfigRef().getString("nats.username") : nats_username, + .password = nats_password.empty() ? getContext()->getConfigRef().getString("nats.password") : nats_password, + .vhost = getContext()->getConfigRef().getString("nats.vhost", getContext()->getMacros()->expand(nats_settings->nats_vhost)), + .secure = nats_settings->nats_secure.value, + .connection_string = getContext()->getMacros()->expand(nats_settings->nats_address) + }; + + if (configuration.secure) + SSL_library_init(); + + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); + + nats_context = addSettings(getContext()); + nats_context->makeQueryContext(); + + if (queue_base.empty()) + { + /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to + * be table-based and not just a random string, because local exchanges should be declared the same for same tables + */ + sharding_exchange = getTableBasedName(exchange_name, table_id_); + + /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better + * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every + * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base + * for the names of later declared queues + */ + queue_base = getTableBasedName("", table_id_); + } + else + { + /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and + * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need to share + * sharding exchange and bridge exchange + */ + sharding_exchange = exchange_name + "_" + queue_base; + } + + bridge_exchange = sharding_exchange + "_bridge"; + + try + { + connection = std::make_unique(configuration, log); + if (connection->connect()) + initNATS(); + else if (!is_attach) + throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to {}", connection->connectionInfoForLog()); + } + catch (...) + { + tryLogCurrentException(log); + if (!is_attach) + throw; + } + + /// One looping task for all consumers as they share the same connection == the same handler == the same event loop + looping_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSLoopingTask", [this]{ loopingFunc(); }); + looping_task->deactivate(); + + streaming_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSStreamingTask", [this]{ streamingToViewsFunc(); }); + streaming_task->deactivate(); + + connection_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSConnectionTask", [this]{ connectionFunc(); }); + connection_task->deactivate(); +} + + +Names StorageNATS::parseSettings(String settings_list) +{ + Names result; + if (settings_list.empty()) + return result; + boost::split(result, settings_list, [](char c){ return c == ','; }); + for (String & key : result) + boost::trim(key); + + return result; +} + + +AMQP::ExchangeType StorageNATS::defineExchangeType(String exchange_type_) +{ + AMQP::ExchangeType type; + if (exchange_type_ != ExchangeType::DEFAULT) + { + if (exchange_type_ == ExchangeType::FANOUT) type = AMQP::ExchangeType::fanout; + else if (exchange_type_ == ExchangeType::DIRECT) type = AMQP::ExchangeType::direct; + else if (exchange_type_ == ExchangeType::TOPIC) type = AMQP::ExchangeType::topic; + else if (exchange_type_ == ExchangeType::HASH) type = AMQP::ExchangeType::consistent_hash; + else if (exchange_type_ == ExchangeType::HEADERS) type = AMQP::ExchangeType::headers; + else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); + } + else + { + type = AMQP::ExchangeType::fanout; + } + + return type; +} + + +String StorageNATS::getTableBasedName(String name, const StorageID & table_id) +{ + if (name.empty()) + return fmt::format("{}_{}", table_id.database_name, table_id.table_name); + else + return fmt::format("{}_{}_{}", name, table_id.database_name, table_id.table_name); +} + + +ContextMutablePtr StorageNATS::addSettings(ContextPtr local_context) const +{ + auto modified_context = Context::createCopy(local_context); + modified_context->setSetting("input_format_skip_unknown_fields", true); + modified_context->setSetting("input_format_allow_errors_ratio", 0.); + modified_context->setSetting("input_format_allow_errors_num", nats_settings->nats_skip_broken_messages.value); + + if (!schema_name.empty()) + modified_context->setSetting("format_schema", schema_name); + + for (const auto & setting : *nats_settings) + { + const auto & setting_name = setting.getName(); + + /// check for non-nats-related settings + if (!setting_name.starts_with("nats_")) + modified_context->setSetting(setting_name, setting.getValue()); + } + + return modified_context; +} + + +void StorageNATS::loopingFunc() +{ + connection->getHandler().startLoop(); +} + + +void StorageNATS::stopLoop() +{ + connection->getHandler().updateLoopState(Loop::STOP); +} + +void StorageNATS::stopLoopIfNoReaders() +{ + /// Stop the loop if no select was started. + /// There can be a case that selects are finished + /// but not all sources decremented the counter, then + /// it is ok that the loop is not stopped, because + /// there is a background task (streaming_task), which + /// also checks whether there is an idle loop. + std::lock_guard lock(loop_mutex); + if (readers_count) + return; + connection->getHandler().updateLoopState(Loop::STOP); +} + +void StorageNATS::startLoop() +{ + assert(rabbit_is_ready); + connection->getHandler().updateLoopState(Loop::RUN); + looping_task->activateAndSchedule(); +} + + +void StorageNATS::incrementReader() +{ + ++readers_count; +} + + +void StorageNATS::decrementReader() +{ + --readers_count; +} + + +void StorageNATS::connectionFunc() +{ + if (rabbit_is_ready) + return; + + if (connection->reconnect()) + initNATS(); + else + connection_task->scheduleAfter(RESCHEDULE_MS); +} + + +/* Need to deactivate this way because otherwise might get a deadlock when first deactivate streaming task in shutdown and then + * inside streaming task try to deactivate any other task + */ +void StorageNATS::deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop) +{ + if (stop_loop) + stopLoop(); + + std::unique_lock lock(task_mutex, std::defer_lock); + if (lock.try_lock()) + { + task->deactivate(); + lock.unlock(); + } + else if (wait) /// Wait only if deactivating from shutdown + { + lock.lock(); + task->deactivate(); + } +} + + +size_t StorageNATS::getMaxBlockSize() const +{ + return nats_settings->nats_max_block_size.changed + ? nats_settings->nats_max_block_size.value + : (getContext()->getSettingsRef().max_insert_block_size.value / num_consumers); +} + + +void StorageNATS::initNATS() +{ + if (shutdown_called || rabbit_is_ready) + return; + + if (use_user_setup) + { + queues.emplace_back(queue_base); + rabbit_is_ready = true; + return; + } + + try + { + auto rabbit_channel = connection->createChannel(); + + /// Main exchange -> Bridge exchange -> ( Sharding exchange ) -> Queues -> Consumers + + initExchange(*rabbit_channel); + bindExchange(*rabbit_channel); + + for (const auto i : collections::range(0, num_queues)) + bindQueue(i + 1, *rabbit_channel); + + LOG_TRACE(log, "NATS setup completed"); + rabbit_is_ready = true; + rabbit_channel->close(); + } + catch (...) + { + tryLogCurrentException(log); + if (!is_attach) + throw; + } +} + + +void StorageNATS::initExchange(AMQP::TcpChannel & rabbit_channel) +{ + /// Exchange hierarchy: + /// 1. Main exchange (defined with table settings - nats_exchange_name, nats_exchange_type). + /// 2. Bridge exchange (fanout). Used to easily disconnect main exchange and to simplify queue bindings. + /// 3. Sharding (or hash) exchange. Used in case of multiple queues. + /// 4. Consumer exchange. Just an alias for bridge_exchange or sharding exchange to know to what exchange + /// queues will be bound. + + /// All exchanges are declared with options: + /// 1. `durable` (survive NATS server restart) + /// 2. `autodelete` (auto delete in case of queue bindings are dropped). + + rabbit_channel.declareExchange(exchange_name, exchange_type, AMQP::durable) + .onError([&](const char * message) + { + /// This error can be a result of attempt to declare exchange if it was already declared but + /// 1) with different exchange type. + /// 2) with different exchange settings. + throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " + + std::string(message), ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE); + }); + + rabbit_channel.declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable | AMQP::autodelete) + .onError([&](const char * message) + { + /// This error is not supposed to happen as this exchange name is always unique to type and its settings. + throw Exception( + ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, "Unable to declare bridge exchange ({}). Reason: {}", bridge_exchange, std::string(message)); + }); + + if (!hash_exchange) + { + consumer_exchange = bridge_exchange; + return; + } + + AMQP::Table binding_arguments; + + /// Default routing key property in case of hash exchange is a routing key, which is required to be an integer. + /// Support for arbitrary exchange type (i.e. arbitrary pattern of routing keys) requires to eliminate this dependency. + /// This settings changes hash property to message_id. + binding_arguments["hash-property"] = "message_id"; + + /// Declare hash exchange for sharding. + rabbit_channel.declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable | AMQP::autodelete, binding_arguments) + .onError([&](const char * message) + { + /// This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared + /// to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter + /// is bad. + throw Exception( + ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, + "Unable to declare sharding exchange ({}). Reason: {}", sharding_exchange, std::string(message)); + }); + + rabbit_channel.bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) + .onError([&](const char * message) + { + throw Exception( + ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, + "Unable to bind bridge exchange ({}) to sharding exchange ({}). Reason: {}", + bridge_exchange, + sharding_exchange, + std::string(message)); + }); + + consumer_exchange = sharding_exchange; +} + + +void StorageNATS::bindExchange(AMQP::TcpChannel & rabbit_channel) +{ + size_t bound_keys = 0; + + if (exchange_type == AMQP::ExchangeType::headers) + { + AMQP::Table bind_headers; + for (const auto & header : routing_keys) + { + std::vector matching; + boost::split(matching, header, [](char c){ return c == '='; }); + bind_headers[matching[0]] = matching[1]; + } + + rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) + .onSuccess([&]() { connection->getHandler().stopLoop(); }) + .onError([&](const char * message) + { + throw Exception( + ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, + "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", + exchange_name, bridge_exchange, std::string(message)); + }); + } + else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) + { + rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0]) + .onSuccess([&]() { connection->getHandler().stopLoop(); }) + .onError([&](const char * message) + { + throw Exception( + ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, + "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", + exchange_name, bridge_exchange, std::string(message)); + }); + } + else + { + for (const auto & routing_key : routing_keys) + { + rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_key) + .onSuccess([&]() + { + ++bound_keys; + if (bound_keys == routing_keys.size()) + connection->getHandler().stopLoop(); + }) + .onError([&](const char * message) + { + throw Exception( + ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, + "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", + exchange_name, bridge_exchange, std::string(message)); + }); + } + } + + connection->getHandler().startBlockingLoop(); +} + + +void StorageNATS::bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel) +{ + auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) + { + queues.emplace_back(queue_name); + LOG_DEBUG(log, "Queue {} is declared", queue_name); + + if (msgcount) + LOG_INFO(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); + + /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are + * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for + * fanout exchange it can be arbitrary + */ + rabbit_channel.bindQueue(consumer_exchange, queue_name, std::to_string(queue_id)) + .onSuccess([&] { connection->getHandler().stopLoop(); }) + .onError([&](const char * message) + { + throw Exception( + ErrorCodes::CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, + "Failed to create queue binding for exchange {}. Reason: {}", exchange_name, std::string(message)); + }); + }; + + auto error_callback([&](const char * message) + { + /* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a + * given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different + * max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously + * declared queues via any of the various cli tools. + */ + throw Exception("Failed to declare queue. Probably queue settings are conflicting: max_block_size, deadletter_exchange. Attempt \ + specifying differently those settings or use a different queue_base or manually delete previously declared queues, \ + which were declared with the same names. ERROR reason: " + + std::string(message), ErrorCodes::BAD_ARGUMENTS); + }); + + AMQP::Table queue_settings; + + std::unordered_set integer_settings = {"x-max-length", "x-max-length-bytes", "x-message-ttl", "x-expires", "x-priority", "x-max-priority"}; + std::unordered_set string_settings = {"x-overflow", "x-dead-letter-exchange", "x-queue-type"}; + + /// Check user-defined settings. + if (!queue_settings_list.empty()) + { + for (const auto & setting : queue_settings_list) + { + Strings setting_values; + splitInto<'='>(setting_values, setting); + if (setting_values.size() != 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid settings string: {}", setting); + + String key = setting_values[0], value = setting_values[1]; + + if (integer_settings.contains(key)) + queue_settings[key] = parse(value); + else if (string_settings.find(key) != string_settings.end()) + queue_settings[key] = value; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported queue setting: {}", value); + } + } + + /// Impose default settings if there are no user-defined settings. + if (!queue_settings.contains("x-max-length")) + { + queue_settings["x-max-length"] = queue_size; + } + if (!queue_settings.contains("x-overflow")) + { + queue_settings["x-overflow"] = "reject-publish"; + } + + /// If queue_base - a single name, then it can be used as one specific queue, from which to read. + /// Otherwise it is used as a generator (unique for current table) of queue names, because it allows to + /// maximize performance - via setting `nats_num_queues`. + const String queue_name = !hash_exchange ? queue_base : std::to_string(queue_id) + "_" + queue_base; + + /// AMQP::autodelete setting is not allowed, because in case of server restart there will be no consumers + /// and deleting queues should not take place. + rabbit_channel.declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); + connection->getHandler().startBlockingLoop(); +} + + +bool StorageNATS::updateChannel(ChannelPtr & channel) +{ + try + { + channel = connection->createChannel(); + return true; + } + catch (...) + { + tryLogCurrentException(log); + return false; + } +} + + +void StorageNATS::prepareChannelForBuffer(ConsumerBufferPtr buffer) +{ + if (!buffer) + return; + + if (buffer->queuesCount() != queues.size()) + buffer->updateQueues(queues); + + buffer->updateAckTracker(); + + if (updateChannel(buffer->getChannel())) + buffer->setupChannel(); +} + + +void StorageNATS::unbindExchange() +{ + /* This is needed because with NATS (without special adjustments) can't, for example, properly make mv if there was insert query + * on the same table before, and in another direction it will make redundant copies, but most likely nobody will do that. + * As publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in + * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as + * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible + * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local + * queues, then messages will go both ways and in one of them they will remain not consumed. So need to disconnect local exchange + * bindings to remove redunadant message copies, but after that mv cannot work unless those bindings are recreated. Recreating them is + * not difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. + */ + if (!exchange_removed.exchange(true)) + { + try + { + streaming_task->deactivate(); + + stopLoop(); + looping_task->deactivate(); + + auto rabbit_channel = connection->createChannel(); + rabbit_channel->removeExchange(bridge_exchange) + .onSuccess([&]() + { + connection->getHandler().stopLoop(); + }) + .onError([&](const char * message) + { + throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_REMOVE_RABBITMQ_EXCHANGE); + }); + + connection->getHandler().startBlockingLoop(); + rabbit_channel->close(); + } + catch (...) + { + exchange_removed = false; + throw; + } + } +} + + +Pipe StorageNATS::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /* query_info */, + ContextPtr local_context, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + unsigned /* num_streams */) +{ + if (!rabbit_is_ready) + throw Exception("NATS setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_RABBITMQ); + + if (num_created_consumers == 0) + return {}; + + if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + + if (mv_attached) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageNATS with attached materialized views"); + + std::lock_guard lock(loop_mutex); + + auto sample_block = storage_snapshot->getSampleBlockForColumns(column_names); + auto modified_context = addSettings(local_context); + + if (!connection->isConnected()) + { + if (connection->getHandler().loopRunning()) + deactivateTask(looping_task, false, true); + if (!connection->reconnect()) + throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "No connection to {}", connection->connectionInfoForLog()); + } + + initializeBuffers(); + + Pipes pipes; + pipes.reserve(num_created_consumers); + + for (size_t i = 0; i < num_created_consumers; ++i) + { + auto rabbit_source = std::make_shared( + *this, storage_snapshot, modified_context, column_names, 1, nats_settings->nats_commit_on_select); + + auto converting_dag = ActionsDAG::makeConvertingActions( + rabbit_source->getPort().getHeader().getColumnsWithTypeAndName(), + sample_block.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting = std::make_shared(std::move(converting_dag)); + auto converting_transform = std::make_shared(rabbit_source->getPort().getHeader(), std::move(converting)); + + pipes.emplace_back(std::move(rabbit_source)); + pipes.back().addTransform(std::move(converting_transform)); + } + + if (!connection->getHandler().loopRunning() && connection->isConnected()) + startLoop(); + + LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); + auto united_pipe = Pipe::unitePipes(std::move(pipes)); + united_pipe.addInterpreterContext(modified_context); + return united_pipe; +} + + +SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +{ + return std::make_shared(*this, metadata_snapshot, local_context); +} + + +void StorageNATS::startup() +{ + if (!rabbit_is_ready) + { + if (connection->isConnected()) + { + try + { + initNATS(); + } + catch (...) + { + if (!is_attach) + throw; + tryLogCurrentException(log); + } + } + else + { + connection_task->activateAndSchedule(); + } + } + + for (size_t i = 0; i < num_consumers; ++i) + { + try + { + auto buffer = createReadBuffer(); + pushReadBuffer(std::move(buffer)); + ++num_created_consumers; + } + catch (...) + { + if (!is_attach) + throw; + tryLogCurrentException(log); + } + } + + streaming_task->activateAndSchedule(); +} + + +void StorageNATS::shutdown() +{ + shutdown_called = true; + + /// In case it has not yet been able to setup connection; + deactivateTask(connection_task, true, false); + + /// The order of deactivating tasks is important: wait for streamingToViews() func to finish and + /// then wait for background event loop to finish. + deactivateTask(streaming_task, true, false); + deactivateTask(looping_task, true, true); + + /// Just a paranoid try catch, it is not actually needed. + try + { + if (drop_table) + { + for (auto & buffer : buffers) + buffer->closeChannel(); + + cleanupNATS(); + } + + /// It is important to close connection here - before removing consumer buffers, because + /// it will finish and clean callbacks, which might use those buffers data. + connection->disconnect(); + + for (size_t i = 0; i < num_created_consumers; ++i) + popReadBuffer(); + } + catch (...) + { + tryLogCurrentException(log); + } +} + + +/// The only thing publishers are supposed to be aware of is _exchanges_ and queues are a responsibility of a consumer. +/// Therefore, if a table is dropped, a clean up is needed. +void StorageNATS::cleanupNATS() const +{ + if (use_user_setup) + return; + + connection->heartbeat(); + if (!connection->isConnected()) + { + String queue_names; + for (const auto & queue : queues) + { + if (!queue_names.empty()) + queue_names += ", "; + queue_names += queue; + } + LOG_WARNING(log, + "NATS clean up not done, because there is no connection in table's shutdown." + "There are {} queues ({}), which might need to be deleted manually. Exchanges will be auto-deleted", + queues.size(), queue_names); + return; + } + + auto rabbit_channel = connection->createChannel(); + for (const auto & queue : queues) + { + /// AMQP::ifunused is needed, because it is possible to share queues between multiple tables and dropping + /// on of them should not affect others. + /// AMQP::ifempty is not used on purpose. + + rabbit_channel->removeQueue(queue, AMQP::ifunused) + .onSuccess([&](uint32_t num_messages) + { + LOG_TRACE(log, "Successfully deleted queue {}, messages contained {}", queue, num_messages); + connection->getHandler().stopLoop(); + }) + .onError([&](const char * message) + { + LOG_ERROR(log, "Failed to delete queue {}. Error message: {}", queue, message); + connection->getHandler().stopLoop(); + }); + } + connection->getHandler().startBlockingLoop(); + rabbit_channel->close(); + + /// Also there is no need to cleanup exchanges as they were created with AMQP::autodelete option. Once queues + /// are removed, exchanges will also be cleaned. +} + + +void StorageNATS::pushReadBuffer(ConsumerBufferPtr buffer) +{ + std::lock_guard lock(buffers_mutex); + buffers.push_back(buffer); + semaphore.set(); +} + + +ConsumerBufferPtr StorageNATS::popReadBuffer() +{ + return popReadBuffer(std::chrono::milliseconds::zero()); +} + + +ConsumerBufferPtr StorageNATS::popReadBuffer(std::chrono::milliseconds timeout) +{ + // Wait for the first free buffer + if (timeout == std::chrono::milliseconds::zero()) + semaphore.wait(); + else + { + if (!semaphore.tryWait(timeout.count())) + return nullptr; + } + + // Take the first available buffer from the list + std::lock_guard lock(buffers_mutex); + auto buffer = buffers.back(); + buffers.pop_back(); + + return buffer; +} + + +ConsumerBufferPtr StorageNATS::createReadBuffer() +{ + return std::make_shared( + connection->getHandler(), queues, ++consumer_id, + unique_strbase, log, row_delimiter, queue_size, shutdown_called); +} + + +ProducerBufferPtr StorageNATS::createWriteBuffer() +{ + return std::make_shared( + configuration, getContext(), routing_keys, exchange_name, exchange_type, + producer_id.fetch_add(1), persistent, shutdown_called, log, + row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); +} + + +bool StorageNATS::checkDependencies(const StorageID & table_id) +{ + // Check if all dependencies are attached + auto dependencies = DatabaseCatalog::instance().getDependencies(table_id); + if (dependencies.empty()) + return true; + + // Check the dependencies are ready? + for (const auto & db_tab : dependencies) + { + auto table = DatabaseCatalog::instance().tryGetTable(db_tab, getContext()); + if (!table) + return false; + + // If it materialized view, check it's target table + auto * materialized_view = dynamic_cast(table.get()); + if (materialized_view && !materialized_view->tryGetTargetTable()) + return false; + + // Check all its dependencies + if (!checkDependencies(db_tab)) + return false; + } + + return true; +} + + +void StorageNATS::initializeBuffers() +{ + assert(rabbit_is_ready); + if (!initialized) + { + for (const auto & buffer : buffers) + prepareChannelForBuffer(buffer); + initialized = true; + } +} + + +void StorageNATS::streamingToViewsFunc() +{ + if (rabbit_is_ready) + { + try + { + auto table_id = getStorageID(); + + // Check if at least one direct dependency is attached + size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + bool rabbit_connected = connection->isConnected() || connection->reconnect(); + + if (dependencies_count && rabbit_connected) + { + initializeBuffers(); + auto start_time = std::chrono::steady_clock::now(); + + mv_attached.store(true); + + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!shutdown_called && num_created_consumers > 0) + { + if (!checkDependencies(table_id)) + break; + + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + + if (streamToViews()) + { + /// Reschedule with backoff. + if (milliseconds_to_wait < BACKOFF_TRESHOLD) + milliseconds_to_wait *= 2; + stopLoopIfNoReaders(); + break; + } + else + { + milliseconds_to_wait = RESCHEDULE_MS; + } + + auto end_time = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + if (duration.count() > MAX_THREAD_WORK_DURATION_MS) + { + stopLoopIfNoReaders(); + LOG_TRACE(log, "Reschedule streaming. Thread work duration limit exceeded."); + break; + } + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + mv_attached.store(false); + + /// If there is no running select, stop the loop which was + /// activated by previous select. + if (connection->getHandler().loopRunning()) + stopLoopIfNoReaders(); + + if (!shutdown_called) + streaming_task->scheduleAfter(milliseconds_to_wait); +} + + +bool StorageNATS::streamToViews() +{ + auto table_id = getStorageID(); + auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); + if (!table) + throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); + + // Create an INSERT query for streaming data + auto insert = std::make_shared(); + insert->table_id = table_id; + + // Only insert into dependent views and expect that input blocks contain virtual columns + InterpreterInsertQuery interpreter(insert, nats_context, false, true, true); + auto block_io = interpreter.execute(); + + auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); + auto column_names = block_io.pipeline.getHeader().getNames(); + auto sample_block = storage_snapshot->getSampleBlockForColumns(column_names); + + auto block_size = getMaxBlockSize(); + + // Create a stream for each consumer and join them in a union stream + std::vector> sources; + Pipes pipes; + sources.reserve(num_created_consumers); + pipes.reserve(num_created_consumers); + + for (size_t i = 0; i < num_created_consumers; ++i) + { + auto source = std::make_shared( + *this, storage_snapshot, nats_context, column_names, block_size, false); + sources.emplace_back(source); + pipes.emplace_back(source); + + // Limit read batch to maximum block size to allow DDL + StreamLocalLimits limits; + + limits.speed_limits.max_execution_time = nats_settings->nats_flush_interval_ms.changed + ? nats_settings->nats_flush_interval_ms + : getContext()->getSettingsRef().stream_flush_interval_ms; + + limits.timeout_overflow_mode = OverflowMode::BREAK; + + source->setLimits(limits); + } + + block_io.pipeline.complete(Pipe::unitePipes(std::move(pipes))); + + if (!connection->getHandler().loopRunning()) + startLoop(); + + { + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); + } + + /* Note: sending ack() with loop running in another thread will lead to a lot of data races inside the library, but only in case + * error occurs or connection is lost while ack is being sent + */ + deactivateTask(looping_task, false, true); + size_t queue_empty = 0; + + if (!connection->isConnected()) + { + if (shutdown_called) + return true; + + if (connection->reconnect()) + { + LOG_DEBUG(log, "Connection restored, updating channels"); + for (auto & source : sources) + source->updateChannel(); + } + else + { + LOG_TRACE(log, "Reschedule streaming. Unable to restore connection."); + return true; + } + } + else + { + /// Commit + for (auto & source : sources) + { + if (source->queueEmpty()) + ++queue_empty; + + if (source->needChannelUpdate()) + { + auto buffer = source->getBuffer(); + prepareChannelForBuffer(buffer); + } + + /* false is returned by the sendAck function in only two cases: + * 1) if connection failed. In this case all channels will be closed and will be unable to send ack. Also ack is made based on + * delivery tags, which are unique to channels, so if channels fail, those delivery tags will become invalid and there is + * no way to send specific ack from a different channel. Actually once the server realises that it has messages in a queue + * waiting for confirm from a channel which suddenly closed, it will immediately make those messages accessible to other + * consumers. So in this case duplicates are inevitable. + * 2) size of the sent frame (libraries's internal request interface) exceeds max frame - internal library error. This is more + * common for message frames, but not likely to happen to ack frame I suppose. So I do not believe it is likely to happen. + * Also in this case if channel didn't get closed - it is ok if failed to send ack, because the next attempt to send ack on + * the same channel will also commit all previously not-committed messages. Anyway I do not think that for ack frame this + * will ever happen. + */ + if (!source->sendAck()) + { + /// Iterate loop to activate error callbacks if they happened + connection->getHandler().iterateLoop(); + if (!connection->isConnected()) + break; + } + + connection->getHandler().iterateLoop(); + } + } + + if ((queue_empty == num_created_consumers) && (++read_attempts == MAX_FAILED_READ_ATTEMPTS)) + { + connection->heartbeat(); + read_attempts = 0; + LOG_TRACE(log, "Reschedule streaming. Queues are empty."); + return true; + } + else + { + startLoop(); + } + + /// Do not reschedule, do not stop event loop. + return false; +} + + +void registerStorageNATS(StorageFactory & factory) +{ + auto creator_fn = [](const StorageFactory::Arguments & args) + { + auto nats_settings = std::make_unique(); + bool with_named_collection = getExternalDataSourceConfiguration(args.engine_args, *nats_settings, args.getLocalContext()); + if (!with_named_collection && !args.storage_def->settings) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "NATS engine must have settings"); + + nats_settings->loadFromQuery(*args.storage_def); + + if (!nats_settings->nats_host_port.changed + && !nats_settings->nats_address.changed) + throw Exception("You must specify either `nats_host_port` or `nats_address` settings", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!nats_settings->nats_format.changed) + throw Exception("You must specify `nats_format` setting", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return StorageNATS::create(args.table_id, args.getContext(), args.columns, std::move(nats_settings), args.attach); + }; + + factory.registerStorage("NATS", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); +} + + +NamesAndTypesList StorageNATS::getVirtuals() const +{ + return NamesAndTypesList{ + {"_exchange_name", std::make_shared()}, + {"_channel_id", std::make_shared()}, + {"_delivery_tag", std::make_shared()}, + {"_redelivered", std::make_shared()}, + {"_message_id", std::make_shared()}, + {"_timestamp", std::make_shared()} + }; +} + +} diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h new file mode 100644 index 00000000000..e5b397194b7 --- /dev/null +++ b/src/Storages/NATS/StorageNATS.h @@ -0,0 +1,207 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class StorageNATS final: public shared_ptr_helper, public IStorage, WithContext +{ + friend struct shared_ptr_helper; + +public: + std::string getName() const override { return "NATS"; } + + bool noPushingToViews() const override { return true; } + + void startup() override; + void shutdown() override; + + /// This is a bad way to let storage know in shutdown() that table is going to be dropped. There are some actions which need + /// to be done only when table is dropped (not when detached). Also connection must be closed only in shutdown, but those + /// actions require an open connection. Therefore there needs to be a way inside shutdown() method to know whether it is called + /// because of drop query. And drop() method is not suitable at all, because it will not only require to reopen connection, but also + /// it can be called considerable time after table is dropped (for example, in case of Atomic database), which is not appropriate for the case. + void checkTableCanBeDropped() const override { drop_table = true; } + + /// Always return virtual columns in addition to required columns + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context) override; + + void pushReadBuffer(ConsumerBufferPtr buf); + ConsumerBufferPtr popReadBuffer(); + ConsumerBufferPtr popReadBuffer(std::chrono::milliseconds timeout); + + ProducerBufferPtr createWriteBuffer(); + + const String & getFormatName() const { return format_name; } + NamesAndTypesList getVirtuals() const override; + + String getExchange() const { return exchange_name; } + void unbindExchange(); + + bool updateChannel(ChannelPtr & channel); + void updateQueues(std::vector & queues_) { queues_ = queues; } + void prepareChannelForBuffer(ConsumerBufferPtr buffer); + + void incrementReader(); + void decrementReader(); + +protected: + StorageNATS( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr nats_settings_, + bool is_attach_); + +private: + ContextMutablePtr nats_context; + std::unique_ptr nats_settings; + + const String exchange_name; + const String format_name; + AMQP::ExchangeType exchange_type; + Names routing_keys; + char row_delimiter; + const String schema_name; + size_t num_consumers; + size_t num_queues; + String queue_base; + Names queue_settings_list; + + /// For insert query. Mark messages as durable. + const bool persistent; + + /// A table setting. It is possible not to perform any NATS setup, which is supposed to be consumer-side setup: + /// declaring exchanges, queues, bindings. Instead everything needed from NATS table is to connect to a specific queue. + /// This solution disables all optimizations and is not really optimal, but allows user to fully control all NATS setup. + bool use_user_setup; + + bool hash_exchange; + Poco::Logger * log; + + NATSConnectionPtr connection; /// Connection for all consumers + NATSConfiguration configuration; + + size_t num_created_consumers = 0; + Poco::Semaphore semaphore; + std::mutex buffers_mutex; + std::vector buffers; /// available buffers for NATS consumers + + String unique_strbase; /// to make unique consumer channel id + + /// maximum number of messages in NATS queue (x-max-length). Also used + /// to setup size of inner buffer for received messages + uint32_t queue_size; + + String sharding_exchange, bridge_exchange, consumer_exchange; + size_t consumer_id = 0; /// counter for consumer buffer, needed for channel id + + std::vector queues; + + std::once_flag flag; /// remove exchange only once + std::mutex task_mutex; + BackgroundSchedulePool::TaskHolder streaming_task; + BackgroundSchedulePool::TaskHolder looping_task; + BackgroundSchedulePool::TaskHolder connection_task; + + uint64_t milliseconds_to_wait; + + /** + * ╰( ͡° ͜ʖ ͡° )つ──☆* Evil atomics: + */ + /// Needed for tell MV or producer background tasks + /// that they must finish as soon as possible. + std::atomic shutdown_called{false}; + /// Counter for producer buffers, needed for channel id. + /// Needed to generate unique producer buffer identifiers. + std::atomic producer_id = 1; + /// Has connection background task completed successfully? + /// It is started only once -- in constructor. + std::atomic rabbit_is_ready = false; + /// Allow to remove exchange only once. + std::atomic exchange_removed = false; + /// For select query we must be aware of the end of streaming + /// to be able to turn off the loop. + std::atomic readers_count = 0; + std::atomic mv_attached = false; + + /// In select query we start event loop, but do not stop it + /// after that select is finished. Then in a thread, which + /// checks for MV we also check if we have select readers. + /// If not - we turn off the loop. The checks are done under + /// mutex to avoid having a turned off loop when select was + /// started. + std::mutex loop_mutex; + + size_t read_attempts = 0; + mutable bool drop_table = false; + bool is_attach; + + ConsumerBufferPtr createReadBuffer(); + void initializeBuffers(); + bool initialized = false; + + /// Functions working in the background + void streamingToViewsFunc(); + void loopingFunc(); + void connectionFunc(); + + void startLoop(); + void stopLoop(); + void stopLoopIfNoReaders(); + + static Names parseSettings(String settings_list); + static AMQP::ExchangeType defineExchangeType(String exchange_type_); + static String getTableBasedName(String name, const StorageID & table_id); + + ContextMutablePtr addSettings(ContextPtr context) const; + size_t getMaxBlockSize() const; + void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); + + void initNATS(); + void cleanupNATS() const; + + void initExchange(AMQP::TcpChannel & rabbit_channel); + void bindExchange(AMQP::TcpChannel & rabbit_channel); + void bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel); + + bool streamToViews(); + bool checkDependencies(const StorageID & table_id); + + static String getRandomName() + { + std::uniform_int_distribution distribution('a', 'z'); + String random_str(32, ' '); + for (auto & c : random_str) + c = distribution(thread_local_rng); + return random_str; + } +}; + +} diff --git a/src/Storages/NATS/UVLoop.h b/src/Storages/NATS/UVLoop.h new file mode 100644 index 00000000000..4de67cbc206 --- /dev/null +++ b/src/Storages/NATS/UVLoop.h @@ -0,0 +1,44 @@ +#pragma once + +#include + +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYSTEM_ERROR; +} + +/// RAII wrapper around uv event loop +class UVLoop : public boost::noncopyable +{ +public: + UVLoop(): loop_ptr(new uv_loop_t()) + { + int res = uv_loop_init(loop_ptr.get()); + + if (res != 0) + throw Exception("UVLoop could not initialize", ErrorCodes::SYSTEM_ERROR); + } + + ~UVLoop() + { + if (loop_ptr) + uv_loop_close(loop_ptr.get()); + } + + inline uv_loop_t * getLoop() { return loop_ptr.get(); } + + inline const uv_loop_t * getLoop() const { return loop_ptr.get(); } + +private: + std::unique_ptr loop_ptr; +}; + +} diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp new file mode 100644 index 00000000000..2f4ea7bee53 --- /dev/null +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -0,0 +1,317 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +static const auto BATCH = 1000; +static const auto RETURNED_LIMIT = 50000; + +namespace ErrorCodes +{ + extern const int CANNOT_CONNECT_RABBITMQ; + extern const int LOGICAL_ERROR; +} + +WriteBufferToNATSProducer::WriteBufferToNATSProducer( + const NATSConfiguration & configuration_, + ContextPtr global_context, + const Names & routing_keys_, + const String & exchange_name_, + const AMQP::ExchangeType exchange_type_, + const size_t channel_id_base_, + const bool persistent_, + std::atomic & shutdown_called_, + Poco::Logger * log_, + std::optional delimiter, + size_t rows_per_message, + size_t chunk_size_) + : WriteBuffer(nullptr, 0) + , connection(configuration_, log_) + , routing_keys(routing_keys_) + , exchange_name(exchange_name_) + , exchange_type(exchange_type_) + , channel_id_base(std::to_string(channel_id_base_)) + , persistent(persistent_) + , shutdown_called(shutdown_called_) + , payloads(BATCH) + , returned(RETURNED_LIMIT) + , log(log_) + , delim(delimiter) + , max_rows(rows_per_message) + , chunk_size(chunk_size_) +{ + if (connection.connect()) + setupChannel(); + else + throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to NATS {}", connection.connectionInfoForLog()); + + writing_task = global_context->getSchedulePool().createTask("NATSWritingTask", [this]{ writingFunc(); }); + writing_task->deactivate(); + + if (exchange_type == AMQP::ExchangeType::headers) + { + for (const auto & header : routing_keys) + { + std::vector matching; + boost::split(matching, header, [](char c){ return c == '='; }); + key_arguments[matching[0]] = matching[1]; + } + } + + reinitializeChunks(); +} + + +WriteBufferToNATSProducer::~WriteBufferToNATSProducer() +{ + writing_task->deactivate(); + connection.disconnect(); + assert(rows == 0); +} + + +void WriteBufferToNATSProducer::countRow() +{ + if (++rows % max_rows == 0) + { + const std::string & last_chunk = chunks.back(); + size_t last_chunk_size = offset(); + + if (last_chunk_size && delim && last_chunk[last_chunk_size - 1] == delim) + --last_chunk_size; + + std::string payload; + payload.reserve((chunks.size() - 1) * chunk_size + last_chunk_size); + + for (auto i = chunks.begin(), end = --chunks.end(); i != end; ++i) + payload.append(*i); + + payload.append(last_chunk, 0, last_chunk_size); + + reinitializeChunks(); + + ++payload_counter; + if (!payloads.push(std::make_pair(payload_counter, payload))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); + } +} + + +void WriteBufferToNATSProducer::setupChannel() +{ + producer_channel = connection.createChannel(); + + producer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Producer's channel {} error: {}", channel_id, message); + + /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) + producer_channel->close(); + + /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once + * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid + */ + for (const auto & record : delivery_record) + if (!returned.push(record.second)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); + + LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", + channel_id, delivery_record.size(), payloads.size()); + + /// Delivery tags are scoped per channel. + delivery_record.clear(); + delivery_tag = 0; + producer_ready = false; + }); + + producer_channel->onReady([&]() + { + channel_id = channel_id_base + "_" + std::to_string(channel_id_counter++); + LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); + + /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, + * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times + * slower than the second, so default is second and the first is turned on in table setting. + * + * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html + */ + producer_channel->confirmSelect() + .onAck([&](uint64_t acked_delivery_tag, bool multiple) + { + removeRecord(acked_delivery_tag, multiple, false); + }) + .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) + { + removeRecord(nacked_delivery_tag, multiple, true); + }); + producer_ready = true; + }); +} + + +void WriteBufferToNATSProducer::removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish) +{ + auto record_iter = delivery_record.find(received_delivery_tag); + assert(record_iter != delivery_record.end()); + + if (multiple) + { + /// If multiple is true, then all delivery tags up to and including current are confirmed (with ack or nack). + ++record_iter; + + if (republish) + for (auto record = delivery_record.begin(); record != record_iter; ++record) + if (!returned.push(record->second)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); + + /// Delete the records even in case when republished because new delivery tags will be assigned by the server. + delivery_record.erase(delivery_record.begin(), record_iter); + } + else + { + if (republish) + if (!returned.push(record_iter->second)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); + + delivery_record.erase(record_iter); + } +} + + +void WriteBufferToNATSProducer::publish(ConcurrentBoundedQueue> & messages, bool republishing) +{ + std::pair payload; + + /* It is important to make sure that delivery_record.size() is never bigger than returned.size(), i.e. number if unacknowledged + * messages cannot exceed returned.size(), because they all might end up there + */ + while (!messages.empty() && producer_channel->usable() && delivery_record.size() < RETURNED_LIMIT) + { + bool pop_result = messages.pop(payload); + + if (!pop_result) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not pop payload"); + + AMQP::Envelope envelope(payload.second.data(), payload.second.size()); + + /// if headers exchange is used, routing keys are added here via headers, if not - it is just empty + AMQP::Table message_settings = key_arguments; + + /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the + * server, then it means that publisher will never know whether those messages were delivered or not, and therefore those records + * that received no ack/nack before connection loss will be republished (see onError() callback), so there might be duplicates. To + * let consumer know that received message might be a possible duplicate - a "republished" field is added to message metadata + */ + message_settings["republished"] = std::to_string(republishing); + envelope.setHeaders(message_settings); + + /* Adding here a messageID property to message metadata. Since NATS does not guarantee exactly-once delivery, then on the + * consumer side "republished" field of message metadata can be checked and, if it set to 1, consumer might also check "messageID" + * property. This way detection of duplicates is guaranteed + */ + envelope.setMessageID(std::to_string(payload.first)); + + /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse + if (persistent) + envelope.setDeliveryMode(2); + + if (exchange_type == AMQP::ExchangeType::consistent_hash) + { + producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope); + } + else if (exchange_type == AMQP::ExchangeType::headers) + { + producer_channel->publish(exchange_name, "", envelope); + } + else + { + producer_channel->publish(exchange_name, routing_keys[0], envelope); + } + + /// This is needed for "publisher confirms", which guarantees at-least-once delivery + ++delivery_tag; + delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); + + /// Need to break at some point to let event loop run, because no publishing actually happens before looping + if (delivery_tag % BATCH == 0) + break; + } + + iterateEventLoop(); +} + + +void WriteBufferToNATSProducer::writingFunc() +{ + while ((!payloads.empty() || wait_all) && !shutdown_called.load()) + { + /// If onReady callback is not received, producer->usable() will anyway return true, + /// but must publish only after onReady callback. + if (producer_ready) + { + /* Publish main paylods only when there are no returned messages. This way it is ensured that returned messages are republished + * as fast as possible and no new publishes are made before returned messages are handled + */ + if (!returned.empty() && producer_channel->usable()) + publish(returned, true); + else if (!payloads.empty() && producer_channel->usable()) + publish(payloads, false); + } + + iterateEventLoop(); + + if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) + wait_all = false; + else if (!producer_channel->usable()) + { + if (connection.reconnect()) + setupChannel(); + } + } + + LOG_DEBUG(log, "Producer on channel {} completed", channel_id); +} + + +void WriteBufferToNATSProducer::nextImpl() +{ + addChunk(); +} + +void WriteBufferToNATSProducer::addChunk() +{ + chunks.push_back(std::string()); + chunks.back().resize(chunk_size); + set(chunks.back().data(), chunk_size); +} + +void WriteBufferToNATSProducer::reinitializeChunks() +{ + rows = 0; + chunks.clear(); + /// We cannot leave the buffer in the undefined state (i.e. without any + /// underlying buffer), since in this case the WriteBuffeR::next() will + /// not call our nextImpl() (due to available() == 0) + addChunk(); +} + + +void WriteBufferToNATSProducer::iterateEventLoop() +{ + connection.getHandler().iterateLoop(); +} + +} diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h new file mode 100644 index 00000000000..d0d80a6cf9d --- /dev/null +++ b/src/Storages/NATS/WriteBufferToNATSProducer.h @@ -0,0 +1,121 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class WriteBufferToNATSProducer : public WriteBuffer +{ +public: + WriteBufferToNATSProducer( + const NATSConfiguration & configuration_, + ContextPtr global_context, + const Names & routing_keys_, + const String & exchange_name_, + const AMQP::ExchangeType exchange_type_, + const size_t channel_id_base_, + const bool persistent_, + std::atomic & shutdown_called_, + Poco::Logger * log_, + std::optional delimiter, + size_t rows_per_message, + size_t chunk_size_ + ); + + ~WriteBufferToNATSProducer() override; + + void countRow(); + void activateWriting() { writing_task->activateAndSchedule(); } + void updateMaxWait() { wait_num.store(payload_counter); } + +private: + void nextImpl() override; + void addChunk(); + void reinitializeChunks(); + + void iterateEventLoop(); + void writingFunc(); + void setupChannel(); + void removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish); + void publish(ConcurrentBoundedQueue> & message, bool republishing); + + NATSConnection connection; + + const Names routing_keys; + const String exchange_name; + AMQP::ExchangeType exchange_type; + const String channel_id_base; /// Serial number of current producer buffer + const bool persistent; + + /* false: when shutdown is called; needed because table might be dropped before all acks are received + * true: in all other cases + */ + std::atomic & shutdown_called; + + AMQP::Table key_arguments; + BackgroundSchedulePool::TaskHolder writing_task; + + std::unique_ptr producer_channel; + bool producer_ready = false; + + /// Channel errors lead to channel closure, need to count number of recreated channels to update channel id + UInt64 channel_id_counter = 0; + + /// channel id which contains id of current producer buffer and serial number of recreated channel in this buffer + String channel_id; + + /* payloads.queue: + * - payloads are pushed to queue in countRow and popped by another thread in writingFunc, each payload gets into queue only once + * returned.queue: + * - payloads are pushed to queue: + * 1) inside channel->onError() callback if channel becomes unusable and the record of pending acknowledgements from server + * is non-empty. + * 2) inside removeRecord() if received nack() - negative acknowledgement from the server that message failed to be written + * to disk or it was unable to reach the queue. + * - payloads are popped from the queue once republished + */ + ConcurrentBoundedQueue> payloads, returned; + + /* Counter of current delivery on a current channel. Delivery tags are scoped per channel. The server attaches a delivery tag for each + * published message - a serial number of delivery on current channel. Delivery tag is a way of server to notify publisher if it was + * able or unable to process delivery, i.e. it sends back a response with a corresponding delivery tag. + */ + UInt64 delivery_tag = 0; + + /* false: message delivery successfully ended: publisher received confirm from server that all published + * 1) persistent messages were written to disk + * 2) non-persistent messages reached the queue + * true: continue to process deliveries and returned messages + */ + bool wait_all = true; + + /* false: until writeSuffix is called + * true: means payloads.queue will not grow anymore + */ + std::atomic wait_num = 0; + + /// Needed to fill messageID property + UInt64 payload_counter = 0; + + /// Record of pending acknowledgements from the server; its size never exceeds size of returned.queue + std::map> delivery_record; + + Poco::Logger * log; + const std::optional delim; + const size_t max_rows; + const size_t chunk_size; + size_t rows = 0; + std::list chunks; +}; + +} From ec8152fdf88839ccd0f34b950a00d42375e725a7 Mon Sep 17 00:00:00 2001 From: tchepavel Date: Mon, 9 May 2022 23:16:11 +0300 Subject: [PATCH 006/627] MV works; rand() is harmful --- base/harmful/harmful.c | 2 +- .../ExternalDataSourceConfiguration.cpp | 5 + src/Storages/NATS/Buffer_fwd.h | 4 +- src/Storages/NATS/NATSConnection.cpp | 71 +- src/Storages/NATS/NATSConnection.h | 14 +- src/Storages/NATS/NATSHandler.cpp | 13 +- src/Storages/NATS/NATSHandler.h | 4 +- src/Storages/NATS/NATSSettings.cpp | 2 +- src/Storages/NATS/NATSSettings.h | 15 +- src/Storages/NATS/NATSSink.cpp | 4 +- src/Storages/NATS/NATSSource.cpp | 84 +- src/Storages/NATS/NATSSource.h | 8 +- .../NATS/ReadBufferFromNATSConsumer.cpp | 212 ++-- .../NATS/ReadBufferFromNATSConsumer.h | 68 +- src/Storages/NATS/StorageNATS.cpp | 1073 ++++++++--------- src/Storages/NATS/StorageNATS.h | 49 +- .../NATS/WriteBufferToNATSProducer.cpp | 94 +- src/Storages/NATS/WriteBufferToNATSProducer.h | 4 +- src/Storages/registerStorages.cpp | 4 + 19 files changed, 838 insertions(+), 892 deletions(-) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index 5a27cae0383..adc3f7158f6 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -206,7 +206,7 @@ TRAP(lgammal) TRAP(nftw) TRAP(nl_langinfo) TRAP(putc_unlocked) -TRAP(rand) +//TRAP(rand) Used by nats-io at startup /** In the current POSIX.1 specification (POSIX.1-2008), readdir() is not required to be thread-safe. However, in modern * implementations (including the glibc implementation), concurrent calls to readdir() that specify different directory streams * are thread-safe. In cases where multiple threads must read from the same directory stream, using readdir() with external diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index abd20e6e5fd..1cb123120f2 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -19,6 +19,8 @@ #include #endif +#include + #include namespace DB @@ -483,6 +485,9 @@ template bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); #endif +template +bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); + template std::optional getExternalDataSourceConfiguration( const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection, const BaseSettings & storage_settings); diff --git a/src/Storages/NATS/Buffer_fwd.h b/src/Storages/NATS/Buffer_fwd.h index caf7b8cfdbe..3eb52314a79 100644 --- a/src/Storages/NATS/Buffer_fwd.h +++ b/src/Storages/NATS/Buffer_fwd.h @@ -6,9 +6,9 @@ namespace DB { class ReadBufferFromNATSConsumer; -using ConsumerBufferPtr = std::shared_ptr; +using ConsumerBufferPtr = std::shared_ptr; class WriteBufferToNATSProducer; -using ProducerBufferPtr = std::shared_ptr; +using ProducerBufferPtr = std::shared_ptr; } diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index 7b0abba7535..cef8a2eb8c1 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -7,36 +7,36 @@ namespace DB { -static const auto CONNECT_SLEEP = 200; +//static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; -NATSConnection::NATSConnection(const NATSConfiguration & configuration_, Poco::Logger * log_) +NATSConnectionManager::NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_) : configuration(configuration_) , log(log_) , event_handler(loop.getLoop(), log) { } -String NATSConnection::connectionInfoForLog() const +String NATSConnectionManager::connectionInfoForLog() const { return configuration.host + ':' + toString(configuration.port); } -bool NATSConnection::isConnected() +bool NATSConnectionManager::isConnected() { std::lock_guard lock(mutex); return isConnectedImpl(); } -bool NATSConnection::connect() +bool NATSConnectionManager::connect() { std::lock_guard lock(mutex); connectImpl(); return isConnectedImpl(); } -bool NATSConnection::reconnect() +bool NATSConnectionManager::reconnect() { std::lock_guard lock(mutex); if (isConnectedImpl()) @@ -44,69 +44,70 @@ bool NATSConnection::reconnect() disconnectImpl(); - /// This will force immediate closure if not yet closed - if (!connection->closed()) - connection->close(true); - LOG_DEBUG(log, "Trying to restore connection to {}", connectionInfoForLog()); connectImpl(); return isConnectedImpl(); } -SubscriptionPtr NATSConnection::createSubscription(const std::string& subject) +SubscriptionPtr NATSConnectionManager::createSubscription(const std::string& subject, natsMsgHandler handler, ReadBufferFromNATSConsumer * consumer) { std::lock_guard lock(mutex); natsSubscription * ns; - natsConnection_SubscribeSync(&ns, connection, subject.c_str()); + status = natsConnection_Subscribe(&ns, connection, subject.c_str(), handler, static_cast(consumer)); + if (status == NATS_OK) + status = natsSubscription_SetPendingLimits(ns, -1, -1); + if (status == NATS_OK) + LOG_DEBUG(log, "Subscribed to subject {}", subject); return SubscriptionPtr(ns, &natsSubscription_Destroy); } -void NATSConnection::disconnect() +void NATSConnectionManager::disconnect() { std::lock_guard lock(mutex); disconnectImpl(); } -bool NATSConnection::closed() +bool NATSConnectionManager::closed() { std::lock_guard lock(mutex); return natsConnection_IsClosed(connection); } -bool NATSConnection::isConnectedImpl() const +bool NATSConnectionManager::isConnectedImpl() const { - return event_handler.connectionRunning() && !natsConnection_IsClosed(connection); + return event_handler.connectionRunning() && !natsConnection_IsClosed(connection) && status == natsStatus::NATS_OK; } -void NATSConnection::connectImpl() +void NATSConnectionManager::connectImpl() { + natsOptions * options = event_handler.getOptions(); + natsOptions_SetUserInfo(options, configuration.username.c_str(), configuration.password.c_str()); + if (configuration.secure) { + natsOptions_SetSecure(options, true); + natsOptions_SkipServerVerification(options, true); + } + std::string address; if (configuration.connection_string.empty()) { - LOG_DEBUG(log, "Connecting to: {}:{} (user: {})", configuration.host, configuration.port, configuration.username); - AMQP::Login login(configuration.username, configuration.password); - AMQP::Address address(configuration.host, configuration.port, login, configuration.vhost, configuration.secure); - connection = std::make_unique(&event_handler, address); + address = configuration.host + ":" + std::to_string(configuration.port); } else { - AMQP::Address address(configuration.connection_string); - connection = std::make_unique(&event_handler, address); + address = configuration.connection_string; } - - auto cnt_retries = 0; - while (true) + natsOptions_SetURL(options, address.c_str()); + status = natsConnection_Connect(&connection, options); + if (status != NATS_OK) { - event_handler.iterateLoop(); - - if (connection->ready() || cnt_retries++ == RETRIES_MAX) - break; - - std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + LOG_DEBUG(log, "Failed to connect to NATS on address: {}", address); + return; } + + event_handler.changeConnectionStatus(true); } -void NATSConnection::disconnectImpl() +void NATSConnectionManager::disconnectImpl() { natsConnection_Close(connection); @@ -114,8 +115,10 @@ void NATSConnection::disconnectImpl() * an AMQP closing-handshake is performed). But cannot open a new connection until previous one is properly closed */ size_t cnt_retries = 0; - while (!closed() && cnt_retries++ != RETRIES_MAX) + while (!natsConnection_IsClosed(connection) && cnt_retries++ != RETRIES_MAX) event_handler.iterateLoop(); + + event_handler.changeConnectionStatus(false); } } diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h index 2c999e873aa..ab97012061c 100644 --- a/src/Storages/NATS/NATSConnection.h +++ b/src/Storages/NATS/NATSConnection.h @@ -2,14 +2,13 @@ #include #include - +#include namespace DB { struct NATSConfiguration { - String url; String host; UInt16 port; String username; @@ -20,13 +19,12 @@ struct NATSConfiguration String connection_string; }; -using SubscriptionPtr = std::unique_ptr; - -class NATSConnection +class NATSConnectionManager { public: - NATSConnection(const NATSConfiguration & configuration_, Poco::Logger * log_); + NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_); + ~NATSConnectionManager() { natsConnection_Destroy(connection); } bool isConnected(); @@ -38,7 +36,7 @@ public: bool closed(); - SubscriptionPtr createSubscription(const std::string& subject); + SubscriptionPtr createSubscription(const std::string& subject, natsMsgHandler handler, ReadBufferFromNATSConsumer * consumer); /// NATSHandler is thread safe. Any public methods can be called concurrently. NATSHandler & getHandler() { return event_handler; } @@ -63,6 +61,6 @@ private: std::mutex mutex; }; -using NATSConnectionPtr = std::unique_ptr; +using NATSConnectionManagerPtr = std::shared_ptr; } diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index 7fb8ff38c47..cb155522a6a 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -18,7 +19,7 @@ NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : natsLibuv_Init(); natsLibuv_SetThreadLocalLoop(loop); natsOptions_Create(&opts); - status = natsOptions_SetEventLoop(opts, static_cast(loop), + status = natsOptions_SetEventLoop(opts, static_cast(loop), natsLibuv_Attach, natsLibuv_Read, natsLibuv_Write, @@ -28,22 +29,32 @@ NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : void NATSHandler::startLoop() { std::lock_guard lock(startup_mutex); + natsLibuv_SetThreadLocalLoop(loop); LOG_DEBUG(log, "Background loop started"); loop_running.store(true); while (loop_state.load() == Loop::RUN) + { uv_run(loop, UV_RUN_NOWAIT); + } LOG_DEBUG(log, "Background loop ended"); loop_running.store(false); } +void NATSHandler::changeConnectionStatus(bool is_running) { + connection_running.store(is_running); +} + void NATSHandler::iterateLoop() { std::unique_lock lock(startup_mutex, std::defer_lock); if (lock.try_lock()) + { + natsLibuv_SetThreadLocalLoop(loop); uv_run(loop, UV_RUN_NOWAIT); + } } /// Do not need synchronization as in iterateLoop(), because this method is used only for diff --git a/src/Storages/NATS/NATSHandler.h b/src/Storages/NATS/NATSHandler.h index 12ea3454b9d..2854bf9a069 100644 --- a/src/Storages/NATS/NATSHandler.h +++ b/src/Storages/NATS/NATSHandler.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include @@ -20,6 +19,8 @@ namespace Loop static const UInt8 STOP = 2; } +using SubscriptionPtr = std::unique_ptr; + class NATSHandler { @@ -41,6 +42,7 @@ public: void stopLoop(); + void changeConnectionStatus(bool is_running); bool connectionRunning() const { return connection_running.load(); } bool loopRunning() const { return loop_running.load(); } diff --git a/src/Storages/NATS/NATSSettings.cpp b/src/Storages/NATS/NATSSettings.cpp index 11659b105b2..ec1149acee4 100644 --- a/src/Storages/NATS/NATSSettings.cpp +++ b/src/Storages/NATS/NATSSettings.cpp @@ -12,7 +12,7 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_NATS_SETTINGS) void NATSSettings::loadFromQuery(ASTStorage & storage_def) { diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h index 5f4051e06d1..c30869e8ea3 100644 --- a/src/Storages/NATS/NATSSettings.h +++ b/src/Storages/NATS/NATSSettings.h @@ -8,17 +8,13 @@ namespace DB class ASTStorage; -#define RABBITMQ_RELATED_SETTINGS(M) \ +#define NATS_RELATED_SETTINGS(M) \ M(String, nats_host_port, "", "A host-port to connect to NATS server.", 0) \ - M(String, nats_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \ + M(String, nats_subjects, "", "List of subject for NATS table to subscribe/publsh to.", 0) \ M(String, nats_format, "", "The message format.", 0) \ - M(String, nats_exchange_type, "default", "The exchange type.", 0) \ - M(String, nats_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \ M(Char, nats_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ M(String, nats_schema, "", "Schema identifier (used by schema-based formats) for NATS engine", 0) \ M(UInt64, nats_num_consumers, 1, "The number of consumer channels per table.", 0) \ - M(UInt64, nats_num_queues, 1, "The number of queues per consumer.", 0) \ - M(String, nats_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ M(Bool, nats_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \ M(Bool, nats_secure, false, "Use SSL connection", 0) \ M(String, nats_address, "", "Address for connection", 0) \ @@ -26,17 +22,16 @@ namespace DB M(UInt64, nats_max_block_size, 0, "Number of row collected before flushing data from NATS.", 0) \ M(Milliseconds, nats_flush_interval_ms, 0, "Timeout for flushing data from NATS.", 0) \ M(String, nats_vhost, "/", "NATS vhost.", 0) \ - M(String, nats_queue_settings_list, "", "A list of nats queue settings", 0) \ M(Bool, nats_queue_consume, false, "Use user-defined queues and do not make any NATS setup: declaring exchanges, queues, bindings", 0) \ M(String, nats_username, "", "NATS username", 0) \ M(String, nats_password, "", "NATS password", 0) \ M(Bool, nats_commit_on_select, false, "Commit messages when select query is made", 0) \ -#define LIST_OF_RABBITMQ_SETTINGS(M) \ - RABBITMQ_RELATED_SETTINGS(M) \ +#define LIST_OF_NATS_SETTINGS(M) \ + NATS_RELATED_SETTINGS(M) \ FORMAT_FACTORY_SETTINGS(M) -DECLARE_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) +DECLARE_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_NATS_SETTINGS) struct NATSSettings : public BaseSettings { diff --git a/src/Storages/NATS/NATSSink.cpp b/src/Storages/NATS/NATSSink.cpp index db2620d9ed2..00fe93a9e68 100644 --- a/src/Storages/NATS/NATSSink.cpp +++ b/src/Storages/NATS/NATSSink.cpp @@ -18,13 +18,13 @@ NATSSink::NATSSink( , metadata_snapshot(metadata_snapshot_) , context(context_) { - storage.unbindExchange(); +// storage.unbindExchange(); } void NATSSink::onStart() { - buffer = storage.createWriteBuffer(); +// buffer = storage.createWriteBuffer(); buffer->activateWriting(); auto format_settings = getFormatSettings(context); diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index 046b8792ced..d757f7b9a54 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -12,7 +12,7 @@ static std::pair getHeaders(const StorageSnapshotPtr & storage_sna { auto non_virtual_header = storage_snapshot->metadata->getSampleBlockNonMaterialized(); auto virtual_header = storage_snapshot->getSampleBlockForColumns( - {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered", "_message_id", "_timestamp"}); + {"_subject", "_timestamp"}); return {non_virtual_header, virtual_header}; } @@ -51,14 +51,14 @@ NATSSource::NATSSource( ContextPtr context_, const Names & columns, size_t max_block_size_, - bool ack_in_suffix_) + bool /*ack_in_suffix_*/) : SourceWithProgress(getSampleBlock(headers.first, headers.second)) , storage(storage_) , storage_snapshot(storage_snapshot_) , context(context_) , column_names(columns) , max_block_size(max_block_size_) - , ack_in_suffix(ack_in_suffix_) +// , ack_in_suffix(ack_in_suffix_) , non_virtual_header(std::move(headers.first)) , virtual_header(std::move(headers.second)) { @@ -77,31 +77,31 @@ NATSSource::~NATSSource() } -bool NATSSource::needChannelUpdate() -{ - if (!buffer) - return false; - - return buffer->needChannelUpdate(); -} - - -void NATSSource::updateChannel() -{ - if (!buffer) - return; - - buffer->updateAckTracker(); - - if (storage.updateChannel(buffer->getChannel())) - buffer->setupChannel(); -} +//bool NATSSource::needChannelUpdate() +//{ +// if (!buffer) +// return false; +// +// return buffer->needChannelUpdate(); +//} +// +// +//void NATSSource::updateChannel() +//{ +// if (!buffer) +// return; +// +// buffer->updateAckTracker(); +// +// if (storage.updateChannel(buffer->getChannel())) +// buffer->setupChannel(); +//} Chunk NATSSource::generate() { auto chunk = generateImpl(); - if (!chunk && ack_in_suffix) - sendAck(); +// if (!chunk && ack_in_suffix) +// sendAck(); return chunk; } @@ -136,23 +136,13 @@ Chunk NATSSource::generateImpl() if (new_rows) { - auto exchange_name = storage.getExchange(); - auto channel_id = buffer->getChannelID(); - auto delivery_tag = buffer->getDeliveryTag(); - auto redelivered = buffer->getRedelivered(); - auto message_id = buffer->getMessageID(); + auto subject = buffer->getSubject(); auto timestamp = buffer->getTimestamp(); - buffer->updateAckTracker({delivery_tag, channel_id}); - for (size_t i = 0; i < new_rows; ++i) { - virtual_columns[0]->insert(exchange_name); - virtual_columns[1]->insert(channel_id); - virtual_columns[2]->insert(delivery_tag); - virtual_columns[3]->insert(redelivered); - virtual_columns[4]->insert(message_id); - virtual_columns[5]->insert(timestamp); + virtual_columns[0]->insert(subject); + virtual_columns[1]->insert(timestamp); } total_rows = total_rows + new_rows; @@ -175,15 +165,15 @@ Chunk NATSSource::generateImpl() } -bool NATSSource::sendAck() -{ - if (!buffer) - return false; - - if (!buffer->ackMessages()) - return false; - - return true; -} +//bool NATSSource::sendAck() +//{ +// if (!buffer) +// return false; +// +// if (!buffer->ackMessages()) +// return false; +// +// return true; +//} } diff --git a/src/Storages/NATS/NATSSource.h b/src/Storages/NATS/NATSSource.h index ce48e5cf382..263e6824603 100644 --- a/src/Storages/NATS/NATSSource.h +++ b/src/Storages/NATS/NATSSource.h @@ -28,9 +28,9 @@ public: Chunk generate() override; bool queueEmpty() const { return !buffer || buffer->queueEmpty(); } - bool needChannelUpdate(); - void updateChannel(); - bool sendAck(); +// bool needChannelUpdate(); +// void updateChannel(); +// bool sendAck(); private: StorageNATS & storage; @@ -38,7 +38,7 @@ private: ContextPtr context; Names column_names; const size_t max_block_size; - bool ack_in_suffix; +// bool ack_in_suffix; bool is_finished = false; const Block non_virtual_header; diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index ad966401cc3..5edd48d97c8 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -5,8 +5,8 @@ #include #include #include -#include #include +#include #include #include "Poco/Timer.h" #include @@ -21,8 +21,8 @@ namespace ErrorCodes ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( NATSHandler & event_handler_, - std::vector & queues_, - size_t channel_id_base_, + std::shared_ptr connection_, + std::vector & subjects_, const String & channel_base_, Poco::Logger * log_, char row_delimiter_, @@ -30,136 +30,105 @@ ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , event_handler(event_handler_) - , queues(queues_) + , connection(connection_) + , subjects(subjects_) , channel_base(channel_base_) - , channel_id_base(channel_id_base_) , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) , received(queue_size_) { + subscribe(); + LOG_DEBUG(log, "Started NATS consumer"); } ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() { + for (const auto& subscription : subscriptions) { + natsSubscription_Unsubscribe(subscription.get()); + } + BufferBase::set(nullptr, 0, 0); } void ReadBufferFromNATSConsumer::subscribe() { - for (const auto & queue_name : queues) - { - consumer_channel->consume(queue_name) - .onSuccess([&](const std::string & /* consumer_tag */) - { - LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name); - - if (++subscribed == queues.size()) - wait_subscription.store(false); - }) - .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) - { - if (message.bodySize()) - { - String message_received = std::string(message.body(), message.body() + message.bodySize()); - if (row_delimiter != '\0') - message_received += row_delimiter; - - if (!received.push({message_received, message.hasMessageID() ? message.messageID() : "", - message.hasTimestamp() ? message.timestamp() : 0, - redelivered, AckTracker(delivery_tag, channel_id)})) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); - } - }) - .onError([&](const char * message) - { - /* End up here either if channel ends up in an error state (then there will be resubscription) or consume call error, which - * arises from queue settings mismatch or queue level error, which should not happen as no one else is supposed to touch them - */ - LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message); - wait_subscription.store(false); - }); + for (const auto & subject : subjects) { + subscriptions.emplace_back(connection->createSubscription(subject, onMsg, this)); } +// for (const auto & queue_name : subjects) +// { +// consumer_channel->consume(queue_name) +// .onSuccess([&](const std::string & /* consumer_tag */) +// { +// LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name); +// +// if (++subscribed == subjects.size()) +// wait_subscription.store(false); +// }) +// .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) +// { +// if (message.bodySize()) +// { +// String message_received = std::string(message.body(), message.body() + message.bodySize()); +// if (row_delimiter != '\0') +// message_received += row_delimiter; +// +// if (!received.push({message_received, message.hasMessageID() ? message.messageID() : "", +// message.hasTimestamp() ? message.timestamp() : 0, +// redelivered, AckTracker(delivery_tag, channel_id)})) +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); +// } +// }) +// .onError([&](const char * message) +// { +// /* End up here either if channel ends up in an error state (then there will be resubscription) or consume call error, which +// * arises from queue settings mismatch or queue level error, which should not happen as no one else is supposed to touch them +// */ +// LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message); +// wait_subscription.store(false); +// }); +// } } -bool ReadBufferFromNATSConsumer::ackMessages() -{ - AckTracker record_info = last_inserted_record_info; - - /* Do not send ack to server if message's channel is not the same as current running channel because delivery tags are scoped per - * channel, so if channel fails, all previous delivery tags become invalid - */ - if (record_info.channel_id == channel_id && record_info.delivery_tag && record_info.delivery_tag > prev_tag) - { - /// Commit all received messages with delivery tags from last committed to last inserted - if (!consumer_channel->ack(record_info.delivery_tag, AMQP::multiple)) - { - LOG_ERROR(log, "Failed to commit messages with delivery tags from last committed to {} on channel {}", - record_info.delivery_tag, channel_id); - return false; - } - - prev_tag = record_info.delivery_tag; - LOG_TRACE(log, "Consumer committed messages with deliveryTags up to {} on channel {}", record_info.delivery_tag, channel_id); - } - - return true; -} +//bool ReadBufferFromNATSConsumer::ackMessages() +//{ +// AckTracker record_info = last_inserted_record_info; +// +// /* Do not send ack to server if message's channel is not the same as current running channel because delivery tags are scoped per +// * channel, so if channel fails, all previous delivery tags become invalid +// */ +// if (record_info.channel_id == channel_id && record_info.delivery_tag && record_info.delivery_tag > prev_tag) +// { +// /// Commit all received messages with delivery tags from last committed to last inserted +// if (!consumer_channel->ack(record_info.delivery_tag, AMQP::multiple)) +// { +// LOG_ERROR(log, "Failed to commit messages with delivery tags from last committed to {} on channel {}", +// record_info.delivery_tag, channel_id); +// return false; +// } +// +// prev_tag = record_info.delivery_tag; +// LOG_TRACE(log, "Consumer committed messages with deliveryTags up to {} on channel {}", record_info.delivery_tag, channel_id); +// } +// +// return true; +//} -void ReadBufferFromNATSConsumer::updateAckTracker(AckTracker record_info) -{ - if (record_info.delivery_tag && channel_error.load()) - return; - - if (!record_info.delivery_tag) - prev_tag = 0; - - last_inserted_record_info = record_info; -} - - -void ReadBufferFromNATSConsumer::setupChannel() -{ - if (!consumer_channel) - return; - - wait_subscription.store(true); - - consumer_channel->onReady([&]() - { - /* First number indicates current consumer buffer; second number indicates serial number of created channel for current buffer, - * i.e. if channel fails - another one is created and its serial number is incremented; channel_base is to guarantee that - * channel_id is unique for each table - */ - channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base; - LOG_TRACE(log, "Channel {} is created", channel_id); - - subscribed = 0; - subscribe(); - channel_error.store(false); - }); - - consumer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Channel {} error: {}", channel_id, message); - - channel_error.store(true); - wait_subscription.store(false); - }); -} - - -bool ReadBufferFromNATSConsumer::needChannelUpdate() -{ - if (wait_subscription) - return false; - - return channel_error || !consumer_channel || !consumer_channel->usable(); -} +//void ReadBufferFromNATSConsumer::updateAckTracker(AckTracker record_info) +//{ +// if (record_info.delivery_tag && channel_error.load()) +// return; +// +// if (!record_info.delivery_tag) +// prev_tag = 0; +// +// last_inserted_record_info = record_info; +//} void ReadBufferFromNATSConsumer::iterateEventLoop() @@ -185,4 +154,27 @@ bool ReadBufferFromNATSConsumer::nextImpl() return false; } +void ReadBufferFromNATSConsumer::onMsg(natsConnection *, natsSubscription *, natsMsg * msg, void * consumer) +{ + auto * buffer = static_cast(consumer); + const int msg_length = natsMsg_GetDataLength(msg); + LOG_DEBUG(buffer->log, "I'm getting something {} {}", msg_length, natsMsg_GetData(msg)); + + if (msg_length) + { + String message_received = std::string(natsMsg_GetData(msg), msg_length); + if (buffer->row_delimiter != '\0') + message_received += buffer->row_delimiter; + + if (!buffer->received.push({ + .message = message_received, + .subject = natsMsg_GetSubject(msg), + .timestamp = natsMsg_GetTime(msg) + })) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); + } + + natsMsg_Destroy(msg); +} + } diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h index 1206ee4e62d..2c1a9704c03 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include namespace Poco @@ -22,8 +22,8 @@ class ReadBufferFromNATSConsumer : public ReadBuffer public: ReadBufferFromNATSConsumer( NATSHandler & event_handler_, - std::vector & queues_, - size_t channel_id_base_, + std::shared_ptr connection_, + std::vector & subjects_, const String & channel_base_, Poco::Logger * log_, char row_delimiter_, @@ -32,73 +32,69 @@ public: ~ReadBufferFromNATSConsumer() override; - struct AckTracker - { - UInt64 delivery_tag; - String channel_id; - - AckTracker() = default; - AckTracker(UInt64 tag, String id) : delivery_tag(tag), channel_id(id) {} - }; +// struct AckTracker +// { +// UInt64 delivery_tag; +// String channel_id; +// +// AckTracker() = default; +// AckTracker(UInt64 tag, String id) : delivery_tag(tag), channel_id(id) {} +// }; struct MessageData { String message; - String message_id; - uint64_t timestamp = 0; - bool redelivered = false; - AckTracker track{}; + String subject; + int64_t timestamp; +// AckTracker track{}; }; - ChannelPtr & getChannel() { return consumer_channel; } - void setupChannel(); - bool needChannelUpdate(); + std::vector & getChannel() { return subscriptions; } void closeChannel() { - if (consumer_channel) - consumer_channel->close(); + for (const auto & subscription : subscriptions) + natsSubscription_Unsubscribe(subscription.get()); } - void updateQueues(std::vector & queues_) { queues = queues_; } - size_t queuesCount() { return queues.size(); } + void updateSubjects(std::vector & subjects_) { subjects = subjects_; } + size_t subjectsCount() { return subjects.size(); } bool isConsumerStopped() { return stopped; } - bool ackMessages(); - void updateAckTracker(AckTracker record = AckTracker()); +// bool ackMessages(); +// void updateAckTracker(AckTracker record = AckTracker()); bool queueEmpty() { return received.empty(); } void allowNext() { allowed = true; } // Allow to read next message. - auto getChannelID() const { return current.track.channel_id; } - auto getDeliveryTag() const { return current.track.delivery_tag; } - auto getRedelivered() const { return current.redelivered; } - auto getMessageID() const { return current.message_id; } + auto getSubject() const { return current.subject; } auto getTimestamp() const { return current.timestamp; } + void iterateEventLoop(); private: bool nextImpl() override; void subscribe(); - void iterateEventLoop(); + static void onMsg(natsConnection *nc, natsSubscription *sub, natsMsg * msg, void * closure); - ChannelPtr consumer_channel; NATSHandler & event_handler; /// Used concurrently, but is thread safe. - std::vector queues; + std::shared_ptr connection; + std::vector subscriptions; + std::vector subjects; const String channel_base; - const size_t channel_id_base; +// const size_t channel_id_base; Poco::Logger * log; char row_delimiter; bool allowed = true; const std::atomic & stopped; String channel_id; - std::atomic channel_error = true, wait_subscription = false; +// std::atomic channel_error = true, wait_subscription = false; ConcurrentBoundedQueue received; MessageData current; - size_t subscribed = 0; +// size_t subscribed = 0; - AckTracker last_inserted_record_info; - UInt64 prev_tag = 0, channel_id_counter = 0; +// AckTracker last_inserted_record_info; +// UInt64 prev_tag = 0, channel_id_counter = 0; }; } diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 41ab2d1fc9b..2cfe9e8be8d 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include @@ -26,6 +25,18 @@ #include #include +//static void +//onMsg(natsConnection *, natsSubscription *, natsMsg *msg, void *) +//{ +// printf("\n\n\nReceived msg: %s - %.*s\n\n\n", +// natsMsg_GetSubject(msg), +// natsMsg_GetDataLength(msg), +// natsMsg_GetData(msg)); +// fflush(stdout); +// +// natsMsg_Destroy(msg); +//} + namespace DB { @@ -48,17 +59,6 @@ namespace ErrorCodes extern const int QUERY_NOT_ALLOWED; } -namespace ExchangeType -{ - /// Note that default here means default by implementation and not by nats settings - static const String DEFAULT = "default"; - static const String FANOUT = "fanout"; - static const String DIRECT = "direct"; - static const String TOPIC = "topic"; - static const String HASH = "consistent_hash"; - static const String HEADERS = "headers"; -} - StorageNATS::StorageNATS( const StorageID & table_id_, @@ -69,19 +69,13 @@ StorageNATS::StorageNATS( : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , nats_settings(std::move(nats_settings_)) - , exchange_name(getContext()->getMacros()->expand(nats_settings->nats_exchange_name)) + , subjects(parseSubjects(getContext()->getMacros()->expand(nats_settings->nats_subjects))) , format_name(getContext()->getMacros()->expand(nats_settings->nats_format)) - , exchange_type(defineExchangeType(getContext()->getMacros()->expand(nats_settings->nats_exchange_type))) - , routing_keys(parseSettings(getContext()->getMacros()->expand(nats_settings->nats_routing_key_list))) , row_delimiter(nats_settings->nats_row_delimiter.value) , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) , num_consumers(nats_settings->nats_num_consumers.value) - , num_queues(nats_settings->nats_num_queues.value) - , queue_base(getContext()->getMacros()->expand(nats_settings->nats_queue_base)) - , queue_settings_list(parseSettings(getContext()->getMacros()->expand(nats_settings->nats_queue_settings_list))) , persistent(nats_settings->nats_persistent.value) - , use_user_setup(nats_settings->nats_queue_consume.value) - , hash_exchange(num_consumers > 1 || num_queues > 1) +// , use_user_setup(nats_settings->nats_queue_consume.value) , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) , semaphore(0, num_consumers) , unique_strbase(getRandomName()) @@ -114,39 +108,56 @@ StorageNATS::StorageNATS( nats_context = addSettings(getContext()); nats_context->makeQueryContext(); - - if (queue_base.empty()) - { - /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to - * be table-based and not just a random string, because local exchanges should be declared the same for same tables - */ - sharding_exchange = getTableBasedName(exchange_name, table_id_); - - /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better - * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every - * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base - * for the names of later declared queues - */ - queue_base = getTableBasedName("", table_id_); - } - else - { - /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and - * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need to share - * sharding exchange and bridge exchange - */ - sharding_exchange = exchange_name + "_" + queue_base; - } - - bridge_exchange = sharding_exchange + "_bridge"; +// +// if (queue_base.empty()) +// { +// /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to +// * be table-based and not just a random string, because local exchanges should be declared the same for same tables +// */ +// sharding_exchange = getTableBasedName(exchange_name, table_id_); +// +// /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better +// * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every +// * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base +// * for the names of later declared queues +// */ +// queue_base = getTableBasedName("", table_id_); +// } +// else +// { +// /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and +// * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need to share +// * sharding exchange and bridge exchange +// */ +// sharding_exchange = exchange_name + "_" + queue_base; +// } +// +// bridge_exchange = sharding_exchange + "_bridge"; try { - connection = std::make_unique(configuration, log); - if (connection->connect()) - initNATS(); - else if (!is_attach) + connection = std::make_shared(configuration, log); + if (!connection->connect() && !is_attach) throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to {}", connection->connectionInfoForLog()); +// if (connection->connect()) +// initNATS(); +// else if (!is_attach) +// throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to {}", connection->connectionInfoForLog()); +// auto sub = connection->createSubscription("foo", onMsg, nullptr); +// int64_t n; +// connection->getHandler().startBlockingLoop(); +// while (true) { +// natsSubscription_GetDelivered(sub.get(), &n); +// printf("Read n : %ld\n", n); +// fflush(stdout); +// std::this_thread::sleep_for(std::chrono::milliseconds(500)); +// } +// auto t2 = std::thread([this](){ +// connection->getHandler().updateLoopState(Loop::RUN); +// LOG_DEBUG(log, "Storteng lup"); +// connection->getHandler().startLoop(); +// }); +// t2.join(); } catch (...) { @@ -162,17 +173,17 @@ StorageNATS::StorageNATS( streaming_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSStreamingTask", [this]{ streamingToViewsFunc(); }); streaming_task->deactivate(); - connection_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSConnectionTask", [this]{ connectionFunc(); }); + connection_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSConnectionManagerTask", [this]{ connectionFunc(); }); connection_task->deactivate(); } -Names StorageNATS::parseSettings(String settings_list) +Names StorageNATS::parseSubjects(String subjects_list) { Names result; - if (settings_list.empty()) + if (subjects_list.empty()) return result; - boost::split(result, settings_list, [](char c){ return c == ','; }); + boost::split(result, subjects_list, [](char c){ return c == ','; }); for (String & key : result) boost::trim(key); @@ -180,27 +191,6 @@ Names StorageNATS::parseSettings(String settings_list) } -AMQP::ExchangeType StorageNATS::defineExchangeType(String exchange_type_) -{ - AMQP::ExchangeType type; - if (exchange_type_ != ExchangeType::DEFAULT) - { - if (exchange_type_ == ExchangeType::FANOUT) type = AMQP::ExchangeType::fanout; - else if (exchange_type_ == ExchangeType::DIRECT) type = AMQP::ExchangeType::direct; - else if (exchange_type_ == ExchangeType::TOPIC) type = AMQP::ExchangeType::topic; - else if (exchange_type_ == ExchangeType::HASH) type = AMQP::ExchangeType::consistent_hash; - else if (exchange_type_ == ExchangeType::HEADERS) type = AMQP::ExchangeType::headers; - else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); - } - else - { - type = AMQP::ExchangeType::fanout; - } - - return type; -} - - String StorageNATS::getTableBasedName(String name, const StorageID & table_id) { if (name.empty()) @@ -260,7 +250,7 @@ void StorageNATS::stopLoopIfNoReaders() void StorageNATS::startLoop() { - assert(rabbit_is_ready); +// assert(nats_is_ready); connection->getHandler().updateLoopState(Loop::RUN); looping_task->activateAndSchedule(); } @@ -280,13 +270,15 @@ void StorageNATS::decrementReader() void StorageNATS::connectionFunc() { - if (rabbit_is_ready) - return; +// if (nats_is_ready) +// return; - if (connection->reconnect()) - initNATS(); - else + if (!connection->reconnect()) connection_task->scheduleAfter(RESCHEDULE_MS); +// if (connection->reconnect()) +// initNATS(); +// else +// connection_task->scheduleAfter(RESCHEDULE_MS); } @@ -320,332 +312,332 @@ size_t StorageNATS::getMaxBlockSize() const } -void StorageNATS::initNATS() -{ - if (shutdown_called || rabbit_is_ready) - return; - - if (use_user_setup) - { - queues.emplace_back(queue_base); - rabbit_is_ready = true; - return; - } - - try - { - auto rabbit_channel = connection->createChannel(); - - /// Main exchange -> Bridge exchange -> ( Sharding exchange ) -> Queues -> Consumers - - initExchange(*rabbit_channel); - bindExchange(*rabbit_channel); - - for (const auto i : collections::range(0, num_queues)) - bindQueue(i + 1, *rabbit_channel); - - LOG_TRACE(log, "NATS setup completed"); - rabbit_is_ready = true; - rabbit_channel->close(); - } - catch (...) - { - tryLogCurrentException(log); - if (!is_attach) - throw; - } -} - - -void StorageNATS::initExchange(AMQP::TcpChannel & rabbit_channel) -{ - /// Exchange hierarchy: - /// 1. Main exchange (defined with table settings - nats_exchange_name, nats_exchange_type). - /// 2. Bridge exchange (fanout). Used to easily disconnect main exchange and to simplify queue bindings. - /// 3. Sharding (or hash) exchange. Used in case of multiple queues. - /// 4. Consumer exchange. Just an alias for bridge_exchange or sharding exchange to know to what exchange - /// queues will be bound. - - /// All exchanges are declared with options: - /// 1. `durable` (survive NATS server restart) - /// 2. `autodelete` (auto delete in case of queue bindings are dropped). - - rabbit_channel.declareExchange(exchange_name, exchange_type, AMQP::durable) - .onError([&](const char * message) - { - /// This error can be a result of attempt to declare exchange if it was already declared but - /// 1) with different exchange type. - /// 2) with different exchange settings. - throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " - + std::string(message), ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE); - }); - - rabbit_channel.declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable | AMQP::autodelete) - .onError([&](const char * message) - { - /// This error is not supposed to happen as this exchange name is always unique to type and its settings. - throw Exception( - ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, "Unable to declare bridge exchange ({}). Reason: {}", bridge_exchange, std::string(message)); - }); - - if (!hash_exchange) - { - consumer_exchange = bridge_exchange; - return; - } - - AMQP::Table binding_arguments; - - /// Default routing key property in case of hash exchange is a routing key, which is required to be an integer. - /// Support for arbitrary exchange type (i.e. arbitrary pattern of routing keys) requires to eliminate this dependency. - /// This settings changes hash property to message_id. - binding_arguments["hash-property"] = "message_id"; - - /// Declare hash exchange for sharding. - rabbit_channel.declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable | AMQP::autodelete, binding_arguments) - .onError([&](const char * message) - { - /// This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared - /// to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter - /// is bad. - throw Exception( - ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, - "Unable to declare sharding exchange ({}). Reason: {}", sharding_exchange, std::string(message)); - }); - - rabbit_channel.bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) - .onError([&](const char * message) - { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind bridge exchange ({}) to sharding exchange ({}). Reason: {}", - bridge_exchange, - sharding_exchange, - std::string(message)); - }); - - consumer_exchange = sharding_exchange; -} - - -void StorageNATS::bindExchange(AMQP::TcpChannel & rabbit_channel) -{ - size_t bound_keys = 0; - - if (exchange_type == AMQP::ExchangeType::headers) - { - AMQP::Table bind_headers; - for (const auto & header : routing_keys) - { - std::vector matching; - boost::split(matching, header, [](char c){ return c == '='; }); - bind_headers[matching[0]] = matching[1]; - } - - rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) - .onSuccess([&]() { connection->getHandler().stopLoop(); }) - .onError([&](const char * message) - { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", - exchange_name, bridge_exchange, std::string(message)); - }); - } - else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) - { - rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0]) - .onSuccess([&]() { connection->getHandler().stopLoop(); }) - .onError([&](const char * message) - { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", - exchange_name, bridge_exchange, std::string(message)); - }); - } - else - { - for (const auto & routing_key : routing_keys) - { - rabbit_channel.bindExchange(exchange_name, bridge_exchange, routing_key) - .onSuccess([&]() - { - ++bound_keys; - if (bound_keys == routing_keys.size()) - connection->getHandler().stopLoop(); - }) - .onError([&](const char * message) - { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", - exchange_name, bridge_exchange, std::string(message)); - }); - } - } - - connection->getHandler().startBlockingLoop(); -} - - -void StorageNATS::bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel) -{ - auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) - { - queues.emplace_back(queue_name); - LOG_DEBUG(log, "Queue {} is declared", queue_name); - - if (msgcount) - LOG_INFO(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); - - /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are - * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for - * fanout exchange it can be arbitrary - */ - rabbit_channel.bindQueue(consumer_exchange, queue_name, std::to_string(queue_id)) - .onSuccess([&] { connection->getHandler().stopLoop(); }) - .onError([&](const char * message) - { - throw Exception( - ErrorCodes::CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, - "Failed to create queue binding for exchange {}. Reason: {}", exchange_name, std::string(message)); - }); - }; - - auto error_callback([&](const char * message) - { - /* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a - * given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different - * max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously - * declared queues via any of the various cli tools. - */ - throw Exception("Failed to declare queue. Probably queue settings are conflicting: max_block_size, deadletter_exchange. Attempt \ - specifying differently those settings or use a different queue_base or manually delete previously declared queues, \ - which were declared with the same names. ERROR reason: " - + std::string(message), ErrorCodes::BAD_ARGUMENTS); - }); - - AMQP::Table queue_settings; - - std::unordered_set integer_settings = {"x-max-length", "x-max-length-bytes", "x-message-ttl", "x-expires", "x-priority", "x-max-priority"}; - std::unordered_set string_settings = {"x-overflow", "x-dead-letter-exchange", "x-queue-type"}; - - /// Check user-defined settings. - if (!queue_settings_list.empty()) - { - for (const auto & setting : queue_settings_list) - { - Strings setting_values; - splitInto<'='>(setting_values, setting); - if (setting_values.size() != 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid settings string: {}", setting); - - String key = setting_values[0], value = setting_values[1]; - - if (integer_settings.contains(key)) - queue_settings[key] = parse(value); - else if (string_settings.find(key) != string_settings.end()) - queue_settings[key] = value; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported queue setting: {}", value); - } - } - - /// Impose default settings if there are no user-defined settings. - if (!queue_settings.contains("x-max-length")) - { - queue_settings["x-max-length"] = queue_size; - } - if (!queue_settings.contains("x-overflow")) - { - queue_settings["x-overflow"] = "reject-publish"; - } - - /// If queue_base - a single name, then it can be used as one specific queue, from which to read. - /// Otherwise it is used as a generator (unique for current table) of queue names, because it allows to - /// maximize performance - via setting `nats_num_queues`. - const String queue_name = !hash_exchange ? queue_base : std::to_string(queue_id) + "_" + queue_base; - - /// AMQP::autodelete setting is not allowed, because in case of server restart there will be no consumers - /// and deleting queues should not take place. - rabbit_channel.declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); - connection->getHandler().startBlockingLoop(); -} - - -bool StorageNATS::updateChannel(ChannelPtr & channel) -{ - try - { - channel = connection->createChannel(); - return true; - } - catch (...) - { - tryLogCurrentException(log); - return false; - } -} - - -void StorageNATS::prepareChannelForBuffer(ConsumerBufferPtr buffer) -{ - if (!buffer) - return; - - if (buffer->queuesCount() != queues.size()) - buffer->updateQueues(queues); - - buffer->updateAckTracker(); - - if (updateChannel(buffer->getChannel())) - buffer->setupChannel(); -} - - -void StorageNATS::unbindExchange() -{ - /* This is needed because with NATS (without special adjustments) can't, for example, properly make mv if there was insert query - * on the same table before, and in another direction it will make redundant copies, but most likely nobody will do that. - * As publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in - * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as - * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible - * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local - * queues, then messages will go both ways and in one of them they will remain not consumed. So need to disconnect local exchange - * bindings to remove redunadant message copies, but after that mv cannot work unless those bindings are recreated. Recreating them is - * not difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. - */ - if (!exchange_removed.exchange(true)) - { - try - { - streaming_task->deactivate(); - - stopLoop(); - looping_task->deactivate(); - - auto rabbit_channel = connection->createChannel(); - rabbit_channel->removeExchange(bridge_exchange) - .onSuccess([&]() - { - connection->getHandler().stopLoop(); - }) - .onError([&](const char * message) - { - throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_REMOVE_RABBITMQ_EXCHANGE); - }); - - connection->getHandler().startBlockingLoop(); - rabbit_channel->close(); - } - catch (...) - { - exchange_removed = false; - throw; - } - } -} +//void StorageNATS::initNATS() +//{ +// if (shutdown_called || nats_is_ready) +// return; +// +// if (use_user_setup) +// { +// queues.emplace_back(queue_base); +// nats_is_ready = true; +// return; +// } +// +// try +// { +// auto nats_channel = connection->createChannel(); +// +// /// Main exchange -> Bridge exchange -> ( Sharding exchange ) -> Queues -> Consumers +// +// initExchange(*nats_channel); +// bindExchange(*nats_channel); +// +// for (const auto i : collections::range(0, num_queues)) +// bindQueue(i + 1, *nats_channel); +// +// LOG_TRACE(log, "NATS setup completed"); +// nats_is_ready = true; +// nats_channel->close(); +// } +// catch (...) +// { +// tryLogCurrentException(log); +// if (!is_attach) +// throw; +// } +//} +// +// +//void StorageNATS::initExchange(AMQP::TcpChannel & nats_channel) +//{ +// /// Exchange hierarchy: +// /// 1. Main exchange (defined with table settings - nats_exchange_name, nats_exchange_type). +// /// 2. Bridge exchange (fanout). Used to easily disconnect main exchange and to simplify queue bindings. +// /// 3. Sharding (or hash) exchange. Used in case of multiple queues. +// /// 4. Consumer exchange. Just an alias for bridge_exchange or sharding exchange to know to what exchange +// /// queues will be bound. +// +// /// All exchanges are declared with options: +// /// 1. `durable` (survive NATS server restart) +// /// 2. `autodelete` (auto delete in case of queue bindings are dropped). +// +// nats_channel.declareExchange(exchange_name, exchange_type, AMQP::durable) +// .onError([&](const char * message) +// { +// /// This error can be a result of attempt to declare exchange if it was already declared but +// /// 1) with different exchange type. +// /// 2) with different exchange settings. +// throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " +// + std::string(message), ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE); +// }); +// +// nats_channel.declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable | AMQP::autodelete) +// .onError([&](const char * message) +// { +// /// This error is not supposed to happen as this exchange name is always unique to type and its settings. +// throw Exception( +// ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, "Unable to declare bridge exchange ({}). Reason: {}", bridge_exchange, std::string(message)); +// }); +// +// if (!hash_exchange) +// { +// consumer_exchange = bridge_exchange; +// return; +// } +// +// AMQP::Table binding_arguments; +// +// /// Default routing key property in case of hash exchange is a routing key, which is required to be an integer. +// /// Support for arbitrary exchange type (i.e. arbitrary pattern of routing keys) requires to eliminate this dependency. +// /// This settings changes hash property to message_id. +// binding_arguments["hash-property"] = "message_id"; +// +// /// Declare hash exchange for sharding. +// nats_channel.declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable | AMQP::autodelete, binding_arguments) +// .onError([&](const char * message) +// { +// /// This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared +// /// to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter +// /// is bad. +// throw Exception( +// ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, +// "Unable to declare sharding exchange ({}). Reason: {}", sharding_exchange, std::string(message)); +// }); +// +// nats_channel.bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) +// .onError([&](const char * message) +// { +// throw Exception( +// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, +// "Unable to bind bridge exchange ({}) to sharding exchange ({}). Reason: {}", +// bridge_exchange, +// sharding_exchange, +// std::string(message)); +// }); +// +// consumer_exchange = sharding_exchange; +//} +// +// +//void StorageNATS::bindExchange(AMQP::TcpChannel & nats_channel) +//{ +// size_t bound_keys = 0; +// +// if (exchange_type == AMQP::ExchangeType::headers) +// { +// AMQP::Table bind_headers; +// for (const auto & header : routing_keys) +// { +// std::vector matching; +// boost::split(matching, header, [](char c){ return c == '='; }); +// bind_headers[matching[0]] = matching[1]; +// } +// +// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) +// .onSuccess([&]() { connection->getHandler().stopLoop(); }) +// .onError([&](const char * message) +// { +// throw Exception( +// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, +// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", +// exchange_name, bridge_exchange, std::string(message)); +// }); +// } +// else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) +// { +// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0]) +// .onSuccess([&]() { connection->getHandler().stopLoop(); }) +// .onError([&](const char * message) +// { +// throw Exception( +// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, +// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", +// exchange_name, bridge_exchange, std::string(message)); +// }); +// } +// else +// { +// for (const auto & routing_key : routing_keys) +// { +// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_key) +// .onSuccess([&]() +// { +// ++bound_keys; +// if (bound_keys == routing_keys.size()) +// connection->getHandler().stopLoop(); +// }) +// .onError([&](const char * message) +// { +// throw Exception( +// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, +// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", +// exchange_name, bridge_exchange, std::string(message)); +// }); +// } +// } +// +// connection->getHandler().startBlockingLoop(); +//} +// +// +//void StorageNATS::bindQueue(size_t queue_id, AMQP::TcpChannel & nats_channel) +//{ +// auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) +// { +// queues.emplace_back(queue_name); +// LOG_DEBUG(log, "Queue {} is declared", queue_name); +// +// if (msgcount) +// LOG_INFO(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); +// +// /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are +// * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for +// * fanout exchange it can be arbitrary +// */ +// nats_channel.bindQueue(consumer_exchange, queue_name, std::to_string(queue_id)) +// .onSuccess([&] { connection->getHandler().stopLoop(); }) +// .onError([&](const char * message) +// { +// throw Exception( +// ErrorCodes::CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, +// "Failed to create queue binding for exchange {}. Reason: {}", exchange_name, std::string(message)); +// }); +// }; +// +// auto error_callback([&](const char * message) +// { +// /* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a +// * given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different +// * max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously +// * declared queues via any of the various cli tools. +// */ +// throw Exception("Failed to declare queue. Probably queue settings are conflicting: max_block_size, deadletter_exchange. Attempt \ +// specifying differently those settings or use a different queue_base or manually delete previously declared queues, \ +// which were declared with the same names. ERROR reason: " +// + std::string(message), ErrorCodes::BAD_ARGUMENTS); +// }); +// +// AMQP::Table queue_settings; +// +// std::unordered_set integer_settings = {"x-max-length", "x-max-length-bytes", "x-message-ttl", "x-expires", "x-priority", "x-max-priority"}; +// std::unordered_set string_settings = {"x-overflow", "x-dead-letter-exchange", "x-queue-type"}; +// +// /// Check user-defined settings. +// if (!queue_settings_list.empty()) +// { +// for (const auto & setting : queue_settings_list) +// { +// Strings setting_values; +// splitInto<'='>(setting_values, setting); +// if (setting_values.size() != 2) +// throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid settings string: {}", setting); +// +// String key = setting_values[0], value = setting_values[1]; +// +// if (integer_settings.contains(key)) +// queue_settings[key] = parse(value); +// else if (string_settings.find(key) != string_settings.end()) +// queue_settings[key] = value; +// else +// throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported queue setting: {}", value); +// } +// } +// +// /// Impose default settings if there are no user-defined settings. +// if (!queue_settings.contains("x-max-length")) +// { +// queue_settings["x-max-length"] = queue_size; +// } +// if (!queue_settings.contains("x-overflow")) +// { +// queue_settings["x-overflow"] = "reject-publish"; +// } +// +// /// If queue_base - a single name, then it can be used as one specific queue, from which to read. +// /// Otherwise it is used as a generator (unique for current table) of queue names, because it allows to +// /// maximize performance - via setting `nats_num_queues`. +// const String queue_name = !hash_exchange ? queue_base : std::to_string(queue_id) + "_" + queue_base; +// +// /// AMQP::autodelete setting is not allowed, because in case of server restart there will be no consumers +// /// and deleting queues should not take place. +// nats_channel.declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); +// connection->getHandler().startBlockingLoop(); +//} +// +// +//bool StorageNATS::updateChannel(ChannelPtr & channel) +//{ +// try +// { +// channel = connection->createChannel(); +// return true; +// } +// catch (...) +// { +// tryLogCurrentException(log); +// return false; +// } +//} +// +// +//void StorageNATS::prepareChannelForBuffer(ConsumerBufferPtr buffer) +//{ +// if (!buffer) +// return; +// +// if (buffer->queuesCount() != queues.size()) +// buffer->updateQueues(queues); +// +// buffer->updateAckTracker(); +// +// if (updateChannel(buffer->getChannel())) +// buffer->setupChannel(); +//} +// +// +//void StorageNATS::unbindExchange() +//{ +// /* This is needed because with NATS (without special adjustments) can't, for example, properly make mv if there was insert query +// * on the same table before, and in another direction it will make redundant copies, but most likely nobody will do that. +// * As publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in +// * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as +// * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible +// * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local +// * queues, then messages will go both ways and in one of them they will remain not consumed. So need to disconnect local exchange +// * bindings to remove redunadant message copies, but after that mv cannot work unless those bindings are recreated. Recreating them is +// * not difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. +// */ +// if (!exchange_removed.exchange(true)) +// { +// try +// { +// streaming_task->deactivate(); +// +// stopLoop(); +// looping_task->deactivate(); +// +// auto nats_channel = connection->createChannel(); +// nats_channel->removeExchange(bridge_exchange) +// .onSuccess([&]() +// { +// connection->getHandler().stopLoop(); +// }) +// .onError([&](const char * message) +// { +// throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_REMOVE_RABBITMQ_EXCHANGE); +// }); +// +// connection->getHandler().startBlockingLoop(); +// nats_channel->close(); +// } +// catch (...) +// { +// exchange_removed = false; +// throw; +// } +// } +//} Pipe StorageNATS::read( @@ -657,8 +649,8 @@ Pipe StorageNATS::read( size_t /* max_block_size */, unsigned /* num_streams */) { - if (!rabbit_is_ready) - throw Exception("NATS setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_RABBITMQ); +// if (!nats_is_ready) +// throw Exception("NATS setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_RABBITMQ); if (num_created_consumers == 0) return {}; @@ -682,25 +674,25 @@ Pipe StorageNATS::read( throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "No connection to {}", connection->connectionInfoForLog()); } - initializeBuffers(); +// initializeBuffers(); Pipes pipes; pipes.reserve(num_created_consumers); for (size_t i = 0; i < num_created_consumers; ++i) { - auto rabbit_source = std::make_shared( + auto nats_source = std::make_shared( *this, storage_snapshot, modified_context, column_names, 1, nats_settings->nats_commit_on_select); auto converting_dag = ActionsDAG::makeConvertingActions( - rabbit_source->getPort().getHeader().getColumnsWithTypeAndName(), + nats_source->getPort().getHeader().getColumnsWithTypeAndName(), sample_block.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); auto converting = std::make_shared(std::move(converting_dag)); - auto converting_transform = std::make_shared(rabbit_source->getPort().getHeader(), std::move(converting)); + auto converting_transform = std::make_shared(nats_source->getPort().getHeader(), std::move(converting)); - pipes.emplace_back(std::move(rabbit_source)); + pipes.emplace_back(std::move(nats_source)); pipes.back().addTransform(std::move(converting_transform)); } @@ -722,13 +714,14 @@ SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & m void StorageNATS::startup() { - if (!rabbit_is_ready) + if (!nats_is_ready) { if (connection->isConnected()) { try { - initNATS(); +// initNATS(); + LOG_DEBUG(log, "Fake init lul"); } catch (...) { @@ -783,11 +776,13 @@ void StorageNATS::shutdown() for (auto & buffer : buffers) buffer->closeChannel(); - cleanupNATS(); +// cleanupNATS(); } /// It is important to close connection here - before removing consumer buffers, because /// it will finish and clean callbacks, which might use those buffers data. + if (connection->getHandler().loopRunning()) + stopLoop(); connection->disconnect(); for (size_t i = 0; i < num_created_consumers; ++i) @@ -802,53 +797,53 @@ void StorageNATS::shutdown() /// The only thing publishers are supposed to be aware of is _exchanges_ and queues are a responsibility of a consumer. /// Therefore, if a table is dropped, a clean up is needed. -void StorageNATS::cleanupNATS() const -{ - if (use_user_setup) - return; - - connection->heartbeat(); - if (!connection->isConnected()) - { - String queue_names; - for (const auto & queue : queues) - { - if (!queue_names.empty()) - queue_names += ", "; - queue_names += queue; - } - LOG_WARNING(log, - "NATS clean up not done, because there is no connection in table's shutdown." - "There are {} queues ({}), which might need to be deleted manually. Exchanges will be auto-deleted", - queues.size(), queue_names); - return; - } - - auto rabbit_channel = connection->createChannel(); - for (const auto & queue : queues) - { - /// AMQP::ifunused is needed, because it is possible to share queues between multiple tables and dropping - /// on of them should not affect others. - /// AMQP::ifempty is not used on purpose. - - rabbit_channel->removeQueue(queue, AMQP::ifunused) - .onSuccess([&](uint32_t num_messages) - { - LOG_TRACE(log, "Successfully deleted queue {}, messages contained {}", queue, num_messages); - connection->getHandler().stopLoop(); - }) - .onError([&](const char * message) - { - LOG_ERROR(log, "Failed to delete queue {}. Error message: {}", queue, message); - connection->getHandler().stopLoop(); - }); - } - connection->getHandler().startBlockingLoop(); - rabbit_channel->close(); - - /// Also there is no need to cleanup exchanges as they were created with AMQP::autodelete option. Once queues - /// are removed, exchanges will also be cleaned. -} +//void StorageNATS::cleanupNATS() const +//{ +// if (use_user_setup) +// return; +// +// connection->heartbeat(); +// if (!connection->isConnected()) +// { +// String queue_names; +// for (const auto & queue : queues) +// { +// if (!queue_names.empty()) +// queue_names += ", "; +// queue_names += queue; +// } +// LOG_WARNING(log, +// "NATS clean up not done, because there is no connection in table's shutdown." +// "There are {} queues ({}), which might need to be deleted manually. Exchanges will be auto-deleted", +// queues.size(), queue_names); +// return; +// } +// +// auto nats_channel = connection->createChannel(); +// for (const auto & queue : queues) +// { +// /// AMQP::ifunused is needed, because it is possible to share queues between multiple tables and dropping +// /// on of them should not affect others. +// /// AMQP::ifempty is not used on purpose. +// +// nats_channel->removeQueue(queue, AMQP::ifunused) +// .onSuccess([&](uint32_t num_messages) +// { +// LOG_TRACE(log, "Successfully deleted queue {}, messages contained {}", queue, num_messages); +// connection->getHandler().stopLoop(); +// }) +// .onError([&](const char * message) +// { +// LOG_ERROR(log, "Failed to delete queue {}. Error message: {}", queue, message); +// connection->getHandler().stopLoop(); +// }); +// } +// connection->getHandler().startBlockingLoop(); +// nats_channel->close(); +// +// /// Also there is no need to cleanup exchanges as they were created with AMQP::autodelete option. Once queues +// /// are removed, exchanges will also be cleaned. +//} void StorageNATS::pushReadBuffer(ConsumerBufferPtr buffer) @@ -888,18 +883,18 @@ ConsumerBufferPtr StorageNATS::popReadBuffer(std::chrono::milliseconds timeout) ConsumerBufferPtr StorageNATS::createReadBuffer() { return std::make_shared( - connection->getHandler(), queues, ++consumer_id, + connection->getHandler(), connection, subjects, unique_strbase, log, row_delimiter, queue_size, shutdown_called); } -ProducerBufferPtr StorageNATS::createWriteBuffer() -{ - return std::make_shared( - configuration, getContext(), routing_keys, exchange_name, exchange_type, - producer_id.fetch_add(1), persistent, shutdown_called, log, - row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); -} +//ProducerBufferPtr StorageNATS::createWriteBuffer() +//{ +// return std::make_shared( +// configuration, getContext(), routing_keys, exchange_name, exchange_type, +// producer_id.fetch_add(1), persistent, shutdown_called, log, +// row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); +//} bool StorageNATS::checkDependencies(const StorageID & table_id) @@ -930,73 +925,69 @@ bool StorageNATS::checkDependencies(const StorageID & table_id) } -void StorageNATS::initializeBuffers() -{ - assert(rabbit_is_ready); - if (!initialized) - { - for (const auto & buffer : buffers) - prepareChannelForBuffer(buffer); - initialized = true; - } -} +//void StorageNATS::initializeBuffers() +//{ +// assert(nats_is_ready); +// if (!initialized) +// { +// for (const auto & buffer : buffers) +// prepareChannelForBuffer(buffer); +// initialized = true; +// } +//} void StorageNATS::streamingToViewsFunc() { - if (rabbit_is_ready) + try { - try + auto table_id = getStorageID(); + + // Check if at least one direct dependency is attached + size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + bool nats_connected = connection->isConnected() || connection->reconnect(); + + if (dependencies_count && nats_connected) { - auto table_id = getStorageID(); + auto start_time = std::chrono::steady_clock::now(); - // Check if at least one direct dependency is attached - size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); - bool rabbit_connected = connection->isConnected() || connection->reconnect(); + mv_attached.store(true); - if (dependencies_count && rabbit_connected) + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!shutdown_called && num_created_consumers > 0) { - initializeBuffers(); - auto start_time = std::chrono::steady_clock::now(); + if (!checkDependencies(table_id)) + break; - mv_attached.store(true); + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - // Keep streaming as long as there are attached views and streaming is not cancelled - while (!shutdown_called && num_created_consumers > 0) + if (streamToViews()) { - if (!checkDependencies(table_id)) - break; + /// Reschedule with backoff. + if (milliseconds_to_wait < BACKOFF_TRESHOLD) + milliseconds_to_wait *= 2; + stopLoopIfNoReaders(); + break; + } + else + { + milliseconds_to_wait = RESCHEDULE_MS; + } - LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - - if (streamToViews()) - { - /// Reschedule with backoff. - if (milliseconds_to_wait < BACKOFF_TRESHOLD) - milliseconds_to_wait *= 2; - stopLoopIfNoReaders(); - break; - } - else - { - milliseconds_to_wait = RESCHEDULE_MS; - } - - auto end_time = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - if (duration.count() > MAX_THREAD_WORK_DURATION_MS) - { - stopLoopIfNoReaders(); - LOG_TRACE(log, "Reschedule streaming. Thread work duration limit exceeded."); - break; - } + auto end_time = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + if (duration.count() > MAX_THREAD_WORK_DURATION_MS) + { + stopLoopIfNoReaders(); + LOG_TRACE(log, "Reschedule streaming. Thread work duration limit exceeded."); + break; } } } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } mv_attached.store(false); @@ -1080,9 +1071,7 @@ bool StorageNATS::streamToViews() if (connection->reconnect()) { - LOG_DEBUG(log, "Connection restored, updating channels"); - for (auto & source : sources) - source->updateChannel(); + LOG_DEBUG(log, "Connection restored"); } else { @@ -1098,39 +1087,13 @@ bool StorageNATS::streamToViews() if (source->queueEmpty()) ++queue_empty; - if (source->needChannelUpdate()) - { - auto buffer = source->getBuffer(); - prepareChannelForBuffer(buffer); - } - - /* false is returned by the sendAck function in only two cases: - * 1) if connection failed. In this case all channels will be closed and will be unable to send ack. Also ack is made based on - * delivery tags, which are unique to channels, so if channels fail, those delivery tags will become invalid and there is - * no way to send specific ack from a different channel. Actually once the server realises that it has messages in a queue - * waiting for confirm from a channel which suddenly closed, it will immediately make those messages accessible to other - * consumers. So in this case duplicates are inevitable. - * 2) size of the sent frame (libraries's internal request interface) exceeds max frame - internal library error. This is more - * common for message frames, but not likely to happen to ack frame I suppose. So I do not believe it is likely to happen. - * Also in this case if channel didn't get closed - it is ok if failed to send ack, because the next attempt to send ack on - * the same channel will also commit all previously not-committed messages. Anyway I do not think that for ack frame this - * will ever happen. - */ - if (!source->sendAck()) - { - /// Iterate loop to activate error callbacks if they happened - connection->getHandler().iterateLoop(); - if (!connection->isConnected()) - break; - } - connection->getHandler().iterateLoop(); } } if ((queue_empty == num_created_consumers) && (++read_attempts == MAX_FAILED_READ_ATTEMPTS)) { - connection->heartbeat(); +// connection->heartbeat(); read_attempts = 0; LOG_TRACE(log, "Reschedule streaming. Queues are empty."); return true; @@ -1174,11 +1137,7 @@ void registerStorageNATS(StorageFactory & factory) NamesAndTypesList StorageNATS::getVirtuals() const { return NamesAndTypesList{ - {"_exchange_name", std::make_shared()}, - {"_channel_id", std::make_shared()}, - {"_delivery_tag", std::make_shared()}, - {"_redelivered", std::make_shared()}, - {"_message_id", std::make_shared()}, + {"_subject", std::make_shared()}, {"_timestamp", std::make_shared()} }; } diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index e5b397194b7..2630df33ae4 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -10,7 +10,6 @@ #include #include #include -#include #include #include @@ -61,12 +60,12 @@ public: const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; - String getExchange() const { return exchange_name; } - void unbindExchange(); - - bool updateChannel(ChannelPtr & channel); - void updateQueues(std::vector & queues_) { queues_ = queues; } - void prepareChannelForBuffer(ConsumerBufferPtr buffer); +// String getExchange() const { return exchange_name; } +// void unbindExchange(); +// +// bool updateChannel(ChannelPtr & channel); + void updateSubjects(std::vector & subjects_) { subjects_ = subjects; } +// void prepareChannelForBuffer(ConsumerBufferPtr buffer); void incrementReader(); void decrementReader(); @@ -82,30 +81,25 @@ protected: private: ContextMutablePtr nats_context; std::unique_ptr nats_settings; + std::vector subjects; - const String exchange_name; const String format_name; - AMQP::ExchangeType exchange_type; - Names routing_keys; char row_delimiter; const String schema_name; size_t num_consumers; - size_t num_queues; - String queue_base; - Names queue_settings_list; /// For insert query. Mark messages as durable. const bool persistent; - /// A table setting. It is possible not to perform any NATS setup, which is supposed to be consumer-side setup: - /// declaring exchanges, queues, bindings. Instead everything needed from NATS table is to connect to a specific queue. - /// This solution disables all optimizations and is not really optimal, but allows user to fully control all NATS setup. - bool use_user_setup; +// /// A table setting. It is possible not to perform any NATS setup, which is supposed to be consumer-side setup: +// /// declaring exchanges, queues, bindings. Instead everything needed from NATS table is to connect to a specific queue. +// /// This solution disables all optimizations and is not really optimal, but allows user to fully control all NATS setup. +// bool use_user_setup; - bool hash_exchange; +// bool hash_exchange; Poco::Logger * log; - NATSConnectionPtr connection; /// Connection for all consumers + NATSConnectionManagerPtr connection; /// Connection for all consumers NATSConfiguration configuration; size_t num_created_consumers = 0; @@ -119,11 +113,8 @@ private: /// to setup size of inner buffer for received messages uint32_t queue_size; - String sharding_exchange, bridge_exchange, consumer_exchange; size_t consumer_id = 0; /// counter for consumer buffer, needed for channel id - std::vector queues; - std::once_flag flag; /// remove exchange only once std::mutex task_mutex; BackgroundSchedulePool::TaskHolder streaming_task; @@ -143,7 +134,7 @@ private: std::atomic producer_id = 1; /// Has connection background task completed successfully? /// It is started only once -- in constructor. - std::atomic rabbit_is_ready = false; + std::atomic nats_is_ready = false; /// Allow to remove exchange only once. std::atomic exchange_removed = false; /// For select query we must be aware of the end of streaming @@ -176,7 +167,7 @@ private: void stopLoop(); void stopLoopIfNoReaders(); - static Names parseSettings(String settings_list); + static Names parseSubjects(String subjects_list); static AMQP::ExchangeType defineExchangeType(String exchange_type_); static String getTableBasedName(String name, const StorageID & table_id); @@ -184,12 +175,12 @@ private: size_t getMaxBlockSize() const; void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); - void initNATS(); - void cleanupNATS() const; +// void initNATS(); +// void cleanupNATS() const; - void initExchange(AMQP::TcpChannel & rabbit_channel); - void bindExchange(AMQP::TcpChannel & rabbit_channel); - void bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel); +// void initExchange(AMQP::TcpChannel & nats_channel); +// void bindExchange(AMQP::TcpChannel & nats_channel); +// void bindQueue(size_t queue_id, AMQP::TcpChannel & nats_channel); bool streamToViews(); bool checkDependencies(const StorageID & table_id); diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index 2f4ea7bee53..175d9458423 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -112,53 +112,53 @@ void WriteBufferToNATSProducer::countRow() void WriteBufferToNATSProducer::setupChannel() { - producer_channel = connection.createChannel(); - - producer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Producer's channel {} error: {}", channel_id, message); - - /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) - producer_channel->close(); - - /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once - * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid - */ - for (const auto & record : delivery_record) - if (!returned.push(record.second)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); - - LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", - channel_id, delivery_record.size(), payloads.size()); - - /// Delivery tags are scoped per channel. - delivery_record.clear(); - delivery_tag = 0; - producer_ready = false; - }); - - producer_channel->onReady([&]() - { - channel_id = channel_id_base + "_" + std::to_string(channel_id_counter++); - LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); - - /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, - * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times - * slower than the second, so default is second and the first is turned on in table setting. - * - * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html - */ - producer_channel->confirmSelect() - .onAck([&](uint64_t acked_delivery_tag, bool multiple) - { - removeRecord(acked_delivery_tag, multiple, false); - }) - .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) - { - removeRecord(nacked_delivery_tag, multiple, true); - }); - producer_ready = true; - }); +// producer_channel = connection.createChannel(); +// +// producer_channel->onError([&](const char * message) +// { +// LOG_ERROR(log, "Producer's channel {} error: {}", channel_id, message); +// +// /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) +// producer_channel->close(); +// +// /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once +// * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid +// */ +// for (const auto & record : delivery_record) +// if (!returned.push(record.second)) +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); +// +// LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", +// channel_id, delivery_record.size(), payloads.size()); +// +// /// Delivery tags are scoped per channel. +// delivery_record.clear(); +// delivery_tag = 0; +// producer_ready = false; +// }); +// +// producer_channel->onReady([&]() +// { +// channel_id = channel_id_base + "_" + std::to_string(channel_id_counter++); +// LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); +// +// /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, +// * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times +// * slower than the second, so default is second and the first is turned on in table setting. +// * +// * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html +// */ +// producer_channel->confirmSelect() +// .onAck([&](uint64_t acked_delivery_tag, bool multiple) +// { +// removeRecord(acked_delivery_tag, multiple, false); +// }) +// .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) +// { +// removeRecord(nacked_delivery_tag, multiple, true); +// }); +// producer_ready = true; +// }); } diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h index d0d80a6cf9d..f2e9e04e227 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.h +++ b/src/Storages/NATS/WriteBufferToNATSProducer.h @@ -49,7 +49,7 @@ private: void removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish); void publish(ConcurrentBoundedQueue> & message, bool republishing); - NATSConnection connection; + NATSConnectionManager connection; const Names routing_keys; const String exchange_name; @@ -69,7 +69,7 @@ private: bool producer_ready = false; /// Channel errors lead to channel closure, need to count number of recreated channels to update channel id - UInt64 channel_id_counter = 0; +// UInt64 channel_id_counter = 0; /// channel id which contains id of current producer buffer and serial number of recreated channel in this buffer String channel_id; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index f567bf6eefc..1e777cbf946 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -51,6 +51,8 @@ void registerStorageMySQL(StorageFactory & factory); void registerStorageMongoDB(StorageFactory & factory); +void registerStorageNATS(StorageFactory & factory); + #if USE_RDKAFKA void registerStorageKafka(StorageFactory & factory); #endif @@ -128,6 +130,8 @@ void registerStorages() registerStorageMongoDB(factory); + registerStorageNATS(factory); + #if USE_RDKAFKA registerStorageKafka(factory); #endif From 30ff0de33e2353af3057af62ade67b50ce290326 Mon Sep 17 00:00:00 2001 From: tchepavel Date: Tue, 10 May 2022 16:00:07 +0300 Subject: [PATCH 007/627] Change submodule to patched --- .gitmodules | 2 +- contrib/nats-io | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index 412fefea899..a2211379314 100644 --- a/.gitmodules +++ b/.gitmodules @@ -267,4 +267,4 @@ url = https://github.com/wangyi-fudan/wyhash.git [submodule "contrib/nats-io"] path = contrib/nats-io - url = https://github.com/nats-io/nats.c.git + url = https://github.com/tchepavel/nats.c.git diff --git a/contrib/nats-io b/contrib/nats-io index d1f59f7bcf8..1eefa3aad67 160000 --- a/contrib/nats-io +++ b/contrib/nats-io @@ -1 +1 @@ -Subproject commit d1f59f7bcf8465526f7e6d9c99982cbd6b209547 +Subproject commit 1eefa3aad671d71a5c18815af1f9f4bdbaee7ec2 From 5f8fcb748feb363aa3d46f5bd23f070f1ede3def Mon Sep 17 00:00:00 2001 From: tchepavel Date: Tue, 10 May 2022 18:39:57 +0300 Subject: [PATCH 008/627] Select, MV, Insert working --- base/harmful/harmful.c | 2 +- contrib/nats-io | 2 +- src/Common/ErrorCodes.cpp | 1 + src/Storages/NATS/NATSConnection.cpp | 28 +- src/Storages/NATS/NATSConnection.h | 9 +- src/Storages/NATS/NATSHandler.cpp | 2 +- src/Storages/NATS/NATSSettings.h | 5 +- src/Storages/NATS/NATSSink.cpp | 5 +- .../NATS/ReadBufferFromNATSConsumer.cpp | 6 +- src/Storages/NATS/StorageNATS.cpp | 457 +----------------- src/Storages/NATS/StorageNATS.h | 19 +- .../NATS/WriteBufferToNATSProducer.cpp | 206 ++------ src/Storages/NATS/WriteBufferToNATSProducer.h | 49 +- 13 files changed, 102 insertions(+), 689 deletions(-) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index adc3f7158f6..5a27cae0383 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -206,7 +206,7 @@ TRAP(lgammal) TRAP(nftw) TRAP(nl_langinfo) TRAP(putc_unlocked) -//TRAP(rand) Used by nats-io at startup +TRAP(rand) /** In the current POSIX.1 specification (POSIX.1-2008), readdir() is not required to be thread-safe. However, in modern * implementations (including the glibc implementation), concurrent calls to readdir() that specify different directory streams * are thread-safe. In cases where multiple threads must read from the same directory stream, using readdir() with external diff --git a/contrib/nats-io b/contrib/nats-io index 1eefa3aad67..6b2227f3675 160000 --- a/contrib/nats-io +++ b/contrib/nats-io @@ -1 +1 @@ -Subproject commit 1eefa3aad671d71a5c18815af1f9f4bdbaee7ec2 +Subproject commit 6b2227f36757da090321e2d317569d2bd42c4cc1 diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index eb84e24b713..2644fa3be95 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -624,6 +624,7 @@ M(653, CANNOT_PARSE_BACKUP_SETTINGS) \ M(654, WRONG_BACKUP_SETTINGS) \ M(655, FAILED_TO_RESTORE_METADATA_ON_OTHER_NODE) \ + M(656, CANNOT_CONNECT_NATS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index cef8a2eb8c1..c76583dae5f 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -1,6 +1,6 @@ #include "NATSConnection.h" -#include +#include #include @@ -58,8 +58,14 @@ SubscriptionPtr NATSConnectionManager::createSubscription(const std::string& sub if (status == NATS_OK) status = natsSubscription_SetPendingLimits(ns, -1, -1); if (status == NATS_OK) + { LOG_DEBUG(log, "Subscribed to subject {}", subject); - return SubscriptionPtr(ns, &natsSubscription_Destroy); + return SubscriptionPtr(ns, &natsSubscription_Destroy); + } + else + { + return SubscriptionPtr(nullptr, &natsSubscription_Destroy); + } } void NATSConnectionManager::disconnect() @@ -96,6 +102,10 @@ void NATSConnectionManager::connectImpl() { address = configuration.connection_string; } + natsOptions_SetMaxReconnect(options, configuration.max_reconnect); + natsOptions_SetReconnectWait(options, configuration.reconnect_wait); +// natsOptions_SetDisconnectedCB(options, disconnectedCallback, this); +// natsOptions_SetReconnectedCB(options, reconnectedCallback, this); natsOptions_SetURL(options, address.c_str()); status = natsConnection_Connect(&connection, options); if (status != NATS_OK) @@ -121,4 +131,18 @@ void NATSConnectionManager::disconnectImpl() event_handler.changeConnectionStatus(false); } +void NATSConnectionManager::reconnectedCallback(natsConnection * nc, void * manager) +{ + char buffer[64]; + + buffer[0] = '\0'; + natsConnection_GetConnectedUrl(nc, buffer, sizeof(buffer)); + LOG_DEBUG(static_cast(manager)->log, "Got reconnected to NATS server: {}.", buffer); +} + +void NATSConnectionManager::disconnectedCallback(natsConnection *, void * manager) +{ + LOG_DEBUG(static_cast(manager)->log, "Got disconnected from NATS server."); +} + } diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h index ab97012061c..813115454e7 100644 --- a/src/Storages/NATS/NATSConnection.h +++ b/src/Storages/NATS/NATSConnection.h @@ -13,7 +13,9 @@ struct NATSConfiguration UInt16 port; String username; String password; - String vhost; + + int max_reconnect; + int reconnect_wait; bool secure; String connection_string; @@ -26,6 +28,8 @@ public: NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_); ~NATSConnectionManager() { natsConnection_Destroy(connection); } + natsConnection * getConnection() { return connection; } + bool isConnected(); bool connect(); @@ -50,6 +54,9 @@ private: void disconnectImpl(); + static void disconnectedCallback(natsConnection * nc, void * storage); + static void reconnectedCallback(natsConnection * nc, void * storage); + NATSConfiguration configuration; Poco::Logger * log; diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index cb155522a6a..cdaa93ef93c 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h index c30869e8ea3..2cb5365a13b 100644 --- a/src/Storages/NATS/NATSSettings.h +++ b/src/Storages/NATS/NATSSettings.h @@ -17,14 +17,15 @@ namespace DB M(UInt64, nats_num_consumers, 1, "The number of consumer channels per table.", 0) \ M(Bool, nats_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \ M(Bool, nats_secure, false, "Use SSL connection", 0) \ + M(UInt64, nats_max_reconnect, 5, "Maximum amount of reconnection attempts.", 0) \ + M(UInt64, nats_reconnect_wait, 2000, "Amount to sleep between each reconnect attempt.", 0) \ + M(String, nats_password, "", "NATS password", 0) \ M(String, nats_address, "", "Address for connection", 0) \ M(UInt64, nats_skip_broken_messages, 0, "Skip at least this number of broken messages from NATS per block", 0) \ M(UInt64, nats_max_block_size, 0, "Number of row collected before flushing data from NATS.", 0) \ M(Milliseconds, nats_flush_interval_ms, 0, "Timeout for flushing data from NATS.", 0) \ - M(String, nats_vhost, "/", "NATS vhost.", 0) \ M(Bool, nats_queue_consume, false, "Use user-defined queues and do not make any NATS setup: declaring exchanges, queues, bindings", 0) \ M(String, nats_username, "", "NATS username", 0) \ - M(String, nats_password, "", "NATS password", 0) \ M(Bool, nats_commit_on_select, false, "Commit messages when select query is made", 0) \ #define LIST_OF_NATS_SETTINGS(M) \ diff --git a/src/Storages/NATS/NATSSink.cpp b/src/Storages/NATS/NATSSink.cpp index 00fe93a9e68..8be95099261 100644 --- a/src/Storages/NATS/NATSSink.cpp +++ b/src/Storages/NATS/NATSSink.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB @@ -18,13 +18,12 @@ NATSSink::NATSSink( , metadata_snapshot(metadata_snapshot_) , context(context_) { -// storage.unbindExchange(); } void NATSSink::onStart() { -// buffer = storage.createWriteBuffer(); + buffer = storage.createWriteBuffer(); buffer->activateWriting(); auto format_settings = getFormatSettings(context); diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index 5edd48d97c8..1a0d72cc559 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include "Poco/Timer.h" #include @@ -56,7 +56,9 @@ ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() void ReadBufferFromNATSConsumer::subscribe() { for (const auto & subject : subjects) { - subscriptions.emplace_back(connection->createSubscription(subject, onMsg, this)); + SubscriptionPtr subscription = connection->createSubscription(subject, onMsg, this); + if (subscription.get()) + subscriptions.emplace_back(std::move(subscription)); } // for (const auto & queue_name : subjects) // { diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 2cfe9e8be8d..afcbcd28a6d 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -23,19 +23,7 @@ #include #include #include -#include - -//static void -//onMsg(natsConnection *, natsSubscription *, natsMsg *msg, void *) -//{ -// printf("\n\n\nReceived msg: %s - %.*s\n\n\n", -// natsMsg_GetSubject(msg), -// natsMsg_GetDataLength(msg), -// natsMsg_GetData(msg)); -// fflush(stdout); -// -// natsMsg_Destroy(msg); -//} +#include namespace DB { @@ -52,10 +40,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int CANNOT_CONNECT_RABBITMQ; - extern const int CANNOT_BIND_RABBITMQ_EXCHANGE; - extern const int CANNOT_DECLARE_RABBITMQ_EXCHANGE; - extern const int CANNOT_REMOVE_RABBITMQ_EXCHANGE; - extern const int CANNOT_CREATE_RABBITMQ_QUEUE_BINDING; extern const int QUERY_NOT_ALLOWED; } @@ -86,6 +70,7 @@ StorageNATS::StorageNATS( auto parsed_address = parseAddress(getContext()->getMacros()->expand(nats_settings->nats_host_port), 5672); context_->getRemoteHostFilter().checkHostAndPort(parsed_address.first, toString(parsed_address.second)); + LOG_DEBUG(log, "Creds: u-{} p-{}", nats_settings->nats_username.value, nats_settings->nats_password.value); auto nats_username = nats_settings->nats_username.value; auto nats_password = nats_settings->nats_password.value; configuration = @@ -94,7 +79,8 @@ StorageNATS::StorageNATS( .port = parsed_address.second, .username = nats_username.empty() ? getContext()->getConfigRef().getString("nats.username") : nats_username, .password = nats_password.empty() ? getContext()->getConfigRef().getString("nats.password") : nats_password, - .vhost = getContext()->getConfigRef().getString("nats.vhost", getContext()->getMacros()->expand(nats_settings->nats_vhost)), + .max_reconnect = static_cast(nats_settings->nats_max_reconnect.value), + .reconnect_wait = static_cast(nats_settings->nats_reconnect_wait.value), .secure = nats_settings->nats_secure.value, .connection_string = getContext()->getMacros()->expand(nats_settings->nats_address) }; @@ -312,334 +298,6 @@ size_t StorageNATS::getMaxBlockSize() const } -//void StorageNATS::initNATS() -//{ -// if (shutdown_called || nats_is_ready) -// return; -// -// if (use_user_setup) -// { -// queues.emplace_back(queue_base); -// nats_is_ready = true; -// return; -// } -// -// try -// { -// auto nats_channel = connection->createChannel(); -// -// /// Main exchange -> Bridge exchange -> ( Sharding exchange ) -> Queues -> Consumers -// -// initExchange(*nats_channel); -// bindExchange(*nats_channel); -// -// for (const auto i : collections::range(0, num_queues)) -// bindQueue(i + 1, *nats_channel); -// -// LOG_TRACE(log, "NATS setup completed"); -// nats_is_ready = true; -// nats_channel->close(); -// } -// catch (...) -// { -// tryLogCurrentException(log); -// if (!is_attach) -// throw; -// } -//} -// -// -//void StorageNATS::initExchange(AMQP::TcpChannel & nats_channel) -//{ -// /// Exchange hierarchy: -// /// 1. Main exchange (defined with table settings - nats_exchange_name, nats_exchange_type). -// /// 2. Bridge exchange (fanout). Used to easily disconnect main exchange and to simplify queue bindings. -// /// 3. Sharding (or hash) exchange. Used in case of multiple queues. -// /// 4. Consumer exchange. Just an alias for bridge_exchange or sharding exchange to know to what exchange -// /// queues will be bound. -// -// /// All exchanges are declared with options: -// /// 1. `durable` (survive NATS server restart) -// /// 2. `autodelete` (auto delete in case of queue bindings are dropped). -// -// nats_channel.declareExchange(exchange_name, exchange_type, AMQP::durable) -// .onError([&](const char * message) -// { -// /// This error can be a result of attempt to declare exchange if it was already declared but -// /// 1) with different exchange type. -// /// 2) with different exchange settings. -// throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " -// + std::string(message), ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE); -// }); -// -// nats_channel.declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable | AMQP::autodelete) -// .onError([&](const char * message) -// { -// /// This error is not supposed to happen as this exchange name is always unique to type and its settings. -// throw Exception( -// ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, "Unable to declare bridge exchange ({}). Reason: {}", bridge_exchange, std::string(message)); -// }); -// -// if (!hash_exchange) -// { -// consumer_exchange = bridge_exchange; -// return; -// } -// -// AMQP::Table binding_arguments; -// -// /// Default routing key property in case of hash exchange is a routing key, which is required to be an integer. -// /// Support for arbitrary exchange type (i.e. arbitrary pattern of routing keys) requires to eliminate this dependency. -// /// This settings changes hash property to message_id. -// binding_arguments["hash-property"] = "message_id"; -// -// /// Declare hash exchange for sharding. -// nats_channel.declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable | AMQP::autodelete, binding_arguments) -// .onError([&](const char * message) -// { -// /// This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared -// /// to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter -// /// is bad. -// throw Exception( -// ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, -// "Unable to declare sharding exchange ({}). Reason: {}", sharding_exchange, std::string(message)); -// }); -// -// nats_channel.bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) -// .onError([&](const char * message) -// { -// throw Exception( -// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, -// "Unable to bind bridge exchange ({}) to sharding exchange ({}). Reason: {}", -// bridge_exchange, -// sharding_exchange, -// std::string(message)); -// }); -// -// consumer_exchange = sharding_exchange; -//} -// -// -//void StorageNATS::bindExchange(AMQP::TcpChannel & nats_channel) -//{ -// size_t bound_keys = 0; -// -// if (exchange_type == AMQP::ExchangeType::headers) -// { -// AMQP::Table bind_headers; -// for (const auto & header : routing_keys) -// { -// std::vector matching; -// boost::split(matching, header, [](char c){ return c == '='; }); -// bind_headers[matching[0]] = matching[1]; -// } -// -// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) -// .onSuccess([&]() { connection->getHandler().stopLoop(); }) -// .onError([&](const char * message) -// { -// throw Exception( -// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, -// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", -// exchange_name, bridge_exchange, std::string(message)); -// }); -// } -// else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) -// { -// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_keys[0]) -// .onSuccess([&]() { connection->getHandler().stopLoop(); }) -// .onError([&](const char * message) -// { -// throw Exception( -// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, -// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", -// exchange_name, bridge_exchange, std::string(message)); -// }); -// } -// else -// { -// for (const auto & routing_key : routing_keys) -// { -// nats_channel.bindExchange(exchange_name, bridge_exchange, routing_key) -// .onSuccess([&]() -// { -// ++bound_keys; -// if (bound_keys == routing_keys.size()) -// connection->getHandler().stopLoop(); -// }) -// .onError([&](const char * message) -// { -// throw Exception( -// ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, -// "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", -// exchange_name, bridge_exchange, std::string(message)); -// }); -// } -// } -// -// connection->getHandler().startBlockingLoop(); -//} -// -// -//void StorageNATS::bindQueue(size_t queue_id, AMQP::TcpChannel & nats_channel) -//{ -// auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) -// { -// queues.emplace_back(queue_name); -// LOG_DEBUG(log, "Queue {} is declared", queue_name); -// -// if (msgcount) -// LOG_INFO(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); -// -// /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are -// * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for -// * fanout exchange it can be arbitrary -// */ -// nats_channel.bindQueue(consumer_exchange, queue_name, std::to_string(queue_id)) -// .onSuccess([&] { connection->getHandler().stopLoop(); }) -// .onError([&](const char * message) -// { -// throw Exception( -// ErrorCodes::CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, -// "Failed to create queue binding for exchange {}. Reason: {}", exchange_name, std::string(message)); -// }); -// }; -// -// auto error_callback([&](const char * message) -// { -// /* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a -// * given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different -// * max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously -// * declared queues via any of the various cli tools. -// */ -// throw Exception("Failed to declare queue. Probably queue settings are conflicting: max_block_size, deadletter_exchange. Attempt \ -// specifying differently those settings or use a different queue_base or manually delete previously declared queues, \ -// which were declared with the same names. ERROR reason: " -// + std::string(message), ErrorCodes::BAD_ARGUMENTS); -// }); -// -// AMQP::Table queue_settings; -// -// std::unordered_set integer_settings = {"x-max-length", "x-max-length-bytes", "x-message-ttl", "x-expires", "x-priority", "x-max-priority"}; -// std::unordered_set string_settings = {"x-overflow", "x-dead-letter-exchange", "x-queue-type"}; -// -// /// Check user-defined settings. -// if (!queue_settings_list.empty()) -// { -// for (const auto & setting : queue_settings_list) -// { -// Strings setting_values; -// splitInto<'='>(setting_values, setting); -// if (setting_values.size() != 2) -// throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid settings string: {}", setting); -// -// String key = setting_values[0], value = setting_values[1]; -// -// if (integer_settings.contains(key)) -// queue_settings[key] = parse(value); -// else if (string_settings.find(key) != string_settings.end()) -// queue_settings[key] = value; -// else -// throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported queue setting: {}", value); -// } -// } -// -// /// Impose default settings if there are no user-defined settings. -// if (!queue_settings.contains("x-max-length")) -// { -// queue_settings["x-max-length"] = queue_size; -// } -// if (!queue_settings.contains("x-overflow")) -// { -// queue_settings["x-overflow"] = "reject-publish"; -// } -// -// /// If queue_base - a single name, then it can be used as one specific queue, from which to read. -// /// Otherwise it is used as a generator (unique for current table) of queue names, because it allows to -// /// maximize performance - via setting `nats_num_queues`. -// const String queue_name = !hash_exchange ? queue_base : std::to_string(queue_id) + "_" + queue_base; -// -// /// AMQP::autodelete setting is not allowed, because in case of server restart there will be no consumers -// /// and deleting queues should not take place. -// nats_channel.declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); -// connection->getHandler().startBlockingLoop(); -//} -// -// -//bool StorageNATS::updateChannel(ChannelPtr & channel) -//{ -// try -// { -// channel = connection->createChannel(); -// return true; -// } -// catch (...) -// { -// tryLogCurrentException(log); -// return false; -// } -//} -// -// -//void StorageNATS::prepareChannelForBuffer(ConsumerBufferPtr buffer) -//{ -// if (!buffer) -// return; -// -// if (buffer->queuesCount() != queues.size()) -// buffer->updateQueues(queues); -// -// buffer->updateAckTracker(); -// -// if (updateChannel(buffer->getChannel())) -// buffer->setupChannel(); -//} -// -// -//void StorageNATS::unbindExchange() -//{ -// /* This is needed because with NATS (without special adjustments) can't, for example, properly make mv if there was insert query -// * on the same table before, and in another direction it will make redundant copies, but most likely nobody will do that. -// * As publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in -// * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as -// * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible -// * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local -// * queues, then messages will go both ways and in one of them they will remain not consumed. So need to disconnect local exchange -// * bindings to remove redunadant message copies, but after that mv cannot work unless those bindings are recreated. Recreating them is -// * not difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. -// */ -// if (!exchange_removed.exchange(true)) -// { -// try -// { -// streaming_task->deactivate(); -// -// stopLoop(); -// looping_task->deactivate(); -// -// auto nats_channel = connection->createChannel(); -// nats_channel->removeExchange(bridge_exchange) -// .onSuccess([&]() -// { -// connection->getHandler().stopLoop(); -// }) -// .onError([&](const char * message) -// { -// throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_REMOVE_RABBITMQ_EXCHANGE); -// }); -// -// connection->getHandler().startBlockingLoop(); -// nats_channel->close(); -// } -// catch (...) -// { -// exchange_removed = false; -// throw; -// } -// } -//} - - Pipe StorageNATS::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -649,9 +307,6 @@ Pipe StorageNATS::read( size_t /* max_block_size */, unsigned /* num_streams */) { -// if (!nats_is_ready) -// throw Exception("NATS setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_RABBITMQ); - if (num_created_consumers == 0) return {}; @@ -674,8 +329,6 @@ Pipe StorageNATS::read( throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "No connection to {}", connection->connectionInfoForLog()); } -// initializeBuffers(); - Pipes pipes; pipes.reserve(num_created_consumers); @@ -714,26 +367,9 @@ SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & m void StorageNATS::startup() { - if (!nats_is_ready) + if (!connection->isConnected()) { - if (connection->isConnected()) - { - try - { -// initNATS(); - LOG_DEBUG(log, "Fake init lul"); - } - catch (...) - { - if (!is_attach) - throw; - tryLogCurrentException(log); - } - } - else - { - connection_task->activateAndSchedule(); - } + connection_task->activateAndSchedule(); } for (size_t i = 0; i < num_consumers; ++i) @@ -794,58 +430,6 @@ void StorageNATS::shutdown() } } - -/// The only thing publishers are supposed to be aware of is _exchanges_ and queues are a responsibility of a consumer. -/// Therefore, if a table is dropped, a clean up is needed. -//void StorageNATS::cleanupNATS() const -//{ -// if (use_user_setup) -// return; -// -// connection->heartbeat(); -// if (!connection->isConnected()) -// { -// String queue_names; -// for (const auto & queue : queues) -// { -// if (!queue_names.empty()) -// queue_names += ", "; -// queue_names += queue; -// } -// LOG_WARNING(log, -// "NATS clean up not done, because there is no connection in table's shutdown." -// "There are {} queues ({}), which might need to be deleted manually. Exchanges will be auto-deleted", -// queues.size(), queue_names); -// return; -// } -// -// auto nats_channel = connection->createChannel(); -// for (const auto & queue : queues) -// { -// /// AMQP::ifunused is needed, because it is possible to share queues between multiple tables and dropping -// /// on of them should not affect others. -// /// AMQP::ifempty is not used on purpose. -// -// nats_channel->removeQueue(queue, AMQP::ifunused) -// .onSuccess([&](uint32_t num_messages) -// { -// LOG_TRACE(log, "Successfully deleted queue {}, messages contained {}", queue, num_messages); -// connection->getHandler().stopLoop(); -// }) -// .onError([&](const char * message) -// { -// LOG_ERROR(log, "Failed to delete queue {}. Error message: {}", queue, message); -// connection->getHandler().stopLoop(); -// }); -// } -// connection->getHandler().startBlockingLoop(); -// nats_channel->close(); -// -// /// Also there is no need to cleanup exchanges as they were created with AMQP::autodelete option. Once queues -// /// are removed, exchanges will also be cleaned. -//} - - void StorageNATS::pushReadBuffer(ConsumerBufferPtr buffer) { std::lock_guard lock(buffers_mutex); @@ -888,13 +472,12 @@ ConsumerBufferPtr StorageNATS::createReadBuffer() } -//ProducerBufferPtr StorageNATS::createWriteBuffer() -//{ -// return std::make_shared( -// configuration, getContext(), routing_keys, exchange_name, exchange_type, -// producer_id.fetch_add(1), persistent, shutdown_called, log, -// row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); -//} +ProducerBufferPtr StorageNATS::createWriteBuffer() +{ + return std::make_shared( + configuration, getContext(), subjects[0], shutdown_called, log, + row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); +} bool StorageNATS::checkDependencies(const StorageID & table_id) @@ -925,18 +508,6 @@ bool StorageNATS::checkDependencies(const StorageID & table_id) } -//void StorageNATS::initializeBuffers() -//{ -// assert(nats_is_ready); -// if (!initialized) -// { -// for (const auto & buffer : buffers) -// prepareChannelForBuffer(buffer); -// initialized = true; -// } -//} - - void StorageNATS::streamingToViewsFunc() { try @@ -1081,7 +652,6 @@ bool StorageNATS::streamToViews() } else { - /// Commit for (auto & source : sources) { if (source->queueEmpty()) @@ -1093,7 +663,6 @@ bool StorageNATS::streamToViews() if ((queue_empty == num_created_consumers) && (++read_attempts == MAX_FAILED_READ_ATTEMPTS)) { -// connection->heartbeat(); read_attempts = 0; LOG_TRACE(log, "Reschedule streaming. Queues are empty."); return true; @@ -1127,7 +696,7 @@ void registerStorageNATS(StorageFactory & factory) if (!nats_settings->nats_format.changed) throw Exception("You must specify `nats_format` setting", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageNATS::create(args.table_id, args.getContext(), args.columns, std::move(nats_settings), args.attach); + return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(nats_settings), args.attach); }; factory.registerStorage("NATS", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 2630df33ae4..7e9fee198a4 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -17,11 +16,17 @@ namespace DB { -class StorageNATS final: public shared_ptr_helper, public IStorage, WithContext +class StorageNATS final: public IStorage, WithContext { - friend struct shared_ptr_helper; public: + StorageNATS( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr nats_settings_, + bool is_attach_); + std::string getName() const override { return "NATS"; } bool noPushingToViews() const override { return true; } @@ -70,14 +75,6 @@ public: void incrementReader(); void decrementReader(); -protected: - StorageNATS( - const StorageID & table_id_, - ContextPtr context_, - const ColumnsDescription & columns_, - std::unique_ptr nats_settings_, - bool is_attach_); - private: ContextMutablePtr nats_context; std::unique_ptr nats_settings; diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index 175d9458423..44095200d7e 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -17,22 +17,17 @@ namespace DB { static const auto BATCH = 1000; -static const auto RETURNED_LIMIT = 50000; namespace ErrorCodes { - extern const int CANNOT_CONNECT_RABBITMQ; + extern const int CANNOT_CONNECT_NATS; extern const int LOGICAL_ERROR; } WriteBufferToNATSProducer::WriteBufferToNATSProducer( const NATSConfiguration & configuration_, ContextPtr global_context, - const Names & routing_keys_, - const String & exchange_name_, - const AMQP::ExchangeType exchange_type_, - const size_t channel_id_base_, - const bool persistent_, + const String & subject_, std::atomic & shutdown_called_, Poco::Logger * log_, std::optional delimiter, @@ -40,37 +35,20 @@ WriteBufferToNATSProducer::WriteBufferToNATSProducer( size_t chunk_size_) : WriteBuffer(nullptr, 0) , connection(configuration_, log_) - , routing_keys(routing_keys_) - , exchange_name(exchange_name_) - , exchange_type(exchange_type_) - , channel_id_base(std::to_string(channel_id_base_)) - , persistent(persistent_) + , subject(subject_) , shutdown_called(shutdown_called_) , payloads(BATCH) - , returned(RETURNED_LIMIT) , log(log_) , delim(delimiter) , max_rows(rows_per_message) , chunk_size(chunk_size_) { - if (connection.connect()) - setupChannel(); - else - throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to NATS {}", connection.connectionInfoForLog()); + if (!connection.connect()) + throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "Cannot connect to NATS {}", connection.connectionInfoForLog()); writing_task = global_context->getSchedulePool().createTask("NATSWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); - if (exchange_type == AMQP::ExchangeType::headers) - { - for (const auto & header : routing_keys) - { - std::vector matching; - boost::split(matching, header, [](char c){ return c == '='; }); - key_arguments[matching[0]] = matching[1]; - } - } - reinitializeChunks(); } @@ -104,152 +82,42 @@ void WriteBufferToNATSProducer::countRow() reinitializeChunks(); ++payload_counter; - if (!payloads.push(std::make_pair(payload_counter, payload))) + if (!payloads.push(payload)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); + LOG_DEBUG(log, "Pushed payload to queue {} {}", payload, payloads.size()); } } - -void WriteBufferToNATSProducer::setupChannel() +void WriteBufferToNATSProducer::publish() { -// producer_channel = connection.createChannel(); -// -// producer_channel->onError([&](const char * message) -// { -// LOG_ERROR(log, "Producer's channel {} error: {}", channel_id, message); -// -// /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) -// producer_channel->close(); -// -// /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once -// * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid -// */ -// for (const auto & record : delivery_record) -// if (!returned.push(record.second)) -// throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); -// -// LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", -// channel_id, delivery_record.size(), payloads.size()); -// -// /// Delivery tags are scoped per channel. -// delivery_record.clear(); -// delivery_tag = 0; -// producer_ready = false; -// }); -// -// producer_channel->onReady([&]() -// { -// channel_id = channel_id_base + "_" + std::to_string(channel_id_counter++); -// LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); -// -// /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, -// * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times -// * slower than the second, so default is second and the first is turned on in table setting. -// * -// * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html -// */ -// producer_channel->confirmSelect() -// .onAck([&](uint64_t acked_delivery_tag, bool multiple) -// { -// removeRecord(acked_delivery_tag, multiple, false); -// }) -// .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) -// { -// removeRecord(nacked_delivery_tag, multiple, true); -// }); -// producer_ready = true; -// }); -} + String payload; - -void WriteBufferToNATSProducer::removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish) -{ - auto record_iter = delivery_record.find(received_delivery_tag); - assert(record_iter != delivery_record.end()); - - if (multiple) + natsStatus status{NATS_OK}; + while (!payloads.empty()) { - /// If multiple is true, then all delivery tags up to and including current are confirmed (with ack or nack). - ++record_iter; - - if (republish) - for (auto record = delivery_record.begin(); record != record_iter; ++record) - if (!returned.push(record->second)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); - - /// Delete the records even in case when republished because new delivery tags will be assigned by the server. - delivery_record.erase(delivery_record.begin(), record_iter); - } - else - { - if (republish) - if (!returned.push(record_iter->second)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); - - delivery_record.erase(record_iter); - } -} - - -void WriteBufferToNATSProducer::publish(ConcurrentBoundedQueue> & messages, bool republishing) -{ - std::pair payload; - - /* It is important to make sure that delivery_record.size() is never bigger than returned.size(), i.e. number if unacknowledged - * messages cannot exceed returned.size(), because they all might end up there - */ - while (!messages.empty() && producer_channel->usable() && delivery_record.size() < RETURNED_LIMIT) - { - bool pop_result = messages.pop(payload); + bool pop_result = payloads.pop(payload); if (!pop_result) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not pop payload"); - AMQP::Envelope envelope(payload.second.data(), payload.second.size()); - - /// if headers exchange is used, routing keys are added here via headers, if not - it is just empty - AMQP::Table message_settings = key_arguments; - - /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the - * server, then it means that publisher will never know whether those messages were delivered or not, and therefore those records - * that received no ack/nack before connection loss will be republished (see onError() callback), so there might be duplicates. To - * let consumer know that received message might be a possible duplicate - a "republished" field is added to message metadata - */ - message_settings["republished"] = std::to_string(republishing); - envelope.setHeaders(message_settings); - - /* Adding here a messageID property to message metadata. Since NATS does not guarantee exactly-once delivery, then on the - * consumer side "republished" field of message metadata can be checked and, if it set to 1, consumer might also check "messageID" - * property. This way detection of duplicates is guaranteed - */ - envelope.setMessageID(std::to_string(payload.first)); - - /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse - if (persistent) - envelope.setDeliveryMode(2); - - if (exchange_type == AMQP::ExchangeType::consistent_hash) + if (status == NATS_OK) { - producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope); - } - else if (exchange_type == AMQP::ExchangeType::headers) - { - producer_channel->publish(exchange_name, "", envelope); + status = natsConnection_PublishString(connection.getConnection(), subject.c_str(), payload.c_str()); } else { - producer_channel->publish(exchange_name, routing_keys[0], envelope); + LOG_DEBUG(log, "Something went wrong during publishing to NATS subject {}.", subject); } - - /// This is needed for "publisher confirms", which guarantees at-least-once delivery - ++delivery_tag; - delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); - - /// Need to break at some point to let event loop run, because no publishing actually happens before looping - if (delivery_tag % BATCH == 0) - break; } + if (status == NATS_OK) + { + status = natsConnection_Flush(connection.getConnection()); + } + else + { + LOG_DEBUG(log, "Something went wrong during publishing to NATS subject {}.", subject); + } iterateEventLoop(); } @@ -258,31 +126,15 @@ void WriteBufferToNATSProducer::writingFunc() { while ((!payloads.empty() || wait_all) && !shutdown_called.load()) { - /// If onReady callback is not received, producer->usable() will anyway return true, - /// but must publish only after onReady callback. - if (producer_ready) - { - /* Publish main paylods only when there are no returned messages. This way it is ensured that returned messages are republished - * as fast as possible and no new publishes are made before returned messages are handled - */ - if (!returned.empty() && producer_channel->usable()) - publish(returned, true); - else if (!payloads.empty() && producer_channel->usable()) - publish(payloads, false); - } + publish(); + + if (wait_num.load() && payloads.empty()) + wait_all = false; iterateEventLoop(); - - if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) - wait_all = false; - else if (!producer_channel->usable()) - { - if (connection.reconnect()) - setupChannel(); - } } - LOG_DEBUG(log, "Producer on channel {} completed", channel_id); + LOG_DEBUG(log, "Producer on subject {} completed", subject); } diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h index f2e9e04e227..779e35519c7 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.h +++ b/src/Storages/NATS/WriteBufferToNATSProducer.h @@ -20,11 +20,7 @@ public: WriteBufferToNATSProducer( const NATSConfiguration & configuration_, ContextPtr global_context, - const Names & routing_keys_, - const String & exchange_name_, - const AMQP::ExchangeType exchange_type_, - const size_t channel_id_base_, - const bool persistent_, + const String & subject_, std::atomic & shutdown_called_, Poco::Logger * log_, std::optional delimiter, @@ -45,52 +41,22 @@ private: void iterateEventLoop(); void writingFunc(); - void setupChannel(); - void removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish); - void publish(ConcurrentBoundedQueue> & message, bool republishing); + void publish(); NATSConnectionManager connection; + const String subject; - const Names routing_keys; - const String exchange_name; - AMQP::ExchangeType exchange_type; - const String channel_id_base; /// Serial number of current producer buffer - const bool persistent; - - /* false: when shutdown is called; needed because table might be dropped before all acks are received + /* false: when shutdown is called * true: in all other cases */ std::atomic & shutdown_called; - AMQP::Table key_arguments; BackgroundSchedulePool::TaskHolder writing_task; - std::unique_ptr producer_channel; - bool producer_ready = false; - - /// Channel errors lead to channel closure, need to count number of recreated channels to update channel id -// UInt64 channel_id_counter = 0; - - /// channel id which contains id of current producer buffer and serial number of recreated channel in this buffer - String channel_id; - /* payloads.queue: * - payloads are pushed to queue in countRow and popped by another thread in writingFunc, each payload gets into queue only once - * returned.queue: - * - payloads are pushed to queue: - * 1) inside channel->onError() callback if channel becomes unusable and the record of pending acknowledgements from server - * is non-empty. - * 2) inside removeRecord() if received nack() - negative acknowledgement from the server that message failed to be written - * to disk or it was unable to reach the queue. - * - payloads are popped from the queue once republished */ - ConcurrentBoundedQueue> payloads, returned; - - /* Counter of current delivery on a current channel. Delivery tags are scoped per channel. The server attaches a delivery tag for each - * published message - a serial number of delivery on current channel. Delivery tag is a way of server to notify publisher if it was - * able or unable to process delivery, i.e. it sends back a response with a corresponding delivery tag. - */ - UInt64 delivery_tag = 0; + ConcurrentBoundedQueue payloads; /* false: message delivery successfully ended: publisher received confirm from server that all published * 1) persistent messages were written to disk @@ -103,13 +69,8 @@ private: * true: means payloads.queue will not grow anymore */ std::atomic wait_num = 0; - - /// Needed to fill messageID property UInt64 payload_counter = 0; - /// Record of pending acknowledgements from the server; its size never exceeds size of returned.queue - std::map> delivery_record; - Poco::Logger * log; const std::optional delim; const size_t max_rows; From f53ec6a3a5ba605d58a09912226ac9f889af7d83 Mon Sep 17 00:00:00 2001 From: tchepavel Date: Thu, 12 May 2022 14:08:45 +0300 Subject: [PATCH 009/627] Preparing for first draft --- contrib/nats-io-cmake/CMakeLists.txt | 0 src/Storages/NATS/NATSConnection.cpp | 74 ++++++------ src/Storages/NATS/NATSConnection.h | 23 ++-- src/Storages/NATS/NATSSettings.h | 10 +- src/Storages/NATS/NATSSource.cpp | 49 +------- src/Storages/NATS/NATSSource.h | 10 +- .../NATS/ReadBufferFromNATSConsumer.cpp | 108 +++-------------- .../NATS/ReadBufferFromNATSConsumer.h | 28 +---- src/Storages/NATS/StorageNATS.cpp | 114 ++++-------------- src/Storages/NATS/StorageNATS.h | 44 +------ .../NATS/WriteBufferToNATSProducer.cpp | 19 +-- src/Storages/NATS/WriteBufferToNATSProducer.h | 6 +- 12 files changed, 120 insertions(+), 365 deletions(-) create mode 100644 contrib/nats-io-cmake/CMakeLists.txt diff --git a/contrib/nats-io-cmake/CMakeLists.txt b/contrib/nats-io-cmake/CMakeLists.txt new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index c76583dae5f..110f4c740d8 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -3,12 +3,15 @@ #include #include +#include + namespace DB { //static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; +static const auto CONNECTED_TO_BUFFER_SIZE = 256; NATSConnectionManager::NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_) @@ -18,9 +21,19 @@ NATSConnectionManager::NATSConnectionManager(const NATSConfiguration & configura { } + +NATSConnectionManager::~NATSConnectionManager() +{ + if (has_connection) + natsConnection_Destroy(connection); +} + String NATSConnectionManager::connectionInfoForLog() const { - return configuration.host + ':' + toString(configuration.port); + if (!configuration.url.empty()) { + return "url : " + configuration.url; + } + return "cluster: " + boost::algorithm::join(configuration.servers, ", "); } bool NATSConnectionManager::isConnected() @@ -50,24 +63,6 @@ bool NATSConnectionManager::reconnect() return isConnectedImpl(); } -SubscriptionPtr NATSConnectionManager::createSubscription(const std::string& subject, natsMsgHandler handler, ReadBufferFromNATSConsumer * consumer) -{ - std::lock_guard lock(mutex); - natsSubscription * ns; - status = natsConnection_Subscribe(&ns, connection, subject.c_str(), handler, static_cast(consumer)); - if (status == NATS_OK) - status = natsSubscription_SetPendingLimits(ns, -1, -1); - if (status == NATS_OK) - { - LOG_DEBUG(log, "Subscribed to subject {}", subject); - return SubscriptionPtr(ns, &natsSubscription_Destroy); - } - else - { - return SubscriptionPtr(nullptr, &natsSubscription_Destroy); - } -} - void NATSConnectionManager::disconnect() { std::lock_guard lock(mutex); @@ -82,7 +77,7 @@ bool NATSConnectionManager::closed() bool NATSConnectionManager::isConnectedImpl() const { - return event_handler.connectionRunning() && !natsConnection_IsClosed(connection) && status == natsStatus::NATS_OK; + return event_handler.connectionRunning() && connection && !natsConnection_IsClosed(connection); } void NATSConnectionManager::connectImpl() @@ -93,27 +88,31 @@ void NATSConnectionManager::connectImpl() natsOptions_SetSecure(options, true); natsOptions_SkipServerVerification(options, true); } - std::string address; - if (configuration.connection_string.empty()) + if (!configuration.url.empty()) { - address = configuration.host + ":" + std::to_string(configuration.port); - } - else - { - address = configuration.connection_string; + natsOptions_SetURL(options, configuration.url.c_str()); + } else { + const char * servers[configuration.servers.size()]; + for (size_t i = 0; i < configuration.servers.size(); ++i) { + servers[i] = configuration.servers[i].c_str(); + } + natsOptions_SetServers(options, servers, configuration.servers.size()); } natsOptions_SetMaxReconnect(options, configuration.max_reconnect); natsOptions_SetReconnectWait(options, configuration.reconnect_wait); -// natsOptions_SetDisconnectedCB(options, disconnectedCallback, this); -// natsOptions_SetReconnectedCB(options, reconnectedCallback, this); - natsOptions_SetURL(options, address.c_str()); - status = natsConnection_Connect(&connection, options); + natsOptions_SetDisconnectedCB(options, disconnectedCallback, log); + natsOptions_SetReconnectedCB(options, reconnectedCallback, log); + auto status = natsConnection_Connect(&connection, options); if (status != NATS_OK) { - LOG_DEBUG(log, "Failed to connect to NATS on address: {}", address); + if (!configuration.url.empty()) + LOG_DEBUG(log, "Failed to connect to NATS on address: {}", configuration.url); + else + LOG_DEBUG(log, "Failed to connect to NATS cluster"); return; } + has_connection = true; event_handler.changeConnectionStatus(true); } @@ -131,18 +130,17 @@ void NATSConnectionManager::disconnectImpl() event_handler.changeConnectionStatus(false); } -void NATSConnectionManager::reconnectedCallback(natsConnection * nc, void * manager) +void NATSConnectionManager::reconnectedCallback(natsConnection * nc, void * log) { - char buffer[64]; - + char buffer[CONNECTED_TO_BUFFER_SIZE]; buffer[0] = '\0'; natsConnection_GetConnectedUrl(nc, buffer, sizeof(buffer)); - LOG_DEBUG(static_cast(manager)->log, "Got reconnected to NATS server: {}.", buffer); + LOG_DEBUG(static_cast(log), "Got reconnected to NATS server: {}.", buffer); } -void NATSConnectionManager::disconnectedCallback(natsConnection *, void * manager) +void NATSConnectionManager::disconnectedCallback(natsConnection *, void * log) { - LOG_DEBUG(static_cast(manager)->log, "Got disconnected from NATS server."); + LOG_DEBUG(static_cast(log), "Got disconnected from NATS server."); } } diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h index 813115454e7..76f89eaeddb 100644 --- a/src/Storages/NATS/NATSConnection.h +++ b/src/Storages/NATS/NATSConnection.h @@ -9,16 +9,17 @@ namespace DB struct NATSConfiguration { - String host; - UInt16 port; + String url; + std::vector servers; + String username; String password; + String token; int max_reconnect; int reconnect_wait; bool secure; - String connection_string; }; class NATSConnectionManager @@ -26,9 +27,7 @@ class NATSConnectionManager public: NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_); - ~NATSConnectionManager() { natsConnection_Destroy(connection); } - - natsConnection * getConnection() { return connection; } + ~NATSConnectionManager(); bool isConnected(); @@ -40,10 +39,9 @@ public: bool closed(); - SubscriptionPtr createSubscription(const std::string& subject, natsMsgHandler handler, ReadBufferFromNATSConsumer * consumer); - /// NATSHandler is thread safe. Any public methods can be called concurrently. NATSHandler & getHandler() { return event_handler; } + natsConnection * getConnection() { return connection; } String connectionInfoForLog() const; @@ -54,8 +52,8 @@ private: void disconnectImpl(); - static void disconnectedCallback(natsConnection * nc, void * storage); - static void reconnectedCallback(natsConnection * nc, void * storage); + static void disconnectedCallback(natsConnection * nc, void * log); + static void reconnectedCallback(natsConnection * nc, void * log); NATSConfiguration configuration; Poco::Logger * log; @@ -63,8 +61,11 @@ private: UVLoop loop; NATSHandler event_handler; + natsConnection * connection; - natsStatus status; + // true if at any point was a successful connection + bool has_connection = false; + std::mutex mutex; }; diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h index 2cb5365a13b..4304d416368 100644 --- a/src/Storages/NATS/NATSSettings.h +++ b/src/Storages/NATS/NATSSettings.h @@ -9,24 +9,22 @@ namespace DB #define NATS_RELATED_SETTINGS(M) \ - M(String, nats_host_port, "", "A host-port to connect to NATS server.", 0) \ + M(String, nats_url, "", "A host-port to connect to NATS server.", 0) \ M(String, nats_subjects, "", "List of subject for NATS table to subscribe/publsh to.", 0) \ M(String, nats_format, "", "The message format.", 0) \ M(Char, nats_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ M(String, nats_schema, "", "Schema identifier (used by schema-based formats) for NATS engine", 0) \ M(UInt64, nats_num_consumers, 1, "The number of consumer channels per table.", 0) \ - M(Bool, nats_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \ M(Bool, nats_secure, false, "Use SSL connection", 0) \ M(UInt64, nats_max_reconnect, 5, "Maximum amount of reconnection attempts.", 0) \ M(UInt64, nats_reconnect_wait, 2000, "Amount to sleep between each reconnect attempt.", 0) \ - M(String, nats_password, "", "NATS password", 0) \ - M(String, nats_address, "", "Address for connection", 0) \ + M(String, nats_server_list, "", "Server list for connection", 0) \ M(UInt64, nats_skip_broken_messages, 0, "Skip at least this number of broken messages from NATS per block", 0) \ M(UInt64, nats_max_block_size, 0, "Number of row collected before flushing data from NATS.", 0) \ M(Milliseconds, nats_flush_interval_ms, 0, "Timeout for flushing data from NATS.", 0) \ - M(Bool, nats_queue_consume, false, "Use user-defined queues and do not make any NATS setup: declaring exchanges, queues, bindings", 0) \ M(String, nats_username, "", "NATS username", 0) \ - M(Bool, nats_commit_on_select, false, "Commit messages when select query is made", 0) \ + M(String, nats_password, "", "NATS password", 0) \ + M(String, nats_token, "", "NATS token", 0) \ #define LIST_OF_NATS_SETTINGS(M) \ NATS_RELATED_SETTINGS(M) \ diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index d757f7b9a54..4d66e43ab46 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -11,8 +11,7 @@ namespace DB static std::pair getHeaders(const StorageSnapshotPtr & storage_snapshot) { auto non_virtual_header = storage_snapshot->metadata->getSampleBlockNonMaterialized(); - auto virtual_header = storage_snapshot->getSampleBlockForColumns( - {"_subject", "_timestamp"}); + auto virtual_header = storage_snapshot->getSampleBlockForColumns({"_subject"}); return {non_virtual_header, virtual_header}; } @@ -31,16 +30,14 @@ NATSSource::NATSSource( const StorageSnapshotPtr & storage_snapshot_, ContextPtr context_, const Names & columns, - size_t max_block_size_, - bool ack_in_suffix_) + size_t max_block_size_) : NATSSource( storage_, storage_snapshot_, getHeaders(storage_snapshot_), context_, columns, - max_block_size_, - ack_in_suffix_) + max_block_size_) { } @@ -50,8 +47,7 @@ NATSSource::NATSSource( std::pair headers, ContextPtr context_, const Names & columns, - size_t max_block_size_, - bool /*ack_in_suffix_*/) + size_t max_block_size_) : SourceWithProgress(getSampleBlock(headers.first, headers.second)) , storage(storage_) , storage_snapshot(storage_snapshot_) @@ -76,32 +72,9 @@ NATSSource::~NATSSource() storage.pushReadBuffer(buffer); } - -//bool NATSSource::needChannelUpdate() -//{ -// if (!buffer) -// return false; -// -// return buffer->needChannelUpdate(); -//} -// -// -//void NATSSource::updateChannel() -//{ -// if (!buffer) -// return; -// -// buffer->updateAckTracker(); -// -// if (storage.updateChannel(buffer->getChannel())) -// buffer->setupChannel(); -//} - Chunk NATSSource::generate() { auto chunk = generateImpl(); -// if (!chunk && ack_in_suffix) -// sendAck(); return chunk; } @@ -137,12 +110,10 @@ Chunk NATSSource::generateImpl() if (new_rows) { auto subject = buffer->getSubject(); - auto timestamp = buffer->getTimestamp(); for (size_t i = 0; i < new_rows; ++i) { virtual_columns[0]->insert(subject); - virtual_columns[1]->insert(timestamp); } total_rows = total_rows + new_rows; @@ -164,16 +135,4 @@ Chunk NATSSource::generateImpl() return Chunk(std::move(result_columns), total_rows); } - -//bool NATSSource::sendAck() -//{ -// if (!buffer) -// return false; -// -// if (!buffer->ackMessages()) -// return false; -// -// return true; -//} - } diff --git a/src/Storages/NATS/NATSSource.h b/src/Storages/NATS/NATSSource.h index 263e6824603..4b6b71348c2 100644 --- a/src/Storages/NATS/NATSSource.h +++ b/src/Storages/NATS/NATSSource.h @@ -17,8 +17,7 @@ public: const StorageSnapshotPtr & storage_snapshot_, ContextPtr context_, const Names & columns, - size_t max_block_size_, - bool ack_in_suffix = false); + size_t max_block_size_); ~NATSSource() override; @@ -28,9 +27,6 @@ public: Chunk generate() override; bool queueEmpty() const { return !buffer || buffer->queueEmpty(); } -// bool needChannelUpdate(); -// void updateChannel(); -// bool sendAck(); private: StorageNATS & storage; @@ -38,7 +34,6 @@ private: ContextPtr context; Names column_names; const size_t max_block_size; -// bool ack_in_suffix; bool is_finished = false; const Block non_virtual_header; @@ -52,8 +47,7 @@ private: std::pair headers, ContextPtr context_, const Names & columns, - size_t max_block_size_, - bool ack_in_suffix); + size_t max_block_size_); Chunk generateImpl(); }; diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index 1a0d72cc559..6c09bc00a23 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -17,28 +17,26 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int CANNOT_CONNECT_NATS; } ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( - NATSHandler & event_handler_, std::shared_ptr connection_, std::vector & subjects_, - const String & channel_base_, + const String & subscribe_queue_name, Poco::Logger * log_, char row_delimiter_, uint32_t queue_size_, const std::atomic & stopped_) : ReadBuffer(nullptr, 0) - , event_handler(event_handler_) , connection(connection_) , subjects(subjects_) - , channel_base(channel_base_) , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) , received(queue_size_) { - subscribe(); + subscribe(subscribe_queue_name); LOG_DEBUG(log, "Started NATS consumer"); } @@ -53,89 +51,22 @@ ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() } -void ReadBufferFromNATSConsumer::subscribe() +void ReadBufferFromNATSConsumer::subscribe(const String & subscribe_queue_name) { for (const auto & subject : subjects) { - SubscriptionPtr subscription = connection->createSubscription(subject, onMsg, this); - if (subscription.get()) - subscriptions.emplace_back(std::move(subscription)); + natsSubscription * ns; + auto status = natsConnection_QueueSubscribe( + &ns, connection->getConnection(), subject.c_str(), subscribe_queue_name.c_str(), onMsg, static_cast(this)); + if (status == NATS_OK) + { + LOG_DEBUG(log, "Subscribed to subject {}", subject); + subscriptions.emplace_back(ns, &natsSubscription_Destroy); + } + else + { + throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "Failed to subscribe to subject {}", subject); + } } -// for (const auto & queue_name : subjects) -// { -// consumer_channel->consume(queue_name) -// .onSuccess([&](const std::string & /* consumer_tag */) -// { -// LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name); -// -// if (++subscribed == subjects.size()) -// wait_subscription.store(false); -// }) -// .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) -// { -// if (message.bodySize()) -// { -// String message_received = std::string(message.body(), message.body() + message.bodySize()); -// if (row_delimiter != '\0') -// message_received += row_delimiter; -// -// if (!received.push({message_received, message.hasMessageID() ? message.messageID() : "", -// message.hasTimestamp() ? message.timestamp() : 0, -// redelivered, AckTracker(delivery_tag, channel_id)})) -// throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); -// } -// }) -// .onError([&](const char * message) -// { -// /* End up here either if channel ends up in an error state (then there will be resubscription) or consume call error, which -// * arises from queue settings mismatch or queue level error, which should not happen as no one else is supposed to touch them -// */ -// LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message); -// wait_subscription.store(false); -// }); -// } -} - - -//bool ReadBufferFromNATSConsumer::ackMessages() -//{ -// AckTracker record_info = last_inserted_record_info; -// -// /* Do not send ack to server if message's channel is not the same as current running channel because delivery tags are scoped per -// * channel, so if channel fails, all previous delivery tags become invalid -// */ -// if (record_info.channel_id == channel_id && record_info.delivery_tag && record_info.delivery_tag > prev_tag) -// { -// /// Commit all received messages with delivery tags from last committed to last inserted -// if (!consumer_channel->ack(record_info.delivery_tag, AMQP::multiple)) -// { -// LOG_ERROR(log, "Failed to commit messages with delivery tags from last committed to {} on channel {}", -// record_info.delivery_tag, channel_id); -// return false; -// } -// -// prev_tag = record_info.delivery_tag; -// LOG_TRACE(log, "Consumer committed messages with deliveryTags up to {} on channel {}", record_info.delivery_tag, channel_id); -// } -// -// return true; -//} - - -//void ReadBufferFromNATSConsumer::updateAckTracker(AckTracker record_info) -//{ -// if (record_info.delivery_tag && channel_error.load()) -// return; -// -// if (!record_info.delivery_tag) -// prev_tag = 0; -// -// last_inserted_record_info = record_info; -//} - - -void ReadBufferFromNATSConsumer::iterateEventLoop() -{ - event_handler.iterateLoop(); } @@ -160,18 +91,17 @@ void ReadBufferFromNATSConsumer::onMsg(natsConnection *, natsSubscription *, nat { auto * buffer = static_cast(consumer); const int msg_length = natsMsg_GetDataLength(msg); - LOG_DEBUG(buffer->log, "I'm getting something {} {}", msg_length, natsMsg_GetData(msg)); if (msg_length) { String message_received = std::string(natsMsg_GetData(msg), msg_length); + String subject = natsMsg_GetSubject(msg); if (buffer->row_delimiter != '\0') message_received += buffer->row_delimiter; if (!buffer->received.push({ - .message = message_received, - .subject = natsMsg_GetSubject(msg), - .timestamp = natsMsg_GetTime(msg) + .message = std::move(message_received), + .subject = std::move(subject), })) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); } diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h index 2c1a9704c03..f1872c64aaf 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -21,10 +21,9 @@ class ReadBufferFromNATSConsumer : public ReadBuffer public: ReadBufferFromNATSConsumer( - NATSHandler & event_handler_, std::shared_ptr connection_, std::vector & subjects_, - const String & channel_base_, + const String & subscribe_queue_name, Poco::Logger * log_, char row_delimiter_, uint32_t queue_size_, @@ -32,21 +31,10 @@ public: ~ReadBufferFromNATSConsumer() override; -// struct AckTracker -// { -// UInt64 delivery_tag; -// String channel_id; -// -// AckTracker() = default; -// AckTracker(UInt64 tag, String id) : delivery_tag(tag), channel_id(id) {} -// }; - struct MessageData { String message; String subject; - int64_t timestamp; -// AckTracker track{}; }; std::vector & getChannel() { return subscriptions; } @@ -60,41 +48,29 @@ public: size_t subjectsCount() { return subjects.size(); } bool isConsumerStopped() { return stopped; } -// bool ackMessages(); -// void updateAckTracker(AckTracker record = AckTracker()); bool queueEmpty() { return received.empty(); } void allowNext() { allowed = true; } // Allow to read next message. auto getSubject() const { return current.subject; } - auto getTimestamp() const { return current.timestamp; } - void iterateEventLoop(); private: bool nextImpl() override; - void subscribe(); + void subscribe(const String & subscribe_queue_name); static void onMsg(natsConnection *nc, natsSubscription *sub, natsMsg * msg, void * closure); - NATSHandler & event_handler; /// Used concurrently, but is thread safe. std::shared_ptr connection; std::vector subscriptions; std::vector subjects; - const String channel_base; -// const size_t channel_id_base; Poco::Logger * log; char row_delimiter; bool allowed = true; const std::atomic & stopped; String channel_id; -// std::atomic channel_error = true, wait_subscription = false; ConcurrentBoundedQueue received; MessageData current; -// size_t subscribed = 0; - -// AckTracker last_inserted_record_info; -// UInt64 prev_tag = 0, channel_id_counter = 0; }; } diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index afcbcd28a6d..7ee6ebfd7e8 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -29,9 +29,8 @@ namespace DB { static const uint32_t QUEUE_SIZE = 100000; -static const auto MAX_FAILED_READ_ATTEMPTS = 10; static const auto RESCHEDULE_MS = 500; -static const auto BACKOFF_TRESHOLD = 32000; +static const auto BACKOFF_TRESHOLD = 8000; static const auto MAX_THREAD_WORK_DURATION_MS = 60000; namespace ErrorCodes @@ -39,7 +38,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int CANNOT_CONNECT_RABBITMQ; + extern const int CANNOT_CONNECT_NATS; extern const int QUERY_NOT_ALLOWED; } @@ -53,36 +52,31 @@ StorageNATS::StorageNATS( : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , nats_settings(std::move(nats_settings_)) - , subjects(parseSubjects(getContext()->getMacros()->expand(nats_settings->nats_subjects))) + , subjects(parseList(getContext()->getMacros()->expand(nats_settings->nats_subjects))) , format_name(getContext()->getMacros()->expand(nats_settings->nats_format)) , row_delimiter(nats_settings->nats_row_delimiter.value) , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) , num_consumers(nats_settings->nats_num_consumers.value) - , persistent(nats_settings->nats_persistent.value) -// , use_user_setup(nats_settings->nats_queue_consume.value) , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) , semaphore(0, num_consumers) - , unique_strbase(getRandomName()) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) , milliseconds_to_wait(RESCHEDULE_MS) , is_attach(is_attach_) { - auto parsed_address = parseAddress(getContext()->getMacros()->expand(nats_settings->nats_host_port), 5672); - context_->getRemoteHostFilter().checkHostAndPort(parsed_address.first, toString(parsed_address.second)); + auto nats_username = getContext()->getMacros()->expand(nats_settings->nats_username); + auto nats_password = getContext()->getMacros()->expand(nats_settings->nats_password); + auto nats_token = getContext()->getMacros()->expand(nats_settings->nats_token); - LOG_DEBUG(log, "Creds: u-{} p-{}", nats_settings->nats_username.value, nats_settings->nats_password.value); - auto nats_username = nats_settings->nats_username.value; - auto nats_password = nats_settings->nats_password.value; configuration = { - .host = parsed_address.first, - .port = parsed_address.second, - .username = nats_username.empty() ? getContext()->getConfigRef().getString("nats.username") : nats_username, - .password = nats_password.empty() ? getContext()->getConfigRef().getString("nats.password") : nats_password, + .url = getContext()->getMacros()->expand(nats_settings->nats_url), + .servers = parseList(getContext()->getMacros()->expand(nats_settings->nats_server_list)), + .username = nats_username.empty() ? getContext()->getConfigRef().getString("nats.username", "") : nats_username, + .password = nats_password.empty() ? getContext()->getConfigRef().getString("nats.password", "") : nats_password, + .token = nats_token.empty() ? getContext()->getConfigRef().getString("nats.token", "") : nats_token, .max_reconnect = static_cast(nats_settings->nats_max_reconnect.value), .reconnect_wait = static_cast(nats_settings->nats_reconnect_wait.value), - .secure = nats_settings->nats_secure.value, - .connection_string = getContext()->getMacros()->expand(nats_settings->nats_address) + .secure = nats_settings->nats_secure.value }; if (configuration.secure) @@ -94,56 +88,12 @@ StorageNATS::StorageNATS( nats_context = addSettings(getContext()); nats_context->makeQueryContext(); -// -// if (queue_base.empty()) -// { -// /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to -// * be table-based and not just a random string, because local exchanges should be declared the same for same tables -// */ -// sharding_exchange = getTableBasedName(exchange_name, table_id_); -// -// /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better -// * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every -// * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base -// * for the names of later declared queues -// */ -// queue_base = getTableBasedName("", table_id_); -// } -// else -// { -// /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and -// * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need to share -// * sharding exchange and bridge exchange -// */ -// sharding_exchange = exchange_name + "_" + queue_base; -// } -// -// bridge_exchange = sharding_exchange + "_bridge"; try { connection = std::make_shared(configuration, log); - if (!connection->connect() && !is_attach) - throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to {}", connection->connectionInfoForLog()); -// if (connection->connect()) -// initNATS(); -// else if (!is_attach) -// throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to {}", connection->connectionInfoForLog()); -// auto sub = connection->createSubscription("foo", onMsg, nullptr); -// int64_t n; -// connection->getHandler().startBlockingLoop(); -// while (true) { -// natsSubscription_GetDelivered(sub.get(), &n); -// printf("Read n : %ld\n", n); -// fflush(stdout); -// std::this_thread::sleep_for(std::chrono::milliseconds(500)); -// } -// auto t2 = std::thread([this](){ -// connection->getHandler().updateLoopState(Loop::RUN); -// LOG_DEBUG(log, "Storteng lup"); -// connection->getHandler().startLoop(); -// }); -// t2.join(); + if (!connection->connect()) + throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "Cannot connect to {}", connection->connectionInfoForLog()); } catch (...) { @@ -164,12 +114,12 @@ StorageNATS::StorageNATS( } -Names StorageNATS::parseSubjects(String subjects_list) +Names StorageNATS::parseList(const String& list) { Names result; - if (subjects_list.empty()) + if (list.empty()) return result; - boost::split(result, subjects_list, [](char c){ return c == ','; }); + boost::split(result, list, [](char c){ return c == ','; }); for (String & key : result) boost::trim(key); @@ -256,15 +206,8 @@ void StorageNATS::decrementReader() void StorageNATS::connectionFunc() { -// if (nats_is_ready) -// return; - if (!connection->reconnect()) connection_task->scheduleAfter(RESCHEDULE_MS); -// if (connection->reconnect()) -// initNATS(); -// else -// connection_task->scheduleAfter(RESCHEDULE_MS); } @@ -326,7 +269,7 @@ Pipe StorageNATS::read( if (connection->getHandler().loopRunning()) deactivateTask(looping_task, false, true); if (!connection->reconnect()) - throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "No connection to {}", connection->connectionInfoForLog()); + throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "No connection to {}", connection->connectionInfoForLog()); } Pipes pipes; @@ -335,7 +278,7 @@ Pipe StorageNATS::read( for (size_t i = 0; i < num_created_consumers; ++i) { auto nats_source = std::make_shared( - *this, storage_snapshot, modified_context, column_names, 1, nats_settings->nats_commit_on_select); + *this, storage_snapshot, modified_context, column_names, 1); auto converting_dag = ActionsDAG::makeConvertingActions( nats_source->getPort().getHeader().getColumnsWithTypeAndName(), @@ -411,8 +354,6 @@ void StorageNATS::shutdown() { for (auto & buffer : buffers) buffer->closeChannel(); - -// cleanupNATS(); } /// It is important to close connection here - before removing consumer buffers, because @@ -467,8 +408,7 @@ ConsumerBufferPtr StorageNATS::popReadBuffer(std::chrono::milliseconds timeout) ConsumerBufferPtr StorageNATS::createReadBuffer() { return std::make_shared( - connection->getHandler(), connection, subjects, - unique_strbase, log, row_delimiter, queue_size, shutdown_called); + connection, subjects, getStorageID().getFullTableName(), log, row_delimiter, queue_size, shutdown_called); } @@ -603,7 +543,7 @@ bool StorageNATS::streamToViews() for (size_t i = 0; i < num_created_consumers; ++i) { auto source = std::make_shared( - *this, storage_snapshot, nats_context, column_names, block_size, false); + *this, storage_snapshot, nats_context, column_names, block_size); sources.emplace_back(source); pipes.emplace_back(source); @@ -661,9 +601,8 @@ bool StorageNATS::streamToViews() } } - if ((queue_empty == num_created_consumers) && (++read_attempts == MAX_FAILED_READ_ATTEMPTS)) + if (queue_empty == num_created_consumers) { - read_attempts = 0; LOG_TRACE(log, "Reschedule streaming. Queues are empty."); return true; } @@ -688,9 +627,9 @@ void registerStorageNATS(StorageFactory & factory) nats_settings->loadFromQuery(*args.storage_def); - if (!nats_settings->nats_host_port.changed - && !nats_settings->nats_address.changed) - throw Exception("You must specify either `nats_host_port` or `nats_address` settings", + if (!nats_settings->nats_url.changed + && !nats_settings->nats_server_list.changed) + throw Exception("You must specify either `nats_url` or `nats_server_list` settings", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (!nats_settings->nats_format.changed) @@ -706,8 +645,7 @@ void registerStorageNATS(StorageFactory & factory) NamesAndTypesList StorageNATS::getVirtuals() const { return NamesAndTypesList{ - {"_subject", std::make_shared()}, - {"_timestamp", std::make_shared()} + {"_subject", std::make_shared()} }; } diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 7e9fee198a4..e536cf6711a 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -65,13 +65,6 @@ public: const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; -// String getExchange() const { return exchange_name; } -// void unbindExchange(); -// -// bool updateChannel(ChannelPtr & channel); - void updateSubjects(std::vector & subjects_) { subjects_ = subjects; } -// void prepareChannelForBuffer(ConsumerBufferPtr buffer); - void incrementReader(); void decrementReader(); @@ -85,15 +78,6 @@ private: const String schema_name; size_t num_consumers; - /// For insert query. Mark messages as durable. - const bool persistent; - -// /// A table setting. It is possible not to perform any NATS setup, which is supposed to be consumer-side setup: -// /// declaring exchanges, queues, bindings. Instead everything needed from NATS table is to connect to a specific queue. -// /// This solution disables all optimizations and is not really optimal, but allows user to fully control all NATS setup. -// bool use_user_setup; - -// bool hash_exchange; Poco::Logger * log; NATSConnectionManagerPtr connection; /// Connection for all consumers @@ -104,14 +88,10 @@ private: std::mutex buffers_mutex; std::vector buffers; /// available buffers for NATS consumers - String unique_strbase; /// to make unique consumer channel id - /// maximum number of messages in NATS queue (x-max-length). Also used /// to setup size of inner buffer for received messages uint32_t queue_size; - size_t consumer_id = 0; /// counter for consumer buffer, needed for channel id - std::once_flag flag; /// remove exchange only once std::mutex task_mutex; BackgroundSchedulePool::TaskHolder streaming_task; @@ -120,20 +100,9 @@ private: uint64_t milliseconds_to_wait; - /** - * ╰( ͡° ͜ʖ ͡° )つ──☆* Evil atomics: - */ /// Needed for tell MV or producer background tasks /// that they must finish as soon as possible. std::atomic shutdown_called{false}; - /// Counter for producer buffers, needed for channel id. - /// Needed to generate unique producer buffer identifiers. - std::atomic producer_id = 1; - /// Has connection background task completed successfully? - /// It is started only once -- in constructor. - std::atomic nats_is_ready = false; - /// Allow to remove exchange only once. - std::atomic exchange_removed = false; /// For select query we must be aware of the end of streaming /// to be able to turn off the loop. std::atomic readers_count = 0; @@ -147,13 +116,10 @@ private: /// started. std::mutex loop_mutex; - size_t read_attempts = 0; mutable bool drop_table = false; bool is_attach; ConsumerBufferPtr createReadBuffer(); - void initializeBuffers(); - bool initialized = false; /// Functions working in the background void streamingToViewsFunc(); @@ -164,21 +130,13 @@ private: void stopLoop(); void stopLoopIfNoReaders(); - static Names parseSubjects(String subjects_list); - static AMQP::ExchangeType defineExchangeType(String exchange_type_); + static Names parseList(const String& list); static String getTableBasedName(String name, const StorageID & table_id); ContextMutablePtr addSettings(ContextPtr context) const; size_t getMaxBlockSize() const; void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); -// void initNATS(); -// void cleanupNATS() const; - -// void initExchange(AMQP::TcpChannel & nats_channel); -// void bindExchange(AMQP::TcpChannel & nats_channel); -// void bindQueue(size_t queue_id, AMQP::TcpChannel & nats_channel); - bool streamToViews(); bool checkDependencies(const StorageID & table_id); diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index 44095200d7e..49b69b34365 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -88,7 +88,7 @@ void WriteBufferToNATSProducer::countRow() } } -void WriteBufferToNATSProducer::publish() +natsStatus WriteBufferToNATSProducer::publish() { String payload; @@ -107,18 +107,18 @@ void WriteBufferToNATSProducer::publish() else { LOG_DEBUG(log, "Something went wrong during publishing to NATS subject {}.", subject); + break; } } if (status == NATS_OK) - { status = natsConnection_Flush(connection.getConnection()); - } - else - { + + if (status != NATS_OK) LOG_DEBUG(log, "Something went wrong during publishing to NATS subject {}.", subject); - } + iterateEventLoop(); + return status; } @@ -126,11 +126,14 @@ void WriteBufferToNATSProducer::writingFunc() { while ((!payloads.empty() || wait_all) && !shutdown_called.load()) { - publish(); + auto status = publish(); - if (wait_num.load() && payloads.empty()) + if (wait_payloads.load() && payloads.empty()) wait_all = false; + if (status != NATS_OK && wait_all) + connection.reconnect(); + iterateEventLoop(); } diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h index 779e35519c7..e973c19e332 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.h +++ b/src/Storages/NATS/WriteBufferToNATSProducer.h @@ -32,7 +32,7 @@ public: void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } - void updateMaxWait() { wait_num.store(payload_counter); } + void updateMaxWait() { wait_payloads.store(true); } private: void nextImpl() override; @@ -41,7 +41,7 @@ private: void iterateEventLoop(); void writingFunc(); - void publish(); + natsStatus publish(); NATSConnectionManager connection; const String subject; @@ -68,7 +68,7 @@ private: /* false: until writeSuffix is called * true: means payloads.queue will not grow anymore */ - std::atomic wait_num = 0; + std::atomic wait_payloads = false; UInt64 payload_counter = 0; Poco::Logger * log; From d9a19910a87341aea03d185559f6ed05b69e49bd Mon Sep 17 00:00:00 2001 From: tchepavel Date: Thu, 12 May 2022 15:32:51 +0300 Subject: [PATCH 010/627] Add nats-io cmake; move subscribe --- contrib/nats-io-cmake/CMakeLists.txt | 54 +++++++++++++++++++ src/Storages/NATS/NATSSource.cpp | 2 +- .../NATS/ReadBufferFromNATSConsumer.cpp | 7 ++- .../NATS/ReadBufferFromNATSConsumer.h | 5 +- 4 files changed, 64 insertions(+), 4 deletions(-) diff --git a/contrib/nats-io-cmake/CMakeLists.txt b/contrib/nats-io-cmake/CMakeLists.txt index e69de29bb2d..6dc94445c46 100644 --- a/contrib/nats-io-cmake/CMakeLists.txt +++ b/contrib/nats-io-cmake/CMakeLists.txt @@ -0,0 +1,54 @@ +option (ENABLE_KAFKA "Enable kafka" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_KAFKA) + message(STATUS "Not using librdkafka") + return() +endif() + +set(NATS_IO_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/nats-io/src") + +if(UNIX) + set(NATS_PLATFORM_INCLUDE "unix") +elseif(WIN32) + set(NATS_PLATFORM_INCLUDE "apple") +endif() + +file(GLOB PS_SOURCES "${NATS_IO_SOURCE_DIR}/${NATS_PLATFORM_INCLUDE}/*.c") +set(SRCS + "${NATS_IO_SOURCE_DIR}/asynccb.c" + "${NATS_IO_SOURCE_DIR}/buf.c" + "${NATS_IO_SOURCE_DIR}/comsock.c" + "${NATS_IO_SOURCE_DIR}/conn.c" + "${NATS_IO_SOURCE_DIR}/crypto.c" + "${NATS_IO_SOURCE_DIR}/hash.c" + "${NATS_IO_SOURCE_DIR}/js.c" + "${NATS_IO_SOURCE_DIR}/jsm.c" + "${NATS_IO_SOURCE_DIR}/kv.c" + "${NATS_IO_SOURCE_DIR}/msg.c" + "${NATS_IO_SOURCE_DIR}/nats.c" + "${NATS_IO_SOURCE_DIR}/natstime.c" + "${NATS_IO_SOURCE_DIR}/nkeys.c" + "${NATS_IO_SOURCE_DIR}/nuid.c" + "${NATS_IO_SOURCE_DIR}/opts.c" + "${NATS_IO_SOURCE_DIR}/parser.c" + "${NATS_IO_SOURCE_DIR}/pub.c" + "${NATS_IO_SOURCE_DIR}/srvpool.c" + "${NATS_IO_SOURCE_DIR}/stats.c" + "${NATS_IO_SOURCE_DIR}/status.c" + "${NATS_IO_SOURCE_DIR}/sub.c" + "${NATS_IO_SOURCE_DIR}/timer.c" + "${NATS_IO_SOURCE_DIR}/url.c" + "${NATS_IO_SOURCE_DIR}/util.c" +) + +add_library(_nats-io ${SRCS} ${PS_SOURCES}) +add_library(ch_contrib::nats-io ALIAS _nats-io) + +target_include_directories(_nats-io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}) +target_include_directories(_nats-io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}/adapters) +target_include_directories(_nats-io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}/include) +target_include_directories(_nats-io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}/${NATS_PLATFORM_INCLUDE}) + +target_link_libraries(_nats-io + PRIVATE OpenSSL::Crypto OpenSSL::SSL ch_contrib::uv +) diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index 4d66e43ab46..bc33542f7ce 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -54,7 +54,6 @@ NATSSource::NATSSource( , context(context_) , column_names(columns) , max_block_size(max_block_size_) -// , ack_in_suffix(ack_in_suffix_) , non_virtual_header(std::move(headers.first)) , virtual_header(std::move(headers.second)) { @@ -85,6 +84,7 @@ Chunk NATSSource::generateImpl() { auto timeout = std::chrono::milliseconds(context->getSettingsRef().rabbitmq_max_wait_ms.totalMilliseconds()); buffer = storage.popReadBuffer(timeout); + buffer->subscribe(); } if (!buffer || is_finished) diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index 6c09bc00a23..2c2c910eb22 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -34,10 +34,9 @@ ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) + , queue_name(subscribe_queue_name) , received(queue_size_) { - subscribe(subscribe_queue_name); - LOG_DEBUG(log, "Started NATS consumer"); } @@ -53,6 +52,9 @@ ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() void ReadBufferFromNATSConsumer::subscribe(const String & subscribe_queue_name) { + if (subscribed) + return; + for (const auto & subject : subjects) { natsSubscription * ns; auto status = natsConnection_QueueSubscribe( @@ -67,6 +69,7 @@ void ReadBufferFromNATSConsumer::subscribe(const String & subscribe_queue_name) throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "Failed to subscribe to subject {}", subject); } } + subscribed = true; } diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h index f1872c64aaf..18404d622d4 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -57,7 +57,7 @@ public: private: bool nextImpl() override; - void subscribe(const String & subscribe_queue_name); + void subscribe(); static void onMsg(natsConnection *nc, natsSubscription *sub, natsMsg * msg, void * closure); std::shared_ptr connection; @@ -68,6 +68,9 @@ private: bool allowed = true; const std::atomic & stopped; + bool subscribed = false; + String queue_name; + String channel_id; ConcurrentBoundedQueue received; MessageData current; From e1a7ccd0698d36bf717750dde709c97c3c058d7e Mon Sep 17 00:00:00 2001 From: tchepavel Date: Thu, 12 May 2022 16:52:14 +0300 Subject: [PATCH 011/627] NATS. Fix some reconnection issues; add settings --- src/Storages/NATS/NATSConnection.cpp | 25 +++++++++++-------- src/Storages/NATS/NATSHandler.cpp | 5 ++++ src/Storages/NATS/NATSHandler.h | 2 ++ src/Storages/NATS/NATSSettings.h | 1 + .../NATS/ReadBufferFromNATSConsumer.cpp | 11 +++++--- .../NATS/ReadBufferFromNATSConsumer.h | 8 ++---- src/Storages/NATS/StorageNATS.cpp | 6 +++-- .../NATS/WriteBufferToNATSProducer.cpp | 19 +++++++------- 8 files changed, 45 insertions(+), 32 deletions(-) diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index 110f4c740d8..5e8659692e5 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -102,22 +102,27 @@ void NATSConnectionManager::connectImpl() natsOptions_SetReconnectWait(options, configuration.reconnect_wait); natsOptions_SetDisconnectedCB(options, disconnectedCallback, log); natsOptions_SetReconnectedCB(options, reconnectedCallback, log); - auto status = natsConnection_Connect(&connection, options); - if (status != NATS_OK) + natsConnection * tmp = nullptr; + event_handler.setThreadLocalLoop(); + auto status = natsConnection_Connect(&tmp, options); + LOG_DEBUG(log, "New connection status {}", static_cast(status)); + if (status == NATS_OK) { - if (!configuration.url.empty()) - LOG_DEBUG(log, "Failed to connect to NATS on address: {}", configuration.url); - else - LOG_DEBUG(log, "Failed to connect to NATS cluster"); - return; + connection = tmp; + has_connection = true; + event_handler.changeConnectionStatus(true); + } + else + { + LOG_DEBUG(log, "New connection failed. Nats status text: {}. Last error message: {}", natsStatus_GetText(status), nats_GetLastError(nullptr)); } - - has_connection = true; - event_handler.changeConnectionStatus(true); } void NATSConnectionManager::disconnectImpl() { + if (!has_connection) + return; + natsConnection_Close(connection); /** Connection is not closed immediately (firstly, all pending operations are completed, and then diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index cdaa93ef93c..b3fc6c2773c 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -57,6 +57,11 @@ void NATSHandler::iterateLoop() } } +void NATSHandler::setThreadLocalLoop() { + std::lock_guard lock(startup_mutex); + natsLibuv_SetThreadLocalLoop(loop); +} + /// Do not need synchronization as in iterateLoop(), because this method is used only for /// initial NATS setup - at this point there is no background loop thread. void NATSHandler::startBlockingLoop() diff --git a/src/Storages/NATS/NATSHandler.h b/src/Storages/NATS/NATSHandler.h index 2854bf9a069..fea03520cea 100644 --- a/src/Storages/NATS/NATSHandler.h +++ b/src/Storages/NATS/NATSHandler.h @@ -40,6 +40,8 @@ public: /// No synchronization is done with the main loop thread. void startBlockingLoop(); + void setThreadLocalLoop(); + void stopLoop(); void changeConnectionStatus(bool is_running); diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h index 4304d416368..3f852d247f2 100644 --- a/src/Storages/NATS/NATSSettings.h +++ b/src/Storages/NATS/NATSSettings.h @@ -15,6 +15,7 @@ namespace DB M(Char, nats_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ M(String, nats_schema, "", "Schema identifier (used by schema-based formats) for NATS engine", 0) \ M(UInt64, nats_num_consumers, 1, "The number of consumer channels per table.", 0) \ + M(String, nats_queue_group, "", "Name for queue group of NATS subscribers.", 0) \ M(Bool, nats_secure, false, "Use SSL connection", 0) \ M(UInt64, nats_max_reconnect, 5, "Maximum amount of reconnection attempts.", 0) \ M(UInt64, nats_reconnect_wait, 2000, "Amount to sleep between each reconnect attempt.", 0) \ diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index 2c2c910eb22..638e6d1f44e 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -39,7 +39,6 @@ ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( { } - ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() { for (const auto& subscription : subscriptions) { @@ -49,8 +48,7 @@ ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() BufferBase::set(nullptr, 0, 0); } - -void ReadBufferFromNATSConsumer::subscribe(const String & subscribe_queue_name) +void ReadBufferFromNATSConsumer::subscribe() { if (subscribed) return; @@ -58,7 +56,7 @@ void ReadBufferFromNATSConsumer::subscribe(const String & subscribe_queue_name) for (const auto & subject : subjects) { natsSubscription * ns; auto status = natsConnection_QueueSubscribe( - &ns, connection->getConnection(), subject.c_str(), subscribe_queue_name.c_str(), onMsg, static_cast(this)); + &ns, connection->getConnection(), subject.c_str(), queue_name.c_str(), onMsg, static_cast(this)); if (status == NATS_OK) { LOG_DEBUG(log, "Subscribed to subject {}", subject); @@ -72,6 +70,11 @@ void ReadBufferFromNATSConsumer::subscribe(const String & subscribe_queue_name) subscribed = true; } +void ReadBufferFromNATSConsumer::unsubscribe() +{ + for (const auto & subscription : subscriptions) + natsSubscription_Unsubscribe(subscription.get()); +} bool ReadBufferFromNATSConsumer::nextImpl() { diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h index 18404d622d4..e31a677203d 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -38,11 +38,8 @@ public: }; std::vector & getChannel() { return subscriptions; } - void closeChannel() - { - for (const auto & subscription : subscriptions) - natsSubscription_Unsubscribe(subscription.get()); - } + void subscribe(); + void unsubscribe(); void updateSubjects(std::vector & subjects_) { subjects = subjects_; } size_t subjectsCount() { return subjects.size(); } @@ -57,7 +54,6 @@ public: private: bool nextImpl() override; - void subscribe(); static void onMsg(natsConnection *nc, natsSubscription *sub, natsMsg * msg, void * closure); std::shared_ptr connection; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 7ee6ebfd7e8..99ad73f088d 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -353,7 +353,7 @@ void StorageNATS::shutdown() if (drop_table) { for (auto & buffer : buffers) - buffer->closeChannel(); + buffer->unsubscribe(); } /// It is important to close connection here - before removing consumer buffers, because @@ -408,7 +408,9 @@ ConsumerBufferPtr StorageNATS::popReadBuffer(std::chrono::milliseconds timeout) ConsumerBufferPtr StorageNATS::createReadBuffer() { return std::make_shared( - connection, subjects, getStorageID().getFullTableName(), log, row_delimiter, queue_size, shutdown_called); + connection, subjects, + nats_settings->nats_queue_group.changed ? nats_settings->nats_queue_group.value : getStorageID().getFullTableName(), + log, row_delimiter, queue_size, shutdown_called); } diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index 49b69b34365..073a027e92d 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -84,7 +84,6 @@ void WriteBufferToNATSProducer::countRow() ++payload_counter; if (!payloads.push(payload)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); - LOG_DEBUG(log, "Pushed payload to queue {} {}", payload, payloads.size()); } } @@ -99,23 +98,23 @@ natsStatus WriteBufferToNATSProducer::publish() if (!pop_result) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not pop payload"); + status = natsConnection_PublishString(connection.getConnection(), subject.c_str(), payload.c_str()); - if (status == NATS_OK) + if (status != NATS_OK) { - status = natsConnection_PublishString(connection.getConnection(), subject.c_str(), payload.c_str()); - } - else - { - LOG_DEBUG(log, "Something went wrong during publishing to NATS subject {}.", subject); + LOG_DEBUG(log, "Something went wrong during publishing to NATS subject. Nats status text: {}. Last error message: {}", + natsStatus_GetText(status), nats_GetLastError(nullptr)); break; } } if (status == NATS_OK) + { status = natsConnection_Flush(connection.getConnection()); - - if (status != NATS_OK) - LOG_DEBUG(log, "Something went wrong during publishing to NATS subject {}.", subject); + if (status != NATS_OK) + LOG_DEBUG(log, "Something went wrong during flushing NATS connection. Nats status text: {}. Last error message: {}", + natsStatus_GetText(status), nats_GetLastError(nullptr)); + } iterateEventLoop(); return status; From cc94378698eecbd2a69cf7e030c28854b5c7e55b Mon Sep 17 00:00:00 2001 From: tchepavel Date: Thu, 12 May 2022 19:11:32 +0300 Subject: [PATCH 012/627] Add nats-io cmake; finally fix publishing --- contrib/nats-io-cmake/CMakeLists.txt | 14 +++--- docker/test/fasttest/run.sh | 1 + src/CMakeLists.txt | 8 +++- src/Core/config_core.h.in | 1 + .../ExternalDataSourceConfiguration.cpp | 5 ++- src/Storages/NATS/NATSConnection.cpp | 5 +-- src/Storages/NATS/NATSHandler.cpp | 1 + .../NATS/WriteBufferToNATSProducer.cpp | 44 +++++++++---------- src/Storages/NATS/WriteBufferToNATSProducer.h | 4 +- src/Storages/registerStorages.cpp | 11 +++-- src/configure_config.cmake | 3 ++ 11 files changed, 57 insertions(+), 40 deletions(-) diff --git a/contrib/nats-io-cmake/CMakeLists.txt b/contrib/nats-io-cmake/CMakeLists.txt index 6dc94445c46..9cd5d0daa51 100644 --- a/contrib/nats-io-cmake/CMakeLists.txt +++ b/contrib/nats-io-cmake/CMakeLists.txt @@ -41,14 +41,14 @@ set(SRCS "${NATS_IO_SOURCE_DIR}/util.c" ) -add_library(_nats-io ${SRCS} ${PS_SOURCES}) -add_library(ch_contrib::nats-io ALIAS _nats-io) +add_library(_nats_io ${SRCS} ${PS_SOURCES}) +add_library(ch_contrib::nats_io ALIAS _nats_io) -target_include_directories(_nats-io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}) -target_include_directories(_nats-io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}/adapters) -target_include_directories(_nats-io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}/include) -target_include_directories(_nats-io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}/${NATS_PLATFORM_INCLUDE}) +target_include_directories(_nats_io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}) +target_include_directories(_nats_io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}/adapters) +target_include_directories(_nats_io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}/include) +target_include_directories(_nats_io SYSTEM PUBLIC ${NATS_IO_SOURCE_DIR}/${NATS_PLATFORM_INCLUDE}) -target_link_libraries(_nats-io +target_link_libraries(_nats_io PRIVATE OpenSSL::Crypto OpenSSL::SSL ch_contrib::uv ) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index c547ae03a52..949ff5602b9 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -177,6 +177,7 @@ function clone_submodules contrib/jemalloc contrib/replxx contrib/wyhash + contrib/nats-io ) git submodule sync diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4e364629f24..ea8b58f07ea 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -107,7 +107,9 @@ if (TARGET ch_contrib::rdkafka) add_headers_and_sources(dbms Storages/Kafka) endif() -add_headers_and_sources(dbms Storages/NATS) +if (TARGET ch_contrib::nats_io) + add_headers_and_sources(dbms Storages/NATS) +endif() if (TARGET ch_contrib::amqp_cpp) add_headers_and_sources(dbms Storages/RabbitMQ) @@ -382,7 +384,9 @@ if (TARGET ch_contrib::rdkafka) dbms_target_link_libraries(PRIVATE ch_contrib::rdkafka ch_contrib::cppkafka) endif() -dbms_target_link_libraries(PRIVATE ch_contrib::nats-io) +if (TARGET ch_contrib::nats_io) + dbms_target_link_libraries(PRIVATE ch_contrib::nats_io) +endif() if (TARGET ch_contrib::sasl2) dbms_target_link_libraries(PRIVATE ch_contrib::sasl2) diff --git a/src/Core/config_core.h.in b/src/Core/config_core.h.in index 3fc2503aaa5..46c77593d4e 100644 --- a/src/Core/config_core.h.in +++ b/src/Core/config_core.h.in @@ -6,6 +6,7 @@ #cmakedefine01 USE_MYSQL #cmakedefine01 USE_RDKAFKA #cmakedefine01 USE_AMQPCPP +#cmakedefine01 USE_NATSIO #cmakedefine01 USE_EMBEDDED_COMPILER #cmakedefine01 USE_SSL #cmakedefine01 USE_LDAP diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 1cb123120f2..683e5608a12 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -18,8 +18,9 @@ #if USE_MYSQL #include #endif - +#if USE_NATSIO #include +#endif #include @@ -485,8 +486,10 @@ template bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); #endif +#if USE_NATSIO template bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); +#endif template std::optional getExternalDataSourceConfiguration( diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index 5e8659692e5..26b95281c21 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -102,13 +102,10 @@ void NATSConnectionManager::connectImpl() natsOptions_SetReconnectWait(options, configuration.reconnect_wait); natsOptions_SetDisconnectedCB(options, disconnectedCallback, log); natsOptions_SetReconnectedCB(options, reconnectedCallback, log); - natsConnection * tmp = nullptr; event_handler.setThreadLocalLoop(); - auto status = natsConnection_Connect(&tmp, options); - LOG_DEBUG(log, "New connection status {}", static_cast(status)); + auto status = natsConnection_Connect(&connection, options); if (status == NATS_OK) { - connection = tmp; has_connection = true; event_handler.changeConnectionStatus(true); } diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index b3fc6c2773c..b83bea6b63c 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -67,6 +67,7 @@ void NATSHandler::setThreadLocalLoop() { void NATSHandler::startBlockingLoop() { LOG_DEBUG(log, "Started blocking loop."); + natsLibuv_SetThreadLocalLoop(loop); uv_run(loop, UV_RUN_DEFAULT); } diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index 073a027e92d..5b885840798 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -5,8 +5,6 @@ #include #include #include -#include -#include #include #include #include @@ -87,37 +85,38 @@ void WriteBufferToNATSProducer::countRow() } } -natsStatus WriteBufferToNATSProducer::publish() +void WriteBufferToNATSProducer::publish() { - String payload; + uv_thread_t flush_thread; - natsStatus status{NATS_OK}; - while (!payloads.empty()) + uv_thread_create(&flush_thread, publishThreadFunc, static_cast(this)); + + connection.getHandler().startLoop(); + uv_thread_join(&flush_thread); +} + +void WriteBufferToNATSProducer::publishThreadFunc(void * arg) { + String payload; + WriteBufferToNATSProducer * buffer = static_cast(arg); + + natsStatus status; + while (!buffer->payloads.empty()) { - bool pop_result = payloads.pop(payload); + bool pop_result = buffer->payloads.pop(payload); if (!pop_result) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not pop payload"); - status = natsConnection_PublishString(connection.getConnection(), subject.c_str(), payload.c_str()); + status = natsConnection_PublishString(buffer->connection.getConnection(), buffer->subject.c_str(), payload.c_str()); if (status != NATS_OK) { - LOG_DEBUG(log, "Something went wrong during publishing to NATS subject. Nats status text: {}. Last error message: {}", + LOG_DEBUG(buffer->log, "Something went wrong during publishing to NATS subject. Nats status text: {}. Last error message: {}", natsStatus_GetText(status), nats_GetLastError(nullptr)); break; } } - if (status == NATS_OK) - { - status = natsConnection_Flush(connection.getConnection()); - if (status != NATS_OK) - LOG_DEBUG(log, "Something went wrong during flushing NATS connection. Nats status text: {}. Last error message: {}", - natsStatus_GetText(status), nats_GetLastError(nullptr)); - } - - iterateEventLoop(); - return status; + nats_ReleaseThreadMemory(); } @@ -125,12 +124,13 @@ void WriteBufferToNATSProducer::writingFunc() { while ((!payloads.empty() || wait_all) && !shutdown_called.load()) { - auto status = publish(); + publish(); - if (wait_payloads.load() && payloads.empty()) + LOG_DEBUG(log, "Writing func {} {} {}", wait_payloads.load(), payloads.empty(), natsConnection_Buffered(connection.getConnection())); + if (wait_payloads.load() && payloads.empty() && natsConnection_Buffered(connection.getConnection()) == 0) wait_all = false; - if (status != NATS_OK && wait_all) + if (!connection.isConnected() && wait_all) connection.reconnect(); iterateEventLoop(); diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h index e973c19e332..e081f1a738d 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.h +++ b/src/Storages/NATS/WriteBufferToNATSProducer.h @@ -41,7 +41,9 @@ private: void iterateEventLoop(); void writingFunc(); - natsStatus publish(); + void publish(); + + static void publishThreadFunc(void * arg); NATSConnectionManager connection; const String subject; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 1e777cbf946..9075c8fe2f9 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -51,7 +51,6 @@ void registerStorageMySQL(StorageFactory & factory); void registerStorageMongoDB(StorageFactory & factory); -void registerStorageNATS(StorageFactory & factory); #if USE_RDKAFKA void registerStorageKafka(StorageFactory & factory); @@ -61,6 +60,10 @@ void registerStorageKafka(StorageFactory & factory); void registerStorageRabbitMQ(StorageFactory & factory); #endif +#if USE_NATSIO +void registerStorageNATS(StorageFactory & factory); +#endif + #if USE_ROCKSDB void registerStorageEmbeddedRocksDB(StorageFactory & factory); #endif @@ -130,8 +133,6 @@ void registerStorages() registerStorageMongoDB(factory); - registerStorageNATS(factory); - #if USE_RDKAFKA registerStorageKafka(factory); #endif @@ -144,6 +145,10 @@ void registerStorages() registerStorageRabbitMQ(factory); #endif + #if USE_NATSIO + registerStorageNATS(factory); + #endif + #if USE_ROCKSDB registerStorageEmbeddedRocksDB(factory); #endif diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 519307ba28a..3f3ddf54716 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -49,6 +49,9 @@ endif() if (TARGET ch_contrib::amqp_cpp) set(USE_AMQPCPP 1) endif() +if (TARGET ch_contrib::nats_io) + set(USE_NATSIO 1) +endif() if (TARGET ch_contrib::cassandra) set(USE_CASSANDRA 1) endif() From fe1cdf8ef19ccddd5820141b25c21d29ec9c4a01 Mon Sep 17 00:00:00 2001 From: tchepavel Date: Thu, 12 May 2022 22:11:59 +0300 Subject: [PATCH 013/627] NATS. Final functional improvements --- src/Storages/NATS/NATSConnection.cpp | 30 ++++++++++++------- src/Storages/NATS/NATSConnection.h | 2 +- src/Storages/NATS/NATSHandler.cpp | 16 +++------- src/Storages/NATS/NATSHandler.h | 11 ++----- .../NATS/ReadBufferFromNATSConsumer.h | 2 -- src/Storages/NATS/StorageNATS.cpp | 1 - 6 files changed, 26 insertions(+), 36 deletions(-) diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index 26b95281c21..cb3c3dc626f 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -57,7 +57,7 @@ bool NATSConnectionManager::reconnect() disconnectImpl(); - LOG_DEBUG(log, "Trying to restore connection to {}", connectionInfoForLog()); + LOG_DEBUG(log, "Trying to restore connection to NATS {}", connectionInfoForLog()); connectImpl(); return isConnectedImpl(); @@ -83,17 +83,24 @@ bool NATSConnectionManager::isConnectedImpl() const void NATSConnectionManager::connectImpl() { natsOptions * options = event_handler.getOptions(); - natsOptions_SetUserInfo(options, configuration.username.c_str(), configuration.password.c_str()); - if (configuration.secure) { + if (!configuration.username.empty() && !configuration.password.empty()) + natsOptions_SetUserInfo(options, configuration.username.c_str(), configuration.password.c_str()); + if (!configuration.token.empty()) + natsOptions_SetToken(options, configuration.token.c_str()); + + if (configuration.secure) + { natsOptions_SetSecure(options, true); natsOptions_SkipServerVerification(options, true); } if (!configuration.url.empty()) { natsOptions_SetURL(options, configuration.url.c_str()); - } else { + } else + { const char * servers[configuration.servers.size()]; - for (size_t i = 0; i < configuration.servers.size(); ++i) { + for (size_t i = 0; i < configuration.servers.size(); ++i) + { servers[i] = configuration.servers[i].c_str(); } natsOptions_SetServers(options, servers, configuration.servers.size()); @@ -102,8 +109,11 @@ void NATSConnectionManager::connectImpl() natsOptions_SetReconnectWait(options, configuration.reconnect_wait); natsOptions_SetDisconnectedCB(options, disconnectedCallback, log); natsOptions_SetReconnectedCB(options, reconnectedCallback, log); - event_handler.setThreadLocalLoop(); - auto status = natsConnection_Connect(&connection, options); + natsStatus status; + { + auto lock = event_handler.setThreadLocalLoop(); + status = natsConnection_Connect(&connection, options); + } if (status == NATS_OK) { has_connection = true; @@ -111,7 +121,8 @@ void NATSConnectionManager::connectImpl() } else { - LOG_DEBUG(log, "New connection failed. Nats status text: {}. Last error message: {}", natsStatus_GetText(status), nats_GetLastError(nullptr)); + LOG_DEBUG(log, "New connection to {} failed. Nats status text: {}. Last error message: {}", + connectionInfoForLog(), natsStatus_GetText(status), nats_GetLastError(nullptr)); } } @@ -122,9 +133,6 @@ void NATSConnectionManager::disconnectImpl() natsConnection_Close(connection); - /** Connection is not closed immediately (firstly, all pending operations are completed, and then - * an AMQP closing-handshake is performed). But cannot open a new connection until previous one is properly closed - */ size_t cnt_retries = 0; while (!natsConnection_IsClosed(connection) && cnt_retries++ != RETRIES_MAX) event_handler.iterateLoop(); diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h index 76f89eaeddb..3045ab193d0 100644 --- a/src/Storages/NATS/NATSConnection.h +++ b/src/Storages/NATS/NATSConnection.h @@ -63,7 +63,7 @@ private: natsConnection * connection; - // true if at any point was a successful connection + // true if at any point successfully connected to NATS bool has_connection = false; std::mutex mutex; diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index b83bea6b63c..ea8e2d7b503 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -19,7 +19,7 @@ NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : natsLibuv_Init(); natsLibuv_SetThreadLocalLoop(loop); natsOptions_Create(&opts); - status = natsOptions_SetEventLoop(opts, static_cast(loop), + natsOptions_SetEventLoop(opts, static_cast(loop), natsLibuv_Attach, natsLibuv_Read, natsLibuv_Write, @@ -57,18 +57,10 @@ void NATSHandler::iterateLoop() } } -void NATSHandler::setThreadLocalLoop() { - std::lock_guard lock(startup_mutex); +LockPtr NATSHandler::setThreadLocalLoop() { + auto lock = std::make_unique>(startup_mutex); natsLibuv_SetThreadLocalLoop(loop); -} - -/// Do not need synchronization as in iterateLoop(), because this method is used only for -/// initial NATS setup - at this point there is no background loop thread. -void NATSHandler::startBlockingLoop() -{ - LOG_DEBUG(log, "Started blocking loop."); - natsLibuv_SetThreadLocalLoop(loop); - uv_run(loop, UV_RUN_DEFAULT); + return lock; } void NATSHandler::stopLoop() diff --git a/src/Storages/NATS/NATSHandler.h b/src/Storages/NATS/NATSHandler.h index fea03520cea..b1a81a81c31 100644 --- a/src/Storages/NATS/NATSHandler.h +++ b/src/Storages/NATS/NATSHandler.h @@ -20,6 +20,7 @@ namespace Loop } using SubscriptionPtr = std::unique_ptr; +using LockPtr = std::unique_ptr>; class NATSHandler { @@ -36,11 +37,7 @@ public: /// Adds synchronization with main background loop. void iterateLoop(); - /// Loop to wait for small tasks in a blocking mode. - /// No synchronization is done with the main loop thread. - void startBlockingLoop(); - - void setThreadLocalLoop(); + LockPtr setThreadLocalLoop(); void stopLoop(); @@ -51,13 +48,11 @@ public: void updateLoopState(UInt8 state) { loop_state.store(state); } UInt8 getLoopState() { return loop_state.load(); } - natsStatus getStatus() { return status; } natsOptions * getOptions() { return opts; } private: uv_loop_t * loop; natsOptions * opts = nullptr; - natsStatus status = NATS_OK; Poco::Logger * log; std::atomic connection_running, loop_running; @@ -65,6 +60,4 @@ private: std::mutex startup_mutex; }; -using NATSHandlerPtr = std::shared_ptr; - } diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h index e31a677203d..78a0a418349 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -37,11 +37,9 @@ public: String subject; }; - std::vector & getChannel() { return subscriptions; } void subscribe(); void unsubscribe(); - void updateSubjects(std::vector & subjects_) { subjects = subjects_; } size_t subjectsCount() { return subjects.size(); } bool isConsumerStopped() { return stopped; } diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 99ad73f088d..7a9e895e829 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -186,7 +186,6 @@ void StorageNATS::stopLoopIfNoReaders() void StorageNATS::startLoop() { -// assert(nats_is_ready); connection->getHandler().updateLoopState(Loop::RUN); looping_task->activateAndSchedule(); } From 1295ee852c489dd8fec6ba914af0ae194092e89a Mon Sep 17 00:00:00 2001 From: tchepavel Date: Thu, 12 May 2022 22:19:11 +0300 Subject: [PATCH 014/627] NATS. Style check --- src/Storages/NATS/NATSConnection.cpp | 8 +- src/Storages/NATS/NATSConnection.h | 5 +- src/Storages/NATS/NATSHandler.cpp | 13 +-- src/Storages/NATS/NATSHandler.h | 6 +- src/Storages/NATS/NATSSettings.cpp | 4 +- src/Storages/NATS/NATSSettings.h | 5 +- src/Storages/NATS/NATSSink.cpp | 6 +- src/Storages/NATS/NATSSink.h | 1 - src/Storages/NATS/NATSSource.cpp | 14 +-- src/Storages/NATS/NATSSource.h | 13 ++- .../NATS/ReadBufferFromNATSConsumer.cpp | 47 +++++----- .../NATS/ReadBufferFromNATSConsumer.h | 26 +++--- src/Storages/NATS/StorageNATS.cpp | 90 +++++++++---------- src/Storages/NATS/StorageNATS.h | 22 ++--- src/Storages/NATS/UVLoop.h | 4 +- .../NATS/WriteBufferToNATSProducer.cpp | 53 +++++------ src/Storages/NATS/WriteBufferToNATSProducer.h | 28 +++--- 17 files changed, 163 insertions(+), 182 deletions(-) diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index cb3c3dc626f..e46900ce57b 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -1,7 +1,7 @@ #include "NATSConnection.h" -#include #include +#include #include @@ -30,7 +30,8 @@ NATSConnectionManager::~NATSConnectionManager() String NATSConnectionManager::connectionInfoForLog() const { - if (!configuration.url.empty()) { + if (!configuration.url.empty()) + { return "url : " + configuration.url; } return "cluster: " + boost::algorithm::join(configuration.servers, ", "); @@ -96,7 +97,8 @@ void NATSConnectionManager::connectImpl() if (!configuration.url.empty()) { natsOptions_SetURL(options, configuration.url.c_str()); - } else + } + else { const char * servers[configuration.servers.size()]; for (size_t i = 0; i < configuration.servers.size(); ++i) diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h index 3045ab193d0..2315b0f1b12 100644 --- a/src/Storages/NATS/NATSConnection.h +++ b/src/Storages/NATS/NATSConnection.h @@ -1,8 +1,8 @@ #pragma once -#include -#include #include +#include +#include namespace DB { @@ -24,7 +24,6 @@ struct NATSConfiguration class NATSConnectionManager { - public: NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_); ~NATSConnectionManager(); diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index ea8e2d7b503..87954c2b6d1 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -1,7 +1,7 @@ -#include -#include #include #include +#include +#include namespace DB { @@ -43,7 +43,8 @@ void NATSHandler::startLoop() loop_running.store(false); } -void NATSHandler::changeConnectionStatus(bool is_running) { +void NATSHandler::changeConnectionStatus(bool is_running) +{ connection_running.store(is_running); } @@ -57,7 +58,8 @@ void NATSHandler::iterateLoop() } } -LockPtr NATSHandler::setThreadLocalLoop() { +LockPtr NATSHandler::setThreadLocalLoop() +{ auto lock = std::make_unique>(startup_mutex); natsLibuv_SetThreadLocalLoop(loop); return lock; @@ -69,7 +71,8 @@ void NATSHandler::stopLoop() uv_stop(loop); } -NATSHandler::~NATSHandler() { +NATSHandler::~NATSHandler() +{ natsOptions_Destroy(opts); } diff --git a/src/Storages/NATS/NATSHandler.h b/src/Storages/NATS/NATSHandler.h index b1a81a81c31..9ffd58fc07a 100644 --- a/src/Storages/NATS/NATSHandler.h +++ b/src/Storages/NATS/NATSHandler.h @@ -1,13 +1,10 @@ #pragma once -#include #include #include +#include #include -#include -#include #include -#include #include namespace DB @@ -24,7 +21,6 @@ using LockPtr = std::unique_ptr>; class NATSHandler { - public: NATSHandler(uv_loop_t * loop_, Poco::Logger * log_); diff --git a/src/Storages/NATS/NATSSettings.cpp b/src/Storages/NATS/NATSSettings.cpp index ec1149acee4..ffdb79247d2 100644 --- a/src/Storages/NATS/NATSSettings.cpp +++ b/src/Storages/NATS/NATSSettings.cpp @@ -1,7 +1,7 @@ -#include #include -#include #include +#include +#include #include namespace DB diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h index 3f852d247f2..c06d92c0dec 100644 --- a/src/Storages/NATS/NATSSettings.h +++ b/src/Storages/NATS/NATSSettings.h @@ -5,8 +5,7 @@ namespace DB { - class ASTStorage; - +class ASTStorage; #define NATS_RELATED_SETTINGS(M) \ M(String, nats_url, "", "A host-port to connect to NATS server.", 0) \ @@ -25,7 +24,7 @@ namespace DB M(Milliseconds, nats_flush_interval_ms, 0, "Timeout for flushing data from NATS.", 0) \ M(String, nats_username, "", "NATS username", 0) \ M(String, nats_password, "", "NATS password", 0) \ - M(String, nats_token, "", "NATS token", 0) \ + M(String, nats_token, "", "NATS token", 0) #define LIST_OF_NATS_SETTINGS(M) \ NATS_RELATED_SETTINGS(M) \ diff --git a/src/Storages/NATS/NATSSink.cpp b/src/Storages/NATS/NATSSink.cpp index 8be95099261..bae6712a0e3 100644 --- a/src/Storages/NATS/NATSSink.cpp +++ b/src/Storages/NATS/NATSSink.cpp @@ -1,8 +1,8 @@ -#include -#include -#include #include #include +#include +#include +#include #include diff --git a/src/Storages/NATS/NATSSink.h b/src/Storages/NATS/NATSSink.h index 2f887dd3a4f..3023d0c90ea 100644 --- a/src/Storages/NATS/NATSSink.h +++ b/src/Storages/NATS/NATSSink.h @@ -12,7 +12,6 @@ using IOutputFormatPtr = std::shared_ptr; class NATSSink : public SinkToStorage { - public: explicit NATSSink(StorageNATS & storage_, const StorageMetadataPtr & metadata_snapshot_, ContextPtr context_); diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index bc33542f7ce..7219db3e99c 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -31,13 +31,7 @@ NATSSource::NATSSource( ContextPtr context_, const Names & columns, size_t max_block_size_) - : NATSSource( - storage_, - storage_snapshot_, - getHeaders(storage_snapshot_), - context_, - columns, - max_block_size_) + : NATSSource(storage_, storage_snapshot_, getHeaders(storage_snapshot_), context_, columns, max_block_size_) { } @@ -93,8 +87,8 @@ Chunk NATSSource::generateImpl() is_finished = true; MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); - auto input_format = FormatFactory::instance().getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + auto input_format + = FormatFactory::instance().getInputFormat(storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); StreamingFormatExecutor executor(non_virtual_header, input_format); @@ -128,7 +122,7 @@ Chunk NATSSource::generateImpl() if (total_rows == 0) return {}; - auto result_columns = executor.getResultColumns(); + auto result_columns = executor.getResultColumns(); for (auto & column : virtual_columns) result_columns.push_back(std::move(column)); diff --git a/src/Storages/NATS/NATSSource.h b/src/Storages/NATS/NATSSource.h index 4b6b71348c2..08fcf97114a 100644 --- a/src/Storages/NATS/NATSSource.h +++ b/src/Storages/NATS/NATSSource.h @@ -1,8 +1,8 @@ #pragma once #include -#include #include +#include namespace DB @@ -10,14 +10,13 @@ namespace DB class NATSSource : public SourceWithProgress { - public: NATSSource( - StorageNATS & storage_, - const StorageSnapshotPtr & storage_snapshot_, - ContextPtr context_, - const Names & columns, - size_t max_block_size_); + StorageNATS & storage_, + const StorageSnapshotPtr & storage_snapshot_, + ContextPtr context_, + const Names & columns, + size_t max_block_size_); ~NATSSource() override; diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index 638e6d1f44e..4f11a051029 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -1,15 +1,14 @@ -#include -#include -#include -#include #include +#include #include +#include +#include +#include #include #include #include -#include #include "Poco/Timer.h" -#include +#include namespace DB { @@ -21,27 +20,28 @@ namespace ErrorCodes } ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( - std::shared_ptr connection_, - std::vector & subjects_, - const String & subscribe_queue_name, - Poco::Logger * log_, - char row_delimiter_, - uint32_t queue_size_, - const std::atomic & stopped_) - : ReadBuffer(nullptr, 0) - , connection(connection_) - , subjects(subjects_) - , log(log_) - , row_delimiter(row_delimiter_) - , stopped(stopped_) - , queue_name(subscribe_queue_name) - , received(queue_size_) + std::shared_ptr connection_, + std::vector & subjects_, + const String & subscribe_queue_name, + Poco::Logger * log_, + char row_delimiter_, + uint32_t queue_size_, + const std::atomic & stopped_) + : ReadBuffer(nullptr, 0) + , connection(connection_) + , subjects(subjects_) + , log(log_) + , row_delimiter(row_delimiter_) + , stopped(stopped_) + , queue_name(subscribe_queue_name) + , received(queue_size_) { } ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() { - for (const auto& subscription : subscriptions) { + for (const auto & subscription : subscriptions) + { natsSubscription_Unsubscribe(subscription.get()); } @@ -53,7 +53,8 @@ void ReadBufferFromNATSConsumer::subscribe() if (subscribed) return; - for (const auto & subject : subjects) { + for (const auto & subject : subjects) + { natsSubscription * ns; auto status = natsConnection_QueueSubscribe( &ns, connection->getConnection(), subject.c_str(), queue_name.c_str(), onMsg, static_cast(this)); diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h index 78a0a418349..d75bf1c0376 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -1,16 +1,15 @@ #pragma once -#include -#include -#include -#include #include +#include +#include #include +#include #include namespace Poco { - class Logger; +class Logger; } namespace DB @@ -18,16 +17,15 @@ namespace DB class ReadBufferFromNATSConsumer : public ReadBuffer { - public: ReadBufferFromNATSConsumer( - std::shared_ptr connection_, - std::vector & subjects_, - const String & subscribe_queue_name, - Poco::Logger * log_, - char row_delimiter_, - uint32_t queue_size_, - const std::atomic & stopped_); + std::shared_ptr connection_, + std::vector & subjects_, + const String & subscribe_queue_name, + Poco::Logger * log_, + char row_delimiter_, + uint32_t queue_size_, + const std::atomic & stopped_); ~ReadBufferFromNATSConsumer() override; @@ -52,7 +50,7 @@ public: private: bool nextImpl() override; - static void onMsg(natsConnection *nc, natsSubscription *sub, natsMsg * msg, void * closure); + static void onMsg(natsConnection * nc, natsSubscription * sub, natsMsg * msg, void * closure); std::shared_ptr connection; std::vector subscriptions; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 7a9e895e829..31f22e7ebed 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -9,21 +8,21 @@ #include #include #include +#include #include #include #include #include -#include #include #include #include #include #include #include +#include #include #include #include -#include namespace DB { @@ -44,24 +43,24 @@ namespace ErrorCodes StorageNATS::StorageNATS( - const StorageID & table_id_, - ContextPtr context_, - const ColumnsDescription & columns_, - std::unique_ptr nats_settings_, - bool is_attach_) - : IStorage(table_id_) - , WithContext(context_->getGlobalContext()) - , nats_settings(std::move(nats_settings_)) - , subjects(parseList(getContext()->getMacros()->expand(nats_settings->nats_subjects))) - , format_name(getContext()->getMacros()->expand(nats_settings->nats_format)) - , row_delimiter(nats_settings->nats_row_delimiter.value) - , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) - , num_consumers(nats_settings->nats_num_consumers.value) - , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) - , semaphore(0, num_consumers) - , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) - , milliseconds_to_wait(RESCHEDULE_MS) - , is_attach(is_attach_) + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr nats_settings_, + bool is_attach_) + : IStorage(table_id_) + , WithContext(context_->getGlobalContext()) + , nats_settings(std::move(nats_settings_)) + , subjects(parseList(getContext()->getMacros()->expand(nats_settings->nats_subjects))) + , format_name(getContext()->getMacros()->expand(nats_settings->nats_format)) + , row_delimiter(nats_settings->nats_row_delimiter.value) + , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) + , num_consumers(nats_settings->nats_num_consumers.value) + , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) + , semaphore(0, num_consumers) + , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) + , milliseconds_to_wait(RESCHEDULE_MS) + , is_attach(is_attach_) { auto nats_username = getContext()->getMacros()->expand(nats_settings->nats_username); auto nats_password = getContext()->getMacros()->expand(nats_settings->nats_password); @@ -103,23 +102,23 @@ StorageNATS::StorageNATS( } /// One looping task for all consumers as they share the same connection == the same handler == the same event loop - looping_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSLoopingTask", [this]{ loopingFunc(); }); + looping_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSLoopingTask", [this] { loopingFunc(); }); looping_task->deactivate(); - streaming_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSStreamingTask", [this]{ streamingToViewsFunc(); }); + streaming_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSStreamingTask", [this] { streamingToViewsFunc(); }); streaming_task->deactivate(); - connection_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSConnectionManagerTask", [this]{ connectionFunc(); }); + connection_task = getContext()->getMessageBrokerSchedulePool().createTask("NATSConnectionManagerTask", [this] { connectionFunc(); }); connection_task->deactivate(); } -Names StorageNATS::parseList(const String& list) +Names StorageNATS::parseList(const String & list) { Names result; if (list.empty()) return result; - boost::split(result, list, [](char c){ return c == ','; }); + boost::split(result, list, [](char c) { return c == ','; }); for (String & key : result) boost::trim(key); @@ -234,26 +233,26 @@ void StorageNATS::deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool size_t StorageNATS::getMaxBlockSize() const { - return nats_settings->nats_max_block_size.changed - ? nats_settings->nats_max_block_size.value - : (getContext()->getSettingsRef().max_insert_block_size.value / num_consumers); + return nats_settings->nats_max_block_size.changed ? nats_settings->nats_max_block_size.value + : (getContext()->getSettingsRef().max_insert_block_size.value / num_consumers); } Pipe StorageNATS::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & /* query_info */, - ContextPtr local_context, - QueryProcessingStage::Enum /* processed_stage */, - size_t /* max_block_size */, - unsigned /* num_streams */) + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /* query_info */, + ContextPtr local_context, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + unsigned /* num_streams */) { if (num_created_consumers == 0) return {}; if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + throw Exception( + ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); if (mv_attached) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageNATS with attached materialized views"); @@ -276,8 +275,7 @@ Pipe StorageNATS::read( for (size_t i = 0; i < num_created_consumers; ++i) { - auto nats_source = std::make_shared( - *this, storage_snapshot, modified_context, column_names, 1); + auto nats_source = std::make_shared(*this, storage_snapshot, modified_context, column_names, 1); auto converting_dag = ActionsDAG::makeConvertingActions( nats_source->getPort().getHeader().getColumnsWithTypeAndName(), @@ -543,8 +541,7 @@ bool StorageNATS::streamToViews() for (size_t i = 0; i < num_created_consumers; ++i) { - auto source = std::make_shared( - *this, storage_snapshot, nats_context, column_names, block_size); + auto source = std::make_shared(*this, storage_snapshot, nats_context, column_names, block_size); sources.emplace_back(source); pipes.emplace_back(source); @@ -552,8 +549,8 @@ bool StorageNATS::streamToViews() StreamLocalLimits limits; limits.speed_limits.max_execution_time = nats_settings->nats_flush_interval_ms.changed - ? nats_settings->nats_flush_interval_ms - : getContext()->getSettingsRef().stream_flush_interval_ms; + ? nats_settings->nats_flush_interval_ms + : getContext()->getSettingsRef().stream_flush_interval_ms; limits.timeout_overflow_mode = OverflowMode::BREAK; @@ -628,10 +625,9 @@ void registerStorageNATS(StorageFactory & factory) nats_settings->loadFromQuery(*args.storage_def); - if (!nats_settings->nats_url.changed - && !nats_settings->nats_server_list.changed) - throw Exception("You must specify either `nats_url` or `nats_server_list` settings", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (!nats_settings->nats_url.changed && !nats_settings->nats_server_list.changed) + throw Exception( + "You must specify either `nats_url` or `nats_server_list` settings", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (!nats_settings->nats_format.changed) throw Exception("You must specify `nats_format` setting", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index e536cf6711a..7ff91268b39 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -1,24 +1,23 @@ #pragma once +#include +#include +#include +#include #include #include -#include -#include -#include #include -#include #include +#include +#include #include -#include -#include namespace DB { -class StorageNATS final: public IStorage, WithContext +class StorageNATS final : public IStorage, WithContext { - public: StorageNATS( const StorageID & table_id_, @@ -51,10 +50,7 @@ public: size_t max_block_size, unsigned num_streams) override; - SinkToStoragePtr write( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; void pushReadBuffer(ConsumerBufferPtr buf); ConsumerBufferPtr popReadBuffer(); @@ -130,7 +126,7 @@ private: void stopLoop(); void stopLoopIfNoReaders(); - static Names parseList(const String& list); + static Names parseList(const String & list); static String getTableBasedName(String name, const StorageID & table_id); ContextMutablePtr addSettings(ContextPtr context) const; diff --git a/src/Storages/NATS/UVLoop.h b/src/Storages/NATS/UVLoop.h index 4de67cbc206..66668739dd7 100644 --- a/src/Storages/NATS/UVLoop.h +++ b/src/Storages/NATS/UVLoop.h @@ -2,8 +2,8 @@ #include -#include #include +#include #include @@ -19,7 +19,7 @@ namespace ErrorCodes class UVLoop : public boost::noncopyable { public: - UVLoop(): loop_ptr(new uv_loop_t()) + UVLoop() : loop_ptr(new uv_loop_t()) { int res = uv_loop_init(loop_ptr.get()); diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index 5b885840798..54b5b3b1042 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -1,14 +1,14 @@ #include -#include -#include -#include -#include -#include -#include +#include #include #include -#include +#include +#include +#include +#include +#include +#include namespace DB @@ -23,28 +23,28 @@ namespace ErrorCodes } WriteBufferToNATSProducer::WriteBufferToNATSProducer( - const NATSConfiguration & configuration_, - ContextPtr global_context, - const String & subject_, - std::atomic & shutdown_called_, - Poco::Logger * log_, - std::optional delimiter, - size_t rows_per_message, - size_t chunk_size_) - : WriteBuffer(nullptr, 0) - , connection(configuration_, log_) - , subject(subject_) - , shutdown_called(shutdown_called_) - , payloads(BATCH) - , log(log_) - , delim(delimiter) - , max_rows(rows_per_message) - , chunk_size(chunk_size_) + const NATSConfiguration & configuration_, + ContextPtr global_context, + const String & subject_, + std::atomic & shutdown_called_, + Poco::Logger * log_, + std::optional delimiter, + size_t rows_per_message, + size_t chunk_size_) + : WriteBuffer(nullptr, 0) + , connection(configuration_, log_) + , subject(subject_) + , shutdown_called(shutdown_called_) + , payloads(BATCH) + , log(log_) + , delim(delimiter) + , max_rows(rows_per_message) + , chunk_size(chunk_size_) { if (!connection.connect()) throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "Cannot connect to NATS {}", connection.connectionInfoForLog()); - writing_task = global_context->getSchedulePool().createTask("NATSWritingTask", [this]{ writingFunc(); }); + writing_task = global_context->getSchedulePool().createTask("NATSWritingTask", [this] { writingFunc(); }); writing_task->deactivate(); reinitializeChunks(); @@ -95,7 +95,8 @@ void WriteBufferToNATSProducer::publish() uv_thread_join(&flush_thread); } -void WriteBufferToNATSProducer::publishThreadFunc(void * arg) { +void WriteBufferToNATSProducer::publishThreadFunc(void * arg) +{ String payload; WriteBufferToNATSProducer * buffer = static_cast(arg); diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h index e081f1a738d..0cffd35d414 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.h +++ b/src/Storages/NATS/WriteBufferToNATSProducer.h @@ -1,15 +1,14 @@ #pragma once -#include -#include +#include #include #include -#include -#include -#include -#include +#include #include #include +#include +#include +#include namespace DB { @@ -18,15 +17,14 @@ class WriteBufferToNATSProducer : public WriteBuffer { public: WriteBufferToNATSProducer( - const NATSConfiguration & configuration_, - ContextPtr global_context, - const String & subject_, - std::atomic & shutdown_called_, - Poco::Logger * log_, - std::optional delimiter, - size_t rows_per_message, - size_t chunk_size_ - ); + const NATSConfiguration & configuration_, + ContextPtr global_context, + const String & subject_, + std::atomic & shutdown_called_, + Poco::Logger * log_, + std::optional delimiter, + size_t rows_per_message, + size_t chunk_size_); ~WriteBufferToNATSProducer() override; From 35cbe48ca39e52559d34f306cd3a84e3058b83bf Mon Sep 17 00:00:00 2001 From: tchepavel Date: Mon, 16 May 2022 13:03:28 +0300 Subject: [PATCH 015/627] Fixing; preparing test --- docker/test/integration/runner/Dockerfile | 2 ++ .../runner/compose/docker_compose_nats.yml | 24 +++++++++++++++++++ src/Storages/NATS/NATSHandler.cpp | 1 - src/Storages/NATS/NATSHandler.h | 1 + src/Storages/NATS/StorageNATS.cpp | 2 ++ src/Storages/NATS/StorageNATS.h | 11 --------- 6 files changed, 29 insertions(+), 12 deletions(-) create mode 100644 docker/test/integration/runner/compose/docker_compose_nats.yml diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index b5c6a39a965..1a98e3be37d 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -62,6 +62,7 @@ RUN python3 -m pip install \ PyMySQL \ aerospike==4.0.0 \ avro==1.10.2 \ + asyncio \ cassandra-driver \ confluent-kafka==1.5.0 \ dict2xml \ @@ -74,6 +75,7 @@ RUN python3 -m pip install \ kazoo \ lz4 \ minio \ + nats-py \ protobuf \ psycopg2-binary==2.8.6 \ pymongo==3.11.0 \ diff --git a/docker/test/integration/runner/compose/docker_compose_nats.yml b/docker/test/integration/runner/compose/docker_compose_nats.yml new file mode 100644 index 00000000000..76d157e3454 --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_nats.yml @@ -0,0 +1,24 @@ +version: '2.3' +services: + nats: + image: nats + command: "--cluster_name NATS --cluster nats://0.0.0.0:6222 --http_port 8222 " + networks: ["nats"] + nats-1: + ports: + - "${NATS_EXTERNAL_PORT_USER}:${NATS_INTERNAL_PORT}" + image: nats + command: "--cluster_name NATS --cluster nats://0.0.0.0:6222 --routes=nats://ruser:T0pS3cr3t@nats:6222 --user click --pass house" + networks: ["nats"] + depends_on: ["nats"] + nats-2: + ports: + - "${NATS_EXTERNAL_PORT_TOKEN}:${NATS_INTERNAL_PORT}" + image: nats + command: "--cluster_name NATS --cluster nats://0.0.0.0:6222 --routes=nats://ruser:T0pS3cr3t@nats:6222 --auth clickhouse" + networks: ["nats"] + depends_on: ["nats"] + +networks: + nats: + name: nats \ No newline at end of file diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index 87954c2b6d1..2db3115c46f 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -1,5 +1,4 @@ #include -#include #include #include diff --git a/src/Storages/NATS/NATSHandler.h b/src/Storages/NATS/NATSHandler.h index 9ffd58fc07a..57b9e30421b 100644 --- a/src/Storages/NATS/NATSHandler.h +++ b/src/Storages/NATS/NATSHandler.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 31f22e7ebed..a40b4d04307 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -24,6 +24,8 @@ #include #include +#include + namespace DB { diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 7ff91268b39..fcae5687535 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -12,7 +11,6 @@ #include #include - namespace DB { @@ -135,15 +133,6 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); - - static String getRandomName() - { - std::uniform_int_distribution distribution('a', 'z'); - String random_str(32, ' '); - for (auto & c : random_str) - c = distribution(thread_local_rng); - return random_str; - } }; } From 2f0013934754a51bf354d8c1a998a1298b9f4b83 Mon Sep 17 00:00:00 2001 From: tchepavel Date: Mon, 16 May 2022 16:35:17 +0300 Subject: [PATCH 016/627] Fix review issues; add insert queue parameter --- .gitmodules | 2 +- src/Core/Settings.h | 3 +- src/Storages/NATS/NATSConnection.cpp | 9 +- src/Storages/NATS/NATSConnection.h | 2 +- src/Storages/NATS/NATSHandler.cpp | 7 +- src/Storages/NATS/NATSHandler.h | 7 +- src/Storages/NATS/NATSSink.cpp | 5 +- src/Storages/NATS/NATSSink.h | 2 +- src/Storages/NATS/NATSSource.cpp | 13 +-- src/Storages/NATS/NATSSource.h | 2 - .../NATS/ReadBufferFromNATSConsumer.cpp | 12 --- .../NATS/ReadBufferFromNATSConsumer.h | 2 - src/Storages/NATS/StorageNATS.cpp | 84 +++++++++++++++++-- src/Storages/NATS/StorageNATS.h | 7 +- src/Storages/RabbitMQ/RabbitMQConnection.h | 2 +- src/Storages/RabbitMQ/UVLoop.h | 44 ---------- src/Storages/{NATS => }/UVLoop.h | 0 17 files changed, 93 insertions(+), 110 deletions(-) delete mode 100644 src/Storages/RabbitMQ/UVLoop.h rename src/Storages/{NATS => }/UVLoop.h (100%) diff --git a/.gitmodules b/.gitmodules index 38041e7f414..ba97de232e7 100644 --- a/.gitmodules +++ b/.gitmodules @@ -270,4 +270,4 @@ url = https://github.com/eigen-mirror/eigen [submodule "contrib/nats-io"] path = contrib/nats-io - url = https://github.com/tchepavel/nats.c.git + url = https://github.com/ClickHouse/nats.c.git diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e364db7e108..7027ab916f7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -95,7 +95,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited. Only has meaning at server startup.", 0) \ M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited. Only has meaning at server startup.", 0) \ - M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ and FileLog engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \ + M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \ + M(String, stream_like_engine_insert_queue, "", "When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.", 0) \ \ M(Milliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \ M(Milliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.", 0) \ diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index e46900ce57b..359754bb144 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -78,7 +78,7 @@ bool NATSConnectionManager::closed() bool NATSConnectionManager::isConnectedImpl() const { - return event_handler.connectionRunning() && connection && !natsConnection_IsClosed(connection); + return connection && has_connection && !natsConnection_IsClosed(connection); } void NATSConnectionManager::connectImpl() @@ -117,15 +117,10 @@ void NATSConnectionManager::connectImpl() status = natsConnection_Connect(&connection, options); } if (status == NATS_OK) - { has_connection = true; - event_handler.changeConnectionStatus(true); - } else - { LOG_DEBUG(log, "New connection to {} failed. Nats status text: {}. Last error message: {}", connectionInfoForLog(), natsStatus_GetText(status), nats_GetLastError(nullptr)); - } } void NATSConnectionManager::disconnectImpl() @@ -138,8 +133,6 @@ void NATSConnectionManager::disconnectImpl() size_t cnt_retries = 0; while (!natsConnection_IsClosed(connection) && cnt_retries++ != RETRIES_MAX) event_handler.iterateLoop(); - - event_handler.changeConnectionStatus(false); } void NATSConnectionManager::reconnectedCallback(natsConnection * nc, void * log) diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h index 2315b0f1b12..78a273164db 100644 --- a/src/Storages/NATS/NATSConnection.h +++ b/src/Storages/NATS/NATSConnection.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index 2db3115c46f..79d0adf0c6b 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -11,7 +12,6 @@ namespace DB NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : loop(loop_), log(log_), - connection_running(false), loop_running(false), loop_state(Loop::STOP) { @@ -42,11 +42,6 @@ void NATSHandler::startLoop() loop_running.store(false); } -void NATSHandler::changeConnectionStatus(bool is_running) -{ - connection_running.store(is_running); -} - void NATSHandler::iterateLoop() { std::unique_lock lock(startup_mutex, std::defer_lock); diff --git a/src/Storages/NATS/NATSHandler.h b/src/Storages/NATS/NATSHandler.h index 57b9e30421b..e3894c888a3 100644 --- a/src/Storages/NATS/NATSHandler.h +++ b/src/Storages/NATS/NATSHandler.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -37,9 +37,6 @@ public: LockPtr setThreadLocalLoop(); void stopLoop(); - - void changeConnectionStatus(bool is_running); - bool connectionRunning() const { return connection_running.load(); } bool loopRunning() const { return loop_running.load(); } void updateLoopState(UInt8 state) { loop_state.store(state); } @@ -52,7 +49,7 @@ private: natsOptions * opts = nullptr; Poco::Logger * log; - std::atomic connection_running, loop_running; + std::atomic loop_running; std::atomic loop_state; std::mutex startup_mutex; }; diff --git a/src/Storages/NATS/NATSSink.cpp b/src/Storages/NATS/NATSSink.cpp index bae6712a0e3..44cf51072e6 100644 --- a/src/Storages/NATS/NATSSink.cpp +++ b/src/Storages/NATS/NATSSink.cpp @@ -12,18 +12,19 @@ namespace DB NATSSink::NATSSink( StorageNATS & storage_, const StorageMetadataPtr & metadata_snapshot_, - ContextPtr context_) + ContextPtr context_, + ProducerBufferPtr buffer_) : SinkToStorage(metadata_snapshot_->getSampleBlockNonMaterialized()) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) + , buffer(buffer_) { } void NATSSink::onStart() { - buffer = storage.createWriteBuffer(); buffer->activateWriting(); auto format_settings = getFormatSettings(context); diff --git a/src/Storages/NATS/NATSSink.h b/src/Storages/NATS/NATSSink.h index 3023d0c90ea..d94575de0e7 100644 --- a/src/Storages/NATS/NATSSink.h +++ b/src/Storages/NATS/NATSSink.h @@ -13,7 +13,7 @@ using IOutputFormatPtr = std::shared_ptr; class NATSSink : public SinkToStorage { public: - explicit NATSSink(StorageNATS & storage_, const StorageMetadataPtr & metadata_snapshot_, ContextPtr context_); + explicit NATSSink(StorageNATS & storage_, const StorageMetadataPtr & metadata_snapshot_, ContextPtr context_, ProducerBufferPtr buffer_); void onStart() override; void consume(Chunk chunk) override; diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index 7219db3e99c..3fdba292923 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -66,13 +66,6 @@ NATSSource::~NATSSource() } Chunk NATSSource::generate() -{ - auto chunk = generateImpl(); - - return chunk; -} - -Chunk NATSSource::generateImpl() { if (!buffer) { @@ -104,11 +97,7 @@ Chunk NATSSource::generateImpl() if (new_rows) { auto subject = buffer->getSubject(); - - for (size_t i = 0; i < new_rows; ++i) - { - virtual_columns[0]->insert(subject); - } + virtual_columns[0]->insertMany(subject, new_rows); total_rows = total_rows + new_rows; } diff --git a/src/Storages/NATS/NATSSource.h b/src/Storages/NATS/NATSSource.h index 08fcf97114a..6c1ea5bed0d 100644 --- a/src/Storages/NATS/NATSSource.h +++ b/src/Storages/NATS/NATSSource.h @@ -47,8 +47,6 @@ private: ContextPtr context_, const Names & columns, size_t max_block_size_); - - Chunk generateImpl(); }; } diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index 4f11a051029..e8abe9a5ed4 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -1,8 +1,6 @@ #include #include #include -#include -#include #include #include #include @@ -38,16 +36,6 @@ ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( { } -ReadBufferFromNATSConsumer::~ReadBufferFromNATSConsumer() -{ - for (const auto & subscription : subscriptions) - { - natsSubscription_Unsubscribe(subscription.get()); - } - - BufferBase::set(nullptr, 0, 0); -} - void ReadBufferFromNATSConsumer::subscribe() { if (subscribed) diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h index d75bf1c0376..6f2ee071c81 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -27,8 +27,6 @@ public: uint32_t queue_size_, const std::atomic & stopped_); - ~ReadBufferFromNATSConsumer() override; - struct MessageData { String message; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index a40b4d04307..c7c25e6fd03 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -21,7 +20,6 @@ #include #include #include -#include #include #include @@ -53,7 +51,7 @@ StorageNATS::StorageNATS( : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , nats_settings(std::move(nats_settings_)) - , subjects(parseList(getContext()->getMacros()->expand(nats_settings->nats_subjects))) + , subjects(parseList(getContext()->getMacros()->expand(nats_settings->nats_subjects), ',')) , format_name(getContext()->getMacros()->expand(nats_settings->nats_format)) , row_delimiter(nats_settings->nats_row_delimiter.value) , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) @@ -71,7 +69,7 @@ StorageNATS::StorageNATS( configuration = { .url = getContext()->getMacros()->expand(nats_settings->nats_url), - .servers = parseList(getContext()->getMacros()->expand(nats_settings->nats_server_list)), + .servers = parseList(getContext()->getMacros()->expand(nats_settings->nats_server_list), ','), .username = nats_username.empty() ? getContext()->getConfigRef().getString("nats.username", "") : nats_username, .password = nats_password.empty() ? getContext()->getConfigRef().getString("nats.password", "") : nats_password, .token = nats_token.empty() ? getContext()->getConfigRef().getString("nats.token", "") : nats_token, @@ -115,12 +113,12 @@ StorageNATS::StorageNATS( } -Names StorageNATS::parseList(const String & list) +Names StorageNATS::parseList(const String & list, char delim) { Names result; if (list.empty()) return result; - boost::split(result, list, [](char c) { return c == ','; }); + boost::split(result, list, [delim](char c) { return c == delim; }); for (String & key : result) boost::trim(key); @@ -303,7 +301,32 @@ Pipe StorageNATS::read( SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - return std::make_shared(*this, metadata_snapshot, local_context); + auto modified_context = addSettings(local_context); + std::string subject = modified_context->getSettingsRef().stream_like_engine_insert_queue.changed + ? modified_context->getSettingsRef().stream_like_engine_insert_queue.value + : ""; + if (subject.empty()) + { + if (subjects.size() > 1) + { + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "This NATS engine reads from multiple subjects. You must specify `stream_like_engine_insert_queue` to choose the subject to write to"); + } + else + { + subject = subjects[0]; + } + } + + auto pos = subject.find('*'); + if (pos != std::string::npos || subject.back() == '>') + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can not publish to wildcard subject"); + + if (!isSubjectInSubscriptions(subject)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Selected subject is not among engine subjects"); + + return std::make_shared(*this, metadata_snapshot, local_context, createWriteBuffer(subject)); } @@ -413,13 +436,53 @@ ConsumerBufferPtr StorageNATS::createReadBuffer() } -ProducerBufferPtr StorageNATS::createWriteBuffer() +ProducerBufferPtr StorageNATS::createWriteBuffer(const std::string & subject) { return std::make_shared( - configuration, getContext(), subjects[0], shutdown_called, log, + configuration, getContext(), subject, shutdown_called, log, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } +bool StorageNATS::isSubjectInSubscriptions(const std::string & subject) { + auto subject_levels = parseList(subject, '.'); + + for (const auto & nats_subject : subjects) + { + auto nats_subject_levels = parseList(nats_subject, '.'); + size_t levels_to_check = 0; + if (!nats_subject_levels.empty() && nats_subject_levels.back() == ">") + levels_to_check = nats_subject_levels.size() - 1; + if (levels_to_check) + { + if (subject_levels.size() < levels_to_check) + continue; + } + else + { + if (subject_levels.size() != nats_subject_levels.size()) + continue; + levels_to_check = nats_subject_levels.size(); + } + + bool is_same = true; + for (size_t i = 0; i < levels_to_check; ++i) + { + if (nats_subject_levels[i] == "*") + continue; + + if (subject_levels[i] != nats_subject_levels[i]) + { + is_same = false; + break; + } + } + if (is_same) + return true; + } + + return false; +} + bool StorageNATS::checkDependencies(const StorageID & table_id) { @@ -634,6 +697,9 @@ void registerStorageNATS(StorageFactory & factory) if (!nats_settings->nats_format.changed) throw Exception("You must specify `nats_format` setting", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (!nats_settings->nats_subjects.changed) + throw Exception("You must specify `nats_subjects` setting", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(nats_settings), args.attach); }; diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index fcae5687535..4197e600ea6 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -54,8 +54,6 @@ public: ConsumerBufferPtr popReadBuffer(); ConsumerBufferPtr popReadBuffer(std::chrono::milliseconds timeout); - ProducerBufferPtr createWriteBuffer(); - const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; @@ -114,6 +112,9 @@ private: bool is_attach; ConsumerBufferPtr createReadBuffer(); + ProducerBufferPtr createWriteBuffer(const std::string & subject); + + bool isSubjectInSubscriptions(const std::string & subject); /// Functions working in the background void streamingToViewsFunc(); @@ -124,7 +125,7 @@ private: void stopLoop(); void stopLoopIfNoReaders(); - static Names parseList(const String & list); + static Names parseList(const String & list, char delim); static String getTableBasedName(String name, const StorageID & table_id); ContextMutablePtr addSettings(ContextPtr context) const; diff --git a/src/Storages/RabbitMQ/RabbitMQConnection.h b/src/Storages/RabbitMQ/RabbitMQConnection.h index acc3c48f85b..7a355afea0e 100644 --- a/src/Storages/RabbitMQ/RabbitMQConnection.h +++ b/src/Storages/RabbitMQ/RabbitMQConnection.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/src/Storages/RabbitMQ/UVLoop.h b/src/Storages/RabbitMQ/UVLoop.h deleted file mode 100644 index 4de67cbc206..00000000000 --- a/src/Storages/RabbitMQ/UVLoop.h +++ /dev/null @@ -1,44 +0,0 @@ -#pragma once - -#include - -#include -#include - -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int SYSTEM_ERROR; -} - -/// RAII wrapper around uv event loop -class UVLoop : public boost::noncopyable -{ -public: - UVLoop(): loop_ptr(new uv_loop_t()) - { - int res = uv_loop_init(loop_ptr.get()); - - if (res != 0) - throw Exception("UVLoop could not initialize", ErrorCodes::SYSTEM_ERROR); - } - - ~UVLoop() - { - if (loop_ptr) - uv_loop_close(loop_ptr.get()); - } - - inline uv_loop_t * getLoop() { return loop_ptr.get(); } - - inline const uv_loop_t * getLoop() const { return loop_ptr.get(); } - -private: - std::unique_ptr loop_ptr; -}; - -} diff --git a/src/Storages/NATS/UVLoop.h b/src/Storages/UVLoop.h similarity index 100% rename from src/Storages/NATS/UVLoop.h rename to src/Storages/UVLoop.h From a3af94d49e4e342a1cdf4eca26bf4d4d923d5f8e Mon Sep 17 00:00:00 2001 From: tchepavel Date: Tue, 17 May 2022 16:58:09 +0300 Subject: [PATCH 017/627] Documentation; integration tests; subscribe on startup --- .../runner/compose/docker_compose_nats.yml | 23 +- .../table-engines/integrations/nats.md | 143 ++ src/Storages/NATS/NATSSettings.h | 2 +- src/Storages/NATS/StorageNATS.cpp | 49 +- src/Storages/NATS/StorageNATS.h | 5 + .../NATS/WriteBufferToNATSProducer.cpp | 2 + tests/integration/helpers/cluster.py | 67 + .../integration/test_storage_nats/__init__.py | 0 .../clickhouse_path/format_schemas/nats.proto | 6 + .../test_storage_nats/configs/macros.xml | 7 + .../configs/named_collection.xml | 13 + .../test_storage_nats/configs/nats.xml | 6 + .../test_storage_nats/configs/users.xml | 8 + .../integration/test_storage_nats/nats_pb2.py | 34 + tests/integration/test_storage_nats/test.py | 1459 +++++++++++++++++ .../test_nats_json.reference | 50 + 16 files changed, 1845 insertions(+), 29 deletions(-) create mode 100644 docs/en/engines/table-engines/integrations/nats.md create mode 100644 tests/integration/test_storage_nats/__init__.py create mode 100755 tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto create mode 100644 tests/integration/test_storage_nats/configs/macros.xml create mode 100644 tests/integration/test_storage_nats/configs/named_collection.xml create mode 100644 tests/integration/test_storage_nats/configs/nats.xml create mode 100644 tests/integration/test_storage_nats/configs/users.xml create mode 100644 tests/integration/test_storage_nats/nats_pb2.py create mode 100644 tests/integration/test_storage_nats/test.py create mode 100644 tests/integration/test_storage_nats/test_nats_json.reference diff --git a/docker/test/integration/runner/compose/docker_compose_nats.yml b/docker/test/integration/runner/compose/docker_compose_nats.yml index 76d157e3454..19ae4c162b1 100644 --- a/docker/test/integration/runner/compose/docker_compose_nats.yml +++ b/docker/test/integration/runner/compose/docker_compose_nats.yml @@ -1,24 +1,7 @@ version: '2.3' services: - nats: + nats1: image: nats - command: "--cluster_name NATS --cluster nats://0.0.0.0:6222 --http_port 8222 " - networks: ["nats"] - nats-1: ports: - - "${NATS_EXTERNAL_PORT_USER}:${NATS_INTERNAL_PORT}" - image: nats - command: "--cluster_name NATS --cluster nats://0.0.0.0:6222 --routes=nats://ruser:T0pS3cr3t@nats:6222 --user click --pass house" - networks: ["nats"] - depends_on: ["nats"] - nats-2: - ports: - - "${NATS_EXTERNAL_PORT_TOKEN}:${NATS_INTERNAL_PORT}" - image: nats - command: "--cluster_name NATS --cluster nats://0.0.0.0:6222 --routes=nats://ruser:T0pS3cr3t@nats:6222 --auth clickhouse" - networks: ["nats"] - depends_on: ["nats"] - -networks: - nats: - name: nats \ No newline at end of file + - "${NATS_EXTERNAL_PORT}:${NATS_INTERNAL_PORT}" + command: "-p 4444 --user click --pass house" \ No newline at end of file diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md new file mode 100644 index 00000000000..dc99c3515b9 --- /dev/null +++ b/docs/en/engines/table-engines/integrations/nats.md @@ -0,0 +1,143 @@ +--- +sidebar_position: 14 +sidebar_label: NATS +--- + +# NATS Engine {#redisstreams-engine} + +This engine allows integrating ClickHouse with [NATS](https://nats.io/). + +`NATS` lets you: + +- Publish or subcribe to message subjects. +- Process new messages as they become available. + +## Creating a Table {#table_engine-redisstreams-creating-a-table} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = NATS SETTINGS + nats_url = 'host:port', + nats_subjects = 'subject1,subject2,...', + nats_format = 'data_format'[,] + [nats_row_delimiter = 'delimiter_symbol',] + [nats_schema = '',] + [nats_num_consumers = N,] + [nats_queue_group = 'group_name',] + [nats_secure = false,] + [nats_max_reconnect = N,] + [nats_reconnect_wait = N,] + [nats_server_list = 'host1:port1,host2:port2,...',] + [nats_skip_broken_messages = N,] + [nats_max_block_size = N,] + [nats_flush_interval_ms = N,] + [nats_username = 'user',] + [nats_password = 'password'] + [redis_password = 'clickhouse'] +``` + +Required parameters: + +- `nats_url` – host:port (for example, `localhost:5672`).. +- `nats_subjects` – List of subject for NATS table to subscribe/publsh to. Supports wildcard subjects like `foo.*.bar` or `baz.>` +- `nats_format` – Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section. + +Optional parameters: + +- `nats_row_delimiter` – Delimiter character, which ends the message. +- `nats_schema` – Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object. +- `nats_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. +- `nats_queue_group` – Name for queue group of NATS subscribers. Default is the table name. +- `nats_max_reconnect` – Maximum amount of reconnection attempts per try to connect to NATS. Default: `5`. +- `nats_reconnect_wait` – Amount of time in milliseconds to sleep between each reconnect attempt. Default: `5000`. +- `nats_server_list` - Server list for connection. Can be specified to connect to NATS cluster. +- `nats_skip_broken_messages` - NATS message parser tolerance to schema-incompatible messages per block. Default: `0`. If `nats_skip_broken_messages = N` then the engine skips *N* RabbitMQ messages that cannot be parsed (a message equals a row of data). +- `nats_max_block_size` - Number of row collected by poll(s) for flushing data from NATS. +- `nats_flush_interval_ms` - Timeout for flushing data read from NATS. +- `nats_username` - NATS username. +- `nats_password` - NATS password. +- `nats_token` - NATS auth token. + +SSL connection: + +For secure connection use `nats_secure = 1`. +The default behaviour of the used library is not to check if the created TLS connection is sufficiently secure. Whether the certificate is expired, self-signed, missing or invalid: the connection is simply permitted. More strict checking of certificates can possibly be implemented in the future. + + +Also format settings can be added along with nats-related settings. + +Example: + +``` sql + CREATE TABLE queue ( + key UInt64, + value UInt64, + date DateTime + ) ENGINE = NATS + SETTINGS nats_url = 'localhost:4444', + nats_subject = 'subject1', + nats_format = 'JSONEachRow', + date_time_input_format = 'best_effort'; +``` + +The NATS server configuration can be added using the ClickHouse config file. + More specifically you can add Redis password for NATS engine: + +``` xml + + click + house + clickhouse + +``` + +## Description {#description} + +`SELECT` is not particularly useful for reading messages (except for debugging), because each message can be read only once. It is more practical to create real-time threads using [materialized views](../../../sql-reference/statements/create/view.md). To do this: + +1. Use the engine to create a NATS consumer and consider it a data stream. +2. Create a table with the desired structure. +3. Create a materialized view that converts data from the engine and puts it into a previously created table. + +When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background. This allows you to continually receive messages from NATS and convert them to the required format using `SELECT`. +One NATS table can have as many materialized views as you like, they do not read data from the table directly, but receive new records (in blocks), this way you can write to several tables with different detail level (with grouping - aggregation and without). + +Example: + +``` sql + CREATE TABLE queue ( + key UInt64, + value UInt64 + ) ENGINE = NATS + SETTINGS nats_url = 'localhost:4444', + nats_subject = 'subject1', + nats_format = 'JSONEachRow', + date_time_input_format = 'best_effort'; + + CREATE TABLE daily (key UInt64, value UInt64) + ENGINE = MergeTree() ORDER BY key; + + CREATE MATERIALIZED VIEW consumer TO daily + AS SELECT key, value FROM queue; + + SELECT key, value FROM daily ORDER BY key; +``` + +To stop receiving streams data or to change the conversion logic, detach the materialized view: + +``` sql + DETACH TABLE consumer; + ATTACH TABLE consumer; +``` + +If you want to change the target table by using `ALTER`, we recommend disabling the material view to avoid discrepancies between the target table and the data from the view. + +## Virtual Columns {#virtual-columns} + +- `_subject` - NATS message subject. + +[Original article](https://clickhouse.com/docs/en/engines/table-engines/integrations/nats/) diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h index c06d92c0dec..02864e490c2 100644 --- a/src/Storages/NATS/NATSSettings.h +++ b/src/Storages/NATS/NATSSettings.h @@ -17,7 +17,7 @@ class ASTStorage; M(String, nats_queue_group, "", "Name for queue group of NATS subscribers.", 0) \ M(Bool, nats_secure, false, "Use SSL connection", 0) \ M(UInt64, nats_max_reconnect, 5, "Maximum amount of reconnection attempts.", 0) \ - M(UInt64, nats_reconnect_wait, 2000, "Amount to sleep between each reconnect attempt.", 0) \ + M(UInt64, nats_reconnect_wait, 2000, "Amount of time in milliseconds to sleep between each reconnect attempt.", 0) \ M(String, nats_server_list, "", "Server list for connection", 0) \ M(UInt64, nats_skip_broken_messages, 0, "Skip at least this number of broken messages from NATS per block", 0) \ M(UInt64, nats_max_block_size, 0, "Number of row collected before flushing data from NATS.", 0) \ diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index c7c25e6fd03..74791fddd47 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -70,7 +70,7 @@ StorageNATS::StorageNATS( { .url = getContext()->getMacros()->expand(nats_settings->nats_url), .servers = parseList(getContext()->getMacros()->expand(nats_settings->nats_server_list), ','), - .username = nats_username.empty() ? getContext()->getConfigRef().getString("nats.username", "") : nats_username, + .username = nats_username.empty() ? getContext()->getConfigRef().getString("nats.user", "") : nats_username, .password = nats_password.empty() ? getContext()->getConfigRef().getString("nats.password", "") : nats_password, .token = nats_token.empty() ? getContext()->getConfigRef().getString("nats.token", "") : nats_token, .max_reconnect = static_cast(nats_settings->nats_max_reconnect.value), @@ -92,7 +92,8 @@ StorageNATS::StorageNATS( { connection = std::make_shared(configuration, log); if (!connection->connect()) - throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "Cannot connect to {}", connection->connectionInfoForLog()); + throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "Cannot connect to {}. Nats last error: {}", + connection->connectionInfoForLog(), nats_GetLastError(nullptr)); } catch (...) { @@ -204,10 +205,41 @@ void StorageNATS::decrementReader() void StorageNATS::connectionFunc() { - if (!connection->reconnect()) + if (consumers_ready) + return; + + bool needs_rescheduling = true; + if (connection->reconnect()) + needs_rescheduling &= !initBuffers(); + + if (needs_rescheduling) connection_task->scheduleAfter(RESCHEDULE_MS); } +bool StorageNATS::initBuffers() +{ + size_t num_initialized = 0; + for (auto & buffer : buffers) + { + try + { + buffer->subscribe(); + ++num_initialized; + } + catch (...) + { + tryLogCurrentException(log); + break; + } + } + + startLoop(); + const bool are_buffers_initialized = num_initialized == num_created_consumers; + if (are_buffers_initialized) + consumers_ready.store(true); + return are_buffers_initialized; +} + /* Need to deactivate this way because otherwise might get a deadlock when first deactivate streaming task in shutdown and then * inside streaming task try to deactivate any other task @@ -247,6 +279,9 @@ Pipe StorageNATS::read( size_t /* max_block_size */, unsigned /* num_streams */) { + if (!consumers_ready) + throw Exception("NATS consumers setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_NATS); + if (num_created_consumers == 0) return {}; @@ -332,11 +367,6 @@ SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & m void StorageNATS::startup() { - if (!connection->isConnected()) - { - connection_task->activateAndSchedule(); - } - for (size_t i = 0; i < num_consumers; ++i) { try @@ -353,6 +383,9 @@ void StorageNATS::startup() } } + if (!connection->isConnected() || !initBuffers()) + connection_task->activateAndSchedule(); + streaming_task->activateAndSchedule(); } diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 4197e600ea6..8dff8ecb533 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -92,6 +92,8 @@ private: uint64_t milliseconds_to_wait; + /// True if consumers have subscribed to all subjects + std::atomic consumers_ready{false}; /// Needed for tell MV or producer background tasks /// that they must finish as soon as possible. std::atomic shutdown_called{false}; @@ -116,11 +118,14 @@ private: bool isSubjectInSubscriptions(const std::string & subject); + /// Functions working in the background void streamingToViewsFunc(); void loopingFunc(); void connectionFunc(); + bool initBuffers(); + void startLoop(); void stopLoop(); void stopLoopIfNoReaders(); diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index 54b5b3b1042..f4464db1564 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -113,6 +113,8 @@ void WriteBufferToNATSProducer::publishThreadFunc(void * arg) { LOG_DEBUG(buffer->log, "Something went wrong during publishing to NATS subject. Nats status text: {}. Last error message: {}", natsStatus_GetText(status), nats_GetLastError(nullptr)); + if (!buffer->payloads.push(std::move(payload))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); break; } } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0a6f93f048e..c80e4b861f4 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -22,12 +22,14 @@ try: # Please, add modules that required for specific tests only here. # So contributors will be able to run most tests locally # without installing tons of unneeded packages that may be not so easy to install. + import asyncio from cassandra.policies import RoundRobinPolicy import cassandra.cluster import psycopg2 from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT import pymongo import pymysql + import nats from confluent_kafka.avro.cached_schema_registry_client import ( CachedSchemaRegistryClient, ) @@ -212,6 +214,11 @@ def check_rabbitmq_is_available(rabbitmq_id): return p.returncode == 0 +async def check_nats_is_available(nats_ip): + nc = await nats.connect('{}:4444'.format(nats_ip), user='click', password='house') + return nc.is_connected + + def enable_consistent_hash_plugin(rabbitmq_id): p = subprocess.Popen( ( @@ -335,6 +342,7 @@ class ClickHouseCluster: self.base_kafka_cmd = [] self.base_kerberized_kafka_cmd = [] self.base_rabbitmq_cmd = [] + self.base_nats_cmd = [] self.base_cassandra_cmd = [] self.base_jdbc_bridge_cmd = [] self.base_redis_cmd = [] @@ -351,6 +359,7 @@ class ClickHouseCluster: self.with_kafka = False self.with_kerberized_kafka = False self.with_rabbitmq = False + self.with_nats = False self.with_odbc_drivers = False self.with_hdfs = False self.with_kerberized_hdfs = False @@ -431,6 +440,12 @@ class ClickHouseCluster: self.rabbitmq_dir = p.abspath(p.join(self.instances_dir, "rabbitmq")) self.rabbitmq_logs_dir = os.path.join(self.rabbitmq_dir, "logs") + self.nats_host = "nats1" + self.nats_ip = None + self.nats_port = 4444 + self.nats_docker_id = None + + # available when with_nginx == True self.nginx_host = "nginx" self.nginx_ip = None @@ -1004,6 +1019,26 @@ class ClickHouseCluster: ] return self.base_rabbitmq_cmd + def setup_nats_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_nats = True + env_variables["NATS_HOST"] = self.nats_host + env_variables["NATS_INTERNAL_PORT"] = "4444" + env_variables["NATS_EXTERNAL_PORT"] = str(self.nats_port) + + self.base_cmd.extend( + ["--file", p.join(docker_compose_yml_dir, "docker_compose_nats.yml")] + ) + self.base_nats_cmd = [ + "docker-compose", + "--env-file", + instance.env_file, + "--project-name", + self.project_name, + "--file", + p.join(docker_compose_yml_dir, "docker_compose_nats.yml"), + ] + return self.base_nats_cmd + def setup_mongo_secure_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_mongo = self.with_mongo_secure = True env_variables["MONGO_HOST"] = self.mongo_host @@ -1170,6 +1205,7 @@ class ClickHouseCluster: with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, + with_nats=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, @@ -1258,6 +1294,7 @@ class ClickHouseCluster: with_kafka=with_kafka, with_kerberized_kafka=with_kerberized_kafka, with_rabbitmq=with_rabbitmq, + with_nats=with_nats, with_nginx=with_nginx, with_kerberized_hdfs=with_kerberized_hdfs, with_mongo=with_mongo or with_mongo_secure, @@ -1393,6 +1430,11 @@ class ClickHouseCluster: self.setup_rabbitmq_cmd(instance, env_variables, docker_compose_yml_dir) ) + if with_nats and not self.with_nats: + cmds.append( + self.setup_nats_cmd(instance, env_variables, docker_compose_yml_dir) + ) + if with_nginx and not self.with_nginx: cmds.append( self.setup_nginx_cmd(instance, env_variables, docker_compose_yml_dir) @@ -1836,6 +1878,18 @@ class ClickHouseCluster: raise Exception("Cannot wait RabbitMQ container") return False + def wait_nats_is_available(self, nats_ip, max_retries=5): + retries = 0 + while True: + if asyncio.run(check_nats_is_available(nats_ip)): + break + else: + retries += 1 + if retries > max_retries: + raise Exception("NATS is not available") + logging.debug("Waiting for NATS to start up") + time.sleep(1) + def wait_nginx_to_start(self, timeout=60): self.nginx_ip = self.get_instance_ip(self.nginx_host) start = time.time() @@ -2284,6 +2338,14 @@ class ClickHouseCluster: if self.wait_rabbitmq_to_start(throw=(i == 4)): break + if self.with_nats and self.base_nats_cmd: + logging.debug("Setup NATS") + subprocess_check_call(self.base_nats_cmd + common_opts) + self.nats_docker_id = self.get_instance_docker_id("nats1") + self.up_called = True + self.nats_ip = self.get_instance_ip("nats1") + self.wait_nats_is_available(self.nats_ip) + if self.with_hdfs and self.base_hdfs_cmd: logging.debug("Setup HDFS") os.makedirs(self.hdfs_logs_dir) @@ -2639,6 +2701,7 @@ class ClickHouseInstance: with_kafka, with_kerberized_kafka, with_rabbitmq, + with_nats, with_nginx, with_kerberized_hdfs, with_mongo, @@ -2719,6 +2782,7 @@ class ClickHouseInstance: self.with_kafka = with_kafka self.with_kerberized_kafka = with_kerberized_kafka self.with_rabbitmq = with_rabbitmq + self.with_nats = with_nats self.with_nginx = with_nginx self.with_kerberized_hdfs = with_kerberized_hdfs self.with_mongo = with_mongo @@ -3689,6 +3753,9 @@ class ClickHouseInstance: if self.with_rabbitmq: depends_on.append("rabbitmq1") + if self.with_nats: + depends_on.append("nats1") + if self.with_zookeeper: depends_on.append("zoo1") depends_on.append("zoo2") diff --git a/tests/integration/test_storage_nats/__init__.py b/tests/integration/test_storage_nats/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto b/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto new file mode 100755 index 00000000000..44a29facd13 --- /dev/null +++ b/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + + message KeyValueProto { + uint64 key = 1; + string value = 2; + } diff --git a/tests/integration/test_storage_nats/configs/macros.xml b/tests/integration/test_storage_nats/configs/macros.xml new file mode 100644 index 00000000000..4aa547e049e --- /dev/null +++ b/tests/integration/test_storage_nats/configs/macros.xml @@ -0,0 +1,7 @@ + + + nats1:4444 + macro + JSONEachRow + + diff --git a/tests/integration/test_storage_nats/configs/named_collection.xml b/tests/integration/test_storage_nats/configs/named_collection.xml new file mode 100644 index 00000000000..15817f321f0 --- /dev/null +++ b/tests/integration/test_storage_nats/configs/named_collection.xml @@ -0,0 +1,13 @@ + + + + nats1:4444 + named + JSONEachRow + 111 + 12 + click + house + + + diff --git a/tests/integration/test_storage_nats/configs/nats.xml b/tests/integration/test_storage_nats/configs/nats.xml new file mode 100644 index 00000000000..0a8be9122ad --- /dev/null +++ b/tests/integration/test_storage_nats/configs/nats.xml @@ -0,0 +1,6 @@ + + + click + house + + diff --git a/tests/integration/test_storage_nats/configs/users.xml b/tests/integration/test_storage_nats/configs/users.xml new file mode 100644 index 00000000000..797dfebba0e --- /dev/null +++ b/tests/integration/test_storage_nats/configs/users.xml @@ -0,0 +1,8 @@ + + + + + 1 + + + diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py new file mode 100644 index 00000000000..663a7dfc8c6 --- /dev/null +++ b/tests/integration/test_storage_nats/nats_pb2.py @@ -0,0 +1,34 @@ +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: clickhouse_path/format_schemas/nats.proto +"""Generated protocol buffer code.""" +from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n)clickhouse_path/format_schemas/nats.proto\"+\n\rKeyValueProto\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3') + + + +_KEYVALUEPROTO = DESCRIPTOR.message_types_by_name['KeyValueProto'] +KeyValueProto = _reflection.GeneratedProtocolMessageType('KeyValueProto', (_message.Message,), { + 'DESCRIPTOR' : _KEYVALUEPROTO, + '__module__' : 'clickhouse_path.format_schemas.nats_pb2' + # @@protoc_insertion_point(class_scope:KeyValueProto) + }) +_sym_db.RegisterMessage(KeyValueProto) + +if _descriptor._USE_C_DESCRIPTORS == False: + + DESCRIPTOR._options = None + _KEYVALUEPROTO._serialized_start=45 + _KEYVALUEPROTO._serialized_end=88 +# @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py new file mode 100644 index 00000000000..985302cc7a5 --- /dev/null +++ b/tests/integration/test_storage_nats/test.py @@ -0,0 +1,1459 @@ +import json +import os.path as p +import random +import subprocess +import threading +import logging +import time +from random import randrange +import math + +import asyncio +import nats +import pytest +from google.protobuf.internal.encoder import _VarintBytes +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster, check_nats_is_available +from helpers.test_tools import TSV + +from . import nats_pb2 + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "instance", + main_configs=[ + "configs/nats.xml", + "configs/macros.xml", + "configs/named_collection.xml", + ], + user_configs=["configs/users.xml"], + with_nats=True, + clickhouse_path_dir="clickhouse_path", +) + + +# Helpers + +def wait_nats_to_start(nats_ip, timeout=180): + start = time.time() + while time.time() - start < timeout: + try: + if asyncio.run(check_nats_is_available(nats_ip)): + logging.debug("NATS is available") + return + time.sleep(0.5) + except Exception as ex: + logging.debug("Can't connect to NATS " + str(ex)) + time.sleep(0.5) + + +def nats_check_result(result, check=False, ref_file="test_nats_json.reference"): + fpath = p.join(p.dirname(__file__), ref_file) + with open(fpath) as reference: + if check: + assert TSV(result) == TSV(reference) + else: + return TSV(result) == TSV(reference) + + +def kill_nats(nats_id): + p = subprocess.Popen(("docker", "stop", nats_id), stdout=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + + +def revive_nats(nats_id, nats_ip): + p = subprocess.Popen(("docker", "start", nats_id), stdout=subprocess.PIPE) + p.communicate() + wait_nats_to_start(nats_ip) + + +# Fixtures + + +@pytest.fixture(scope="module") +def nats_cluster(): + try: + cluster.start() + logging.debug("nats_id is {}".format(instance.cluster.nats_docker_id)) + instance.query("CREATE DATABASE test") + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def nats_setup_teardown(): + print("NATS is available - running test") + yield # run test + instance.query("DROP DATABASE test NO DELAY") + instance.query("CREATE DATABASE test") + + +# Tests + +async def nats_produce_messages(ip, subject, messages=(), bytes=None): + nc = await nats.connect('{}:4444'.format(ip), user='click', password='house') + logging.debug("NATS connection status: " + str(nc.is_connected)) + + for message in messages: + await nc.publish(subject, message.encode()) + if bytes is not None: + await nc.publish(subject, bytes) + logging.debug("Finished publising to " + subject) + + await nc.close() + return messages + + +def check_table_is_ready(instance, table_name): + try: + instance.query("SELECT * FROM {}".format(table_name)) + return True + except Exception: + return False + + + +def test_nats_select(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'select', + nats_format = 'JSONEachRow', + nats_row_delimiter = '\\n'; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'select', messages)) + + # The order of messages in select * from test.nats is not guaranteed, so sleep to collect everything in one select + time.sleep(1) + + result = "" + while True: + result += instance.query( + "SELECT * FROM test.nats ORDER BY key", ignore_error=True + ) + if nats_check_result(result): + break + + nats_check_result(result, True) + + +def test_nats_select_empty(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'empty', + nats_format = 'TSV', + nats_row_delimiter = '\\n'; + """ + ) + + assert int(instance.query("SELECT count() FROM test.nats")) == 0 + + +def test_nats_json_without_delimiter(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'json', + nats_format = 'JSONEachRow'; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + messages = "" + for i in range(25): + messages += json.dumps({"key": i, "value": i}) + "\n" + + all_messages = [messages] + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'json', all_messages)) + + messages = "" + for i in range(25, 50): + messages += json.dumps({"key": i, "value": i}) + "\n" + all_messages = [messages] + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'json', all_messages)) + + time.sleep(1) + + result = "" + time_limit_sec = 60 + deadline = time.monotonic() + time_limit_sec + + while time.monotonic() < deadline: + result += instance.query( + "SELECT * FROM test.nats ORDER BY key", ignore_error=True + ) + if nats_check_result(result): + break + + nats_check_result(result, True) + + +def test_nats_csv_with_delimiter(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'csv', + nats_format = 'CSV', + nats_row_delimiter = '\\n'; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + messages = [] + for i in range(50): + messages.append("{i}, {i}".format(i=i)) + + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'csv', messages)) + + time.sleep(1) + + result = "" + for _ in range(60): + result += instance.query( + "SELECT * FROM test.nats ORDER BY key", ignore_error=True + ) + if nats_check_result(result): + break + + nats_check_result(result, True) + + +def test_nats_tsv_with_delimiter(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'tsv', + nats_format = 'TSV', + nats_row_delimiter = '\\n'; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.nats; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + + messages = [] + for i in range(50): + messages.append("{i}\t{i}".format(i=i)) + + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'tsv', messages)) + + result = "" + for _ in range(60): + result = instance.query("SELECT * FROM test.view ORDER BY key") + if nats_check_result(result): + break + + nats_check_result(result, True) +# + +def test_nats_macros(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = '{nats_url}', + nats_subjects = '{nats_subjects}', + nats_format = '{nats_format}' + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + + message = "" + for i in range(50): + message += json.dumps({"key": i, "value": i}) + "\n" + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'macro', [message])) + + time.sleep(1) + + result = "" + for _ in range(60): + result += instance.query( + "SELECT * FROM test.nats ORDER BY key", ignore_error=True + ) + if nats_check_result(result): + break + + nats_check_result(result, True) + + +def test_nats_materialized_view(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'mv', + nats_format = 'JSONEachRow', + nats_row_delimiter = '\\n'; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.nats; + + CREATE TABLE test.view2 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS + SELECT * FROM test.nats group by (key, value); + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mv', messages)) + + time_limit_sec = 60 + deadline = time.monotonic() + time_limit_sec + + while time.monotonic() < deadline: + result = instance.query("SELECT * FROM test.view ORDER BY key") + if nats_check_result(result): + break + + nats_check_result(result, True) + + deadline = time.monotonic() + time_limit_sec + + while time.monotonic() < deadline: + result = instance.query("SELECT * FROM test.view2 ORDER BY key") + if nats_check_result(result): + break + + nats_check_result(result, True) + + +def test_nats_materialized_view_with_subquery(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'mvsq', + nats_format = 'JSONEachRow', + nats_row_delimiter = '\\n'; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM (SELECT * FROM test.nats); + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mvsq', messages)) + + time_limit_sec = 60 + deadline = time.monotonic() + time_limit_sec + + while time.monotonic() < deadline: + result = instance.query("SELECT * FROM test.view ORDER BY key") + if nats_check_result(result): + break + + nats_check_result(result, True) + + +def test_nats_many_materialized_views(nats_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.view1; + DROP TABLE IF EXISTS test.view2; + DROP TABLE IF EXISTS test.consumer1; + DROP TABLE IF EXISTS test.consumer2; + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'mmv', + nats_format = 'JSONEachRow', + nats_row_delimiter = '\\n'; + CREATE TABLE test.view1 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE TABLE test.view2 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer1 TO test.view1 AS + SELECT * FROM test.nats; + CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS + SELECT * FROM test.nats; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mmv', messages)) + + time_limit_sec = 60 + deadline = time.monotonic() + time_limit_sec + + while time.monotonic() < deadline: + result1 = instance.query("SELECT * FROM test.view1 ORDER BY key") + result2 = instance.query("SELECT * FROM test.view2 ORDER BY key") + if nats_check_result(result1) and nats_check_result(result2): + break + + instance.query( + """ + DROP TABLE test.consumer1; + DROP TABLE test.consumer2; + DROP TABLE test.view1; + DROP TABLE test.view2; + """ + ) + + nats_check_result(result1, True) + nats_check_result(result2, True) + + +def test_nats_protobuf(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value String) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'pb', + nats_format = 'Protobuf', + nats_schema = 'nats.proto:KeyValueProto'; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.nats; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + + data = b"" + for i in range(0, 20): + msg = nats_pb2.KeyValueProto() + msg.key = i + msg.value = str(i) + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'pb', bytes=data)) + data = b"" + for i in range(20, 21): + msg = nats_pb2.KeyValueProto() + msg.key = i + msg.value = str(i) + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'pb', bytes=data)) + data = b"" + for i in range(21, 50): + msg = nats_pb2.KeyValueProto() + msg.key = i + msg.value = str(i) + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'pb', bytes=data)) + + + result = "" + time_limit_sec = 60 + deadline = time.monotonic() + time_limit_sec + + while time.monotonic() < deadline: + result = instance.query("SELECT * FROM test.view ORDER BY key") + if nats_check_result(result): + break + + nats_check_result(result, True) + + +def test_nats_big_message(nats_cluster): + # Create batchs of messages of size ~100Kb + nats_messages = 1000 + batch_messages = 1000 + messages = [ + json.dumps({"key": i, "value": "x" * 100}) * batch_messages + for i in range(nats_messages) + ] + + + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value String) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'big', + nats_format = 'JSONEachRow'; + CREATE TABLE test.view (key UInt64, value String) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.nats; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'big', messages)) + + while True: + result = instance.query("SELECT count() FROM test.view") + if int(result) == batch_messages * nats_messages: + break + + + assert ( + int(result) == nats_messages * batch_messages + ), "ClickHouse lost some messages: {}".format(result) + + +def test_nats_mv_combo(nats_cluster): + NUM_MV = 5 + NUM_CONSUMERS = 4 + + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'combo', + nats_num_consumers = {}, + nats_format = 'JSONEachRow', + nats_row_delimiter = '\\n'; + """.format(NUM_CONSUMERS) + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + for mv_id in range(NUM_MV): + instance.query( + """ + DROP TABLE IF EXISTS test.combo_{0}; + DROP TABLE IF EXISTS test.combo_{0}_mv; + CREATE TABLE test.combo_{0} (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.combo_{0}_mv TO test.combo_{0} AS + SELECT * FROM test.nats; + """.format( + mv_id + ) + ) + + time.sleep(2) + + i = [0] + messages_num = 10000 + + def produce(): + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({"key": i[0], "value": i[0]})) + i[0] += 1 + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'combo', messages)) + + threads = [] + threads_num = 20 + + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + while True: + result = 0 + for mv_id in range(NUM_MV): + result += int( + instance.query("SELECT count() FROM test.combo_{0}".format(mv_id)) + ) + if int(result) == messages_num * threads_num * NUM_MV: + break + time.sleep(1) + + for thread in threads: + thread.join() + + for mv_id in range(NUM_MV): + instance.query( + """ + DROP TABLE test.combo_{0}_mv; + DROP TABLE test.combo_{0}; + """.format( + mv_id + ) + ) + + assert ( + int(result) == messages_num * threads_num * NUM_MV + ), "ClickHouse lost some messages: {}".format(result) + + +def test_nats_insert(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'insert', + nats_format = 'TSV', + nats_row_delimiter = '\\n'; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + values = [] + for i in range(50): + values.append("({i}, {i})".format(i=i)) + values = ",".join(values) + + insert_messages = [] + + async def sub_to_nats(): + nc = await nats.connect('{}:4444'.format(nats_cluster.nats_ip), user='click', password='house') + sub = await nc.subscribe('insert') + await sub.unsubscribe(50) + async for msg in sub.messages: + insert_messages.append(msg.data.decode()) + + await sub.drain() + await nc.drain() + + def run_sub(): + asyncio.run(sub_to_nats()) + + thread = threading.Thread(target=run_sub) + thread.start() + time.sleep(1) + + while True: + try: + instance.query("INSERT INTO test.nats VALUES {}".format(values)) + break + except QueryRuntimeException as e: + if "Local: Timed out." in str(e): + continue + else: + raise + thread.join() + + result = "\n".join(insert_messages) + nats_check_result(result, True) + + +def test_nats_many_subjects_insert_wrong(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'insert1,insert2.>,insert3.*.foo', + nats_format = 'TSV', + nats_row_delimiter = '\\n'; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + values = [] + for i in range(50): + values.append("({i}, {i})".format(i=i)) + values = ",".join(values) + + # no subject specified + instance.query_and_get_error("INSERT INTO test.nats VALUES {}".format(values)) + + # can't insert into wildcard subjects + instance.query_and_get_error("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert2.>' VALUES {}".format(values)) + instance.query_and_get_error("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.*.foo' VALUES {}".format(values)) + + # specified subject is not among engine's subjects + instance.query_and_get_error("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert4' VALUES {}".format(values)) + instance.query_and_get_error("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.foo.baz' VALUES {}".format(values)) + instance.query_and_get_error("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='foo.insert2' VALUES {}".format(values)) + +def test_nats_many_subjects_insert_right(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'right_insert1,right_insert2', + nats_format = 'TSV', + nats_row_delimiter = '\\n'; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + values = [] + for i in range(50): + values.append("({i}, {i})".format(i=i)) + values = ",".join(values) + + insert_messages = [] + + async def sub_to_nats(): + nc = await nats.connect('{}:4444'.format(nats_cluster.nats_ip), user='click', password='house') + sub = await nc.subscribe('right_insert1') + await sub.unsubscribe(50) + async for msg in sub.messages: + insert_messages.append(msg.data.decode()) + + await sub.drain() + await nc.drain() + + def run_sub(): + asyncio.run(sub_to_nats()) + + thread = threading.Thread(target=run_sub) + thread.start() + time.sleep(1) + + while True: + try: + instance.query("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='right_insert1' VALUES {}".format(values)) + break + except QueryRuntimeException as e: + if "Local: Timed out." in str(e): + continue + else: + raise + thread.join() + + result = "\n".join(insert_messages) + nats_check_result(result, True) + + +def test_nats_many_inserts(nats_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.nats_many; + DROP TABLE IF EXISTS test.nats_consume; + DROP TABLE IF EXISTS test.view_many; + DROP TABLE IF EXISTS test.consumer_many; + CREATE TABLE test.nats_many (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'many_inserts', + nats_format = 'TSV', + nats_row_delimiter = '\\n'; + CREATE TABLE test.nats_consume (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'many_inserts', + nats_format = 'TSV', + nats_row_delimiter = '\\n'; + """ + ) + while not check_table_is_ready(instance, 'test.nats_consume'): + logging.debug("Table test.nats_consume is not yet ready") + time.sleep(0.5) + + messages_num = 10000 + values = [] + for i in range(messages_num): + values.append("({i}, {i})".format(i=i)) + values = ",".join(values) + + def insert(): + while True: + try: + instance.query( + "INSERT INTO test.nats_many VALUES {}".format(values) + ) + break + except QueryRuntimeException as e: + if "Local: Timed out." in str(e): + continue + else: + raise + + threads = [] + threads_num = 10 + for _ in range(threads_num): + threads.append(threading.Thread(target=insert)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + instance.query( + """ + CREATE TABLE test.view_many (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer_many TO test.view_many AS + SELECT * FROM test.nats_consume; + """ + ) + + for thread in threads: + thread.join() + + while True: + result = instance.query("SELECT count() FROM test.view_many") + print(result, messages_num * threads_num) + if int(result) == messages_num * threads_num: + break + time.sleep(1) + + instance.query( + """ + DROP TABLE test.nats_consume; + DROP TABLE test.nats_many; + DROP TABLE test.consumer_many; + DROP TABLE test.view_many; + """ + ) + + assert ( + int(result) == messages_num * threads_num + ), "ClickHouse lost some messages: {}".format(result) + + +def test_nats_overloaded_insert(nats_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.view_overload; + DROP TABLE IF EXISTS test.consumer_overload; + DROP TABLE IF EXISTS test.nats_consume; + CREATE TABLE test.nats_consume (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'over', + nats_num_consumers = 5, + nats_max_block_size = 10000, + nats_format = 'TSV', + nats_row_delimiter = '\\n'; + CREATE TABLE test.nats_overload (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'over', + nats_format = 'TSV', + nats_row_delimiter = '\\n'; + CREATE TABLE test.view_overload (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key + SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; + CREATE MATERIALIZED VIEW test.consumer_overload TO test.view_overload AS + SELECT * FROM test.nats_consume; + """ + ) + while not check_table_is_ready(instance, 'test.nats_consume'): + logging.debug("Table test.nats_consume is not yet ready") + time.sleep(0.5) + + messages_num = 100000 + + def insert(): + values = [] + for i in range(messages_num): + values.append("({i}, {i})".format(i=i)) + values = ",".join(values) + + while True: + try: + instance.query( + "INSERT INTO test.nats_overload VALUES {}".format(values) + ) + break + except QueryRuntimeException as e: + if "Local: Timed out." in str(e): + continue + else: + raise + + threads = [] + threads_num = 5 + for _ in range(threads_num): + threads.append(threading.Thread(target=insert)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + while True: + result = instance.query("SELECT count() FROM test.view_overload") + time.sleep(1) + if int(result) == messages_num * threads_num: + break + + instance.query( + """ + DROP TABLE test.consumer_overload; + DROP TABLE test.view_overload; + DROP TABLE test.nats_consume; + DROP TABLE test.nats_overload; + """ + ) + + for thread in threads: + thread.join() + + assert ( + int(result) == messages_num * threads_num + ), "ClickHouse lost some messages: {}".format(result) + + +def test_nats_virtual_column(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats_virtuals (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'virtuals', + nats_format = 'JSONEachRow'; + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT value, key, _subject FROM test.nats_virtuals; + """ + ) + while not check_table_is_ready(instance, 'test.nats_virtuals'): + logging.debug("Table test.nats_virtuals is not yet ready") + time.sleep(0.5) + + + message_num = 10 + i = 0 + messages = [] + for _ in range(message_num): + messages.append(json.dumps({"key": i, "value": i})) + i += 1 + + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'virtuals', messages)) + + while True: + result = instance.query("SELECT count() FROM test.view") + time.sleep(1) + if int(result) == message_num: + break + + + result = instance.query( + """ + SELECT key, value, _subject + FROM test.view ORDER BY key + """ + ) + + expected = """\ +0 0 virtuals +1 1 virtuals +2 2 virtuals +3 3 virtuals +4 4 virtuals +5 5 virtuals +6 6 virtuals +7 7 virtuals +8 8 virtuals +9 9 virtuals +""" + + instance.query( + """ + DROP TABLE test.nats_virtuals; + DROP TABLE test.view; + """ + ) + + assert TSV(result) == TSV(expected) + + +def test_nats_virtual_column_with_materialized_view(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats_virtuals_mv (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'virtuals_mv', + nats_format = 'JSONEachRow'; + CREATE TABLE test.view (key UInt64, value UInt64, subject String) ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT *, _subject as subject + FROM test.nats_virtuals_mv; + """ + ) + while not check_table_is_ready(instance, 'test.nats_virtuals_mv'): + logging.debug("Table test.nats_virtuals_mv is not yet ready") + time.sleep(0.5) + + message_num = 10 + i = 0 + messages = [] + for _ in range(message_num): + messages.append(json.dumps({"key": i, "value": i})) + i += 1 + + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'virtuals_mv', messages)) + + while True: + result = instance.query("SELECT count() FROM test.view") + time.sleep(1) + if int(result) == message_num: + break + + + result = instance.query( + "SELECT key, value, subject FROM test.view ORDER BY key" + ) + expected = """\ +0 0 virtuals_mv +1 1 virtuals_mv +2 2 virtuals_mv +3 3 virtuals_mv +4 4 virtuals_mv +5 5 virtuals_mv +6 6 virtuals_mv +7 7 virtuals_mv +8 8 virtuals_mv +9 9 virtuals_mv +""" + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + DROP TABLE test.nats_virtuals_mv + """ + ) + + assert TSV(result) == TSV(expected) + + +def test_nats_many_consumers_to_each_queue(nats_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.destination; + CREATE TABLE test.destination(key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + """ + ) + + num_tables = 4 + for table_id in range(num_tables): + print(("Setting up table {}".format(table_id))) + instance.query( + """ + DROP TABLE IF EXISTS test.many_consumers_{0}; + DROP TABLE IF EXISTS test.many_consumers_{0}_mv; + CREATE TABLE test.many_consumers_{0} (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'many_consumers', + nats_num_consumers = 2, + nats_queue_group = 'many_consumers', + nats_format = 'JSONEachRow', + nats_row_delimiter = '\\n'; + CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS + SELECT key, value FROM test.many_consumers_{0}; + """.format( + table_id + ) + ) + while not check_table_is_ready(instance, 'test.many_consumers_{}'.format(table_id)): + logging.debug("Table test.many_consumers_{} is not yet ready".format(table_id)) + time.sleep(0.5) + + i = [0] + messages_num = 1000 + + + def produce(): + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({"key": i[0], "value": i[0]})) + i[0] += 1 + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'many_consumers', messages)) + + threads = [] + threads_num = 20 + + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + result1 = "" + while True: + result1 = instance.query("SELECT count() FROM test.destination") + time.sleep(1) + if int(result1) == messages_num * threads_num: + break + + for thread in threads: + thread.join() + + for consumer_id in range(num_tables): + instance.query( + """ + DROP TABLE test.many_consumers_{0}; + DROP TABLE test.many_consumers_{0}_mv; + """.format( + consumer_id + ) + ) + + instance.query( + """ + DROP TABLE test.destination; + """ + ) + + assert ( + int(result1) == messages_num * threads_num + ), "ClickHouse lost some messages: {}".format(result1) + + +def test_nats_restore_failed_connection_without_losses_on_write(nats_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.consume; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE TABLE test.consume (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'producer_reconnect', + nats_format = 'JSONEachRow', + nats_num_consumers = 2, + nats_row_delimiter = '\\n'; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.consume; + DROP TABLE IF EXISTS test.producer_reconnect; + CREATE TABLE test.producer_reconnect (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'producer_reconnect', + nats_format = 'JSONEachRow', + nats_row_delimiter = '\\n'; + """ + ) + while not check_table_is_ready(instance, 'test.consume'): + logging.debug("Table test.consume is not yet ready") + time.sleep(0.5) + + + messages_num = 100000 + values = [] + for i in range(messages_num): + values.append("({i}, {i})".format(i=i)) + values = ",".join(values) + + while True: + try: + instance.query( + "INSERT INTO test.producer_reconnect VALUES {}".format(values) + ) + break + except QueryRuntimeException as e: + if "Local: Timed out." in str(e): + continue + else: + raise + + while int(instance.query("SELECT count() FROM test.view")) == 0: + time.sleep(0.1) + + kill_nats(nats_cluster.nats_docker_id) + time.sleep(4) + revive_nats(nats_cluster.nats_docker_id, nats_cluster.nats_ip) + + while True: + result = instance.query("SELECT count(DISTINCT key) FROM test.view") + time.sleep(1) + if int(result) == messages_num: + break + + instance.query( + """ + DROP TABLE test.consume; + DROP TABLE test.producer_reconnect; + """ + ) + + assert int(result) == messages_num, "ClickHouse lost some messages: {}".format( + result + ) + + +def test_nats_no_connection_at_startup_1(nats_cluster): + # no connection when table is initialized + nats_cluster.pause_container("nats1") + instance.query_and_get_error( + """ + CREATE TABLE test.cs (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'cs', + nats_format = 'JSONEachRow', + nats_num_consumers = '5', + nats_row_delimiter = '\\n'; + """ + ) + nats_cluster.unpause_container("nats1") + + +def test_nats_no_connection_at_startup_2(nats_cluster): + instance.query( + """ + CREATE TABLE test.cs (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'cs', + nats_format = 'JSONEachRow', + nats_num_consumers = '5', + nats_row_delimiter = '\\n'; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.cs; + """ + ) + while not check_table_is_ready(instance, 'test.cs'): + logging.debug("Table test.cs is not yet ready") + time.sleep(0.5) + + instance.query("DETACH TABLE test.cs") + nats_cluster.pause_container("nats1") + instance.query("ATTACH TABLE test.cs") + nats_cluster.unpause_container("nats1") + + messages_num = 1000 + messages = [] + for i in range(messages_num): + messages.append(json.dumps({"key": i, "value": i})) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'cs', messages)) + + while True: + result = instance.query("SELECT count() FROM test.view") + time.sleep(1) + if int(result) == messages_num: + break + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.cs; + """ + ) + + assert int(result) == messages_num, "ClickHouse lost some messages: {}".format( + result + ) + + +def test_nats_format_factory_settings(nats_cluster): + instance.query( + """ + CREATE TABLE test.format_settings ( + id String, date DateTime + ) ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'format_settings', + nats_format = 'JSONEachRow', + date_time_input_format = 'best_effort'; + """ + ) + while not check_table_is_ready(instance, 'test.format_settings'): + logging.debug("Table test.format_settings is not yet ready") + time.sleep(0.5) + + + message = json.dumps( + {"id": "format_settings_test", "date": "2021-01-19T14:42:33.1829214Z"} + ) + expected = instance.query( + """SELECT parseDateTimeBestEffort(CAST('2021-01-19T14:42:33.1829214Z', 'String'))""" + ) + + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'format_settings', [message])) + + while True: + result = instance.query("SELECT date FROM test.format_settings") + if result == expected: + break + + instance.query( + """ + CREATE TABLE test.view ( + id String, date DateTime + ) ENGINE = MergeTree ORDER BY id; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.format_settings; + """ + ) + + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'format_settings', [message])) + while True: + result = instance.query("SELECT date FROM test.view") + if result == expected: + break + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.format_settings; + """ + ) + + assert result == expected + + +def test_nats_bad_args(nats_cluster): + instance.query_and_get_error( + """ + CREATE TABLE test.drop (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_secure = true, + nats_format = 'JSONEachRow'; + """ + ) + + +def test_nats_drop_mv(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'mv', + nats_format = 'JSONEachRow'; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.nats; + """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + messages = [] + for i in range(20): + messages.append(json.dumps({"key": i, "value": i})) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mv', messages)) + + instance.query("DROP VIEW test.consumer") + messages = [] + for i in range(20, 40): + messages.append(json.dumps({"key": i, "value": i})) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mv', messages)) + + instance.query( + """ + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.nats; + """ + ) + messages = [] + for i in range(40, 50): + messages.append(json.dumps({"key": i, "value": i})) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mv', messages)) + + while True: + result = instance.query("SELECT * FROM test.view ORDER BY key") + if nats_check_result(result): + break + + nats_check_result(result, True) + + instance.query("DROP VIEW test.consumer") + messages = [] + for i in range(50, 60): + messages.append(json.dumps({"key": i, "value": i})) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mv', messages)) + + count = 0 + while True: + count = int(instance.query("SELECT count() FROM test.nats")) + if count: + break + + assert count > 0 + + +def test_nats_predefined_configuration(nats_cluster): + instance.query( + """ + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS(nats1) """ + ) + while not check_table_is_ready(instance, 'test.nats'): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'named', [json.dumps({"key": 1, "value": 2})])) + while True: + result = instance.query( + "SELECT * FROM test.nats ORDER BY key", ignore_error=True + ) + if result == "1\t2\n": + break + + +if __name__ == "__main__": + cluster.start() + input("Cluster created, press any key to destroy...") + cluster.shutdown() diff --git a/tests/integration/test_storage_nats/test_nats_json.reference b/tests/integration/test_storage_nats/test_nats_json.reference new file mode 100644 index 00000000000..959bb2aad74 --- /dev/null +++ b/tests/integration/test_storage_nats/test_nats_json.reference @@ -0,0 +1,50 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +21 21 +22 22 +23 23 +24 24 +25 25 +26 26 +27 27 +28 28 +29 29 +30 30 +31 31 +32 32 +33 33 +34 34 +35 35 +36 36 +37 37 +38 38 +39 39 +40 40 +41 41 +42 42 +43 43 +44 44 +45 45 +46 46 +47 47 +48 48 +49 49 From 51ce27acfc6e398e61630e5322dffd92f6f8d493 Mon Sep 17 00:00:00 2001 From: Vladislav V Date: Tue, 17 May 2022 17:00:25 +0300 Subject: [PATCH 018/627] Update materializedview.md --- docs/ru/engines/table-engines/special/materializedview.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/special/materializedview.md b/docs/ru/engines/table-engines/special/materializedview.md index 091e45518ad..a8782c57ca8 100644 --- a/docs/ru/engines/table-engines/special/materializedview.md +++ b/docs/ru/engines/table-engines/special/materializedview.md @@ -5,5 +5,5 @@ sidebar_label: MaterializedView # MaterializedView {#materializedview} -Используется для реализации материализованных представлений (подробнее см. запрос [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query)). Для хранения данных, использует другой движок, который был указан при создании представления. При чтении из таблицы, просто использует этот движок. +Используется для реализации материализованных представлений (подробнее см. запрос [CREATE VIEW](../../../sql-reference/statements/create/view.md#материализованные-представления-materialized)). Для хранения данных, использует другой движок, который был указан при создании представления. При чтении из таблицы, просто использует этот движок. From d0df471a2d9e4c956e725c584fe9ddb2d0f11658 Mon Sep 17 00:00:00 2001 From: Vladislav V Date: Tue, 17 May 2022 17:03:12 +0300 Subject: [PATCH 019/627] Update materializedview.md --- docs/ru/engines/table-engines/special/materializedview.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/special/materializedview.md b/docs/ru/engines/table-engines/special/materializedview.md index a8782c57ca8..9500d6197e7 100644 --- a/docs/ru/engines/table-engines/special/materializedview.md +++ b/docs/ru/engines/table-engines/special/materializedview.md @@ -5,5 +5,5 @@ sidebar_label: MaterializedView # MaterializedView {#materializedview} -Используется для реализации материализованных представлений (подробнее см. запрос [CREATE VIEW](../../../sql-reference/statements/create/view.md#материализованные-представления-materialized)). Для хранения данных, использует другой движок, который был указан при создании представления. При чтении из таблицы, просто использует этот движок. +Используется для реализации материализованных представлений (подробнее см. запрос [CREATE VIEW](../../../sql-reference/statements/create/view.md#materialized)). Для хранения данных, использует другой движок, который был указан при создании представления. При чтении из таблицы, просто использует этот движок. From 87217ce6ccfd861116565c4a07c9a9c613568cda Mon Sep 17 00:00:00 2001 From: tchepavel Date: Mon, 23 May 2022 14:57:39 +0300 Subject: [PATCH 020/627] NATS. Add new setting to doc; clean up code. --- .../table-engines/integrations/nats.md | 24 +++++++++++++++++-- .../NATS/WriteBufferToNATSProducer.cpp | 24 ++++++++++++------- 2 files changed, 38 insertions(+), 10 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md index dc99c3515b9..7c975653f0e 100644 --- a/docs/en/engines/table-engines/integrations/nats.md +++ b/docs/en/engines/table-engines/integrations/nats.md @@ -67,6 +67,26 @@ SSL connection: For secure connection use `nats_secure = 1`. The default behaviour of the used library is not to check if the created TLS connection is sufficiently secure. Whether the certificate is expired, self-signed, missing or invalid: the connection is simply permitted. More strict checking of certificates can possibly be implemented in the future. +Writing to NATS table: + +If table reads only from one subject, any insert will publish to the same subject. +However, if table reads from multiple subjects, we need to specify which subject we want to publish to. +That is why whenever inserting into table with multiple subjects, setting `stream_like_engine_insert_queue` is needed. +You can select one of the subjects the table reads from and publish your data there. For example: + +``` sql + CREATE TABLE queue ( + key UInt64, + value UInt64 + ) ENGINE = NATS + SETTINGS nats_url = 'localhost:4444', + nats_subjects = 'subject1,subject2', + nats_format = 'JSONEachRow'; + + INSERT INTO queue + SETTINGS stream_like_engine_insert_queue = 'subject2' + VALUES (1, 1); +``` Also format settings can be added along with nats-related settings. @@ -79,7 +99,7 @@ Example: date DateTime ) ENGINE = NATS SETTINGS nats_url = 'localhost:4444', - nats_subject = 'subject1', + nats_subjects = 'subject1', nats_format = 'JSONEachRow', date_time_input_format = 'best_effort'; ``` @@ -114,7 +134,7 @@ Example: value UInt64 ) ENGINE = NATS SETTINGS nats_url = 'localhost:4444', - nats_subject = 'subject1', + nats_subjects = 'subject1', nats_format = 'JSONEachRow', date_time_input_format = 'best_effort'; diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index f4464db1564..a5ac74018f2 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -125,18 +125,26 @@ void WriteBufferToNATSProducer::publishThreadFunc(void * arg) void WriteBufferToNATSProducer::writingFunc() { - while ((!payloads.empty() || wait_all) && !shutdown_called.load()) + try { - publish(); + while ((!payloads.empty() || wait_all) && !shutdown_called.load()) + { + publish(); - LOG_DEBUG(log, "Writing func {} {} {}", wait_payloads.load(), payloads.empty(), natsConnection_Buffered(connection.getConnection())); - if (wait_payloads.load() && payloads.empty() && natsConnection_Buffered(connection.getConnection()) == 0) - wait_all = false; + LOG_DEBUG( + log, "Writing func {} {} {}", wait_payloads.load(), payloads.empty(), natsConnection_Buffered(connection.getConnection())); + if (wait_payloads.load() && payloads.empty() && natsConnection_Buffered(connection.getConnection()) == 0) + wait_all = false; - if (!connection.isConnected() && wait_all) - connection.reconnect(); + if (!connection.isConnected() && wait_all) + connection.reconnect(); - iterateEventLoop(); + iterateEventLoop(); + } + } + catch (...) + { + tryLogCurrentException(log); } LOG_DEBUG(log, "Producer on subject {} completed", subject); From fc83f97199553002e6ed62cc4eb5bd668944c18d Mon Sep 17 00:00:00 2001 From: tchepavel Date: Tue, 24 May 2022 18:14:16 +0300 Subject: [PATCH 021/627] Fix same names in proto test --- .../clickhouse_path/format_schemas/nats.proto | 2 +- tests/integration/test_storage_nats/nats_pb2.py | 16 ++++++++-------- tests/integration/test_storage_nats/test.py | 8 ++++---- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto b/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto index 44a29facd13..090ed917cdd 100755 --- a/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto +++ b/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto @@ -1,6 +1,6 @@ syntax = "proto3"; - message KeyValueProto { + message ProtoKeyValue { uint64 key = 1; string value = 2; } diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py index 663a7dfc8c6..b20c32b14a0 100644 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ b/tests/integration/test_storage_nats/nats_pb2.py @@ -14,21 +14,21 @@ _sym_db = _symbol_database.Default() -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n)clickhouse_path/format_schemas/nats.proto\"+\n\rKeyValueProto\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n)clickhouse_path/format_schemas/nats.proto\"+\n\rProtoKeyValue\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3') -_KEYVALUEPROTO = DESCRIPTOR.message_types_by_name['KeyValueProto'] -KeyValueProto = _reflection.GeneratedProtocolMessageType('KeyValueProto', (_message.Message,), { - 'DESCRIPTOR' : _KEYVALUEPROTO, +_PROTOKEYVALUE = DESCRIPTOR.message_types_by_name['ProtoKeyValue'] +ProtoKeyValue = _reflection.GeneratedProtocolMessageType('ProtoKeyValue', (_message.Message,), { + 'DESCRIPTOR' : _PROTOKEYVALUE, '__module__' : 'clickhouse_path.format_schemas.nats_pb2' - # @@protoc_insertion_point(class_scope:KeyValueProto) + # @@protoc_insertion_point(class_scope:ProtoKeyValue) }) -_sym_db.RegisterMessage(KeyValueProto) +_sym_db.RegisterMessage(ProtoKeyValue) if _descriptor._USE_C_DESCRIPTORS == False: DESCRIPTOR._options = None - _KEYVALUEPROTO._serialized_start=45 - _KEYVALUEPROTO._serialized_end=88 + _PROTOKEYVALUE._serialized_start=45 + _PROTOKEYVALUE._serialized_end=88 # @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py index 985302cc7a5..0470e29af91 100644 --- a/tests/integration/test_storage_nats/test.py +++ b/tests/integration/test_storage_nats/test.py @@ -467,7 +467,7 @@ def test_nats_protobuf(nats_cluster): SETTINGS nats_url = 'nats1:4444', nats_subjects = 'pb', nats_format = 'Protobuf', - nats_schema = 'nats.proto:KeyValueProto'; + nats_schema = 'nats.proto:ProtoKeyValue'; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; @@ -482,7 +482,7 @@ def test_nats_protobuf(nats_cluster): data = b"" for i in range(0, 20): - msg = nats_pb2.KeyValueProto() + msg = nats_pb2.ProtoKeyValue() msg.key = i msg.value = str(i) serialized_msg = msg.SerializeToString() @@ -490,7 +490,7 @@ def test_nats_protobuf(nats_cluster): asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'pb', bytes=data)) data = b"" for i in range(20, 21): - msg = nats_pb2.KeyValueProto() + msg = nats_pb2.ProtoKeyValue() msg.key = i msg.value = str(i) serialized_msg = msg.SerializeToString() @@ -498,7 +498,7 @@ def test_nats_protobuf(nats_cluster): asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'pb', bytes=data)) data = b"" for i in range(21, 50): - msg = nats_pb2.KeyValueProto() + msg = nats_pb2.ProtoKeyValue() msg.key = i msg.value = str(i) serialized_msg = msg.SerializeToString() From 681695b7069d086d6a14ac58fa4d988b87f82e30 Mon Sep 17 00:00:00 2001 From: tchepavel Date: Wed, 25 May 2022 21:31:14 +0300 Subject: [PATCH 022/627] Black python --- tests/integration/test_storage_nats/test.py | 198 +++++++++++--------- 1 file changed, 114 insertions(+), 84 deletions(-) diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py index 0470e29af91..a29a72ea909 100644 --- a/tests/integration/test_storage_nats/test.py +++ b/tests/integration/test_storage_nats/test.py @@ -34,6 +34,7 @@ instance = cluster.add_instance( # Helpers + def wait_nats_to_start(nats_ip, timeout=180): start = time.time() while time.time() - start < timeout: @@ -94,8 +95,9 @@ def nats_setup_teardown(): # Tests + async def nats_produce_messages(ip, subject, messages=(), bytes=None): - nc = await nats.connect('{}:4444'.format(ip), user='click', password='house') + nc = await nats.connect("{}:4444".format(ip), user="click", password="house") logging.debug("NATS connection status: " + str(nc.is_connected)) for message in messages: @@ -116,7 +118,6 @@ def check_table_is_ready(instance, table_name): return False - def test_nats_select(nats_cluster): instance.query( """ @@ -128,14 +129,14 @@ def test_nats_select(nats_cluster): nats_row_delimiter = '\\n'; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) messages = [] for i in range(50): messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'select', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "select", messages)) # The order of messages in select * from test.nats is not guaranteed, so sleep to collect everything in one select time.sleep(1) @@ -176,7 +177,7 @@ def test_nats_json_without_delimiter(nats_cluster): nats_format = 'JSONEachRow'; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) @@ -185,13 +186,13 @@ def test_nats_json_without_delimiter(nats_cluster): messages += json.dumps({"key": i, "value": i}) + "\n" all_messages = [messages] - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'json', all_messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "json", all_messages)) messages = "" for i in range(25, 50): messages += json.dumps({"key": i, "value": i}) + "\n" all_messages = [messages] - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'json', all_messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "json", all_messages)) time.sleep(1) @@ -220,7 +221,7 @@ def test_nats_csv_with_delimiter(nats_cluster): nats_row_delimiter = '\\n'; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) @@ -228,7 +229,7 @@ def test_nats_csv_with_delimiter(nats_cluster): for i in range(50): messages.append("{i}, {i}".format(i=i)) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'csv', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "csv", messages)) time.sleep(1) @@ -259,16 +260,15 @@ def test_nats_tsv_with_delimiter(nats_cluster): SELECT * FROM test.nats; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) - messages = [] for i in range(50): messages.append("{i}\t{i}".format(i=i)) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'tsv', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "tsv", messages)) result = "" for _ in range(60): @@ -277,8 +277,11 @@ def test_nats_tsv_with_delimiter(nats_cluster): break nats_check_result(result, True) + + # + def test_nats_macros(nats_cluster): instance.query( """ @@ -289,15 +292,14 @@ def test_nats_macros(nats_cluster): nats_format = '{nats_format}' """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) - message = "" for i in range(50): message += json.dumps({"key": i, "value": i}) + "\n" - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'macro', [message])) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "macro", [message])) time.sleep(1) @@ -334,16 +336,15 @@ def test_nats_materialized_view(nats_cluster): SELECT * FROM test.nats group by (key, value); """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) - messages = [] for i in range(50): messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mv', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "mv", messages)) time_limit_sec = 60 deadline = time.monotonic() + time_limit_sec @@ -381,15 +382,14 @@ def test_nats_materialized_view_with_subquery(nats_cluster): SELECT * FROM (SELECT * FROM test.nats); """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) - messages = [] for i in range(50): messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mvsq', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "mvsq", messages)) time_limit_sec = 60 deadline = time.monotonic() + time_limit_sec @@ -427,15 +427,14 @@ def test_nats_many_materialized_views(nats_cluster): SELECT * FROM test.nats; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) - messages = [] for i in range(50): messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mmv', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "mmv", messages)) time_limit_sec = 60 deadline = time.monotonic() + time_limit_sec @@ -475,11 +474,10 @@ def test_nats_protobuf(nats_cluster): SELECT * FROM test.nats; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) - data = b"" for i in range(0, 20): msg = nats_pb2.ProtoKeyValue() @@ -487,7 +485,7 @@ def test_nats_protobuf(nats_cluster): msg.value = str(i) serialized_msg = msg.SerializeToString() data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'pb', bytes=data)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "pb", bytes=data)) data = b"" for i in range(20, 21): msg = nats_pb2.ProtoKeyValue() @@ -495,7 +493,7 @@ def test_nats_protobuf(nats_cluster): msg.value = str(i) serialized_msg = msg.SerializeToString() data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'pb', bytes=data)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "pb", bytes=data)) data = b"" for i in range(21, 50): msg = nats_pb2.ProtoKeyValue() @@ -503,8 +501,7 @@ def test_nats_protobuf(nats_cluster): msg.value = str(i) serialized_msg = msg.SerializeToString() data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'pb', bytes=data)) - + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "pb", bytes=data)) result = "" time_limit_sec = 60 @@ -527,7 +524,6 @@ def test_nats_big_message(nats_cluster): for i in range(nats_messages) ] - instance.query( """ CREATE TABLE test.nats (key UInt64, value String) @@ -542,18 +538,17 @@ def test_nats_big_message(nats_cluster): SELECT * FROM test.nats; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'big', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "big", messages)) while True: result = instance.query("SELECT count() FROM test.view") if int(result) == batch_messages * nats_messages: break - assert ( int(result) == nats_messages * batch_messages ), "ClickHouse lost some messages: {}".format(result) @@ -572,9 +567,11 @@ def test_nats_mv_combo(nats_cluster): nats_num_consumers = {}, nats_format = 'JSONEachRow', nats_row_delimiter = '\\n'; - """.format(NUM_CONSUMERS) + """.format( + NUM_CONSUMERS + ) ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) @@ -603,7 +600,7 @@ def test_nats_mv_combo(nats_cluster): for _ in range(messages_num): messages.append(json.dumps({"key": i[0], "value": i[0]})) i[0] += 1 - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'combo', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "combo", messages)) threads = [] threads_num = 20 @@ -653,7 +650,7 @@ def test_nats_insert(nats_cluster): nats_row_delimiter = '\\n'; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) @@ -665,8 +662,10 @@ def test_nats_insert(nats_cluster): insert_messages = [] async def sub_to_nats(): - nc = await nats.connect('{}:4444'.format(nats_cluster.nats_ip), user='click', password='house') - sub = await nc.subscribe('insert') + nc = await nats.connect( + "{}:4444".format(nats_cluster.nats_ip), user="click", password="house" + ) + sub = await nc.subscribe("insert") await sub.unsubscribe(50) async for msg in sub.messages: insert_messages.append(msg.data.decode()) @@ -707,7 +706,7 @@ def test_nats_many_subjects_insert_wrong(nats_cluster): nats_row_delimiter = '\\n'; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) @@ -720,13 +719,34 @@ def test_nats_many_subjects_insert_wrong(nats_cluster): instance.query_and_get_error("INSERT INTO test.nats VALUES {}".format(values)) # can't insert into wildcard subjects - instance.query_and_get_error("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert2.>' VALUES {}".format(values)) - instance.query_and_get_error("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.*.foo' VALUES {}".format(values)) + instance.query_and_get_error( + "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert2.>' VALUES {}".format( + values + ) + ) + instance.query_and_get_error( + "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.*.foo' VALUES {}".format( + values + ) + ) # specified subject is not among engine's subjects - instance.query_and_get_error("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert4' VALUES {}".format(values)) - instance.query_and_get_error("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.foo.baz' VALUES {}".format(values)) - instance.query_and_get_error("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='foo.insert2' VALUES {}".format(values)) + instance.query_and_get_error( + "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert4' VALUES {}".format( + values + ) + ) + instance.query_and_get_error( + "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.foo.baz' VALUES {}".format( + values + ) + ) + instance.query_and_get_error( + "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='foo.insert2' VALUES {}".format( + values + ) + ) + def test_nats_many_subjects_insert_right(nats_cluster): instance.query( @@ -739,7 +759,7 @@ def test_nats_many_subjects_insert_right(nats_cluster): nats_row_delimiter = '\\n'; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) @@ -751,8 +771,10 @@ def test_nats_many_subjects_insert_right(nats_cluster): insert_messages = [] async def sub_to_nats(): - nc = await nats.connect('{}:4444'.format(nats_cluster.nats_ip), user='click', password='house') - sub = await nc.subscribe('right_insert1') + nc = await nats.connect( + "{}:4444".format(nats_cluster.nats_ip), user="click", password="house" + ) + sub = await nc.subscribe("right_insert1") await sub.unsubscribe(50) async for msg in sub.messages: insert_messages.append(msg.data.decode()) @@ -769,7 +791,11 @@ def test_nats_many_subjects_insert_right(nats_cluster): while True: try: - instance.query("INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='right_insert1' VALUES {}".format(values)) + instance.query( + "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='right_insert1' VALUES {}".format( + values + ) + ) break except QueryRuntimeException as e: if "Local: Timed out." in str(e): @@ -803,7 +829,7 @@ def test_nats_many_inserts(nats_cluster): nats_row_delimiter = '\\n'; """ ) - while not check_table_is_ready(instance, 'test.nats_consume'): + while not check_table_is_ready(instance, "test.nats_consume"): logging.debug("Table test.nats_consume is not yet ready") time.sleep(0.5) @@ -816,9 +842,7 @@ def test_nats_many_inserts(nats_cluster): def insert(): while True: try: - instance.query( - "INSERT INTO test.nats_many VALUES {}".format(values) - ) + instance.query("INSERT INTO test.nats_many VALUES {}".format(values)) break except QueryRuntimeException as e: if "Local: Timed out." in str(e): @@ -896,7 +920,7 @@ def test_nats_overloaded_insert(nats_cluster): SELECT * FROM test.nats_consume; """ ) - while not check_table_is_ready(instance, 'test.nats_consume'): + while not check_table_is_ready(instance, "test.nats_consume"): logging.debug("Table test.nats_consume is not yet ready") time.sleep(0.5) @@ -963,11 +987,10 @@ def test_nats_virtual_column(nats_cluster): SELECT value, key, _subject FROM test.nats_virtuals; """ ) - while not check_table_is_ready(instance, 'test.nats_virtuals'): + while not check_table_is_ready(instance, "test.nats_virtuals"): logging.debug("Table test.nats_virtuals is not yet ready") time.sleep(0.5) - message_num = 10 i = 0 messages = [] @@ -975,7 +998,7 @@ def test_nats_virtual_column(nats_cluster): messages.append(json.dumps({"key": i, "value": i})) i += 1 - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'virtuals', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "virtuals", messages)) while True: result = instance.query("SELECT count() FROM test.view") @@ -983,7 +1006,6 @@ def test_nats_virtual_column(nats_cluster): if int(result) == message_num: break - result = instance.query( """ SELECT key, value, _subject @@ -1029,7 +1051,7 @@ def test_nats_virtual_column_with_materialized_view(nats_cluster): FROM test.nats_virtuals_mv; """ ) - while not check_table_is_ready(instance, 'test.nats_virtuals_mv'): + while not check_table_is_ready(instance, "test.nats_virtuals_mv"): logging.debug("Table test.nats_virtuals_mv is not yet ready") time.sleep(0.5) @@ -1040,7 +1062,7 @@ def test_nats_virtual_column_with_materialized_view(nats_cluster): messages.append(json.dumps({"key": i, "value": i})) i += 1 - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'virtuals_mv', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "virtuals_mv", messages)) while True: result = instance.query("SELECT count() FROM test.view") @@ -1048,10 +1070,7 @@ def test_nats_virtual_column_with_materialized_view(nats_cluster): if int(result) == message_num: break - - result = instance.query( - "SELECT key, value, subject FROM test.view ORDER BY key" - ) + result = instance.query("SELECT key, value, subject FROM test.view ORDER BY key") expected = """\ 0 0 virtuals_mv 1 1 virtuals_mv @@ -1107,20 +1126,25 @@ def test_nats_many_consumers_to_each_queue(nats_cluster): table_id ) ) - while not check_table_is_ready(instance, 'test.many_consumers_{}'.format(table_id)): - logging.debug("Table test.many_consumers_{} is not yet ready".format(table_id)) + while not check_table_is_ready( + instance, "test.many_consumers_{}".format(table_id) + ): + logging.debug( + "Table test.many_consumers_{} is not yet ready".format(table_id) + ) time.sleep(0.5) i = [0] messages_num = 1000 - def produce(): messages = [] for _ in range(messages_num): messages.append(json.dumps({"key": i[0], "value": i[0]})) i[0] += 1 - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'many_consumers', messages)) + asyncio.run( + nats_produce_messages(nats_cluster.nats_ip, "many_consumers", messages) + ) threads = [] threads_num = 20 @@ -1187,11 +1211,10 @@ def test_nats_restore_failed_connection_without_losses_on_write(nats_cluster): nats_row_delimiter = '\\n'; """ ) - while not check_table_is_ready(instance, 'test.consume'): + while not check_table_is_ready(instance, "test.consume"): logging.debug("Table test.consume is not yet ready") time.sleep(0.5) - messages_num = 100000 values = [] for i in range(messages_num): @@ -1269,7 +1292,7 @@ def test_nats_no_connection_at_startup_2(nats_cluster): SELECT * FROM test.cs; """ ) - while not check_table_is_ready(instance, 'test.cs'): + while not check_table_is_ready(instance, "test.cs"): logging.debug("Table test.cs is not yet ready") time.sleep(0.5) @@ -1282,7 +1305,7 @@ def test_nats_no_connection_at_startup_2(nats_cluster): messages = [] for i in range(messages_num): messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'cs', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "cs", messages)) while True: result = instance.query("SELECT count() FROM test.view") @@ -1314,11 +1337,10 @@ def test_nats_format_factory_settings(nats_cluster): date_time_input_format = 'best_effort'; """ ) - while not check_table_is_ready(instance, 'test.format_settings'): + while not check_table_is_ready(instance, "test.format_settings"): logging.debug("Table test.format_settings is not yet ready") time.sleep(0.5) - message = json.dumps( {"id": "format_settings_test", "date": "2021-01-19T14:42:33.1829214Z"} ) @@ -1326,7 +1348,9 @@ def test_nats_format_factory_settings(nats_cluster): """SELECT parseDateTimeBestEffort(CAST('2021-01-19T14:42:33.1829214Z', 'String'))""" ) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'format_settings', [message])) + asyncio.run( + nats_produce_messages(nats_cluster.nats_ip, "format_settings", [message]) + ) while True: result = instance.query("SELECT date FROM test.format_settings") @@ -1343,7 +1367,9 @@ def test_nats_format_factory_settings(nats_cluster): """ ) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'format_settings', [message])) + asyncio.run( + nats_produce_messages(nats_cluster.nats_ip, "format_settings", [message]) + ) while True: result = instance.query("SELECT date FROM test.view") if result == expected: @@ -1386,20 +1412,20 @@ def test_nats_drop_mv(nats_cluster): SELECT * FROM test.nats; """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) messages = [] for i in range(20): messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mv', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "mv", messages)) instance.query("DROP VIEW test.consumer") messages = [] for i in range(20, 40): messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mv', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "mv", messages)) instance.query( """ @@ -1410,7 +1436,7 @@ def test_nats_drop_mv(nats_cluster): messages = [] for i in range(40, 50): messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mv', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "mv", messages)) while True: result = instance.query("SELECT * FROM test.view ORDER BY key") @@ -1423,7 +1449,7 @@ def test_nats_drop_mv(nats_cluster): messages = [] for i in range(50, 60): messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'mv', messages)) + asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "mv", messages)) count = 0 while True: @@ -1440,11 +1466,15 @@ def test_nats_predefined_configuration(nats_cluster): CREATE TABLE test.nats (key UInt64, value UInt64) ENGINE = NATS(nats1) """ ) - while not check_table_is_ready(instance, 'test.nats'): + while not check_table_is_ready(instance, "test.nats"): logging.debug("Table test.nats is not yet ready") time.sleep(0.5) - asyncio.run(nats_produce_messages(nats_cluster.nats_ip, 'named', [json.dumps({"key": 1, "value": 2})])) + asyncio.run( + nats_produce_messages( + nats_cluster.nats_ip, "named", [json.dumps({"key": 1, "value": 2})] + ) + ) while True: result = instance.query( "SELECT * FROM test.nats ORDER BY key", ignore_error=True From d4afc36f8e96f9447968c661a5213f51b7a1575e Mon Sep 17 00:00:00 2001 From: tchepavel Date: Wed, 25 May 2022 22:02:47 +0300 Subject: [PATCH 023/627] Style+build fixes; make long tests time limited --- .../NATS/ReadBufferFromNATSConsumer.cpp | 9 ++--- .../NATS/ReadBufferFromNATSConsumer.h | 2 +- src/Storages/NATS/StorageNATS.cpp | 3 +- tests/integration/test_storage_nats/test.py | 33 ++++++++++--------- 4 files changed, 25 insertions(+), 22 deletions(-) diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index e8abe9a5ed4..55c3c18677d 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -94,10 +94,11 @@ void ReadBufferFromNATSConsumer::onMsg(natsConnection *, natsSubscription *, nat if (buffer->row_delimiter != '\0') message_received += buffer->row_delimiter; - if (!buffer->received.push({ - .message = std::move(message_received), - .subject = std::move(subject), - })) + MessageData data = { + .message = message_received, + .subject = subject, + }; + if (!buffer->received.push(std::move(data))) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); } diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h index 6f2ee071c81..dd38c08224d 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -48,7 +48,7 @@ public: private: bool nextImpl() override; - static void onMsg(natsConnection * nc, natsSubscription * sub, natsMsg * msg, void * closure); + static void onMsg(natsConnection * nc, natsSubscription * sub, natsMsg * msg, void * consumer); std::shared_ptr connection; std::vector subscriptions; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 74791fddd47..0f14044df7f 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -476,7 +476,8 @@ ProducerBufferPtr StorageNATS::createWriteBuffer(const std::string & subject) row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } -bool StorageNATS::isSubjectInSubscriptions(const std::string & subject) { +bool StorageNATS::isSubjectInSubscriptions(const std::string & subject) +{ auto subject_levels = parseList(subject, '.'); for (const auto & nats_subject : subjects) diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py index a29a72ea909..23b6817410c 100644 --- a/tests/integration/test_storage_nats/test.py +++ b/tests/integration/test_storage_nats/test.py @@ -827,6 +827,11 @@ def test_nats_many_inserts(nats_cluster): nats_subjects = 'many_inserts', nats_format = 'TSV', nats_row_delimiter = '\\n'; + CREATE TABLE test.view_many (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer_many TO test.view_many AS + SELECT * FROM test.nats_consume; """ ) while not check_table_is_ready(instance, "test.nats_consume"): @@ -858,23 +863,16 @@ def test_nats_many_inserts(nats_cluster): time.sleep(random.uniform(0, 1)) thread.start() - instance.query( - """ - CREATE TABLE test.view_many (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer_many TO test.view_many AS - SELECT * FROM test.nats_consume; - """ - ) - for thread in threads: thread.join() - while True: + time_limit_sec = 300 + deadline = time.monotonic() + time_limit_sec + + while time.monotonic() < deadline: result = instance.query("SELECT count() FROM test.view_many") print(result, messages_num * threads_num) - if int(result) == messages_num * threads_num: + if int(result) >= messages_num * threads_num: break time.sleep(1) @@ -889,7 +887,7 @@ def test_nats_many_inserts(nats_cluster): assert ( int(result) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result) + ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format(result) def test_nats_overloaded_insert(nats_cluster): @@ -952,10 +950,13 @@ def test_nats_overloaded_insert(nats_cluster): time.sleep(random.uniform(0, 1)) thread.start() - while True: + time_limit_sec = 300 + deadline = time.monotonic() + time_limit_sec + + while time.monotonic() < deadline: result = instance.query("SELECT count() FROM test.view_overload") time.sleep(1) - if int(result) == messages_num * threads_num: + if int(result) >= messages_num * threads_num: break instance.query( @@ -972,7 +973,7 @@ def test_nats_overloaded_insert(nats_cluster): assert ( int(result) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result) + ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format(result) def test_nats_virtual_column(nats_cluster): From 531bda5a65f73553c1e62760d2b8ae2b4211311c Mon Sep 17 00:00:00 2001 From: tchepavel Date: Wed, 25 May 2022 22:13:26 +0300 Subject: [PATCH 024/627] Spelling and more black python --- src/Storages/NATS/NATSSettings.h | 2 +- tests/integration/helpers/cluster.py | 3 +- .../integration/test_storage_nats/nats_pb2.py | 30 +++++++++++-------- tests/integration/test_storage_nats/test.py | 8 +++-- 4 files changed, 25 insertions(+), 18 deletions(-) diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h index 02864e490c2..6029aaea9f6 100644 --- a/src/Storages/NATS/NATSSettings.h +++ b/src/Storages/NATS/NATSSettings.h @@ -9,7 +9,7 @@ class ASTStorage; #define NATS_RELATED_SETTINGS(M) \ M(String, nats_url, "", "A host-port to connect to NATS server.", 0) \ - M(String, nats_subjects, "", "List of subject for NATS table to subscribe/publsh to.", 0) \ + M(String, nats_subjects, "", "List of subject for NATS table to subscribe/publish to.", 0) \ M(String, nats_format, "", "The message format.", 0) \ M(Char, nats_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ M(String, nats_schema, "", "Schema identifier (used by schema-based formats) for NATS engine", 0) \ diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9be416d7b05..315c78b6ed1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -221,7 +221,7 @@ def check_rabbitmq_is_available(rabbitmq_id): async def check_nats_is_available(nats_ip): - nc = await nats.connect('{}:4444'.format(nats_ip), user='click', password='house') + nc = await nats.connect("{}:4444".format(nats_ip), user="click", password="house") return nc.is_connected @@ -458,7 +458,6 @@ class ClickHouseCluster: self.nats_port = 4444 self.nats_docker_id = None - # available when with_nginx == True self.nginx_host = "nginx" self.nginx_ip = None diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py index b20c32b14a0..4330ff57950 100644 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ b/tests/integration/test_storage_nats/nats_pb2.py @@ -7,28 +7,32 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import message as _message from google.protobuf import reflection as _reflection from google.protobuf import symbol_database as _symbol_database + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n)clickhouse_path/format_schemas/nats.proto"+\n\rProtoKeyValue\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3' +) -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n)clickhouse_path/format_schemas/nats.proto\"+\n\rProtoKeyValue\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3') - - - -_PROTOKEYVALUE = DESCRIPTOR.message_types_by_name['ProtoKeyValue'] -ProtoKeyValue = _reflection.GeneratedProtocolMessageType('ProtoKeyValue', (_message.Message,), { - 'DESCRIPTOR' : _PROTOKEYVALUE, - '__module__' : 'clickhouse_path.format_schemas.nats_pb2' - # @@protoc_insertion_point(class_scope:ProtoKeyValue) - }) +_PROTOKEYVALUE = DESCRIPTOR.message_types_by_name["ProtoKeyValue"] +ProtoKeyValue = _reflection.GeneratedProtocolMessageType( + "ProtoKeyValue", + (_message.Message,), + { + "DESCRIPTOR": _PROTOKEYVALUE, + "__module__": "clickhouse_path.format_schemas.nats_pb2" + # @@protoc_insertion_point(class_scope:ProtoKeyValue) + }, +) _sym_db.RegisterMessage(ProtoKeyValue) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _PROTOKEYVALUE._serialized_start=45 - _PROTOKEYVALUE._serialized_end=88 + DESCRIPTOR._options = None + _PROTOKEYVALUE._serialized_start = 45 + _PROTOKEYVALUE._serialized_end = 88 # @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py index 23b6817410c..b3638570f88 100644 --- a/tests/integration/test_storage_nats/test.py +++ b/tests/integration/test_storage_nats/test.py @@ -887,7 +887,9 @@ def test_nats_many_inserts(nats_cluster): assert ( int(result) == messages_num * threads_num - ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format(result) + ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format( + result + ) def test_nats_overloaded_insert(nats_cluster): @@ -973,7 +975,9 @@ def test_nats_overloaded_insert(nats_cluster): assert ( int(result) == messages_num * threads_num - ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format(result) + ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format( + result + ) def test_nats_virtual_column(nats_cluster): From 75db37cb57aa905d3d89bae01739768c03fc62f7 Mon Sep 17 00:00:00 2001 From: tchepavel Date: Wed, 1 Jun 2022 21:30:08 +0300 Subject: [PATCH 025/627] Small engine overhaul --- src/Storages/NATS/NATSHandler.cpp | 12 +++- src/Storages/NATS/NATSSource.cpp | 1 + .../NATS/ReadBufferFromNATSConsumer.cpp | 5 ++ .../NATS/ReadBufferFromNATSConsumer.h | 4 ++ src/Storages/NATS/StorageNATS.cpp | 57 +++++-------------- src/Storages/NATS/StorageNATS.h | 6 +- .../NATS/WriteBufferToNATSProducer.cpp | 7 ++- src/Storages/NATS/WriteBufferToNATSProducer.h | 1 - 8 files changed, 41 insertions(+), 52 deletions(-) diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index 79d0adf0c6b..b5812bc3349 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -9,6 +9,9 @@ namespace DB /* The object of this class is shared between concurrent consumers (who share the same connection == share the same * event loop and handler). */ + +static const auto MAX_THREAD_WORK_DURATION_MS = 60000; + NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : loop(loop_), log(log_), @@ -23,6 +26,8 @@ NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : natsLibuv_Read, natsLibuv_Write, natsLibuv_Detach); + natsOptions_SetIOBufSize(opts, INT_MAX); + natsOptions_SetSendAsap(opts, true); } void NATSHandler::startLoop() @@ -32,10 +37,15 @@ void NATSHandler::startLoop() LOG_DEBUG(log, "Background loop started"); loop_running.store(true); + auto start_time = std::chrono::steady_clock::now(); + auto end_time = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); - while (loop_state.load() == Loop::RUN) + while (loop_state.load() == Loop::RUN && duration.count() < MAX_THREAD_WORK_DURATION_MS) { uv_run(loop, UV_RUN_NOWAIT); + end_time = std::chrono::steady_clock::now(); + duration = std::chrono::duration_cast(end_time - start_time); } LOG_DEBUG(log, "Background loop ended"); diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index 3fdba292923..5479856411e 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -62,6 +62,7 @@ NATSSource::~NATSSource() if (!buffer) return; + buffer->allowNext(); storage.pushReadBuffer(buffer); } diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp index 55c3c18677d..fa6e60ac213 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp @@ -19,6 +19,7 @@ namespace ErrorCodes ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( std::shared_ptr connection_, + StorageNATS & storage_, std::vector & subjects_, const String & subscribe_queue_name, Poco::Logger * log_, @@ -27,6 +28,7 @@ ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , connection(connection_) + , storage(storage_) , subjects(subjects_) , log(log_) , row_delimiter(row_delimiter_) @@ -49,6 +51,7 @@ void ReadBufferFromNATSConsumer::subscribe() if (status == NATS_OK) { LOG_DEBUG(log, "Subscribed to subject {}", subject); + natsSubscription_SetPendingLimits(ns, -1, -1); subscriptions.emplace_back(ns, &natsSubscription_Destroy); } else @@ -100,6 +103,8 @@ void ReadBufferFromNATSConsumer::onMsg(natsConnection *, natsSubscription *, nat }; if (!buffer->received.push(std::move(data))) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); + + buffer->storage.startStreaming(); } natsMsg_Destroy(msg); diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/ReadBufferFromNATSConsumer.h index dd38c08224d..306c0aff3bf 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/ReadBufferFromNATSConsumer.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace Poco { @@ -20,6 +21,7 @@ class ReadBufferFromNATSConsumer : public ReadBuffer public: ReadBufferFromNATSConsumer( std::shared_ptr connection_, + StorageNATS & storage_, std::vector & subjects_, const String & subscribe_queue_name, Poco::Logger * log_, @@ -41,6 +43,7 @@ public: bool isConsumerStopped() { return stopped; } bool queueEmpty() { return received.empty(); } + size_t queueSize() { return received.size(); } void allowNext() { allowed = true; } // Allow to read next message. auto getSubject() const { return current.subject; } @@ -51,6 +54,7 @@ private: static void onMsg(natsConnection * nc, natsSubscription * sub, natsMsg * msg, void * consumer); std::shared_ptr connection; + StorageNATS & storage; std::vector subscriptions; std::vector subjects; Poco::Logger * log; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 0f14044df7f..308ded9a6c2 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -29,7 +29,6 @@ namespace DB static const uint32_t QUEUE_SIZE = 100000; static const auto RESCHEDULE_MS = 500; -static const auto BACKOFF_TRESHOLD = 8000; static const auto MAX_THREAD_WORK_DURATION_MS = 60000; namespace ErrorCodes @@ -59,7 +58,6 @@ StorageNATS::StorageNATS( , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) , semaphore(0, num_consumers) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) - , milliseconds_to_wait(RESCHEDULE_MS) , is_attach(is_attach_) { auto nats_username = getContext()->getMacros()->expand(nats_settings->nats_username); @@ -162,6 +160,7 @@ ContextMutablePtr StorageNATS::addSettings(ContextPtr local_context) const void StorageNATS::loopingFunc() { connection->getHandler().startLoop(); + looping_task->activateAndSchedule(); } @@ -244,22 +243,14 @@ bool StorageNATS::initBuffers() /* Need to deactivate this way because otherwise might get a deadlock when first deactivate streaming task in shutdown and then * inside streaming task try to deactivate any other task */ -void StorageNATS::deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop) +void StorageNATS::deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool stop_loop) { if (stop_loop) stopLoop(); std::unique_lock lock(task_mutex, std::defer_lock); - if (lock.try_lock()) - { - task->deactivate(); - lock.unlock(); - } - else if (wait) /// Wait only if deactivating from shutdown - { - lock.lock(); - task->deactivate(); - } + lock.lock(); + task->deactivate(); } @@ -299,8 +290,6 @@ Pipe StorageNATS::read( if (!connection->isConnected()) { - if (connection->getHandler().loopRunning()) - deactivateTask(looping_task, false, true); if (!connection->reconnect()) throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "No connection to {}", connection->connectionInfoForLog()); } @@ -385,8 +374,6 @@ void StorageNATS::startup() if (!connection->isConnected() || !initBuffers()) connection_task->activateAndSchedule(); - - streaming_task->activateAndSchedule(); } @@ -395,12 +382,12 @@ void StorageNATS::shutdown() shutdown_called = true; /// In case it has not yet been able to setup connection; - deactivateTask(connection_task, true, false); + deactivateTask(connection_task, false); /// The order of deactivating tasks is important: wait for streamingToViews() func to finish and /// then wait for background event loop to finish. - deactivateTask(streaming_task, true, false); - deactivateTask(looping_task, true, true); + deactivateTask(streaming_task, false); + deactivateTask(looping_task, true); /// Just a paranoid try catch, it is not actually needed. try @@ -411,10 +398,6 @@ void StorageNATS::shutdown() buffer->unsubscribe(); } - /// It is important to close connection here - before removing consumer buffers, because - /// it will finish and clean callbacks, which might use those buffers data. - if (connection->getHandler().loopRunning()) - stopLoop(); connection->disconnect(); for (size_t i = 0; i < num_created_consumers; ++i) @@ -463,7 +446,7 @@ ConsumerBufferPtr StorageNATS::popReadBuffer(std::chrono::milliseconds timeout) ConsumerBufferPtr StorageNATS::createReadBuffer() { return std::make_shared( - connection, subjects, + connection, *this, subjects, nats_settings->nats_queue_group.changed ? nats_settings->nats_queue_group.value : getStorageID().getFullTableName(), log, row_delimiter, queue_size, shutdown_called); } @@ -548,6 +531,7 @@ bool StorageNATS::checkDependencies(const StorageID & table_id) void StorageNATS::streamingToViewsFunc() { + bool do_reschedule = true; try { auto table_id = getStorageID(); @@ -573,21 +557,14 @@ void StorageNATS::streamingToViewsFunc() if (streamToViews()) { /// Reschedule with backoff. - if (milliseconds_to_wait < BACKOFF_TRESHOLD) - milliseconds_to_wait *= 2; - stopLoopIfNoReaders(); + do_reschedule = false; break; } - else - { - milliseconds_to_wait = RESCHEDULE_MS; - } auto end_time = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(end_time - start_time); if (duration.count() > MAX_THREAD_WORK_DURATION_MS) { - stopLoopIfNoReaders(); LOG_TRACE(log, "Reschedule streaming. Thread work duration limit exceeded."); break; } @@ -601,13 +578,8 @@ void StorageNATS::streamingToViewsFunc() mv_attached.store(false); - /// If there is no running select, stop the loop which was - /// activated by previous select. - if (connection->getHandler().loopRunning()) - stopLoopIfNoReaders(); - - if (!shutdown_called) - streaming_task->scheduleAfter(milliseconds_to_wait); + if (!shutdown_called && do_reschedule) + streaming_task->scheduleAfter(RESCHEDULE_MS); } @@ -640,6 +612,7 @@ bool StorageNATS::streamToViews() for (size_t i = 0; i < num_created_consumers; ++i) { + LOG_DEBUG(log, "Current queue size: {}", buffers[0]->queueSize()); auto source = std::make_shared(*this, storage_snapshot, nats_context, column_names, block_size); sources.emplace_back(source); pipes.emplace_back(source); @@ -666,10 +639,6 @@ bool StorageNATS::streamToViews() executor.execute(); } - /* Note: sending ack() with loop running in another thread will lead to a lot of data races inside the library, but only in case - * error occurs or connection is lost while ack is being sent - */ - deactivateTask(looping_task, false, true); size_t queue_empty = 0; if (!connection->isConnected()) diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 8dff8ecb533..b1030cbb788 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -60,6 +60,8 @@ public: void incrementReader(); void decrementReader(); + void startStreaming() { if (!mv_attached) { streaming_task->activateAndSchedule(); } } + private: ContextMutablePtr nats_context; std::unique_ptr nats_settings; @@ -90,8 +92,6 @@ private: BackgroundSchedulePool::TaskHolder looping_task; BackgroundSchedulePool::TaskHolder connection_task; - uint64_t milliseconds_to_wait; - /// True if consumers have subscribed to all subjects std::atomic consumers_ready{false}; /// Needed for tell MV or producer background tasks @@ -135,7 +135,7 @@ private: ContextMutablePtr addSettings(ContextPtr context) const; size_t getMaxBlockSize() const; - void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); + void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool stop_loop); bool streamToViews(); bool checkDependencies(const StorageID & table_id); diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp index a5ac74018f2..af76247d903 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ b/src/Storages/NATS/WriteBufferToNATSProducer.cpp @@ -15,6 +15,7 @@ namespace DB { static const auto BATCH = 1000; +static const auto MAX_BUFFERED = 131072; namespace ErrorCodes { @@ -54,7 +55,6 @@ WriteBufferToNATSProducer::WriteBufferToNATSProducer( WriteBufferToNATSProducer::~WriteBufferToNATSProducer() { writing_task->deactivate(); - connection.disconnect(); assert(rows == 0); } @@ -79,7 +79,6 @@ void WriteBufferToNATSProducer::countRow() reinitializeChunks(); - ++payload_counter; if (!payloads.push(payload)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); } @@ -97,12 +96,14 @@ void WriteBufferToNATSProducer::publish() void WriteBufferToNATSProducer::publishThreadFunc(void * arg) { - String payload; WriteBufferToNATSProducer * buffer = static_cast(arg); + String payload; natsStatus status; while (!buffer->payloads.empty()) { + if (natsConnection_Buffered(buffer->connection.getConnection()) > MAX_BUFFERED) + break; bool pop_result = buffer->payloads.pop(payload); if (!pop_result) diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h index 0cffd35d414..484d80598db 100644 --- a/src/Storages/NATS/WriteBufferToNATSProducer.h +++ b/src/Storages/NATS/WriteBufferToNATSProducer.h @@ -69,7 +69,6 @@ private: * true: means payloads.queue will not grow anymore */ std::atomic wait_payloads = false; - UInt64 payload_counter = 0; Poco::Logger * log; const std::optional delim; From 6353b58b6a6bd9c611b367defdc6bfa0d1eb8d53 Mon Sep 17 00:00:00 2001 From: Chebarykov Pavel Date: Thu, 2 Jun 2022 09:09:24 +0000 Subject: [PATCH 026/627] Fix startup test --- tests/integration/test_storage_nats/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py index b3638570f88..a952f4b78a6 100644 --- a/tests/integration/test_storage_nats/test.py +++ b/tests/integration/test_storage_nats/test.py @@ -1297,14 +1297,14 @@ def test_nats_no_connection_at_startup_2(nats_cluster): SELECT * FROM test.cs; """ ) - while not check_table_is_ready(instance, "test.cs"): - logging.debug("Table test.cs is not yet ready") - time.sleep(0.5) instance.query("DETACH TABLE test.cs") nats_cluster.pause_container("nats1") instance.query("ATTACH TABLE test.cs") nats_cluster.unpause_container("nats1") + while not check_table_is_ready(instance, "test.cs"): + logging.debug("Table test.cs is not yet ready") + time.sleep(0.5) messages_num = 1000 messages = [] @@ -1312,7 +1312,7 @@ def test_nats_no_connection_at_startup_2(nats_cluster): messages.append(json.dumps({"key": i, "value": i})) asyncio.run(nats_produce_messages(nats_cluster.nats_ip, "cs", messages)) - while True: + for _ in range(20): result = instance.query("SELECT count() FROM test.view") time.sleep(1) if int(result) == messages_num: From a5afb73c529ef85034294e2591f1539c4ed12f89 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 5 Jun 2022 17:02:58 +0800 Subject: [PATCH 027/627] Remove duplicated peak mem log Also log "Processed in {} sec" a little bit earlier so that it's also sent to client and have the log entry related to the query_id. --- src/Common/MemoryTracker.cpp | 5 +++-- src/Common/MemoryTracker.h | 4 +++- src/Server/TCPHandler.cpp | 13 ++++++++++--- 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 0e7803aaa71..153ba1362b3 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -68,7 +68,7 @@ MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : MemoryTracker::~MemoryTracker() { - if ((level == VariableContext::Process || level == VariableContext::User) && peak) + if ((level == VariableContext::Process || level == VariableContext::User) && peak && log_peak_memory_usage_in_destructor) { try { @@ -82,8 +82,9 @@ MemoryTracker::~MemoryTracker() } -void MemoryTracker::logPeakMemoryUsage() const +void MemoryTracker::logPeakMemoryUsage() { + log_peak_memory_usage_in_destructor = false; const auto * description = description_ptr.load(std::memory_order_relaxed); LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak)); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 73af2ab8857..258b9b1fb6e 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -75,6 +75,8 @@ private: std::atomic overcommit_tracker = nullptr; + bool log_peak_memory_usage_in_destructor = true; + bool updatePeak(Int64 will_be, bool log_memory_usage); void logMemoryUsage(Int64 current) const; @@ -206,7 +208,7 @@ public: void set(Int64 to); /// Prints info about peak memory consumption into log. - void logPeakMemoryUsage() const; + void logPeakMemoryUsage(); }; extern MemoryTracker total_memory_tracker; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f056842926d..9874de59f83 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -204,6 +204,7 @@ void TCPHandler::runImpl() */ std::unique_ptr exception; bool network_error = false; + bool query_duration_already_logged = false; try { @@ -381,6 +382,10 @@ void TCPHandler::runImpl() /// Do it before sending end of stream, to have a chance to show log message in client. query_scope->logPeakMemoryUsage(); + watch.stop(); + LOG_DEBUG(log, "Processed in {} sec.", watch.elapsedSeconds()); + query_duration_already_logged = true; + if (state.is_connection_closed) break; @@ -507,9 +512,11 @@ void TCPHandler::runImpl() */ } - watch.stop(); - - LOG_DEBUG(log, "Processed in {} sec.", watch.elapsedSeconds()); + if (!query_duration_already_logged) + { + watch.stop(); + LOG_DEBUG(log, "Processed in {} sec.", watch.elapsedSeconds()); + } /// It is important to destroy query context here. We do not want it to live arbitrarily longer than the query. query_context.reset(); From ea5e37abefb6c2e4a78f8c7f81ab254545ad9b41 Mon Sep 17 00:00:00 2001 From: Wangyang Guo Date: Mon, 16 May 2022 15:01:22 +0800 Subject: [PATCH 028/627] Align branches within a 32B boundary for AMD64 arch to make benchmark more stable --- CMakeLists.txt | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index a6a09afc489..5f8ad137795 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -319,6 +319,22 @@ if (COMPILER_GCC OR COMPILER_CLANG) set(COMPILER_FLAGS "${COMPILER_FLAGS} -falign-functions=32") endif () +if (ARCH_AMD64) + # align branches within a 32-Byte boundary to avoid the potential performance loss when code layout change, + # which makes benchmark results more stable. + set(BRANCHES_WITHIN_32B_BOUNDARIES "-mbranches-within-32B-boundaries") + if (COMPILER_GCC) + # gcc is in assembler, need to add "-Wa," prefix + set(BRANCHES_WITHIN_32B_BOUNDARIES "-Wa,${BRANCHES_WITHIN_32B_BOUNDARIES}") + endif() + + include(CheckCXXCompilerFlag) + check_cxx_compiler_flag("${BRANCHES_WITHIN_32B_BOUNDARIES}" HAS_BRANCHES_WITHIN_32B_BOUNDARIES) + if (HAS_BRANCHES_WITHIN_32B_BOUNDARIES) + set(COMPILER_FLAGS "${COMPILER_FLAGS} ${BRANCHES_WITHIN_32B_BOUNDARIES}") + endif() +endif() + if (COMPILER_GCC) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fcoroutines") endif () From f06b19bdbfc02e8dc85f564146208bbd29f38771 Mon Sep 17 00:00:00 2001 From: lingpeng0314 Date: Fri, 10 Jun 2022 15:49:46 +0800 Subject: [PATCH 029/627] Add keeper monitoring data --- src/Common/CurrentMetrics.cpp | 2 + src/Common/ProfileEvents.cpp | 15 +++- src/Coordination/KeeperConnectionStats.cpp | 13 +++ src/Coordination/KeeperDispatcher.cpp | 19 +++++ src/Coordination/KeeperStateMachine.cpp | 28 +++++++ src/Interpreters/AsynchronousMetrics.cpp | 96 ++++++++++++++++++++++ 6 files changed, 172 insertions(+), 1 deletion(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index fa9c60c6f79..f479e4cc140 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -93,6 +93,8 @@ M(CacheFileSegments, "Number of existing cache file segments") \ M(CacheDetachedFileSegments, "Number of existing detached cache file segments") \ M(S3Requests, "S3 requests") \ + M(KeeperAliveConnections, "Number of alive connections") \ + M(KeeperOutstandingRequets, "Number of outstanding requests") \ namespace CurrentMetrics { diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index b8e552f6023..a2886330ce9 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -343,7 +343,20 @@ \ M(ScalarSubqueriesGlobalCacheHit, "Number of times a read from a scalar subquery was done using the global cache") \ M(ScalarSubqueriesLocalCacheHit, "Number of times a read from a scalar subquery was done using the local cache") \ - M(ScalarSubqueriesCacheMiss, "Number of times a read from a scalar subquery was not cached and had to be calculated completely") + M(ScalarSubqueriesCacheMiss, "Number of times a read from a scalar subquery was not cached and had to be calculated completely") \ + M(KeeperPacketsSent, "Packets sent by keeper server") \ + M(KeeperPacketsReceived, "Packets received by keeper server") \ + M(KeeperRequestTotal, "Total requests number on keeper server") \ + M(KeeperLatency, "Keeper latency") \ + M(KeeperCommits, "Number of commit count") \ + M(KeeperCommitsFailed, "Number of commit failed count") \ + M(KeeperSnapshotCreations, "Number of snapshot creations count")\ + M(KeeperSnapshotCreationsFailed, "Number of snapshot creations failed count")\ + M(KeeperSnapshotApplys, "Number of snapshot applying")\ + M(KeeperSnapshotApplysFailed, "Number of snapshot applying failed")\ + M(KeeperReadSnapshot, "Number of snapshot read(serialization)")\ + M(KeeperSaveSnapshot, "Number of snapshot save") + namespace ProfileEvents { diff --git a/src/Coordination/KeeperConnectionStats.cpp b/src/Coordination/KeeperConnectionStats.cpp index b4edfe45159..71f24f1ed02 100644 --- a/src/Coordination/KeeperConnectionStats.cpp +++ b/src/Coordination/KeeperConnectionStats.cpp @@ -1,5 +1,14 @@ #include #include +#include + +namespace ProfileEvents +{ + extern const Event KeeperPacketsSent; + extern const Event KeeperPacketsReceived; + extern const Event KeeperRequestTotal; + extern const Event KeeperLatency; +} namespace DB { @@ -40,18 +49,22 @@ uint64_t KeeperConnectionStats::getPacketsSent() const void KeeperConnectionStats::incrementPacketsReceived() { packets_received.fetch_add(1, std::memory_order_relaxed); + ProfileEvents::increment(ProfileEvents::KeeperPacketsReceived, 1); } void KeeperConnectionStats::incrementPacketsSent() { packets_sent.fetch_add(1, std::memory_order_relaxed); + ProfileEvents::increment(ProfileEvents::KeeperPacketsSent, 1); } void KeeperConnectionStats::updateLatency(uint64_t latency_ms) { last_latency.store(latency_ms, std::memory_order_relaxed); total_latency.fetch_add(latency_ms, std::memory_order_relaxed); + ProfileEvents::increment(ProfileEvents::KeeperLatency, latency_ms); count.fetch_add(1, std::memory_order_relaxed); + ProfileEvents::increment(ProfileEvents::KeeperRequestTotal, 1); uint64_t prev_val = min_latency.load(std::memory_order_relaxed); while (prev_val > latency_ms && !min_latency.compare_exchange_weak(prev_val, latency_ms, std::memory_order_relaxed)) {} diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9ad5fe9e8ed..4798e0995a2 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -7,6 +7,13 @@ #include #include #include +#include + +namespace CurrentMetrics +{ + extern const Metric KeeperAliveConnections; + extern const Metric KeeperOutstandingRequets; +} namespace fs = std::filesystem; @@ -57,6 +64,7 @@ void KeeperDispatcher::requestThread() { if (requests_queue->tryPop(request, max_wait)) { + CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequets); if (shutdown_called) break; @@ -78,6 +86,7 @@ void KeeperDispatcher::requestThread() /// Trying to get batch requests as fast as possible if (requests_queue->tryPop(request, 1)) { + CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequets); /// Don't append read request into batch, we have to process them separately if (!coordination_settings->quorum_reads && request.request->isReadRequest()) { @@ -225,6 +234,7 @@ void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKe if (response->xid != Coordination::WATCH_XID && response->getOpNum() == Coordination::OpNum::Close) { session_to_response_callback.erase(session_response_callback); + CurrentMetrics::sub(CurrentMetrics::KeeperAliveConnections); } } } @@ -259,6 +269,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ { throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); } + CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequets); return true; } @@ -350,6 +361,7 @@ void KeeperDispatcher::shutdown() /// Set session expired for all pending requests while (requests_queue && requests_queue->tryPop(request_for_session)) { + CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequets); auto response = request_for_session.request->makeResponse(); response->error = Coordination::Error::ZSESSIONEXPIRED; setResponse(request_for_session.session_id, response); @@ -358,6 +370,7 @@ void KeeperDispatcher::shutdown() /// Clear all registered sessions std::lock_guard lock(session_to_response_callback_mutex); session_to_response_callback.clear(); + CurrentMetrics::set(CurrentMetrics::KeeperAliveConnections, 0); } catch (...) { @@ -382,6 +395,7 @@ void KeeperDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCall std::lock_guard lock(session_to_response_callback_mutex); if (!session_to_response_callback.try_emplace(session_id, callback).second) throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session with id {} already registered in dispatcher", session_id); + CurrentMetrics::add(CurrentMetrics::KeeperAliveConnections); } void KeeperDispatcher::sessionCleanerTask() @@ -414,6 +428,7 @@ void KeeperDispatcher::sessionCleanerTask() std::lock_guard lock(push_request_mutex); if (!requests_queue->push(std::move(request_info))) LOG_INFO(log, "Cannot push close request to queue while cleaning outdated sessions"); + CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequets); } /// Remove session from registered sessions @@ -437,7 +452,10 @@ void KeeperDispatcher::finishSession(int64_t session_id) std::lock_guard lock(session_to_response_callback_mutex); auto session_it = session_to_response_callback.find(session_id); if (session_it != session_to_response_callback.end()) + { session_to_response_callback.erase(session_it); + CurrentMetrics::sub(CurrentMetrics::KeeperAliveConnections); + } } void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error) @@ -520,6 +538,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) std::lock_guard lock(push_request_mutex); if (!requests_queue->tryPush(std::move(request_info), session_timeout_ms)) throw Exception("Cannot push session id request to queue within session timeout", ErrorCodes::TIMEOUT_EXCEEDED); + CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequets); } if (future.wait_for(std::chrono::milliseconds(session_timeout_ms)) != std::future_status::ready) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 368b23f34d2..b0f790e5c1c 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -8,8 +8,21 @@ #include #include "Common/ZooKeeper/ZooKeeperCommon.h" #include +#include #include "Coordination/KeeperStorage.h" +namespace ProfileEvents +{ + extern const Event KeeperCommits; + extern const Event KeeperCommitsFailed; + extern const Event KeeperSnapshotCreations; + extern const Event KeeperSnapshotCreationsFailed; + extern const Event KeeperSnapshotApplys; + extern const Event KeeperSnapshotApplysFailed; + extern const Event KeeperReadSnapshot; + extern const Event KeeperSaveSnapshot; +} + namespace DB { @@ -219,7 +232,10 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); response->session_id = session_id; if (!responses_queue.push(response_for_session)) + { + ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push response with session id {} into responses queue", session_id); + } } } else @@ -229,10 +245,13 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n request_for_session.request, request_for_session.session_id, request_for_session.zxid); for (auto & response_for_session : responses_for_sessions) if (!responses_queue.push(response_for_session)) + { + ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); throw Exception( ErrorCodes::SYSTEM_ERROR, "Could not push response with session id {} into responses queue", response_for_session.session_id); + } if (digest_enabled && request_for_session.digest) { @@ -240,6 +259,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n } } + ProfileEvents::increment(ProfileEvents::KeeperCommits); last_committed_idx = log_idx; return nullptr; } @@ -251,11 +271,14 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) { /// save snapshot into memory std::lock_guard lock(snapshots_lock); if (s.get_last_log_idx() != latest_snapshot_meta->get_last_log_idx()) + { + ProfileEvents::increment(ProfileEvents::KeeperSnapshotApplysFailed); throw Exception( ErrorCodes::LOGICAL_ERROR, "Required to apply snapshot with last log index {}, but our last log index is {}", s.get_last_log_idx(), latest_snapshot_meta->get_last_log_idx()); + } latest_snapshot_ptr = latest_snapshot_buf; } @@ -268,6 +291,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) cluster_config = snapshot_deserialization_result.cluster_config; } + ProfileEvents::increment(ProfileEvents::KeeperSnapshotApplys); last_committed_idx = s.get_last_log_idx(); return true; } @@ -332,6 +356,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res } latest_snapshot_path = path; latest_snapshot_meta = snapshot->snapshot_meta; + ProfileEvents::increment(ProfileEvents::KeeperSnapshotCreations); LOG_DEBUG(log, "Created persistent snapshot {} with path {}", latest_snapshot_meta->get_last_log_idx(), path); } @@ -347,6 +372,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res } catch (...) { + ProfileEvents::increment(ProfileEvents::KeeperSnapshotCreationsFailed); LOG_TRACE(log, "Exception happened during snapshot"); tryLogCurrentException(log); ret = false; @@ -380,6 +406,7 @@ void KeeperStateMachine::save_logical_snp_obj( latest_snapshot_meta = cloned_meta; LOG_DEBUG(log, "Saved snapshot {} to path {}", s.get_last_log_idx(), result_path); obj_id++; + ProfileEvents::increment(ProfileEvents::KeeperSaveSnapshot); } catch (...) { @@ -441,6 +468,7 @@ int KeeperStateMachine::read_logical_snp_obj( return -1; } is_last_obj = true; + ProfileEvents::increment(ProfileEvents::KeeperReadSnapshot); return 1; } diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 4ac5acfd60f..505f956fd5a 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include #include @@ -11,6 +13,8 @@ #include #include #include +#include +#include #include #include #include @@ -1464,6 +1468,98 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti new_values[name] = server_metric.current_threads; } } +#if USE_NURAFT + { + try { + auto keeper_dispatcher = getContext()->getKeeperDispatcher(); + size_t is_leader = 0; + size_t is_follower = 0; + size_t is_observer = 0; + size_t is_standalone = 0; + size_t znode_count = 0; + size_t watch_count =0; + size_t ephemerals_count = 0; + size_t approximate_data_size =0; + size_t key_arena_size = 0; + size_t latest_snapshot_size =0; + size_t open_file_descriptor_count =0; + size_t max_file_descriptor_count =0; + size_t followers =0; + size_t synced_followers = 0; + size_t zxid = 0; + size_t session_with_watches = 0; + size_t paths_watched = 0; + size_t snapshot_dir_size = 0; + size_t log_dir_size = 0; + + if (keeper_dispatcher && keeper_dispatcher->isServerActive()) + { + auto keeper_4LW_info = keeper_dispatcher -> getKeeper4LWInfo(); + if (keeper_4LW_info.is_standalone) + is_standalone = 1; + if (keeper_4LW_info.is_leader) + is_leader = 1; + if (keeper_4LW_info.is_observer) + is_observer = 1; + if (keeper_4LW_info.is_follower) + is_follower = 1; + + zxid = keeper_4LW_info.last_zxid; + const auto & state_machine = keeper_dispatcher->getStateMachine(); + znode_count = state_machine.getNodesCount(); + watch_count = state_machine.getTotalWatchesCount(); + ephemerals_count = state_machine.getTotalEphemeralNodesCount(); + approximate_data_size =state_machine.getApproximateDataSize(); + key_arena_size=state_machine.getKeyArenaSize(); + latest_snapshot_size=state_machine.getLatestSnapshotBufSize(); + session_with_watches = state_machine.getSessionsWithWatchesCount(); + paths_watched = state_machine.getWatchedPathsCount(); + snapshot_dir_size = keeper_dispatcher->getSnapDirSize(); + log_dir_size = keeper_dispatcher->getLogDirSize(); + + #if defined(__linux__) || defined(__APPLE__) + open_file_descriptor_count = getCurrentProcessFDCount(); + max_file_descriptor_count = getMaxFileDescriptorCount(); + #endif + + if (keeper_4LW_info.is_leader) + { + followers = keeper_4LW_info.follower_count; + synced_followers = keeper_4LW_info.synced_follower_count; + } + } + + new_values["KeeperIsLeader"] = is_leader; + new_values["KeeperIsFollower"] = is_follower; + new_values["KeeperIsObserver"] = is_observer; + new_values["KeeperIsStandalone"] = is_standalone; + + new_values["KeeperZnodeCount"] = znode_count; + new_values["KeeperWatchCount"] = watch_count; + new_values["KeeperEphemeralsCount"] = ephemerals_count; + + new_values["KeeperApproximateDataSize"] = approximate_data_size; + new_values["KeeperKeyArenaSize"] = key_arena_size; + + new_values["KeeperLatestSnapshotSize"] = latest_snapshot_size; + + new_values["KeeperOpenFileDescriptorCount"] = open_file_descriptor_count; + new_values["KeeperMaxFileDescriptorCount"] = max_file_descriptor_count; + + new_values["KeeperFollowers"] = followers; + new_values["KeeperSyncedFollowers"] = synced_followers; + new_values["KeeperZxid"] = zxid; + new_values["KeeperSessionWithWatches"] = session_with_watches; + new_values["KeeperPathsWatched"] = paths_watched; + new_values["KeeperSnapshotDirSize"] = snapshot_dir_size; + new_values["KeeperLogDirSize"] = log_dir_size; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +#endif #if USE_JEMALLOC // 'epoch' is a special mallctl -- it updates the statistics. Without it, all From efec6f9ac58d7c47dc0ba1fbbf1eb6784ef20e66 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 20 Jun 2022 12:19:50 +0800 Subject: [PATCH 030/627] support alter on storagehive --- src/Storages/Hive/StorageHive.cpp | 23 +++++++++ src/Storages/Hive/StorageHive.h | 2 + tests/integration/test_hive_query/test.py | 61 +++++++++++++++++++++++ 3 files changed, 86 insertions(+) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 6046dd58677..1498e9080c9 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -889,6 +890,28 @@ std::optional StorageHive::totalRowsByPartitionPredicate(const SelectQue return totalRowsImpl(context_->getSettingsRef(), query_info, context_, PruneLevel::Partition); } +void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /*local_context*/) const +{ + for (const auto & command : commands) + { + if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN + && command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN + && command.type != AlterCommand::Type::COMMENT_TABLE) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", command.type, getName()); + } +} + +void StorageHive::alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder & /*alter_lock_holder*/) +{ + auto table_id = getStorageID(); + checkAlterIsPossible(params, local_context); + auto metadata_snapshot = getInMemoryMetadataPtr(); + StorageInMemoryMetadata new_metadata = *metadata_snapshot; + params.apply(new_metadata, local_context); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); + setInMemoryMetadata(new_metadata); +} + std::optional StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const { diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index d92d2dbd745..af18135639f 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -67,6 +67,8 @@ public: std::optional totalRows(const Settings & settings) const override; std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const override; + void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override; + void alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder & alter_lock_holder) override; private: using FileFormat = IHiveFile::FileFormat; diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index fd4d91d6f78..72919cfd006 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -400,3 +400,64 @@ def test_cache_dir_use(started_cluster): ["bash", "-c", "ls /tmp/clickhouse_local_cache1 | wc -l"] ) assert result0 != "0" and result1 != "0" + + +def test_table_alter_add(started_cluster): + node = started_cluster.instances["h0_0_0"] + result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") + result = node.query( + """ +CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) + """ + ) + result = node.query( + """ +ALTER TABLE default.demo_parquet_1 ADD COLUMN id Nullable(String) FIRST + """ + ) + result = node.query( + """DESC default.demo_parquet_1 FORMAT TSV""" + ) + + expected_result = "id\tNullable(String)\t\t\t\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)" + assert result.strip() == expected_result + + +def test_table_alter_drop(started_cluster): + node = started_cluster.instances["h0_0_0"] + result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") + result = node.query( + """ +CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) + """ + ) + result = node.query( + """ +ALTER TABLE default.demo_parquet_1 DROP COLUMN id + """ + ) + + result = node.query( + """DESC default.demo_parquet_1 FORMAT TSV""" + ) + expected_result = """score\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" + assert result.strip() == expected_result + + +def test_table_alter_comment(started_cluster): + node = started_cluster.instances["h0_0_0"] + result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") + result = node.query( + """ +CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) + """ + ) + + result = node.query( + """ALTER TABLE default.demo_parquet_1 COMMENT COLUMN id 'Text comment'""" + ) + result = node.query( + """DESC default.demo_parquet_1 FORMAT TSV""" + ) + expected_result = """id\tNullable(String)\t\t\tText comment\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" + assert result.strip() == expected_result From b78adb4492a9cb657643f9c49cd51cb0f02fbd3c Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 20 Jun 2022 15:07:08 +0800 Subject: [PATCH 031/627] fixed code style --- tests/integration/test_hive_query/test.py | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 72919cfd006..a2bf3dde284 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -415,9 +415,7 @@ CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`score` Nullable(Int32), `day ALTER TABLE default.demo_parquet_1 ADD COLUMN id Nullable(String) FIRST """ ) - result = node.query( - """DESC default.demo_parquet_1 FORMAT TSV""" - ) + result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") expected_result = "id\tNullable(String)\t\t\t\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)" assert result.strip() == expected_result @@ -437,9 +435,7 @@ ALTER TABLE default.demo_parquet_1 DROP COLUMN id """ ) - result = node.query( - """DESC default.demo_parquet_1 FORMAT TSV""" - ) + result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") expected_result = """score\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" assert result.strip() == expected_result @@ -456,8 +452,6 @@ CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score result = node.query( """ALTER TABLE default.demo_parquet_1 COMMENT COLUMN id 'Text comment'""" ) - result = node.query( - """DESC default.demo_parquet_1 FORMAT TSV""" - ) + result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") expected_result = """id\tNullable(String)\t\t\tText comment\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" assert result.strip() == expected_result From 1d463718cabd63eba74ae4c51368bb73ddb45bf1 Mon Sep 17 00:00:00 2001 From: lingpeng0314 Date: Mon, 20 Jun 2022 16:02:31 +0800 Subject: [PATCH 032/627] Change getKeeperDispatcher error type --- programs/keeper/TinyContext.cpp | 6 +++--- src/Interpreters/Context.cpp | 5 +++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/programs/keeper/TinyContext.cpp b/programs/keeper/TinyContext.cpp index 386fb1e0c1d..20b24893ebd 100644 --- a/programs/keeper/TinyContext.cpp +++ b/programs/keeper/TinyContext.cpp @@ -8,7 +8,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int KEEPER_EXCEPTION; } void TinyContext::setConfig(const ConfigurationPtr & config_) @@ -31,7 +31,7 @@ void TinyContext::initializeKeeperDispatcher([[maybe_unused]] bool start_async) std::lock_guard lock(keeper_dispatcher_mutex); if (keeper_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); + throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Trying to initialize Keeper multiple times"); if (config_ref.has("keeper_server")) { @@ -44,7 +44,7 @@ std::shared_ptr TinyContext::getKeeperDispatcher() const { std::lock_guard lock(keeper_dispatcher_mutex); if (!keeper_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests"); + throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Keeper must be initialized before requests"); return keeper_dispatcher; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ca6ff02b994..8b468c47b5c 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -135,6 +135,7 @@ namespace ErrorCodes extern const int INVALID_SETTING_VALUE; extern const int UNKNOWN_READ_METHOD; extern const int NOT_IMPLEMENTED; + extern const int KEEPER_EXCEPTION; } @@ -2068,7 +2069,7 @@ void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) cons std::lock_guard lock(shared->keeper_dispatcher_mutex); if (shared->keeper_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); + throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Trying to initialize Keeper multiple times"); const auto & config = getConfigRef(); if (config.has("keeper_server")) @@ -2097,7 +2098,7 @@ std::shared_ptr & Context::getKeeperDispatcher() const { std::lock_guard lock(shared->keeper_dispatcher_mutex); if (!shared->keeper_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests"); + throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Keeper must be initialized before requests"); return shared->keeper_dispatcher; } From c18dd55400201bf912cefcd07893f94b59e3774e Mon Sep 17 00:00:00 2001 From: lingpeng0314 Date: Tue, 21 Jun 2022 10:30:19 +0800 Subject: [PATCH 033/627] Resolve comments - refine metric description --- src/Common/ProfileEvents.cpp | 12 ++++++------ src/Interpreters/AsynchronousMetrics.cpp | 19 +++++++------------ 2 files changed, 13 insertions(+), 18 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index a2886330ce9..02762c1ac9d 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -348,14 +348,14 @@ M(KeeperPacketsReceived, "Packets received by keeper server") \ M(KeeperRequestTotal, "Total requests number on keeper server") \ M(KeeperLatency, "Keeper latency") \ - M(KeeperCommits, "Number of commit count") \ - M(KeeperCommitsFailed, "Number of commit failed count") \ - M(KeeperSnapshotCreations, "Number of snapshot creations count")\ - M(KeeperSnapshotCreationsFailed, "Number of snapshot creations failed count")\ + M(KeeperCommits, "Number of successful commits") \ + M(KeeperCommitsFailed, "Number of failed commits") \ + M(KeeperSnapshotCreations, "Number of snapshots creations")\ + M(KeeperSnapshotCreationsFailed, "Number of failed snapshot creations")\ M(KeeperSnapshotApplys, "Number of snapshot applying")\ - M(KeeperSnapshotApplysFailed, "Number of snapshot applying failed")\ + M(KeeperSnapshotApplysFailed, "Number of failed snapshot applying")\ M(KeeperReadSnapshot, "Number of snapshot read(serialization)")\ - M(KeeperSaveSnapshot, "Number of snapshot save") + M(KeeperSaveSnapshot, "Number of snapshot save")\ namespace ProfileEvents diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 505f956fd5a..4b3c37129c2 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1495,23 +1495,19 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti if (keeper_dispatcher && keeper_dispatcher->isServerActive()) { auto keeper_4LW_info = keeper_dispatcher -> getKeeper4LWInfo(); - if (keeper_4LW_info.is_standalone) - is_standalone = 1; - if (keeper_4LW_info.is_leader) - is_leader = 1; - if (keeper_4LW_info.is_observer) - is_observer = 1; - if (keeper_4LW_info.is_follower) - is_follower = 1; + is_standalone = static_cast(keeper_4LW_info.is_standalone; + is_leader = static_cast(keeper_4LW_info.is_leader; + is_observer = static_cast(keeper_4LW_info.is_observer; + is_follower = static_cast(keeper_4LW_info.is_follower; zxid = keeper_4LW_info.last_zxid; const auto & state_machine = keeper_dispatcher->getStateMachine(); znode_count = state_machine.getNodesCount(); watch_count = state_machine.getTotalWatchesCount(); ephemerals_count = state_machine.getTotalEphemeralNodesCount(); - approximate_data_size =state_machine.getApproximateDataSize(); - key_arena_size=state_machine.getKeyArenaSize(); - latest_snapshot_size=state_machine.getLatestSnapshotBufSize(); + approximate_data_size = state_machine.getApproximateDataSize(); + key_arena_size = state_machine.getKeyArenaSize(); + latest_snapshot_size = state_machine.getLatestSnapshotBufSize(); session_with_watches = state_machine.getSessionsWithWatchesCount(); paths_watched = state_machine.getWatchedPathsCount(); snapshot_dir_size = keeper_dispatcher->getSnapDirSize(); @@ -1540,7 +1536,6 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti new_values["KeeperApproximateDataSize"] = approximate_data_size; new_values["KeeperKeyArenaSize"] = key_arena_size; - new_values["KeeperLatestSnapshotSize"] = latest_snapshot_size; new_values["KeeperOpenFileDescriptorCount"] = open_file_descriptor_count; From 1a23dfa827ad378aeebe1e53c516f165e36d129d Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Tue, 21 Jun 2022 06:50:02 -0700 Subject: [PATCH 034/627] Update thrift to 0.13.0 --- contrib/thrift | 2 +- contrib/thrift-cmake/CMakeLists.txt | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/contrib/thrift b/contrib/thrift index 010ccf0a0c7..cecee50308f 160000 --- a/contrib/thrift +++ b/contrib/thrift @@ -1 +1 @@ -Subproject commit 010ccf0a0c7023fea0f6bf4e4078ebdff7e61982 +Subproject commit cecee50308fc7e6f77f55b3fd906c1c6c471fa2f diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 6f94c1ebdc0..33c15a700be 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -15,7 +15,6 @@ set(thriftcpp_SOURCES "${LIBRARY_DIR}/src/thrift/async/TConcurrentClientSyncInfo.cpp" "${LIBRARY_DIR}/src/thrift/concurrency/ThreadManager.cpp" "${LIBRARY_DIR}/src/thrift/concurrency/TimerManager.cpp" - "${LIBRARY_DIR}/src/thrift/concurrency/Util.cpp" "${LIBRARY_DIR}/src/thrift/processor/PeekProcessor.cpp" "${LIBRARY_DIR}/src/thrift/protocol/TBase64Utils.cpp" "${LIBRARY_DIR}/src/thrift/protocol/TDebugProtocol.cpp" From 4deddbf9005546a54e8c69af7d54de4a057c825e Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Tue, 21 Jun 2022 12:28:40 -0700 Subject: [PATCH 035/627] Update thrift to 0.16.0(latest) --- contrib/thrift | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/thrift b/contrib/thrift index cecee50308f..2a93df80f27 160000 --- a/contrib/thrift +++ b/contrib/thrift @@ -1 +1 @@ -Subproject commit cecee50308fc7e6f77f55b3fd906c1c6c471fa2f +Subproject commit 2a93df80f27739ccabb5b885cb12a8dc7595ecdf From 3a71b63b5d387f17a3edfaa020e23c1aba7f70ff Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 21 Jun 2022 12:35:58 +0000 Subject: [PATCH 036/627] Add list request type --- src/Common/ZooKeeper/IKeeper.h | 9 +++++++++ src/Common/ZooKeeper/TestKeeper.cpp | 2 ++ src/Common/ZooKeeper/TestKeeper.h | 1 + src/Common/ZooKeeper/ZooKeeper.cpp | 5 +++-- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 9 +++++++++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 ++ src/Common/ZooKeeper/ZooKeeperImpl.h | 1 + src/Coordination/KeeperStorage.cpp | 22 +++++++++++++++++++++- utils/keeper-bench/Generator.cpp | 2 +- 9 files changed, 49 insertions(+), 4 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 73c7da25a8b..9bd3ff3653c 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -281,9 +281,17 @@ struct SetResponse : virtual Response size_t bytesSize() const override { return sizeof(stat); } }; +enum class ListRequestType : uint8_t +{ + ALL, + PERSISTENT_ONLY, + EPHEMERAL_ONLY +}; + struct ListRequest : virtual Request { String path; + ListRequestType list_request_type{ListRequestType::ALL}; void addRootPath(const String & root_path) override; String getPath() const override { return path; } @@ -492,6 +500,7 @@ public: virtual void list( const String & path, + ListRequestType list_request_type, ListCallback callback, WatchCallback watch) = 0; diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 3d2d5fcb667..af2007e20c4 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -768,11 +768,13 @@ void TestKeeper::set( void TestKeeper::list( const String & path, + ListRequestType list_request_type, ListCallback callback, WatchCallback watch) { TestKeeperListRequest request; request.path = path; + request.list_request_type = list_request_type; RequestInfo request_info; request_info.request = std::make_shared(std::move(request)); diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 40cac3094f1..6e77a5d38c1 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -71,6 +71,7 @@ public: void list( const String & path, + ListRequestType list_request_type, ListCallback callback, WatchCallback watch) override; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index c8ae6b72c3e..c8adb1a18fd 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -9,6 +9,7 @@ #include #include #include +#include "Common/ZooKeeper/IKeeper.h" #include #include #include @@ -1059,7 +1060,7 @@ std::future ZooKeeper::asyncGetChildren(const std::s promise->set_value(response); }; - impl->list(path, std::move(callback), watch_callback); + impl->list(path, Coordination::ListRequestType::ALL, std::move(callback), watch_callback); return future; } @@ -1073,7 +1074,7 @@ std::future ZooKeeper::asyncTryGetChildrenNoThrow(co promise->set_value(response); }; - impl->list(path, std::move(callback), watch_callback); + impl->list(path, Coordination::ListRequestType::ALL, std::move(callback), watch_callback); return future; } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index de2fb630848..f2b8fe59e99 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -1,3 +1,4 @@ +#include "Common/ZooKeeper/IKeeper.h" #include #include #include @@ -285,12 +286,20 @@ void ZooKeeperListRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); Coordination::write(has_watch, out); + Coordination::write(static_cast(list_request_type), out); } void ZooKeeperListRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); Coordination::read(has_watch, in); + + if (!in.eof()) + { + int32_t read_request_type; + Coordination::read(read_request_type, in); + list_request_type = static_cast(read_request_type); + } } std::string ZooKeeperListRequest::toStringImpl() const diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index bd284ed0c91..7ab2d1960ab 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1168,11 +1168,13 @@ void ZooKeeper::set( void ZooKeeper::list( const String & path, + ListRequestType list_request_type, ListCallback callback, WatchCallback watch) { ZooKeeperListRequest request; request.path = path; + request.list_request_type = list_request_type; RequestInfo request_info; request_info.request = std::make_shared(std::move(request)); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index c4acaf8d1ee..aa27b0eefe9 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -164,6 +164,7 @@ public: void list( const String & path, + ListRequestType list_request_type, ListCallback callback, WatchCallback watch) override; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 21265f0bd61..b0090b5e960 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1161,6 +1161,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc } auto & container = storage.container; + auto node_it = container.find(request.path); if (node_it == container.end()) { @@ -1178,8 +1179,27 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc const auto & children = node_it->value.getChildren(); response.names.reserve(children.size()); + const auto add_child = [&](const auto child) + { + using enum Coordination::ListRequestType; + + if (request.list_request_type == ALL) + return true; + + auto child_it = container.find(fmt::format("{}/{}", request.path, child)); + if (child_it == container.end()) + onStorageInconsistency(); + + const auto is_ephemeral = child_it->value.stat.ephemeralOwner != 0; + + return (is_ephemeral && request.list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && request.list_request_type == PERSISTENT_ONLY); + }; + for (const auto child : children) - response.names.push_back(child.toString()); + { + if (add_child(child)) + response.names.push_back(child.toString()); + } response.stat = node_it->value.stat; response.error = Coordination::Error::ZOK; diff --git a/utils/keeper-bench/Generator.cpp b/utils/keeper-bench/Generator.cpp index d3a8323b81f..5d1d0f8a491 100644 --- a/utils/keeper-bench/Generator.cpp +++ b/utils/keeper-bench/Generator.cpp @@ -62,7 +62,7 @@ void removeRecursive(Coordination::ZooKeeper & zookeeper, const std::string & pa promise->set_value(); }; - zookeeper.list(path, list_callback, nullptr); + zookeeper.list(path, ListRequestType::ALL, list_callback, nullptr); future.get(); while (!children.empty()) From 568c957eb2694d50b1d4a1903131a1119c08494d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 23 Jun 2022 07:53:07 +0000 Subject: [PATCH 037/627] Fix TestKeeper --- src/Common/ZooKeeper/TestKeeper.cpp | 9 ++++++++- src/Coordination/KeeperStorage.cpp | 2 +- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index af2007e20c4..d6fddb4d4b0 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -1,3 +1,4 @@ +#include "Common/ZooKeeper/IKeeper.h" #include #include #include @@ -390,8 +391,14 @@ std::pair TestKeeperListRequest::process(TestKeeper::Containe child_it != container.end() && startsWith(child_it->first, path_prefix); ++child_it) { + using enum ListRequestType; if (parentPath(child_it->first) == path) - response.names.emplace_back(baseName(child_it->first)); + { + const auto is_ephemeral = child_it->second.stat.ephemeralOwner != 0; + if (list_request_type == ALL || (is_ephemeral && list_request_type == EPHEMERAL_ONLY) + || (!is_ephemeral && list_request_type == PERSISTENT_ONLY)) + response.names.emplace_back(baseName(child_it->first)); + } } response.stat = it->second.stat; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index b0090b5e960..ed7670c2730 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1643,7 +1643,7 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc void KeeperStorage::finalize() { if (finalized) - throw DB::Exception("Testkeeper storage already finalized", ErrorCodes::LOGICAL_ERROR); + throw DB::Exception("KeeperStorage already finalized", ErrorCodes::LOGICAL_ERROR); finalized = true; From ae3d9fd9623a829bfca4c09bc258e33e0859743f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 23 Jun 2022 10:25:34 +0000 Subject: [PATCH 038/627] Expose list request type in internal client --- src/Common/ZooKeeper/ZooKeeper.cpp | 41 ++++++++++++++++++------------ src/Common/ZooKeeper/ZooKeeper.h | 22 ++++++++++++---- src/Coordination/KeeperStorage.cpp | 3 +-- 3 files changed, 43 insertions(+), 23 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index c8adb1a18fd..5a0be0f76ff 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -313,9 +313,10 @@ static Coordination::WatchCallback callbackForEvent(const EventPtr & watch) Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings & res, Coordination::Stat * stat, - Coordination::WatchCallback watch_callback) + Coordination::WatchCallback watch_callback, + Coordination::ListRequestType list_request_type) { - auto future_result = asyncTryGetChildrenNoThrow(path, watch_callback); + auto future_result = asyncTryGetChildrenNoThrow(path, watch_callback, list_request_type); if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { @@ -336,26 +337,28 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings } } -Strings ZooKeeper::getChildren( - const std::string & path, Coordination::Stat * stat, const EventPtr & watch) +Strings ZooKeeper::getChildren(const std::string & path, Coordination::Stat * stat, const EventPtr & watch) { Strings res; check(tryGetChildren(path, res, stat, watch), path); return res; } -Strings ZooKeeper::getChildrenWatch( - const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback) +Strings ZooKeeper::getChildrenWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback) { Strings res; check(tryGetChildrenWatch(path, res, stat, watch_callback), path); return res; } -Coordination::Error ZooKeeper::tryGetChildren(const std::string & path, Strings & res, - Coordination::Stat * stat, const EventPtr & watch) +Coordination::Error ZooKeeper::tryGetChildren( + const std::string & path, + Strings & res, + Coordination::Stat * stat, + const EventPtr & watch, + Coordination::ListRequestType list_request_type) { - Coordination::Error code = getChildrenImpl(path, res, stat, callbackForEvent(watch)); + Coordination::Error code = getChildrenImpl(path, res, stat, callbackForEvent(watch), list_request_type); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) throw KeeperException(code, path); @@ -363,10 +366,14 @@ Coordination::Error ZooKeeper::tryGetChildren(const std::string & path, Strings return code; } -Coordination::Error ZooKeeper::tryGetChildrenWatch(const std::string & path, Strings & res, - Coordination::Stat * stat, Coordination::WatchCallback watch_callback) +Coordination::Error ZooKeeper::tryGetChildrenWatch( + const std::string & path, + Strings & res, + Coordination::Stat * stat, + Coordination::WatchCallback watch_callback, + Coordination::ListRequestType list_request_type) { - Coordination::Error code = getChildrenImpl(path, res, stat, watch_callback); + Coordination::Error code = getChildrenImpl(path, res, stat, watch_callback, list_request_type); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) throw KeeperException(code, path); @@ -1047,7 +1054,8 @@ std::future ZooKeeper::asyncTrySetNoThrow(const std:: return future; } -std::future ZooKeeper::asyncGetChildren(const std::string & path, Coordination::WatchCallback watch_callback) +std::future ZooKeeper::asyncGetChildren( + const std::string & path, Coordination::WatchCallback watch_callback, Coordination::ListRequestType list_request_type) { auto promise = std::make_shared>(); auto future = promise->get_future(); @@ -1060,11 +1068,12 @@ std::future ZooKeeper::asyncGetChildren(const std::s promise->set_value(response); }; - impl->list(path, Coordination::ListRequestType::ALL, std::move(callback), watch_callback); + impl->list(path, list_request_type, std::move(callback), watch_callback); return future; } -std::future ZooKeeper::asyncTryGetChildrenNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) +std::future ZooKeeper::asyncTryGetChildrenNoThrow( + const std::string & path, Coordination::WatchCallback watch_callback, Coordination::ListRequestType list_request_type) { auto promise = std::make_shared>(); auto future = promise->get_future(); @@ -1074,7 +1083,7 @@ std::future ZooKeeper::asyncTryGetChildrenNoThrow(co promise->set_value(response); }; - impl->list(path, Coordination::ListRequestType::ALL, std::move(callback), watch_callback); + impl->list(path, list_request_type, std::move(callback), watch_callback); return future; } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 6aebccd2b4e..d2f92b6b4c3 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -194,11 +194,13 @@ public: /// * The node doesn't exist. Coordination::Error tryGetChildren(const std::string & path, Strings & res, Coordination::Stat * stat = nullptr, - const EventPtr & watch = nullptr); + const EventPtr & watch = nullptr, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); Coordination::Error tryGetChildrenWatch(const std::string & path, Strings & res, Coordination::Stat * stat, - Coordination::WatchCallback watch_callback); + Coordination::WatchCallback watch_callback, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); /// Performs several operations in a transaction. /// Throws on every error. @@ -279,9 +281,15 @@ public: FutureExists asyncTryExistsNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); using FutureGetChildren = std::future; - FutureGetChildren asyncGetChildren(const std::string & path, Coordination::WatchCallback watch_callback = {}); + FutureGetChildren asyncGetChildren( + const std::string & path, + Coordination::WatchCallback watch_callback = {}, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); /// Like the previous one but don't throw any exceptions on future.get() - FutureGetChildren asyncTryGetChildrenNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); + FutureGetChildren asyncTryGetChildrenNoThrow( + const std::string & path, + Coordination::WatchCallback watch_callback = {}, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); using FutureSet = std::future; FutureSet asyncSet(const std::string & path, const std::string & data, int32_t version = -1); @@ -335,7 +343,11 @@ private: const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback); Coordination::Error setImpl(const std::string & path, const std::string & data, int32_t version, Coordination::Stat * stat); Coordination::Error getChildrenImpl( - const std::string & path, Strings & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback); + const std::string & path, + Strings & res, + Coordination::Stat * stat, + Coordination::WatchCallback watch_callback, + Coordination::ListRequestType list_request_type); Coordination::Error multiImpl(const Coordination::Requests & requests, Coordination::Responses & responses); Coordination::Error existsImpl(const std::string & path, Coordination::Stat * stat_, Coordination::WatchCallback watch_callback); Coordination::Error syncImpl(const std::string & path, std::string & returned_path); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index ed7670c2730..f15c10038c7 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1186,12 +1186,11 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc if (request.list_request_type == ALL) return true; - auto child_it = container.find(fmt::format("{}/{}", request.path, child)); + auto child_it = container.find(fmt::format("{}{}{}", request.path, (request.path.ends_with('/') ? "" : "/"), child)); if (child_it == container.end()) onStorageInconsistency(); const auto is_ephemeral = child_it->value.stat.ephemeralOwner != 0; - return (is_ephemeral && request.list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && request.list_request_type == PERSISTENT_ONLY); }; From 2e71ae97a17224ec7729f1c9e60d5fc998ef45f1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 23 Jun 2022 10:25:49 +0000 Subject: [PATCH 039/627] Add unit test for different list request types --- src/Coordination/tests/gtest_coordination.cpp | 79 +++++++++++++++++++ 1 file changed, 79 insertions(+) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index ee75f2a0860..61578b32912 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include @@ -1956,6 +1957,84 @@ TEST_P(CoordinationTest, TestUncommittedStateBasicCrud) ASSERT_FALSE(get_committed_data()); } +TEST_P(CoordinationTest, TestListRequestTypes) +{ + using namespace DB; + using namespace Coordination; + + KeeperStorage storage{500, "", true}; + + int64_t zxid = 0; + + static constexpr std::string_view path = "/test"; + + const auto create_path = [&](bool is_ephemeral) + { + const auto create_request = std::make_shared(); + int new_zxid = ++zxid; + create_request->path = path; + create_request->is_sequential = true; + create_request->is_ephemeral = is_ephemeral; + storage.preprocessRequest(create_request, 1, 0, new_zxid); + auto responses = storage.processRequest(create_request, 1, new_zxid); + + EXPECT_GE(responses.size(), 1); + const auto & create_response = dynamic_cast(*responses[0].response); + return create_response.path_created; + }; + + static constexpr size_t persistent_num = 5; + std::unordered_set expected_persistent_children; + for (size_t i = 0; i < persistent_num; ++i) + { + expected_persistent_children.insert(getBaseName(create_path(false)).toString()); + } + ASSERT_EQ(expected_persistent_children.size(), persistent_num); + + static constexpr size_t ephemeral_num = 5; + std::unordered_set expected_ephemeral_children; + for (size_t i = 0; i < ephemeral_num; ++i) + { + expected_ephemeral_children.insert(getBaseName(create_path(true)).toString()); + } + ASSERT_EQ(expected_ephemeral_children.size(), ephemeral_num); + + const auto get_children = [&](const auto list_request_type) + { + const auto list_request = std::make_shared(); + int new_zxid = ++zxid; + list_request->path = parentPath(StringRef{path}).toString(); + list_request->list_request_type = list_request_type; + storage.preprocessRequest(list_request, 1, 0, new_zxid); + auto responses = storage.processRequest(list_request, 1, new_zxid); + + EXPECT_GE(responses.size(), 1); + const auto & list_response = dynamic_cast(*responses[0].response); + return list_response.names; + }; + + const auto persistent_children = get_children(ListRequestType::PERSISTENT_ONLY); + EXPECT_EQ(persistent_children.size(), persistent_num); + for (const auto & child : persistent_children) + { + EXPECT_TRUE(expected_persistent_children.contains(child)) << "Missing persistent child " << child; + } + + const auto ephemeral_children = get_children(ListRequestType::EPHEMERAL_ONLY); + EXPECT_EQ(ephemeral_children.size(), ephemeral_num); + for (const auto & child : ephemeral_children) + { + EXPECT_TRUE(expected_ephemeral_children.contains(child)) << "Missing ephemeral child " << child; + } + + const auto all_children = get_children(ListRequestType::ALL); + EXPECT_EQ(all_children.size(), ephemeral_num + persistent_num); + for (const auto & child : all_children) + { + EXPECT_TRUE(expected_ephemeral_children.contains(child) || expected_persistent_children.contains(child)) << "Missing child " << child; + } +} + INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, CoordinationTest, ::testing::ValuesIn(std::initializer_list{ From 59c1c472cb1ee8fcf03134865d45cb38b770022a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 23 Jun 2022 20:04:06 +0000 Subject: [PATCH 040/627] Better exception messages on wrong table engines/functions argument types --- src/Core/Field.cpp | 27 +++++++++++++ src/Core/Field.h | 2 + src/Interpreters/getClusterName.cpp | 3 +- src/Storages/FileLog/StorageFileLog.cpp | 8 ++-- src/Storages/HDFS/StorageHDFS.cpp | 7 ++-- src/Storages/Hive/StorageHive.cpp | 7 ++-- src/Storages/IStorage.cpp | 2 - .../MeiliSearch/StorageMeiliSearch.cpp | 9 ++--- src/Storages/StorageBuffer.cpp | 5 ++- src/Storages/StorageDictionary.cpp | 3 +- src/Storages/StorageDistributed.cpp | 11 ++---- src/Storages/StorageExecutable.cpp | 7 ++-- src/Storages/StorageExternalDistributed.cpp | 27 ++++++------- src/Storages/StorageFile.cpp | 39 ++++++++++--------- src/Storages/StorageGenerateRandom.cpp | 12 +++--- src/Storages/StorageMerge.cpp | 22 +++++------ src/Storages/StorageMongoDB.cpp | 21 +++++----- src/Storages/StorageMySQL.cpp | 22 +++++------ src/Storages/StoragePostgreSQL.cpp | 20 +++++----- src/Storages/StorageS3.cpp | 35 ++++++++--------- src/Storages/StorageSQLite.cpp | 7 ++-- src/Storages/StorageSet.cpp | 2 - src/Storages/StorageSnapshot.cpp | 1 - src/Storages/StorageStripeLog.cpp | 6 --- src/Storages/StorageURL.cpp | 26 ++++++------- src/Storages/StorageXDBC.cpp | 16 ++++---- src/Storages/checkAndGetLiteralArgument.cpp | 37 ++++++++++++++++++ src/Storages/checkAndGetLiteralArgument.h | 21 ++++++++++ src/TableFunctions/Hive/TableFunctionHive.cpp | 13 +++---- src/TableFunctions/ITableFunctionFileLike.cpp | 20 +++++----- src/TableFunctions/ITableFunctionFileLike.h | 2 +- .../TableFunctionDictionary.cpp | 3 +- .../TableFunctionExecutable.cpp | 10 ++--- src/TableFunctions/TableFunctionFile.cpp | 39 +++++++------------ src/TableFunctions/TableFunctionFile.h | 2 +- src/TableFunctions/TableFunctionFormat.cpp | 5 ++- .../TableFunctionGenerateRandom.cpp | 13 ++++--- .../TableFunctionHDFSCluster.cpp | 6 +-- src/TableFunctions/TableFunctionInput.cpp | 4 +- src/TableFunctions/TableFunctionMerge.cpp | 5 ++- src/TableFunctions/TableFunctionNull.cpp | 4 +- src/TableFunctions/TableFunctionRemote.cpp | 11 +++--- src/TableFunctions/TableFunctionS3.cpp | 18 +++++---- src/TableFunctions/TableFunctionS3Cluster.cpp | 7 ++-- src/TableFunctions/TableFunctionSQLite.cpp | 8 ++-- src/TableFunctions/TableFunctionZeros.cpp | 4 +- 46 files changed, 320 insertions(+), 259 deletions(-) create mode 100644 src/Storages/checkAndGetLiteralArgument.cpp create mode 100644 src/Storages/checkAndGetLiteralArgument.h diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 2f37d2ea951..3a4b66e6266 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -559,4 +559,31 @@ String toString(const Field & x) x); } +String fieldTypeToString(Field::Types::Which type) +{ + switch (type) + { + case Field::Types::Which::Null: return "Null"; + case Field::Types::Which::Array: return "Array"; + case Field::Types::Which::Tuple: return "Tuple"; + case Field::Types::Which::Map: return "Map"; + case Field::Types::Which::Object: return "Object"; + case Field::Types::Which::AggregateFunctionState: return "AggregateFunctionState"; + case Field::Types::Which::Bool: return "Bool"; + case Field::Types::Which::String: return "String"; + case Field::Types::Which::Decimal32: return "Decimal32"; + case Field::Types::Which::Decimal64: return "Decimal64"; + case Field::Types::Which::Decimal128: return "Decimal128"; + case Field::Types::Which::Decimal256: return "Decimal256"; + case Field::Types::Which::Float64: return "Float64"; + case Field::Types::Which::Int64: return "Int64"; + case Field::Types::Which::Int128: return "Int128"; + case Field::Types::Which::Int256: return "Int256"; + case Field::Types::Which::UInt64: return "UInt64"; + case Field::Types::Which::UInt128: return "UInt128"; + case Field::Types::Which::UInt256: return "UInt256"; + case Field::Types::Which::UUID: return "UUID"; + } +} + } diff --git a/src/Core/Field.h b/src/Core/Field.h index 2a925a9e2a6..36564c69fbe 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -1011,6 +1011,8 @@ void writeFieldText(const Field & x, WriteBuffer & buf); String toString(const Field & x); +String fieldTypeToString(Field::Types::Which type); + } template <> diff --git a/src/Interpreters/getClusterName.cpp b/src/Interpreters/getClusterName.cpp index fee10e32d70..d3c53b28cdf 100644 --- a/src/Interpreters/getClusterName.cpp +++ b/src/Interpreters/getClusterName.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -22,7 +23,7 @@ std::string getClusterName(const IAST & node) return ast_id->name(); if (const auto * ast_lit = node.as()) - return ast_lit->value.safeGet(); + return checkAndGetLiteralArgument(*ast_lit, "cluster_name"); /// A hack to support hyphens in cluster names. if (const auto * ast_func = node.as()) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 4bf77792559..323bcdc100d 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -9,16 +9,14 @@ #include #include #include -#include #include -#include #include #include #include -#include #include #include #include +#include #include #include #include @@ -805,8 +803,8 @@ void registerStorageFileLog(StorageFactory & factory) auto path_ast = evaluateConstantExpressionAsLiteral(engine_args[0], args.getContext()); auto format_ast = evaluateConstantExpressionAsLiteral(engine_args[1], args.getContext()); - auto path = path_ast->as().value.safeGet(); - auto format = format_ast->as().value.safeGet(); + auto path = checkAndGetLiteralArgument(path_ast, "path"); + auto format = checkAndGetLiteralArgument(format_ast, "format"); return std::make_shared( args.table_id, diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 708bfd5ef8b..5e811f8e42c 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include @@ -664,13 +665,13 @@ void registerStorageHDFS(StorageFactory & factory) engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.getLocalContext()); - String url = engine_args[0]->as().value.safeGet(); + String url = checkAndGetLiteralArgument(engine_args[0], "url"); String format_name = "auto"; if (engine_args.size() > 1) { engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.getLocalContext()); - format_name = engine_args[1]->as().value.safeGet(); + format_name = checkAndGetLiteralArgument(engine_args[1], "format_name"); } if (format_name == "auto") @@ -680,7 +681,7 @@ void registerStorageHDFS(StorageFactory & factory) if (engine_args.size() == 3) { engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.getLocalContext()); - compression_method = engine_args[2]->as().value.safeGet(); + compression_method = checkAndGetLiteralArgument(engine_args[2], "compression_method"); } else compression_method = "auto"; ASTPtr partition_by; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 6046dd58677..2f7219941a2 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -37,6 +37,7 @@ #include #include #include +#include namespace DB { @@ -938,9 +939,9 @@ void registerStorageHive(StorageFactory & factory) for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext()); - const String & hive_metastore_url = engine_args[0]->as().value.safeGet(); - const String & hive_database = engine_args[1]->as().value.safeGet(); - const String & hive_table = engine_args[2]->as().value.safeGet(); + const String & hive_metastore_url = checkAndGetLiteralArgument(engine_args[0], "hive_metastore_url"); + const String & hive_database = checkAndGetLiteralArgument(engine_args[1], "hive_database"); + const String & hive_table = checkAndGetLiteralArgument(engine_args[2], "hive_table"); return std::make_shared( hive_metastore_url, hive_database, diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index cd6c49d6e3b..7607c395ce8 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -1,11 +1,9 @@ #include #include -#include #include #include #include -#include #include #include #include diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp index 02dca993436..c5966d9e322 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -14,7 +13,7 @@ #include #include #include -#include +#include #include #include @@ -156,11 +155,11 @@ MeiliSearchConfiguration StorageMeiliSearch::getConfiguration(ASTs engine_args, for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); - String url = engine_args[0]->as().value.safeGet(); - String index = engine_args[1]->as().value.safeGet(); + String url = checkAndGetLiteralArgument(engine_args[0], "url"); + String index = checkAndGetLiteralArgument(engine_args[1], "index"); String key; if (engine_args.size() == 3) - key = engine_args[2]->as().value.safeGet(); + key = checkAndGetLiteralArgument(engine_args[2], "key"); return MeiliSearchConfiguration(url, index, key); } } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 4c962f36e4f..85fb20d6571 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -1079,8 +1080,8 @@ void registerStorageBuffer(StorageFactory & factory) size_t i = 0; - String destination_database = engine_args[i++]->as().value.safeGet(); - String destination_table = engine_args[i++]->as().value.safeGet(); + String destination_database = checkAndGetLiteralArgument(engine_args[i++], "destination_database"); + String destination_table = checkAndGetLiteralArgument(engine_args[i++], "destination_table"); UInt64 num_buckets = applyVisitor(FieldVisitorConvertToNumber(), engine_args[i++]->as().value); diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index fda6da6c1ff..2839ac03a5b 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB @@ -339,7 +340,7 @@ void registerStorageDictionary(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); args.engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args.engine_args[0], local_context); - String dictionary_name = args.engine_args[0]->as().value.safeGet(); + String dictionary_name = checkAndGetLiteralArgument(args.engine_args[0], "dictionary_name"); if (!args.attach) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1c785df9be4..03eb400a8ad 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include @@ -34,10 +35,6 @@ #include #include #include -#include -#include -#include -#include #include #include @@ -1437,15 +1434,15 @@ void registerStorageDistributed(StorageFactory & factory) engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], local_context); engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], local_context); - String remote_database = engine_args[1]->as().value.safeGet(); - String remote_table = engine_args[2]->as().value.safeGet(); + String remote_database = checkAndGetLiteralArgument(engine_args[1], "remote_database"); + String remote_table = checkAndGetLiteralArgument(engine_args[2], "remote_table"); const auto & sharding_key = engine_args.size() >= 4 ? engine_args[3] : nullptr; String storage_policy = "default"; if (engine_args.size() >= 5) { engine_args[4] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[4], local_context); - storage_policy = engine_args[4]->as().value.safeGet(); + storage_policy = checkAndGetLiteralArgument(engine_args[4], "storage_policy"); } /// Check that sharding_key exists in the table and has numeric type. diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index e0cbdbe98af..2931e62b7ef 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -22,6 +22,7 @@ #include #include #include +#include namespace DB @@ -179,14 +180,14 @@ void registerStorageExecutable(StorageFactory & factory) for (size_t i = 0; i < 2; ++i) args.engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args.engine_args[i], local_context); - auto scipt_name_with_arguments_value = args.engine_args[0]->as().value.safeGet(); + auto script_name_with_arguments_value = checkAndGetLiteralArgument(args.engine_args[0], "script_name_with_arguments_value"); std::vector script_name_with_arguments; - boost::split(script_name_with_arguments, scipt_name_with_arguments_value, [](char c) { return c == ' '; }); + boost::split(script_name_with_arguments, script_name_with_arguments_value, [](char c) { return c == ' '; }); auto script_name = script_name_with_arguments[0]; script_name_with_arguments.erase(script_name_with_arguments.begin()); - auto format = args.engine_args[1]->as().value.safeGet(); + auto format = checkAndGetLiteralArgument(args.engine_args[1], "format"); std::vector input_queries; for (size_t i = 2; i < args.engine_args.size(); ++i) diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 181cf0ca183..04a30769c4e 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -3,13 +3,9 @@ #include #include -#include -#include #include -#include #include #include -#include #include #include #include @@ -17,6 +13,7 @@ #include #include #include +#include #include #include @@ -229,7 +226,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) if (engine_args.size() < 2) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine ExternalDistributed must have at least 2 arguments: engine_name, named_collection and/or description"); - auto engine_name = engine_args[0]->as().value.safeGet(); + auto engine_name = checkAndGetLiteralArgument(engine_args[0], "engine_name"); StorageExternalDistributed::ExternalStorageEngine table_engine; if (engine_name == "URL") table_engine = StorageExternalDistributed::ExternalStorageEngine::URL; @@ -256,7 +253,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) for (const auto & [name, value] : storage_specific_args) { if (name == "description") - cluster_description = value->as()->value.safeGet(); + cluster_description = checkAndGetLiteralArgument(value, "cluster_description"); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown key-value argument {} for table engine URL", name); @@ -271,11 +268,11 @@ void registerStorageExternalDistributed(StorageFactory & factory) for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext()); - cluster_description = engine_args[1]->as().value.safeGet(); - configuration.format = engine_args[2]->as().value.safeGet(); + cluster_description = checkAndGetLiteralArgument(engine_args[1], "cluster_description"); + configuration.format = checkAndGetLiteralArgument(engine_args[2], "format"); configuration.compression_method = "auto"; if (engine_args.size() == 4) - configuration.compression_method = engine_args[3]->as().value.safeGet(); + configuration.compression_method = checkAndGetLiteralArgument(engine_args[3], "compression_method"); } @@ -302,7 +299,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) for (const auto & [name, value] : storage_specific_args) { if (name == "description") - cluster_description = value->as()->value.safeGet(); + cluster_description = checkAndGetLiteralArgument(value, "cluster_description"); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown key-value argument {} for table function URL", name); @@ -320,11 +317,11 @@ void registerStorageExternalDistributed(StorageFactory & factory) "ExternalDistributed('engine_name', 'cluster_description', 'database', 'table', 'user', 'password').", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - cluster_description = engine_args[1]->as().value.safeGet(); - configuration.database = engine_args[2]->as().value.safeGet(); - configuration.table = engine_args[3]->as().value.safeGet(); - configuration.username = engine_args[4]->as().value.safeGet(); - configuration.password = engine_args[5]->as().value.safeGet(); + cluster_description = checkAndGetLiteralArgument(engine_args[1], "cluster_description"); + configuration.database = checkAndGetLiteralArgument(engine_args[2], "database"); + configuration.table = checkAndGetLiteralArgument(engine_args[3], "table"); + configuration.username = checkAndGetLiteralArgument(engine_args[4], "username"); + configuration.password = checkAndGetLiteralArgument(engine_args[5], "password"); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 2fa6003c0eb..d138104018a 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1,5 +1,10 @@ #include #include +#include +#include +#include +#include +#include #include #include @@ -20,30 +25,26 @@ #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 +#include namespace fs = std::filesystem; @@ -1103,7 +1104,7 @@ void registerStorageFile(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); engine_args_ast[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[0], factory_args.getLocalContext()); - storage_args.format_name = engine_args_ast[0]->as().value.safeGet(); + storage_args.format_name = checkAndGetLiteralArgument(engine_args_ast[0], "format_name"); // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current @@ -1171,7 +1172,7 @@ void registerStorageFile(StorageFactory & factory) if (engine_args_ast.size() == 3) { engine_args_ast[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[2], factory_args.getLocalContext()); - storage_args.compression_method = engine_args_ast[2]->as().value.safeGet(); + storage_args.compression_method = checkAndGetLiteralArgument(engine_args_ast[2], "compression_method"); } else storage_args.compression_method = "auto"; diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index fa0baea40cd..d875b4ee80c 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -12,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -469,16 +469,16 @@ void registerStorageGenerateRandom(StorageFactory & factory) if (!engine_args.empty()) { - const Field & value = engine_args[0]->as().value; - if (!value.isNull()) - random_seed = value.safeGet(); + const auto & ast_literal = engine_args[0]->as(); + if (!ast_literal.value.isNull()) + random_seed = checkAndGetLiteralArgument(ast_literal, "random_seed"); } if (engine_args.size() >= 2) - max_string_length = engine_args[1]->as().value.safeGet(); + max_string_length = checkAndGetLiteralArgument(engine_args[1], "max_string_length"); if (engine_args.size() == 3) - max_array_length = engine_args[2]->as().value.safeGet(); + max_array_length = checkAndGetLiteralArgument(engine_args[2], "max_array_length"); return std::make_shared(args.table_id, args.columns, args.comment, max_array_length, max_string_length, random_seed); }); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f6d7e8e7afd..0afc7a0df7e 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1,9 +1,10 @@ #include -#include +#include #include #include #include #include +#include #include #include #include @@ -22,17 +23,16 @@ #include #include #include -#include "Processors/QueryPlan/BuildQueryPipelineSettings.h" -#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" -#include -#include -#include -#include +#include +#include #include #include #include #include -#include +#include +#include +#include +#include namespace DB @@ -848,7 +848,7 @@ std::tuple StorageMerge::evaluateDatabaseName(cons throw Exception("REGEXP in Merge ENGINE takes only one argument", ErrorCodes::BAD_ARGUMENTS); auto * literal = func->arguments->children[0]->as(); - if (!literal || literal->value.safeGet().empty()) + if (!literal || literal->value.getType() != Field::Types::Which::String || literal->value.safeGet().empty()) throw Exception("Argument for REGEXP in Merge ENGINE should be a non empty String Literal", ErrorCodes::BAD_ARGUMENTS); return {true, func->arguments->children[0]}; @@ -879,10 +879,10 @@ void registerStorageMerge(StorageFactory & factory) if (!is_regexp) engine_args[0] = database_ast; - String source_database_name_or_regexp = database_ast->as().value.safeGet(); + String source_database_name_or_regexp = checkAndGetLiteralArgument(database_ast, "database_name"); engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.getLocalContext()); - String table_name_regexp = engine_args[1]->as().value.safeGet(); + String table_name_regexp = checkAndGetLiteralArgument(engine_args[1], "table_name_regexp"); return std::make_shared( args.table_id, args.columns, args.comment, source_database_name_or_regexp, is_regexp, table_name_regexp, args.getContext()); diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 11a1f8ba4d6..1f2523c8645 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -1,11 +1,12 @@ -#include "StorageMongoDB.h" -#include "StorageMongoDBSocketFactory.h" +#include +#include +#include +#include #include #include #include #include -#include #include #include #include @@ -120,7 +121,7 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C for (const auto & [arg_name, arg_value] : storage_specific_args) { if (arg_name == "options") - configuration.options = arg_value->as()->value.safeGet(); + configuration.options = checkAndGetLiteralArgument(arg_value, "options"); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key-value argument." @@ -139,17 +140,17 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); /// 27017 is the default MongoDB port. - auto parsed_host_port = parseAddress(engine_args[0]->as().value.safeGet(), 27017); + auto parsed_host_port = parseAddress(checkAndGetLiteralArgument(engine_args[0], "host:port"), 27017); configuration.host = parsed_host_port.first; configuration.port = parsed_host_port.second; - configuration.database = engine_args[1]->as().value.safeGet(); - configuration.table = engine_args[2]->as().value.safeGet(); - configuration.username = engine_args[3]->as().value.safeGet(); - configuration.password = engine_args[4]->as().value.safeGet(); + configuration.database = checkAndGetLiteralArgument(engine_args[1], "database"); + configuration.table = checkAndGetLiteralArgument(engine_args[2], "table"); + configuration.username = checkAndGetLiteralArgument(engine_args[3], "username"); + configuration.password = checkAndGetLiteralArgument(engine_args[4], "password"); if (engine_args.size() >= 6) - configuration.options = engine_args[5]->as().value.safeGet(); + configuration.options = checkAndGetLiteralArgument(engine_args[5], "database"); } diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 3ed97712292..7fe008eead4 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -5,14 +5,12 @@ #include #include #include +#include #include #include -#include -#include #include #include #include -#include #include #include #include @@ -253,9 +251,9 @@ StorageMySQLConfiguration StorageMySQL::getConfiguration(ASTs engine_args, Conte for (const auto & [arg_name, arg_value] : storage_specific_args) { if (arg_name == "replace_query") - configuration.replace_query = arg_value->as()->value.safeGet(); + configuration.replace_query = checkAndGetLiteralArgument(arg_value, "replace_query"); else if (arg_name == "on_duplicate_clause") - configuration.on_duplicate_clause = arg_value->as()->value.safeGet(); + configuration.on_duplicate_clause = checkAndGetLiteralArgument(arg_value, "on_duplicate_clause"); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key-value argument." @@ -273,18 +271,18 @@ StorageMySQLConfiguration StorageMySQL::getConfiguration(ASTs engine_args, Conte for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context_); - const auto & host_port = engine_args[0]->as().value.safeGet(); + const auto & host_port = checkAndGetLiteralArgument(engine_args[0], "host:port"); size_t max_addresses = context_->getSettingsRef().glob_expansion_max_elements; configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306); - configuration.database = engine_args[1]->as().value.safeGet(); - configuration.table = engine_args[2]->as().value.safeGet(); - configuration.username = engine_args[3]->as().value.safeGet(); - configuration.password = engine_args[4]->as().value.safeGet(); + configuration.database = checkAndGetLiteralArgument(engine_args[1], "database"); + configuration.table = checkAndGetLiteralArgument(engine_args[2], "table"); + configuration.username = checkAndGetLiteralArgument(engine_args[3], "username"); + configuration.password = checkAndGetLiteralArgument(engine_args[4], "password"); if (engine_args.size() >= 6) - configuration.replace_query = engine_args[5]->as().value.safeGet(); + configuration.replace_query = checkAndGetLiteralArgument(engine_args[5], "replace_query"); if (engine_args.size() == 7) - configuration.on_duplicate_clause = engine_args[6]->as().value.safeGet(); + configuration.on_duplicate_clause = checkAndGetLiteralArgument(engine_args[6], "on_duplicate_clause"); } for (const auto & address : configuration.addresses) context_->getRemoteHostFilter().checkHostAndPort(address.first, toString(address.second)); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 5b57384c1dd..53f405bacfd 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -11,8 +11,6 @@ #include #include -#include -#include #include #include @@ -31,7 +29,6 @@ #include #include -#include #include #include @@ -40,6 +37,7 @@ #include #include +#include namespace DB @@ -400,7 +398,7 @@ StoragePostgreSQLConfiguration StoragePostgreSQL::getConfiguration(ASTs engine_a for (const auto & [arg_name, arg_value] : storage_specific_args) { if (arg_name == "on_conflict") - configuration.on_conflict = arg_value->as()->value.safeGet(); + configuration.on_conflict = checkAndGetLiteralArgument(arg_value, "on_conflict"); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key-value argument." @@ -418,7 +416,7 @@ StoragePostgreSQLConfiguration StoragePostgreSQL::getConfiguration(ASTs engine_a for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); - const auto & host_port = engine_args[0]->as().value.safeGet(); + const auto & host_port = checkAndGetLiteralArgument(engine_args[0], "host:port"); size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432); @@ -427,15 +425,15 @@ StoragePostgreSQLConfiguration StoragePostgreSQL::getConfiguration(ASTs engine_a configuration.host = configuration.addresses[0].first; configuration.port = configuration.addresses[0].second; } - configuration.database = engine_args[1]->as().value.safeGet(); - configuration.table = engine_args[2]->as().value.safeGet(); - configuration.username = engine_args[3]->as().value.safeGet(); - configuration.password = engine_args[4]->as().value.safeGet(); + configuration.database = checkAndGetLiteralArgument(engine_args[1], "host:port"); + configuration.table = checkAndGetLiteralArgument(engine_args[2], "table"); + configuration.username = checkAndGetLiteralArgument(engine_args[3], "username"); + configuration.password = checkAndGetLiteralArgument(engine_args[4], "password"); if (engine_args.size() >= 6) - configuration.schema = engine_args[5]->as().value.safeGet(); + configuration.schema = checkAndGetLiteralArgument(engine_args[5], "schema"); if (engine_args.size() >= 7) - configuration.on_conflict = engine_args[6]->as().value.safeGet(); + configuration.on_conflict = checkAndGetLiteralArgument(engine_args[6], "on_conflict"); } for (const auto & address : configuration.addresses) context->getRemoteHostFilter().checkHostAndPort(address.first, toString(address.second)); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a7d9641d5c4..bed21a9affc 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -11,14 +11,12 @@ #include -#include #include #include #include #include #include -#include #include #include @@ -27,6 +25,7 @@ #include #include #include +#include #include #include @@ -1051,25 +1050,25 @@ void StorageS3::processNamedCollectionResult(StorageS3Configuration & configurat for (const auto & [arg_name, arg_value] : key_value_args) { if (arg_name == "access_key_id") - configuration.auth_settings.access_key_id = arg_value->as()->value.safeGet(); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(arg_value, "access_key_id"); else if (arg_name == "secret_access_key") - configuration.auth_settings.secret_access_key = arg_value->as()->value.safeGet(); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(arg_value, "secret_access_key"); else if (arg_name == "filename") - configuration.url = std::filesystem::path(configuration.url) / arg_value->as()->value.safeGet(); + configuration.url = std::filesystem::path(configuration.url) / checkAndGetLiteralArgument(arg_value, "filename"); else if (arg_name == "use_environment_credentials") - configuration.auth_settings.use_environment_credentials = arg_value->as()->value.safeGet(); + configuration.auth_settings.use_environment_credentials = checkAndGetLiteralArgument(arg_value, "use_environment_credentials"); else if (arg_name == "max_single_read_retries") - configuration.rw_settings.max_single_read_retries = arg_value->as()->value.safeGet(); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_single_read_retries"); else if (arg_name == "min_upload_part_size") - configuration.rw_settings.max_single_read_retries = arg_value->as()->value.safeGet(); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "min_upload_part_size"); else if (arg_name == "upload_part_size_multiply_factor") - configuration.rw_settings.max_single_read_retries = arg_value->as()->value.safeGet(); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_factor"); else if (arg_name == "upload_part_size_multiply_parts_count_threshold") - configuration.rw_settings.max_single_read_retries = arg_value->as()->value.safeGet(); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_parts_count_threshold"); else if (arg_name == "max_single_part_upload_size") - configuration.rw_settings.max_single_read_retries = arg_value->as()->value.safeGet(); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_single_part_upload_size"); else if (arg_name == "max_connections") - configuration.rw_settings.max_single_read_retries = arg_value->as()->value.safeGet(); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_connections"); else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Unknown key-value argument `{}` for StorageS3, expected: url, [access_key_id, secret_access_key], name of used format and [compression_method].", @@ -1098,22 +1097,22 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); - configuration.url = engine_args[0]->as().value.safeGet(); + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); if (engine_args.size() >= 4) { - configuration.auth_settings.access_key_id = engine_args[1]->as().value.safeGet(); - configuration.auth_settings.secret_access_key = engine_args[2]->as().value.safeGet(); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); } if (engine_args.size() == 3 || engine_args.size() == 5) { - configuration.compression_method = engine_args.back()->as().value.safeGet(); - configuration.format = engine_args[engine_args.size() - 2]->as().value.safeGet(); + configuration.compression_method = checkAndGetLiteralArgument(engine_args.back(), "compression_method"); + configuration.format = checkAndGetLiteralArgument(engine_args[engine_args.size() - 2], "format"); } else if (engine_args.size() != 1) { configuration.compression_method = "auto"; - configuration.format = engine_args.back()->as().value.safeGet(); + configuration.format = checkAndGetLiteralArgument(engine_args.back(), "format"); } } diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 1eb473af80d..a86ed7646b3 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -1,12 +1,10 @@ #include "StorageSQLite.h" #if USE_SQLITE -#include #include #include #include #include -#include #include #include #include @@ -16,6 +14,7 @@ #include #include #include +#include #include #include @@ -168,8 +167,8 @@ void registerStorageSQLite(StorageFactory & factory) for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext()); - const auto database_path = engine_args[0]->as().value.safeGet(); - const auto table_name = engine_args[1]->as().value.safeGet(); + const auto database_path = checkAndGetLiteralArgument(engine_args[0], "database_path"); + const auto table_name = checkAndGetLiteralArgument(engine_args[1], "table_name"); auto sqlite_db = openSQLiteDB(database_path, args.getContext(), /* throw_on_error */!args.attach); diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index ad63499acfa..2f586a3c26c 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -11,7 +10,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index d935d73d03d..b47623db50b 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index d569a81c4a7..c78ab54ea14 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -1,8 +1,6 @@ #include #include -#include -#include #include #include @@ -12,7 +10,6 @@ #include #include #include -#include #include #include @@ -21,11 +18,8 @@ #include -#include - #include -#include #include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index cbec49865a1..15ae23305f3 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1,6 +1,8 @@ #include +#include +#include +#include -#include #include #include #include @@ -21,17 +23,15 @@ #include #include #include - -#include -#include -#include "Common/ThreadStatus.h" -#include -#include "IO/HTTPCommon.h" -#include "IO/ReadWriteBufferFromHTTP.h" - -#include #include #include + +#include +#include +#include +#include + +#include #include #include #include @@ -960,11 +960,11 @@ URLBasedDataSourceConfiguration StorageURL::getConfiguration(ASTs & args, Contex if (header_it != args.end()) args.erase(header_it); - configuration.url = args[0]->as().value.safeGet(); + configuration.url = checkAndGetLiteralArgument(args[0], "url"); if (args.size() > 1) - configuration.format = args[1]->as().value.safeGet(); + configuration.format = checkAndGetLiteralArgument(args[1], "format"); if (args.size() == 3) - configuration.compression_method = args[2]->as().value.safeGet(); + configuration.compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); } if (configuration.format == "auto") diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index f44daf2557e..0b7a1ae75d4 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -1,16 +1,16 @@ -#include "StorageXDBC.h" +#include +#include +#include +#include +#include #include -#include #include #include #include #include #include #include -#include -#include -#include #include #include @@ -173,11 +173,11 @@ namespace BridgeHelperPtr bridge_helper = std::make_shared>(args.getContext(), args.getContext()->getSettingsRef().http_receive_timeout.value, - engine_args[0]->as().value.safeGet()); + checkAndGetLiteralArgument(engine_args[0], "connection_string")); return std::make_shared( args.table_id, - engine_args[1]->as().value.safeGet(), - engine_args[2]->as().value.safeGet(), + checkAndGetLiteralArgument(engine_args[1], "database_name"), + checkAndGetLiteralArgument(engine_args[2], "table_name"), args.columns, args.comment, args.getContext(), diff --git a/src/Storages/checkAndGetLiteralArgument.cpp b/src/Storages/checkAndGetLiteralArgument.cpp new file mode 100644 index 00000000000..fc986e98d6f --- /dev/null +++ b/src/Storages/checkAndGetLiteralArgument.cpp @@ -0,0 +1,37 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +template +T checkAndGetLiteralArgument(const ASTPtr & arg, const String & arg_name) +{ + return checkAndGetLiteralArgument(*arg->as(), arg_name); +} + +template +T checkAndGetLiteralArgument(const ASTLiteral & arg, const String & arg_name) +{ + T res; + if (arg.value.tryGet(res)) + return res; + + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Argument '{}' must be a literal with type {}, got {}", + arg_name, + fieldTypeToString(Field::TypeToEnum::value), + fieldTypeToString(arg.value.getType())); +} + +template String checkAndGetLiteralArgument(const ASTPtr &, const String &); +template UInt64 checkAndGetLiteralArgument(const ASTPtr &, const String &); +template String checkAndGetLiteralArgument(const ASTLiteral &, const String &); + +} diff --git a/src/Storages/checkAndGetLiteralArgument.h b/src/Storages/checkAndGetLiteralArgument.h new file mode 100644 index 00000000000..e18e932b903 --- /dev/null +++ b/src/Storages/checkAndGetLiteralArgument.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +template +T checkAndGetLiteralArgument(const ASTPtr & arg, const String & arg_name); + +template +T checkAndGetLiteralArgument(const ASTLiteral & arg, const String & arg_name); + + + +} diff --git a/src/TableFunctions/Hive/TableFunctionHive.cpp b/src/TableFunctions/Hive/TableFunctionHive.cpp index 99dded030e5..275f0ec6ffa 100644 --- a/src/TableFunctions/Hive/TableFunctionHive.cpp +++ b/src/TableFunctions/Hive/TableFunctionHive.cpp @@ -2,11 +2,9 @@ #if USE_HIVE #include -#include #include #include #include -#include #include #include #include @@ -14,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -44,11 +43,11 @@ namespace DB for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context_); - hive_metastore_url = args[0]->as().value.safeGet(); - hive_database = args[1]->as().value.safeGet(); - hive_table = args[2]->as().value.safeGet(); - table_structure = args[3]->as().value.safeGet(); - partition_by_def = args[4]->as().value.safeGet(); + hive_metastore_url = checkAndGetLiteralArgument(args[0], "hive_url"); + hive_database = checkAndGetLiteralArgument(args[0], "hive_database"); + hive_table = checkAndGetLiteralArgument(args[0], "hive_table"); + table_structure = checkAndGetLiteralArgument(args[0], "structure"); + partition_by_def = checkAndGetLiteralArgument(args[0], "partition_by_keys"); actual_columns = parseColumnsListFromString(table_structure, context_); } diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 7fa3ccda195..e2391787726 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -8,11 +8,10 @@ #include #include +#include #include -#include - #include namespace DB @@ -25,10 +24,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, ContextPtr context) +void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, const ContextPtr &) { - auto ast = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - filename = ast->as().value.safeGet(); + filename = checkAndGetLiteralArgument(arg, "source"); } String ITableFunctionFileLike::getFormatFromFirstArgument() @@ -49,13 +47,13 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context if (args.empty()) throw Exception("Table function '" + getName() + "' requires at least 1 argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + parseFirstArguments(args[0], context); - for (size_t i = 1; i < args.size(); ++i) - args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context); - if (args.size() > 1) - format = args[1]->as().value.safeGet(); + format = checkAndGetLiteralArgument(args[1], "format"); if (format == "auto") format = getFormatFromFirstArgument(); @@ -67,7 +65,7 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context throw Exception("Table function '" + getName() + "' requires 1, 2, 3 or 4 arguments: filename, format (default auto), structure (default auto) and compression method (default auto)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - structure = args[2]->as().value.safeGet(); + structure = checkAndGetLiteralArgument(args[2], "structure"); if (structure.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -75,7 +73,7 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context ast_function->formatForErrorMessage()); if (args.size() == 4) - compression_method = args[3]->as().value.safeGet(); + compression_method = checkAndGetLiteralArgument(args[3], "compression_method"); } StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 88ad75b1018..c2f32eb0aa3 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -20,7 +20,7 @@ public: protected: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - virtual void parseFirstArguments(const ASTPtr & arg, ContextPtr context); + virtual void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context); virtual String getFormatFromFirstArgument(); String filename; diff --git a/src/TableFunctions/TableFunctionDictionary.cpp b/src/TableFunctions/TableFunctionDictionary.cpp index c251b2703e1..54c23cfb64b 100644 --- a/src/TableFunctions/TableFunctionDictionary.cpp +++ b/src/TableFunctions/TableFunctionDictionary.cpp @@ -7,6 +7,7 @@ #include #include +#include #include @@ -35,7 +36,7 @@ void TableFunctionDictionary::parseArguments(const ASTPtr & ast_function, Contex for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - dictionary_name = args[0]->as().value.safeGet(); + dictionary_name = checkAndGetLiteralArgument(args[0], "dictionary_name"); } ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr context) const diff --git a/src/TableFunctions/TableFunctionExecutable.cpp b/src/TableFunctions/TableFunctionExecutable.cpp index dc88cca51e6..b84008f5ac8 100644 --- a/src/TableFunctions/TableFunctionExecutable.cpp +++ b/src/TableFunctions/TableFunctionExecutable.cpp @@ -4,8 +4,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -43,16 +43,16 @@ void TableFunctionExecutable::parseArguments(const ASTPtr & ast_function, Contex for (size_t i = 0; i <= 2; ++i) args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context); - auto scipt_name_with_arguments_value = args[0]->as().value.safeGet(); + auto script_name_with_arguments_value = checkAndGetLiteralArgument(args[0], "script_name_with_arguments_value"); std::vector script_name_with_arguments; - boost::split(script_name_with_arguments, scipt_name_with_arguments_value, [](char c){ return c == ' '; }); + boost::split(script_name_with_arguments, script_name_with_arguments_value, [](char c){ return c == ' '; }); script_name = script_name_with_arguments[0]; script_name_with_arguments.erase(script_name_with_arguments.begin()); arguments = std::move(script_name_with_arguments); - format = args[1]->as().value.safeGet(); - structure = args[2]->as().value.safeGet(); + format = checkAndGetLiteralArgument(args[1], "format"); + structure = checkAndGetLiteralArgument(args[2], "structure"); for (size_t i = 3; i < args.size(); ++i) { diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 507b3406cb8..6f8f0db46a0 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, ContextPtr context) +void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr & context) { if (context->getApplicationType() != Context::ApplicationType::LOCAL) { @@ -29,36 +29,27 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, ContextPtr conte return; } - if (auto opt_name = tryGetIdentifierName(arg)) + const auto * literal = arg->as(); + auto type = literal->value.getType(); + if (type == Field::Types::String) { - if (*opt_name == "stdin") + filename = literal->value.safeGet(); + if (filename == "stdin" || filename == "-") fd = STDIN_FILENO; - else if (*opt_name == "stdout") + else if (filename == "stdout") fd = STDOUT_FILENO; - else if (*opt_name == "stderr") + else if (filename == "stderr") fd = STDERR_FILENO; - else - filename = *opt_name; } - else if (const auto * literal = arg->as()) + else if (type == Field::Types::Int64 || type == Field::Types::UInt64) { - auto type = literal->value.getType(); - if (type == Field::Types::Int64 || type == Field::Types::UInt64) - { - fd = (type == Field::Types::Int64) ? static_cast(literal->value.get()) : static_cast(literal->value.get()); - if (fd < 0) - throw Exception("File descriptor must be non-negative", ErrorCodes::BAD_ARGUMENTS); - } - else if (type == Field::Types::String) - { - filename = literal->value.get(); - if (filename == "-") - fd = STDIN_FILENO; - } - else - throw Exception( - "The first argument of table function '" + getName() + "' mush be path or file descriptor", ErrorCodes::BAD_ARGUMENTS); + fd = (type == Field::Types::Int64) ? literal->value.get() : literal->value.get(); + if (fd < 0) + throw Exception("File descriptor must be non-negative", ErrorCodes::BAD_ARGUMENTS); } + else + throw Exception( + "The first argument of table function '" + getName() + "' mush be path or file descriptor", ErrorCodes::BAD_ARGUMENTS); } String TableFunctionFile::getFormatFromFirstArgument() diff --git a/src/TableFunctions/TableFunctionFile.h b/src/TableFunctions/TableFunctionFile.h index f956043e69a..20ecdb6222c 100644 --- a/src/TableFunctions/TableFunctionFile.h +++ b/src/TableFunctions/TableFunctionFile.h @@ -24,7 +24,7 @@ public: protected: int fd = -1; - void parseFirstArguments(const ASTPtr & arg, ContextPtr context) override; + void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context) override; String getFormatFromFirstArgument() override; private: diff --git a/src/TableFunctions/TableFunctionFormat.cpp b/src/TableFunctions/TableFunctionFormat.cpp index d3ce9627598..d47f8353e18 100644 --- a/src/TableFunctions/TableFunctionFormat.cpp +++ b/src/TableFunctions/TableFunctionFormat.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include @@ -43,8 +44,8 @@ void TableFunctionFormat::parseArguments(const ASTPtr & ast_function, ContextPtr for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - format = args[0]->as().value.safeGet(); - data = args[1]->as().value.safeGet(); + format = checkAndGetLiteralArgument(args[0], "format"); + data = checkAndGetLiteralArgument(args[1], "data"); } ColumnsDescription TableFunctionFormat::getActualTableStructure(ContextPtr context) const diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index ad766c6c66e..083e4a54190 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include @@ -58,20 +59,20 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co } /// Parsing first argument as table structure and creating a sample block - structure = args[0]->as().value.safeGet(); + structure = checkAndGetLiteralArgument(args[0], "structure"); if (args.size() >= 2) { - const Field & value = args[1]->as().value; - if (!value.isNull()) - random_seed = value.safeGet(); + const auto & literal = args[1]->as(); + if (!literal.value.isNull()) + random_seed = checkAndGetLiteralArgument(literal, "random_seed"); } if (args.size() >= 3) - max_string_length = args[2]->as().value.safeGet(); + max_string_length = checkAndGetLiteralArgument(args[2], "max_string_length"); if (args.size() == 4) - max_array_length = args[3]->as().value.safeGet(); + max_array_length = checkAndGetLiteralArgument(args[3], "max_string_length"); } ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextPtr context) const diff --git a/src/TableFunctions/TableFunctionHDFSCluster.cpp b/src/TableFunctions/TableFunctionHDFSCluster.cpp index 80f19cd015a..b5e14a91b91 100644 --- a/src/TableFunctions/TableFunctionHDFSCluster.cpp +++ b/src/TableFunctions/TableFunctionHDFSCluster.cpp @@ -5,8 +5,8 @@ #include #include -#include #include +#include #include #include #include @@ -15,8 +15,6 @@ #include #include #include -#include -#include #include #include "registerTableFunctions.h" @@ -61,7 +59,7 @@ void TableFunctionHDFSCluster::parseArguments(const ASTPtr & ast_function, Conte arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); /// This argument is always the first - cluster_name = args[0]->as().value.safeGet(); + cluster_name = checkAndGetLiteralArgument(args[0], "cluster_name"); if (!context->tryGetCluster(cluster_name)) throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", cluster_name); diff --git a/src/TableFunctions/TableFunctionInput.cpp b/src/TableFunctions/TableFunctionInput.cpp index 0ff56fefb68..0f26cab3683 100644 --- a/src/TableFunctions/TableFunctionInput.cpp +++ b/src/TableFunctions/TableFunctionInput.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -6,6 +5,7 @@ #include #include #include +#include #include #include #include "registerTableFunctions.h" @@ -40,7 +40,7 @@ void TableFunctionInput::parseArguments(const ASTPtr & ast_function, ContextPtr throw Exception("Table function '" + getName() + "' requires exactly 1 argument: structure", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - structure = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context)->as().value.safeGet(); + structure = checkAndGetLiteralArgument(evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context), "structure"); } ColumnsDescription TableFunctionInput::getActualTableStructure(ContextPtr context) const diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 28aed2f03ed..b055e241459 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -58,10 +59,10 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, ContextPtr if (!is_regexp) args[0] = database_ast; - source_database_name_or_regexp = database_ast->as().value.safeGet(); + source_database_name_or_regexp = checkAndGetLiteralArgument(database_ast, "database_name"); args[1] = evaluateConstantExpressionAsLiteral(args[1], context); - source_table_regexp = args[1]->as().value.safeGet(); + source_table_regexp = checkAndGetLiteralArgument(args[1], "table_name_regexp"); } diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp index dea95b86ffd..f5d5a92ec1a 100644 --- a/src/TableFunctions/TableFunctionNull.cpp +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -1,9 +1,9 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -30,7 +30,7 @@ void TableFunctionNull::parseArguments(const ASTPtr & ast_function, ContextPtr c ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (!arguments.empty()) - structure = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[0], context)->as()->value.safeGet(); + structure = checkAndGetLiteralArgument(evaluateConstantExpressionOrIdentifierAsLiteral(arguments[0], context), "structure"); } ColumnsDescription TableFunctionNull::getActualTableStructure(ContextPtr context) const diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index f06831f191e..098756bcd7c 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include #include @@ -13,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -79,7 +80,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr else { auto database_literal = evaluateConstantExpressionOrIdentifierAsLiteral(arg_value, context); - configuration.database = database_literal->as()->value.safeGet(); + configuration.database = checkAndGetLiteralArgument(database_literal, "database"); } } else @@ -113,7 +114,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (is_cluster_function) { args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); - cluster_name = args[arg_num]->as().value.safeGet(); + cluster_name = checkAndGetLiteralArgument(args[arg_num], "cluster_name"); } else { @@ -134,7 +135,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr else { args[arg_num] = evaluateConstantExpressionForDatabaseName(args[arg_num], context); - configuration.database = args[arg_num]->as().value.safeGet(); + configuration.database = checkAndGetLiteralArgument(args[arg_num], "database"); ++arg_num; @@ -149,7 +150,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { std::swap(qualified_name.database, qualified_name.table); args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); - qualified_name.table = args[arg_num]->as().value.safeGet(); + qualified_name.table = checkAndGetLiteralArgument(args[arg_num], "table"); ++arg_num; } } diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index d081ec4319d..101d946a3f9 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include "registerTableFunctions.h" @@ -56,7 +57,7 @@ void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & ar /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. if (args.size() == 4) { - auto second_arg = args[1]->as().value.safeGet(); + auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id"); if (FormatFactory::instance().getAllFormats().contains(second_arg)) args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; @@ -68,7 +69,8 @@ void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & ar /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. else if (args.size() == 3) { - auto second_arg = args[1]->as().value.safeGet(); + + auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id"); if (FormatFactory::instance().getAllFormats().contains(second_arg)) args_to_idx = {{"format", 1}, {"structure", 2}}; else @@ -80,22 +82,22 @@ void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & ar } /// This argument is always the first - s3_configuration.url = args[0]->as().value.safeGet(); + s3_configuration.url = checkAndGetLiteralArgument(args[0], "url"); if (args_to_idx.contains("format")) - s3_configuration.format = args[args_to_idx["format"]]->as().value.safeGet(); + s3_configuration.format = checkAndGetLiteralArgument(args[args_to_idx["format"]], "format"); if (args_to_idx.contains("structure")) - s3_configuration.structure = args[args_to_idx["structure"]]->as().value.safeGet(); + s3_configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); if (args_to_idx.contains("compression_method")) - s3_configuration.compression_method = args[args_to_idx["compression_method"]]->as().value.safeGet(); + s3_configuration.compression_method = checkAndGetLiteralArgument(args[args_to_idx["compression_method"]], "compression_method"); if (args_to_idx.contains("access_key_id")) - s3_configuration.auth_settings.access_key_id = args[args_to_idx["access_key_id"]]->as().value.safeGet(); + s3_configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(args[args_to_idx["access_key_id"]], "access_key_id"); if (args_to_idx.contains("secret_access_key")) - s3_configuration.auth_settings.secret_access_key = args[args_to_idx["secret_access_key"]]->as().value.safeGet(); + s3_configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); } if (s3_configuration.format == "auto") diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 2f558c58352..fab74c07e11 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -3,11 +3,11 @@ #if USE_AWS_S3 #include +#include +#include #include -#include #include -#include #include #include #include @@ -17,7 +17,6 @@ #include #include #include -#include #include #include "registerTableFunctions.h" @@ -65,7 +64,7 @@ void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, Context throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); /// This arguments are always the first - configuration.cluster_name = args[0]->as().value.safeGet(); + configuration.cluster_name = checkAndGetLiteralArgument(args[0], "cluster_name"); if (!context->tryGetCluster(configuration.cluster_name)) throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", configuration.cluster_name); diff --git a/src/TableFunctions/TableFunctionSQLite.cpp b/src/TableFunctions/TableFunctionSQLite.cpp index fb2dc90a1f7..453b135303a 100644 --- a/src/TableFunctions/TableFunctionSQLite.cpp +++ b/src/TableFunctions/TableFunctionSQLite.cpp @@ -10,14 +10,14 @@ #include "registerTableFunctions.h" #include -#include #include -#include #include #include +#include + namespace DB { @@ -73,8 +73,8 @@ void TableFunctionSQLite::parseArguments(const ASTPtr & ast_function, ContextPtr for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - database_path = args[0]->as().value.safeGet(); - remote_table_name = args[1]->as().value.safeGet(); + database_path = checkAndGetLiteralArgument(args[0], "database_path"); + remote_table_name = checkAndGetLiteralArgument(args[0], "table_name"); sqlite_db = openSQLiteDB(database_path, context); } diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index fdc8c4ac911..3baa09a65ea 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include @@ -55,7 +55,7 @@ void registerTableFunctionZeros(TableFunctionFactory & factory) template UInt64 TableFunctionZeros::evaluateArgument(ContextPtr context, ASTPtr & argument) const { - return evaluateConstantExpressionOrIdentifierAsLiteral(argument, context)->as().value.safeGet(); + return checkAndGetLiteralArgument(evaluateConstantExpressionOrIdentifierAsLiteral(argument, context), "length"); } } From 8f53d6634ee88780e53aecac3ee8c37891bc5330 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 23 Jun 2022 20:10:03 +0000 Subject: [PATCH 041/627] Fix Hive table function --- src/TableFunctions/Hive/TableFunctionHive.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/TableFunctions/Hive/TableFunctionHive.cpp b/src/TableFunctions/Hive/TableFunctionHive.cpp index 275f0ec6ffa..12371df4e3c 100644 --- a/src/TableFunctions/Hive/TableFunctionHive.cpp +++ b/src/TableFunctions/Hive/TableFunctionHive.cpp @@ -44,10 +44,10 @@ namespace DB arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context_); hive_metastore_url = checkAndGetLiteralArgument(args[0], "hive_url"); - hive_database = checkAndGetLiteralArgument(args[0], "hive_database"); - hive_table = checkAndGetLiteralArgument(args[0], "hive_table"); - table_structure = checkAndGetLiteralArgument(args[0], "structure"); - partition_by_def = checkAndGetLiteralArgument(args[0], "partition_by_keys"); + hive_database = checkAndGetLiteralArgument(args[1], "hive_database"); + hive_table = checkAndGetLiteralArgument(args[2], "hive_table"); + table_structure = checkAndGetLiteralArgument(args[3], "structure"); + partition_by_def = checkAndGetLiteralArgument(args[4], "partition_by_keys"); actual_columns = parseColumnsListFromString(table_structure, context_); } From 45f94c3d2392135c549357f56281b9dc51063913 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Thu, 23 Jun 2022 14:01:45 -0700 Subject: [PATCH 042/627] Add new files to makefile for thrift 0.16.0 --- contrib/thrift-cmake/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 33c15a700be..e4ed25b27b8 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -32,6 +32,8 @@ set(thriftcpp_SOURCES "${LIBRARY_DIR}/src/thrift/transport/TServerSocket.cpp" "${LIBRARY_DIR}/src/thrift/transport/TTransportUtils.cpp" "${LIBRARY_DIR}/src/thrift/transport/TBufferTransports.cpp" + "${LIBRARY_DIR}/src/thrift/transport/SocketCommon.cpp" + "${LIBRARY_DIR}/src/thrift/transport/TWebSocketServer.cpp" "${LIBRARY_DIR}/src/thrift/server/TConnectedClient.cpp" "${LIBRARY_DIR}/src/thrift/server/TServerFramework.cpp" "${LIBRARY_DIR}/src/thrift/server/TSimpleServer.cpp" From 982fcfc68a5879481542908761d4c5341a8cc908 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 24 Jun 2022 00:54:05 +0200 Subject: [PATCH 043/627] Fix build --- src/Storages/checkAndGetLiteralArgument.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/checkAndGetLiteralArgument.cpp b/src/Storages/checkAndGetLiteralArgument.cpp index fc986e98d6f..333a4541a55 100644 --- a/src/Storages/checkAndGetLiteralArgument.cpp +++ b/src/Storages/checkAndGetLiteralArgument.cpp @@ -32,6 +32,7 @@ T checkAndGetLiteralArgument(const ASTLiteral & arg, const String & arg_name) template String checkAndGetLiteralArgument(const ASTPtr &, const String &); template UInt64 checkAndGetLiteralArgument(const ASTPtr &, const String &); +template bool checkAndGetLiteralArgument(const ASTPtr &, const String &); template String checkAndGetLiteralArgument(const ASTLiteral &, const String &); } From 8b88df8b81b4d52698ce85d72fb4aac33f970d6b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 24 Jun 2022 11:58:12 +0000 Subject: [PATCH 044/627] Fix build --- src/Storages/checkAndGetLiteralArgument.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/checkAndGetLiteralArgument.cpp b/src/Storages/checkAndGetLiteralArgument.cpp index 333a4541a55..16d7b8b46c3 100644 --- a/src/Storages/checkAndGetLiteralArgument.cpp +++ b/src/Storages/checkAndGetLiteralArgument.cpp @@ -26,12 +26,13 @@ T checkAndGetLiteralArgument(const ASTLiteral & arg, const String & arg_name) ErrorCodes::BAD_ARGUMENTS, "Argument '{}' must be a literal with type {}, got {}", arg_name, - fieldTypeToString(Field::TypeToEnum::value), + fieldTypeToString(Field::TypeToEnum>>::value), fieldTypeToString(arg.value.getType())); } template String checkAndGetLiteralArgument(const ASTPtr &, const String &); template UInt64 checkAndGetLiteralArgument(const ASTPtr &, const String &); +template UInt8 checkAndGetLiteralArgument(const ASTPtr &, const String &); template bool checkAndGetLiteralArgument(const ASTPtr &, const String &); template String checkAndGetLiteralArgument(const ASTLiteral &, const String &); From ba69193db1e5de7d1c0e32285cf0dc3728af9435 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 24 Jun 2022 14:00:22 +0000 Subject: [PATCH 045/627] Fix build --- src/Storages/checkAndGetLiteralArgument.cpp | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Storages/checkAndGetLiteralArgument.cpp b/src/Storages/checkAndGetLiteralArgument.cpp index 16d7b8b46c3..3c43ce98920 100644 --- a/src/Storages/checkAndGetLiteralArgument.cpp +++ b/src/Storages/checkAndGetLiteralArgument.cpp @@ -18,16 +18,17 @@ T checkAndGetLiteralArgument(const ASTPtr & arg, const String & arg_name) template T checkAndGetLiteralArgument(const ASTLiteral & arg, const String & arg_name) { - T res; - if (arg.value.tryGet(res)) - return res; + auto requested_type = Field::TypeToEnum>>::value; + auto provided_type = arg.value.getType(); + if (requested_type != provided_type) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Argument '{}' must be a literal with type {}, got {}", + arg_name, + fieldTypeToString(requested_type), + fieldTypeToString(provided_type)); - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Argument '{}' must be a literal with type {}, got {}", - arg_name, - fieldTypeToString(Field::TypeToEnum>>::value), - fieldTypeToString(arg.value.getType())); + return arg.value.safeGet(); } template String checkAndGetLiteralArgument(const ASTPtr &, const String &); From 23022441a686d894eda5faa3fb3a6823bad4b7ae Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 24 Jun 2022 14:01:12 +0000 Subject: [PATCH 046/627] Fix style --- src/Storages/checkAndGetLiteralArgument.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/checkAndGetLiteralArgument.h b/src/Storages/checkAndGetLiteralArgument.h index e18e932b903..6247dc66455 100644 --- a/src/Storages/checkAndGetLiteralArgument.h +++ b/src/Storages/checkAndGetLiteralArgument.h @@ -5,11 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - template T checkAndGetLiteralArgument(const ASTPtr & arg, const String & arg_name); From c124610b5edc50fb5e62b73e16aae00430e40e72 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 00:27:32 +0200 Subject: [PATCH 047/627] Compatible benchmark: ClickHouse --- benchmark/compatible/clickhouse/benchmark.sh | 21 ++++ benchmark/compatible/clickhouse/create.sql | 110 +++++++++++++++++++ benchmark/compatible/clickhouse/queries.sql | 43 ++++++++ benchmark/compatible/clickhouse/run.sh | 20 ++++ 4 files changed, 194 insertions(+) create mode 100755 benchmark/compatible/clickhouse/benchmark.sh create mode 100644 benchmark/compatible/clickhouse/create.sql create mode 100644 benchmark/compatible/clickhouse/queries.sql create mode 100755 benchmark/compatible/clickhouse/run.sh diff --git a/benchmark/compatible/clickhouse/benchmark.sh b/benchmark/compatible/clickhouse/benchmark.sh new file mode 100755 index 00000000000..3ec9a34fd7b --- /dev/null +++ b/benchmark/compatible/clickhouse/benchmark.sh @@ -0,0 +1,21 @@ +#!/bin/bash + +# Install + +curl https://clickhouse.com/ | sh +env DEBIAN_FRONTEND=noninteractive +sudo ./clickhouse install +sudo clickhouse start + +# Load the data + +clickhouse-client < create.sql + +wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gz -d hits.tsv.gz + +clickhouse-client --time --query "INSERT INTO hits FORMAT TSV" < hits.tsv + +# Run the queries + +./run.sh diff --git a/benchmark/compatible/clickhouse/create.sql b/benchmark/compatible/clickhouse/create.sql new file mode 100644 index 00000000000..dda5978f415 --- /dev/null +++ b/benchmark/compatible/clickhouse/create.sql @@ -0,0 +1,110 @@ +CREATE TABLE hits + ( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) + ) + ENGINE = MergeTree; diff --git a/benchmark/compatible/clickhouse/queries.sql b/benchmark/compatible/clickhouse/queries.sql new file mode 100644 index 00000000000..4f4938ecbb7 --- /dev/null +++ b/benchmark/compatible/clickhouse/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits_100m_compatible; +SELECT COUNT(*) FROM hits_100m_compatible WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits_100m_compatible; +SELECT AVG(UserID) FROM hits_100m_compatible; +SELECT COUNT(DISTINCT UserID) FROM hits_100m_compatible; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits_100m_compatible; +SELECT MIN(EventDate), MAX(EventDate) FROM hits_100m_compatible; +SELECT AdvEngineID, COUNT(*) FROM hits_100m_compatible WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits_100m_compatible GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits_100m_compatible GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits_100m_compatible WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits_100m_compatible WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits_100m_compatible WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits_100m_compatible WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits_100m_compatible WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits_100m_compatible GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits_100m_compatible GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits_100m_compatible GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits_100m_compatible GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits_100m_compatible WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits_100m_compatible WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits_100m_compatible WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits_100m_compatible WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits_100m_compatible WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits_100m_compatible WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits_100m_compatible WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits_100m_compatible WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM hits_100m_compatible WHERE URL != '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS key, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits_100m_compatible 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 hits_100m_compatible; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits_100m_compatible WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits_100m_compatible WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits_100m_compatible GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits_100m_compatible GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits_100m_compatible GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM hits_100m_compatible GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/clickhouse/run.sh b/benchmark/compatible/clickhouse/run.sh new file mode 100755 index 00000000000..86bc4c03f72 --- /dev/null +++ b/benchmark/compatible/clickhouse/run.sh @@ -0,0 +1,20 @@ +#!/bin/bash + +TRIES=3 +QUERY_NUM=1 +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + + echo -n "[" + for i in $(seq 1 $TRIES); do + RES=$(clickhouse-client --time --format=Null --query="$query" 2>&1 ||:) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + + echo "${QUERY_NUM},${i},${RES}" >> result.csv + done + echo "]," + + QUERY_NUM=$((QUERY_NUM + 1)) +done From 1d4f069e3e458fba45530349ae000b624a2dcf4d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 00:31:57 +0200 Subject: [PATCH 048/627] Compatible benchmark: ClickHouse --- benchmark/compatible/clickhouse/queries.sql | 86 ++++++++++----------- 1 file changed, 43 insertions(+), 43 deletions(-) diff --git a/benchmark/compatible/clickhouse/queries.sql b/benchmark/compatible/clickhouse/queries.sql index 4f4938ecbb7..439f893a06b 100644 --- a/benchmark/compatible/clickhouse/queries.sql +++ b/benchmark/compatible/clickhouse/queries.sql @@ -1,43 +1,43 @@ -SELECT COUNT(*) FROM hits_100m_compatible; -SELECT COUNT(*) FROM hits_100m_compatible WHERE AdvEngineID != 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits_100m_compatible; -SELECT AVG(UserID) FROM hits_100m_compatible; -SELECT COUNT(DISTINCT UserID) FROM hits_100m_compatible; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits_100m_compatible; -SELECT MIN(EventDate), MAX(EventDate) FROM hits_100m_compatible; -SELECT AdvEngineID, COUNT(*) FROM hits_100m_compatible WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits_100m_compatible GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits_100m_compatible GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits_100m_compatible WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits_100m_compatible WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits_100m_compatible WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits_100m_compatible WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits_100m_compatible WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits_100m_compatible GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits_100m_compatible GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits_100m_compatible GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits_100m_compatible GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits_100m_compatible WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits_100m_compatible WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits_100m_compatible WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits_100m_compatible WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits_100m_compatible WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits_100m_compatible WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits_100m_compatible WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits_100m_compatible WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM hits_100m_compatible WHERE URL != '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS key, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits_100m_compatible 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 hits_100m_compatible; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits_100m_compatible WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits_100m_compatible WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits_100m_compatible GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits_100m_compatible GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits_100m_compatible GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM hits_100m_compatible GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits_100m_compatible WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +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 SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; From 56639806ae3734611448e3348616f4bfbb13a8e7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 06:00:07 +0200 Subject: [PATCH 049/627] Compatible benchmark --- benchmark/compatible/clickhouse/benchmark.sh | 5 +- benchmark/compatible/clickhouse/create.sql | 218 +++++++++--------- .../clickhouse/results/c6a.4xlarge.txt | 47 ++++ benchmark/compatible/duckdb/README.md | 2 + benchmark/compatible/duckdb/benchmark.sh | 18 ++ benchmark/compatible/duckdb/create.sql | 109 +++++++++ benchmark/compatible/duckdb/queries.sql | 43 ++++ benchmark/compatible/duckdb/run.py | 0 .../compatible/mariadb-columnstore/README.md | 9 + .../mariadb-columnstore/benchmark.sh | 22 ++ .../compatible/mariadb-columnstore/create.sql | 108 +++++++++ benchmark/compatible/mariadb/benchmark.sh | 21 ++ benchmark/compatible/mariadb/create.sql | 109 +++++++++ benchmark/compatible/mariadb/queries.sql | 43 ++++ benchmark/compatible/mariadb/run.sh | 12 + benchmark/compatible/monetdb/benchmark.sh | 26 +++ benchmark/compatible/mysql/benchmark.sh | 21 ++ benchmark/compatible/mysql/create.sql | 110 +++++++++ benchmark/compatible/mysql/queries.sql | 43 ++++ benchmark/compatible/mysql/run.sh | 12 + benchmark/compatible/postgresql/benchmark.sh | 18 ++ benchmark/compatible/postgresql/create.sql | 109 +++++++++ benchmark/compatible/postgresql/queries.sql | 43 ++++ benchmark/compatible/postgresql/run.sh | 13 ++ benchmark/compatible/singlestore/benchmark.sh | 40 ++++ benchmark/compatible/singlestore/create.sql | 109 +++++++++ benchmark/compatible/singlestore/queries.sql | 43 ++++ .../singlestore/results/c6a.4xlarge.txt | 45 ++++ benchmark/compatible/singlestore/run.sh | 12 + benchmark/compatible/timescaledb/benchmark.sh | 15 ++ 30 files changed, 1313 insertions(+), 112 deletions(-) create mode 100644 benchmark/compatible/clickhouse/results/c6a.4xlarge.txt create mode 100644 benchmark/compatible/duckdb/README.md create mode 100644 benchmark/compatible/duckdb/benchmark.sh create mode 100644 benchmark/compatible/duckdb/create.sql create mode 100644 benchmark/compatible/duckdb/queries.sql create mode 100644 benchmark/compatible/duckdb/run.py create mode 100644 benchmark/compatible/mariadb-columnstore/README.md create mode 100644 benchmark/compatible/mariadb-columnstore/benchmark.sh create mode 100644 benchmark/compatible/mariadb-columnstore/create.sql create mode 100644 benchmark/compatible/mariadb/benchmark.sh create mode 100644 benchmark/compatible/mariadb/create.sql create mode 100644 benchmark/compatible/mariadb/queries.sql create mode 100644 benchmark/compatible/mariadb/run.sh create mode 100644 benchmark/compatible/monetdb/benchmark.sh create mode 100644 benchmark/compatible/mysql/benchmark.sh create mode 100644 benchmark/compatible/mysql/create.sql create mode 100644 benchmark/compatible/mysql/queries.sql create mode 100644 benchmark/compatible/mysql/run.sh create mode 100644 benchmark/compatible/postgresql/benchmark.sh create mode 100644 benchmark/compatible/postgresql/create.sql create mode 100644 benchmark/compatible/postgresql/queries.sql create mode 100644 benchmark/compatible/postgresql/run.sh create mode 100644 benchmark/compatible/singlestore/benchmark.sh create mode 100644 benchmark/compatible/singlestore/create.sql create mode 100644 benchmark/compatible/singlestore/queries.sql create mode 100644 benchmark/compatible/singlestore/results/c6a.4xlarge.txt create mode 100644 benchmark/compatible/singlestore/run.sh create mode 100644 benchmark/compatible/timescaledb/benchmark.sh diff --git a/benchmark/compatible/clickhouse/benchmark.sh b/benchmark/compatible/clickhouse/benchmark.sh index 3ec9a34fd7b..ee68d2ce738 100755 --- a/benchmark/compatible/clickhouse/benchmark.sh +++ b/benchmark/compatible/clickhouse/benchmark.sh @@ -3,8 +3,7 @@ # Install curl https://clickhouse.com/ | sh -env DEBIAN_FRONTEND=noninteractive -sudo ./clickhouse install +sudo DEBIAN_FRONTEND=noninteractive ./clickhouse install sudo clickhouse start # Load the data @@ -12,7 +11,7 @@ sudo clickhouse start clickhouse-client < create.sql wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gz -d hits.tsv.gz +gzip -d hits.tsv.gz clickhouse-client --time --query "INSERT INTO hits FORMAT TSV" < hits.tsv diff --git a/benchmark/compatible/clickhouse/create.sql b/benchmark/compatible/clickhouse/create.sql index dda5978f415..0297e914a19 100644 --- a/benchmark/compatible/clickhouse/create.sql +++ b/benchmark/compatible/clickhouse/create.sql @@ -1,110 +1,110 @@ CREATE TABLE hits - ( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) - ) - ENGINE = MergeTree; +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +) +ENGINE = MergeTree; diff --git a/benchmark/compatible/clickhouse/results/c6a.4xlarge.txt b/benchmark/compatible/clickhouse/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..afde87dabda --- /dev/null +++ b/benchmark/compatible/clickhouse/results/c6a.4xlarge.txt @@ -0,0 +1,47 @@ +Machine: c6a.4xlarge, EBS 500GB gp2, Ubuntu 22.04. + +Loading time: 475.529 + +[0.027, 0.001, 0.001], +[0.035, 0.015, 0.021], +[0.083, 0.034, 0.033], +[0.171, 0.044, 0.045], +[1.552, 1.495, 1.574], +[1.270, 1.075, 1.063], +[0.045, 0.026, 0.025], +[0.032, 0.016, 0.015], +[0.717, 0.615, 0.607], +[0.843, 0.821, 0.747], +[0.293, 0.219, 0.216], +[0.312, 0.226, 0.235], +[0.804, 0.694, 0.702], +[1.476, 1.047, 1.029], +[1.013, 0.898, 0.911], +[1.043, 0.964, 1.453], +[3.632, 2.715, 2.711], +[1.867, 1.750, 1.714], +[5.187, 4.797, 4.953], +[0.112, 0.068, 0.041], +[8.637, 1.761, 1.212], +[9.902, 0.902, 0.869], +[18.831, 2.067, 1.829], +[41.903, 4.476, 3.486], +[1.801, 0.254, 0.238], +[0.627, 0.214, 0.207], +[2.181, 0.241, 0.246], +[8.868, 0.748, 0.733], +[9.674, 6.891, 5.770], +[2.620, 2.355, 2.368], +[1.395, 0.533, 0.525], +[4.454, 0.730, 0.712], +[5.453, 4.990, 5.922], +[9.955, 3.968, 4.096], +[9.987, 4.035, 4.476], +[1.695, 1.236, 1.241], +[0.142, 0.079, 0.103], +[0.066, 0.033, 0.040], +[0.065, 0.030, 0.033], +[0.246, 0.207, 0.192], +[0.044, 0.019, 0.020], +[0.030, 0.023, 0.012], +[0.030, 0.018, 0.013] diff --git a/benchmark/compatible/duckdb/README.md b/benchmark/compatible/duckdb/README.md new file mode 100644 index 00000000000..00bbea5de76 --- /dev/null +++ b/benchmark/compatible/duckdb/README.md @@ -0,0 +1,2 @@ +DuckDB cannot load parquet file due to OOM. +The only option is to load a CSV file. diff --git a/benchmark/compatible/duckdb/benchmark.sh b/benchmark/compatible/duckdb/benchmark.sh new file mode 100644 index 00000000000..9b7242e2428 --- /dev/null +++ b/benchmark/compatible/duckdb/benchmark.sh @@ -0,0 +1,18 @@ +#!/bin/bash + +# Install + +sudo apt-get update +sudo apt-get install python3-pip +pip install duckdb + +# Load the data + +wget 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +gzip -d hits.csv.gz + +# Run the queries + +./run.expect + +wc -c my-db.duckdb diff --git a/benchmark/compatible/duckdb/create.sql b/benchmark/compatible/duckdb/create.sql new file mode 100644 index 00000000000..1850bffedce --- /dev/null +++ b/benchmark/compatible/duckdb/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/duckdb/queries.sql b/benchmark/compatible/duckdb/queries.sql new file mode 100644 index 00000000000..439f893a06b --- /dev/null +++ b/benchmark/compatible/duckdb/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +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 SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/duckdb/run.py b/benchmark/compatible/duckdb/run.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/benchmark/compatible/mariadb-columnstore/README.md b/benchmark/compatible/mariadb-columnstore/README.md new file mode 100644 index 00000000000..6be1ce43e2b --- /dev/null +++ b/benchmark/compatible/mariadb-columnstore/README.md @@ -0,0 +1,9 @@ +MariaDB ColumnStore failed after 5 minutes of data loading: + +``` +ubuntu@ip-172-31-4-179:~$ time mysql --password="${PASSWORD}" --host 127.0.0.1 test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" +ERROR 1030 (HY000) at line 1: Got error -1 "Internal error < 0 (Not system error)" from storage engine ColumnStore +``` + +They don't have an issue tracker on GitHub, only JIRA. +JIRA requires login, but does not support SSO. diff --git a/benchmark/compatible/mariadb-columnstore/benchmark.sh b/benchmark/compatible/mariadb-columnstore/benchmark.sh new file mode 100644 index 00000000000..98a7a6ccc69 --- /dev/null +++ b/benchmark/compatible/mariadb-columnstore/benchmark.sh @@ -0,0 +1,22 @@ +#!/bin/bash + +# Install + +sudo apt-get update +sudo apt-get install docker.io +sudo docker run -d -p 3306:3306 -e ANALYTICS_ONLY=1 --name mcs_container mariadb/columnstore + +export PASSWORD="tsFgm457%3cj" +sudo docker exec mcs_container mariadb -e "GRANT ALL PRIVILEGES ON *.* TO 'ubuntu'@'%' IDENTIFIED BY '${PASSWORD}';" + +sudo apt install mariadb-client + +mysql --password="${PASSWORD}" --host 127.0.0.1 -e "CREATE DATABASE test" +mysql --password="${PASSWORD}" --host 127.0.0.1 test < create.sql + +# Load the data + +wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +time mysql --password="${PASSWORD}" --host 127.0.0.1 test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" diff --git a/benchmark/compatible/mariadb-columnstore/create.sql b/benchmark/compatible/mariadb-columnstore/create.sql new file mode 100644 index 00000000000..3de1aa0a510 --- /dev/null +++ b/benchmark/compatible/mariadb-columnstore/create.sql @@ -0,0 +1,108 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +) ENGINE=Columnstore; diff --git a/benchmark/compatible/mariadb/benchmark.sh b/benchmark/compatible/mariadb/benchmark.sh new file mode 100644 index 00000000000..1f811c249c7 --- /dev/null +++ b/benchmark/compatible/mariadb/benchmark.sh @@ -0,0 +1,21 @@ +#!/bin/bash + +# Install + +sudo apt-get update +sudo apt-get install mariadb-server +sudo bash -c "echo -e '[mysql]\nlocal-infile=1\n\n[mysqld]\nlocal-infile=1\n' > /etc/mysql/conf.d/local_infile.cnf" +sudo service mariadb restart + +# Load the data + +wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +sudo mysql -e "CREATE DATABASE test" +sudo mysql test < create.sql +time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" + +# 2:23:45 elapsed + +./run.sh | tee log.txt diff --git a/benchmark/compatible/mariadb/create.sql b/benchmark/compatible/mariadb/create.sql new file mode 100644 index 00000000000..1850bffedce --- /dev/null +++ b/benchmark/compatible/mariadb/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/mariadb/queries.sql b/benchmark/compatible/mariadb/queries.sql new file mode 100644 index 00000000000..439f893a06b --- /dev/null +++ b/benchmark/compatible/mariadb/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +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 SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/mariadb/run.sh b/benchmark/compatible/mariadb/run.sh new file mode 100644 index 00000000000..31f31682f32 --- /dev/null +++ b/benchmark/compatible/mariadb/run.sh @@ -0,0 +1,12 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + for i in $(seq 1 $TRIES); do + sudo mysql test -vvv -e "${query}" + done; +done; diff --git a/benchmark/compatible/monetdb/benchmark.sh b/benchmark/compatible/monetdb/benchmark.sh new file mode 100644 index 00000000000..9e800f04cd1 --- /dev/null +++ b/benchmark/compatible/monetdb/benchmark.sh @@ -0,0 +1,26 @@ +#!/bin/bash + +# Install + +echo "deb https://dev.monetdb.org/downloads/deb/ $(lsb_release -cs) monetdb" | sudo tee /etc/apt/sources.list.d/monetdb.list + +sudo wget --output-document=/etc/apt/trusted.gpg.d/monetdb.gpg https://www.monetdb.org/downloads/MonetDB-GPG-KEY.gpg +sudo apt-get update +sudo apt-get install monetdb5-sql monetdb-client + +sudo monetdbd create /var/lib/monetdb +sudo monetdbd start /var/lib/monetdb + +sudo monetdb create test +sudo monetdb release test + +.monetdb + +# Load the data + +wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +sudo mysql -e "CREATE DATABASE test" +sudo mysql test < create.sql +sudo time mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" diff --git a/benchmark/compatible/mysql/benchmark.sh b/benchmark/compatible/mysql/benchmark.sh new file mode 100644 index 00000000000..388afb66248 --- /dev/null +++ b/benchmark/compatible/mysql/benchmark.sh @@ -0,0 +1,21 @@ +#!/bin/bash + +# Install + +sudo apt-get update +sudo apt-get install mysql-server-8.0 +sudo bash -c "echo -e '[mysql]\nlocal-infile=1\n\n[mysqld]\nlocal-infile=1\n' > /etc/mysql/conf.d/local_infile.cnf" +sudo service mysql restart + +# Load the data + +wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +sudo mysql -e "CREATE DATABASE test" +sudo mysql test < create.sql +time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" + +# 2:37:52 elapsed + +./run.sh | tee log.txt diff --git a/benchmark/compatible/mysql/create.sql b/benchmark/compatible/mysql/create.sql new file mode 100644 index 00000000000..7c1b36b4f4a --- /dev/null +++ b/benchmark/compatible/mysql/create.sql @@ -0,0 +1,110 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +) +ENGINE = InnoDB; diff --git a/benchmark/compatible/mysql/queries.sql b/benchmark/compatible/mysql/queries.sql new file mode 100644 index 00000000000..439f893a06b --- /dev/null +++ b/benchmark/compatible/mysql/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +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 SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/mysql/run.sh b/benchmark/compatible/mysql/run.sh new file mode 100644 index 00000000000..31f31682f32 --- /dev/null +++ b/benchmark/compatible/mysql/run.sh @@ -0,0 +1,12 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + for i in $(seq 1 $TRIES); do + sudo mysql test -vvv -e "${query}" + done; +done; diff --git a/benchmark/compatible/postgresql/benchmark.sh b/benchmark/compatible/postgresql/benchmark.sh new file mode 100644 index 00000000000..68c51bcf791 --- /dev/null +++ b/benchmark/compatible/postgresql/benchmark.sh @@ -0,0 +1,18 @@ +#!/bin/bash + +sudo apt-get update +sudo apt-get install postgresql-common +sudo apt-get install postgresql-14 + +wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz +chmod 777 ~ hits.tsv + +sudo -u postgres psql -t -c 'CREATE DATABASE test' +sudo -u postgres psql test -t < create.sql +sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" + +# COPY 99997497 +# Time: 2341543.463 ms (39:01.543) + +./run.sh | tee log.txt diff --git a/benchmark/compatible/postgresql/create.sql b/benchmark/compatible/postgresql/create.sql new file mode 100644 index 00000000000..1850bffedce --- /dev/null +++ b/benchmark/compatible/postgresql/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/postgresql/queries.sql b/benchmark/compatible/postgresql/queries.sql new file mode 100644 index 00000000000..439f893a06b --- /dev/null +++ b/benchmark/compatible/postgresql/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +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 SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/postgresql/run.sh b/benchmark/compatible/postgresql/run.sh new file mode 100644 index 00000000000..ad95151ab55 --- /dev/null +++ b/benchmark/compatible/postgresql/run.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + echo "$query"; + for i in $(seq 1 $TRIES); do + sudo -u postgres psql test -t -c '\timing' -c "$query" | grep 'Time' + done; +done; diff --git a/benchmark/compatible/singlestore/benchmark.sh b/benchmark/compatible/singlestore/benchmark.sh new file mode 100644 index 00000000000..f00a39a9226 --- /dev/null +++ b/benchmark/compatible/singlestore/benchmark.sh @@ -0,0 +1,40 @@ +#!/bin/bash + +# Install + +sudo apt-get update +sudo apt-get install docker.io + +export LICENSE_KEY="BDA4OGMxMGFlNDRkYTQ0MmU4N2NkYjk4Y2MwYTUxMTQ5AAAAAAAAAAAEAAAAAAAAACgwNAIYTJwt51SEitrak4T9P7TyYzWzIRstlokzAhgy+cgwXnsXTU9gzedJ/ztTg1TPdc4jrlQAAA==" +export ROOT_PASSWORD="H@I}xfqKsw}[wfG,oLpH" + +sudo docker run -i --init \ + --name memsql-ciab \ + -e LICENSE_KEY="${LICENSE_KEY}" \ + -e ROOT_PASSWORD="${ROOT_PASSWORD}" \ + -p 3306:3306 -p 8080:8080 \ + memsql/cluster-in-a-box + +sudo docker start memsql-ciab + +sudo docker exec -it memsql-ciab memsql -p"${ROOT_PASSWORD}" + +# Load the data + +wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz +sudo docker cp hits.tsv memsql-ciab:/ + +sudo docker exec -it memsql-ciab memsql -p"${ROOT_PASSWORD}" -e "CREATE DATABASE test" +sudo docker exec memsql-ciab memsql -p"${ROOT_PASSWORD}" --database=test -e "USE test; $(cat create.sql)" +time sudo docker exec -it memsql-ciab memsql -vvv -p"${ROOT_PASSWORD}" --database=test -e "LOAD DATA INFILE '/hits.tsv' INTO TABLE test.hits" + +# Query OK, 99997497 rows affected (11 min 30.11 sec) + +./run.sh 2>&1 | tee log.txt + +cat log.txt | + grep -P 'rows? in set|^ERROR' result.txt | + sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | + awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/singlestore/create.sql b/benchmark/compatible/singlestore/create.sql new file mode 100644 index 00000000000..84dc14ea22f --- /dev/null +++ b/benchmark/compatible/singlestore/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + SORT KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/singlestore/queries.sql b/benchmark/compatible/singlestore/queries.sql new file mode 100644 index 00000000000..439f893a06b --- /dev/null +++ b/benchmark/compatible/singlestore/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +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 SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/singlestore/results/c6a.4xlarge.txt b/benchmark/compatible/singlestore/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..ffbf96d9904 --- /dev/null +++ b/benchmark/compatible/singlestore/results/c6a.4xlarge.txt @@ -0,0 +1,45 @@ +Load time: 11 min 30.11 sec + +[0.09,0.00,0.00], +[0.23,0.04,0.01], +[0.47,0.15,0.15], +[0.63,0.09,0.08], +[1.65,1.23,1.20], +[7.96,2.79,2.63], +[0.10,0.00,0.00], +[0.17,0.02,0.02], +[1.90,1.40,1.39], +[4.79,3.52,3.48], +[0.94,0.22,0.23], +[0.89,0.24,0.23], +[5.82,2.26,2.25], +[6.97,4.62,4.66], +[3.05,2.28,2.31], +[3.92,2.70,2.28], +[5.83,4.48,4.42], +[4.76,4.13,4.12], +[14.03,null,null], +[0.57,0.05,0.04], +[18.18,1.74,1.94], +[20.85,2.17,0.98], +[31.98,3.12,1.22], +[78.96,3.35,108.85], +[2.82,0.39,0.32], +[1.83,0.44,0.35], +[2.81,0.33,0.32], +[18.33,2.57,1.15], +[null,null,null], +[3.56,2.40,2.40], +[3.83,1.11,1.11], +[7.35,1.73,1.70], +[null,null,null], +[null,null,null], +[null,null,null], +[2.53,1.92,1.84], +[0.92,0.23,0.19], +[0.84,0.15,0.08], +[0.70,0.05,0.05], +[3.12,0.38,0.36], +[0.29,0.03,0.03], +[0.22,0.06,0.02], +[0.27,0.11,0.12] diff --git a/benchmark/compatible/singlestore/run.sh b/benchmark/compatible/singlestore/run.sh new file mode 100644 index 00000000000..16cc104e712 --- /dev/null +++ b/benchmark/compatible/singlestore/run.sh @@ -0,0 +1,12 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + for i in $(seq 1 $TRIES); do + sudo docker exec memsql-ciab memsql -vvv -p"${ROOT_PASSWORD}" --database=test -e "USE test; ${query}" + done; +done; diff --git a/benchmark/compatible/timescaledb/benchmark.sh b/benchmark/compatible/timescaledb/benchmark.sh new file mode 100644 index 00000000000..be202c6a0af --- /dev/null +++ b/benchmark/compatible/timescaledb/benchmark.sh @@ -0,0 +1,15 @@ +#!/bin/bash + +# Install + +sudo apt-get update +sudo apt install gnupg postgresql-common apt-transport-https lsb-release wget +sudo /usr/share/postgresql-common/pgdg/apt.postgresql.org.sh +sudo bash -c 'echo "deb https://packagecloud.io/timescale/timescaledb/ubuntu/ $(lsb_release -c -s) main" > /etc/apt/sources.list.d/timescaledb.list' +wget --quiet -O - https://packagecloud.io/timescale/timescaledb/gpgkey | sudo apt-key add - +sudo apt-get update +sudo apt install timescaledb-2-postgresql-14 +sudo systemctl restart postgresql + +sudo su postgres -c psql -c "CREATE DATABASE test" +sudo su postgres -c psql test -c "CREATE EXTENSION IF NOT EXISTS timescaledb" From ba5d9fad079ef019417f434ee058a102547dd110 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 07:31:49 +0200 Subject: [PATCH 050/627] Compatible benchmark --- benchmark/compatible/clickhouse/benchmark.sh | 2 + .../clickhouse/results/c6a.4xlarge.txt | 3 +- benchmark/compatible/duckdb/README.md | 2 +- benchmark/compatible/heavyai/benchmark.sh | 40 +++++++ benchmark/compatible/heavyai/create.sql | 108 ++++++++++++++++++ benchmark/compatible/mariadb/benchmark.sh | 2 +- benchmark/compatible/monetdb/README.md | 1 + benchmark/compatible/monetdb/benchmark.sh | 11 -- benchmark/compatible/mysql/benchmark.sh | 2 +- benchmark/compatible/postgresql/benchmark.sh | 2 +- benchmark/compatible/questdb/benchmark.sh | 12 ++ benchmark/compatible/questdb/create.sql | 108 ++++++++++++++++++ benchmark/compatible/singlestore/benchmark.sh | 4 + .../singlestore/results/c6a.4xlarge.txt | 1 + benchmark/compatible/timescaledb/benchmark.sh | 13 ++- benchmark/compatible/timescaledb/create.sql | 108 ++++++++++++++++++ 16 files changed, 401 insertions(+), 18 deletions(-) create mode 100644 benchmark/compatible/heavyai/benchmark.sh create mode 100644 benchmark/compatible/heavyai/create.sql create mode 100644 benchmark/compatible/monetdb/README.md create mode 100644 benchmark/compatible/questdb/benchmark.sh create mode 100644 benchmark/compatible/questdb/create.sql create mode 100644 benchmark/compatible/timescaledb/create.sql diff --git a/benchmark/compatible/clickhouse/benchmark.sh b/benchmark/compatible/clickhouse/benchmark.sh index ee68d2ce738..347cc264abf 100755 --- a/benchmark/compatible/clickhouse/benchmark.sh +++ b/benchmark/compatible/clickhouse/benchmark.sh @@ -18,3 +18,5 @@ clickhouse-client --time --query "INSERT INTO hits FORMAT TSV" < hits.tsv # Run the queries ./run.sh + +du -bcs /var/lib/clickhouse diff --git a/benchmark/compatible/clickhouse/results/c6a.4xlarge.txt b/benchmark/compatible/clickhouse/results/c6a.4xlarge.txt index afde87dabda..be9ca59be37 100644 --- a/benchmark/compatible/clickhouse/results/c6a.4xlarge.txt +++ b/benchmark/compatible/clickhouse/results/c6a.4xlarge.txt @@ -1,6 +1,7 @@ Machine: c6a.4xlarge, EBS 500GB gp2, Ubuntu 22.04. -Loading time: 475.529 +Loading time: 475.529 seconds +Dataset size: 14 345 515 782 bytes [0.027, 0.001, 0.001], [0.035, 0.015, 0.021], diff --git a/benchmark/compatible/duckdb/README.md b/benchmark/compatible/duckdb/README.md index 00bbea5de76..d2d7b22c81b 100644 --- a/benchmark/compatible/duckdb/README.md +++ b/benchmark/compatible/duckdb/README.md @@ -1,2 +1,2 @@ DuckDB cannot load parquet file due to OOM. -The only option is to load a CSV file. +The only option is to load a CSV file, but sometimes it also fails with OOM. diff --git a/benchmark/compatible/heavyai/benchmark.sh b/benchmark/compatible/heavyai/benchmark.sh new file mode 100644 index 00000000000..6ffdece841f --- /dev/null +++ b/benchmark/compatible/heavyai/benchmark.sh @@ -0,0 +1,40 @@ +#!/bin/bash + +# Install + +sudo apt update +sudo apt install default-jre-headless +sudo apt install apt-transport-https +sudo useradd -U -m heavyai +sudo curl https://releases.heavy.ai/GPG-KEY-heavyai | sudo apt-key add - +echo "deb https://releases.heavy.ai/os/apt/ stable cpu" | sudo tee /etc/apt/sources.list.d/heavyai.list +sudo apt update +sudo apt install heavyai + +export HEAVYAI_USER=heavyai +export HEAVYAI_GROUP=heavyai +export HEAVYAI_STORAGE=/var/lib/heavyai +export HEAVYAI_PATH=/opt/heavyai +export HEAVYAI_LOG=/var/lib/heavyai/data/mapd_log + +cd $HEAVYAI_PATH/systemd +./install_heavy_systemd.sh + +# Press Enter multiple times. + +sudo systemctl start heavydb +sudo systemctl enable heavydb + +# Load the data + +wget 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +gzip -d hits.csv.gz +chmod 777 ~ hits.csv + +sudo bash -c "echo 'allowed-import-paths = [\"/home/ubuntu/\"]' > /var/lib/heavyai/heavy.conf_" +sudo bash -c "cat /var/lib/heavyai/heavy.conf >> /var/lib/heavyai/heavy.conf_" +sudo bash -c "mv /var/lib/heavyai/heavy.conf_ /var/lib/heavyai/heavy.conf && chown heavyai /var/lib/heavyai/heavy.conf" +sudo systemctl restart heavydb + +/opt/heavyai/bin/heavysql -t -p HyperInteractive < create.sql +time /opt/heavyai/bin/heavysql -t -p HyperInteractive <<< "COPY hits FROM '$(pwd)/hits.csv' WITH (HEADER = 'false');" diff --git a/benchmark/compatible/heavyai/create.sql b/benchmark/compatible/heavyai/create.sql new file mode 100644 index 00000000000..41c961c00fc --- /dev/null +++ b/benchmark/compatible/heavyai/create.sql @@ -0,0 +1,108 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +); diff --git a/benchmark/compatible/mariadb/benchmark.sh b/benchmark/compatible/mariadb/benchmark.sh index 1f811c249c7..6dbcb4b11d9 100644 --- a/benchmark/compatible/mariadb/benchmark.sh +++ b/benchmark/compatible/mariadb/benchmark.sh @@ -18,4 +18,4 @@ time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" # 2:23:45 elapsed -./run.sh | tee log.txt +./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/monetdb/README.md b/benchmark/compatible/monetdb/README.md new file mode 100644 index 00000000000..c9c0747c8f9 --- /dev/null +++ b/benchmark/compatible/monetdb/README.md @@ -0,0 +1 @@ +MonetDB failed to install. diff --git a/benchmark/compatible/monetdb/benchmark.sh b/benchmark/compatible/monetdb/benchmark.sh index 9e800f04cd1..b662bb5f59f 100644 --- a/benchmark/compatible/monetdb/benchmark.sh +++ b/benchmark/compatible/monetdb/benchmark.sh @@ -13,14 +13,3 @@ sudo monetdbd start /var/lib/monetdb sudo monetdb create test sudo monetdb release test - -.monetdb - -# Load the data - -wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -sudo mysql -e "CREATE DATABASE test" -sudo mysql test < create.sql -sudo time mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" diff --git a/benchmark/compatible/mysql/benchmark.sh b/benchmark/compatible/mysql/benchmark.sh index 388afb66248..1a58b801f55 100644 --- a/benchmark/compatible/mysql/benchmark.sh +++ b/benchmark/compatible/mysql/benchmark.sh @@ -18,4 +18,4 @@ time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" # 2:37:52 elapsed -./run.sh | tee log.txt +./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/postgresql/benchmark.sh b/benchmark/compatible/postgresql/benchmark.sh index 68c51bcf791..ab7e7bf422c 100644 --- a/benchmark/compatible/postgresql/benchmark.sh +++ b/benchmark/compatible/postgresql/benchmark.sh @@ -15,4 +15,4 @@ sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" # COPY 99997497 # Time: 2341543.463 ms (39:01.543) -./run.sh | tee log.txt +./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/questdb/benchmark.sh b/benchmark/compatible/questdb/benchmark.sh new file mode 100644 index 00000000000..b40df3e8b52 --- /dev/null +++ b/benchmark/compatible/questdb/benchmark.sh @@ -0,0 +1,12 @@ +#!/bin/bash + +# Install + +wget https://github.com/questdb/questdb/releases/download/6.4.1/questdb-6.4.1-rt-linux-amd64.tar.gz +tar xf questdb*.tar.gz +questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start + +# Import the data + +curl -G --data-urlencode "query=$(cat create.sql)" 'http://localhost:9000/exec?timings=true' +time curl -F data=@hits.csv 'http://localhost:9000/imp?name=hits' diff --git a/benchmark/compatible/questdb/create.sql b/benchmark/compatible/questdb/create.sql new file mode 100644 index 00000000000..3194a305b18 --- /dev/null +++ b/benchmark/compatible/questdb/create.sql @@ -0,0 +1,108 @@ +CREATE TABLE hits +( + WatchID long, + JavaEnable int, + Title string, + GoodEvent int, + EventTime timestamp, + Eventdate date, + CounterID int, + ClientIP int, + RegionID int, + UserID long, + CounterClass int, + OS int, + UserAgent int, + URL string, + Referer string, + IsRefresh int, + RefererCategoryID int, + RefererRegionID int, + URLCategoryID int, + URLRegionID int, + ResolutionWidth int, + ResolutionHeight int, + ResolutionDepth int, + FlashMajor int, + FlashMinor int, + FlashMinor2 string, + NetMajor int, + NetMinor int, + UserAgentMajor int, + UserAgentMinor string, + CookieEnable int, + JavascriptEnable int, + IsMobile int, + MobilePhone int, + MobilePhoneModel string, + Params string, + IPNetworkID int, + TraficSourceID int, + SearchEngineID int, + SearchPhrase string, + AdvEngineID int, + IsArtifical int, + WindowClientWidth int, + WindowClientHeight int, + ClientTimeZone int, + ClientEventTime timestamp, + SilverlightVersion1 int, + SilverlightVersion2 int, + SilverlightVersion3 int, + SilverlightVersion4 int, + PageCharset string, + CodeVersion int, + IsLink int, + IsDownload int, + IsNotBounce int, + FUniqID long, + OriginalURL string, + HID int, + IsOldCounter int, + IsEvent int, + IsParameter int, + DontCountHits int, + WithHash int, + HitColor string, + LocalEventTime timestamp, + Age int, + Sex int, + Income int, + Interests int, + Robotness int, + RemoteIP int, + WindowName int, + OpenerName int, + HistoryLength int, + BrowserLanguage string, + BrowserCountry string, + SocialNetwork string, + SocialAction string, + HTTPError int, + SendTiming int, + DNSTiming int, + ConnectTiming int, + ResponseStartTiming int, + ResponseEndTiming int, + FetchTiming int, + SocialSourceNetworkID int, + SocialSourcePage string, + ParamPrice long, + ParamOrderID string, + ParamCurrency string, + ParamCurrencyID int, + OpenstatServiceName string, + OpenstatCampaignID string, + OpenstatAdID string, + OpenstatSourceID string, + UTMSource string, + UTMMedium string, + UTMCampaign string, + UTMContent string, + UTMTerm string, + FromTag string, + HasGCLID int, + RefererHash long, + URLHash long, + CLID int +) timestamp (EventTime); diff --git a/benchmark/compatible/singlestore/benchmark.sh b/benchmark/compatible/singlestore/benchmark.sh index f00a39a9226..c6a10866373 100644 --- a/benchmark/compatible/singlestore/benchmark.sh +++ b/benchmark/compatible/singlestore/benchmark.sh @@ -33,6 +33,10 @@ time sudo docker exec -it memsql-ciab memsql -vvv -p"${ROOT_PASSWORD}" --databas ./run.sh 2>&1 | tee log.txt +sudo docker exec memsql-ciab du -bcs /var/lib/memsql + +# 29836263469 bytes + cat log.txt | grep -P 'rows? in set|^ERROR' result.txt | sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | diff --git a/benchmark/compatible/singlestore/results/c6a.4xlarge.txt b/benchmark/compatible/singlestore/results/c6a.4xlarge.txt index ffbf96d9904..f7c4aaf7a73 100644 --- a/benchmark/compatible/singlestore/results/c6a.4xlarge.txt +++ b/benchmark/compatible/singlestore/results/c6a.4xlarge.txt @@ -1,4 +1,5 @@ Load time: 11 min 30.11 sec +Data size: 29 836 263 469 bytes [0.09,0.00,0.00], [0.23,0.04,0.01], diff --git a/benchmark/compatible/timescaledb/benchmark.sh b/benchmark/compatible/timescaledb/benchmark.sh index be202c6a0af..4b1b779385b 100644 --- a/benchmark/compatible/timescaledb/benchmark.sh +++ b/benchmark/compatible/timescaledb/benchmark.sh @@ -9,7 +9,16 @@ sudo bash -c 'echo "deb https://packagecloud.io/timescale/timescaledb/ubuntu/ $( wget --quiet -O - https://packagecloud.io/timescale/timescaledb/gpgkey | sudo apt-key add - sudo apt-get update sudo apt install timescaledb-2-postgresql-14 +sudo bash -c "echo \"shared_preload_libraries = 'timescaledb'\" >> /etc/postgresql/14/main/postgresql.conf" sudo systemctl restart postgresql -sudo su postgres -c psql -c "CREATE DATABASE test" -sudo su postgres -c psql test -c "CREATE EXTENSION IF NOT EXISTS timescaledb" +sudo -u postgres psql -c "CREATE DATABASE test" +sudo -u postgres psql test -c "CREATE EXTENSION IF NOT EXISTS timescaledb" + +sudo -u postgres psql test < create.sql +sudo -u postgres psql test -c "SELECT create_hypertable('hits', 'eventtime')" +sudo -u postgres psql test -c "CREATE INDEX ix_counterid ON hits (counterid)" +sudo -u postgres psql test -c "ALTER TABLE hits SET (timescaledb.compress, timescaledb.compress_orderby = 'counterid, eventdate, userid, eventtime')" +sudo -u postgres psql test -c "SELECT add_compression_policy('hits', INTERVAL '1s')" + +sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" diff --git a/benchmark/compatible/timescaledb/create.sql b/benchmark/compatible/timescaledb/create.sql new file mode 100644 index 00000000000..41c961c00fc --- /dev/null +++ b/benchmark/compatible/timescaledb/create.sql @@ -0,0 +1,108 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +); From 9ab973a19c8e0bc06416e6806c5bcf792120918e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 07:59:40 +0200 Subject: [PATCH 051/627] Compatible benchmark --- benchmark/compatible/heavyai/benchmark.sh | 4 ++ benchmark/compatible/heavyai/queries.sql | 43 +++++++++++++++++++ benchmark/compatible/heavyai/run.sh | 13 ++++++ benchmark/compatible/questdb/benchmark.sh | 5 +++ benchmark/compatible/questdb/create.sql | 2 +- benchmark/compatible/timescaledb/benchmark.sh | 10 +++++ benchmark/compatible/timescaledb/queries.sql | 43 +++++++++++++++++++ benchmark/compatible/timescaledb/run.sh | 13 ++++++ 8 files changed, 132 insertions(+), 1 deletion(-) create mode 100644 benchmark/compatible/heavyai/queries.sql create mode 100644 benchmark/compatible/heavyai/run.sh create mode 100644 benchmark/compatible/timescaledb/queries.sql create mode 100644 benchmark/compatible/timescaledb/run.sh diff --git a/benchmark/compatible/heavyai/benchmark.sh b/benchmark/compatible/heavyai/benchmark.sh index 6ffdece841f..a1b53a998e8 100644 --- a/benchmark/compatible/heavyai/benchmark.sh +++ b/benchmark/compatible/heavyai/benchmark.sh @@ -38,3 +38,7 @@ sudo systemctl restart heavydb /opt/heavyai/bin/heavysql -t -p HyperInteractive < create.sql time /opt/heavyai/bin/heavysql -t -p HyperInteractive <<< "COPY hits FROM '$(pwd)/hits.csv' WITH (HEADER = 'false');" + +# Loaded: 99997497 recs, Rejected: 0 recs in 572.633000 secs + +./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/heavyai/queries.sql b/benchmark/compatible/heavyai/queries.sql new file mode 100644 index 00000000000..439f893a06b --- /dev/null +++ b/benchmark/compatible/heavyai/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +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 SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/heavyai/run.sh b/benchmark/compatible/heavyai/run.sh new file mode 100644 index 00000000000..2688e576e15 --- /dev/null +++ b/benchmark/compatible/heavyai/run.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + echo "$query"; + for i in $(seq 1 $TRIES); do + /opt/heavyai/bin/heavysql -t -p HyperInteractive <<< "${query}" + done; +done; diff --git a/benchmark/compatible/questdb/benchmark.sh b/benchmark/compatible/questdb/benchmark.sh index b40df3e8b52..b4433122ddd 100644 --- a/benchmark/compatible/questdb/benchmark.sh +++ b/benchmark/compatible/questdb/benchmark.sh @@ -8,5 +8,10 @@ questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start # Import the data +wget 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +gzip -d hits.csv.gz + curl -G --data-urlencode "query=$(cat create.sql)" 'http://localhost:9000/exec?timings=true' time curl -F data=@hits.csv 'http://localhost:9000/imp?name=hits' + +# diff --git a/benchmark/compatible/questdb/create.sql b/benchmark/compatible/questdb/create.sql index 3194a305b18..5c0f777a8b2 100644 --- a/benchmark/compatible/questdb/create.sql +++ b/benchmark/compatible/questdb/create.sql @@ -105,4 +105,4 @@ CREATE TABLE hits RefererHash long, URLHash long, CLID int -) timestamp (EventTime); +); diff --git a/benchmark/compatible/timescaledb/benchmark.sh b/benchmark/compatible/timescaledb/benchmark.sh index 4b1b779385b..89e748f0a3a 100644 --- a/benchmark/compatible/timescaledb/benchmark.sh +++ b/benchmark/compatible/timescaledb/benchmark.sh @@ -15,6 +15,12 @@ sudo systemctl restart postgresql sudo -u postgres psql -c "CREATE DATABASE test" sudo -u postgres psql test -c "CREATE EXTENSION IF NOT EXISTS timescaledb" +# Import the data + +wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz +chmod 777 ~ hits.tsv + sudo -u postgres psql test < create.sql sudo -u postgres psql test -c "SELECT create_hypertable('hits', 'eventtime')" sudo -u postgres psql test -c "CREATE INDEX ix_counterid ON hits (counterid)" @@ -22,3 +28,7 @@ sudo -u postgres psql test -c "ALTER TABLE hits SET (timescaledb.compress, times sudo -u postgres psql test -c "SELECT add_compression_policy('hits', INTERVAL '1s')" sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" + +# 1619875.288 seconds (26:59.875) + +./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/timescaledb/queries.sql b/benchmark/compatible/timescaledb/queries.sql new file mode 100644 index 00000000000..439f893a06b --- /dev/null +++ b/benchmark/compatible/timescaledb/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +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 SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/timescaledb/run.sh b/benchmark/compatible/timescaledb/run.sh new file mode 100644 index 00000000000..ad95151ab55 --- /dev/null +++ b/benchmark/compatible/timescaledb/run.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + echo "$query"; + for i in $(seq 1 $TRIES); do + sudo -u postgres psql test -t -c '\timing' -c "$query" | grep 'Time' + done; +done; From c5a1f70291ca30d20bd6cf3768fadd1f0d83cb92 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 08:04:56 +0200 Subject: [PATCH 052/627] Compatible benchmark --- benchmark/compatible/duckdb/benchmark.sh | 0 benchmark/compatible/heavyai/benchmark.sh | 0 benchmark/compatible/heavyai/run.sh | 0 benchmark/compatible/mariadb-columnstore/benchmark.sh | 0 benchmark/compatible/mariadb/benchmark.sh | 0 benchmark/compatible/mariadb/run.sh | 0 benchmark/compatible/monetdb/benchmark.sh | 0 benchmark/compatible/mysql/benchmark.sh | 0 benchmark/compatible/mysql/run.sh | 0 benchmark/compatible/postgresql/benchmark.sh | 0 benchmark/compatible/postgresql/run.sh | 0 benchmark/compatible/questdb/benchmark.sh | 0 benchmark/compatible/singlestore/benchmark.sh | 0 benchmark/compatible/singlestore/run.sh | 0 benchmark/compatible/timescaledb/benchmark.sh | 0 benchmark/compatible/timescaledb/run.sh | 0 16 files changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 benchmark/compatible/duckdb/benchmark.sh mode change 100644 => 100755 benchmark/compatible/heavyai/benchmark.sh mode change 100644 => 100755 benchmark/compatible/heavyai/run.sh mode change 100644 => 100755 benchmark/compatible/mariadb-columnstore/benchmark.sh mode change 100644 => 100755 benchmark/compatible/mariadb/benchmark.sh mode change 100644 => 100755 benchmark/compatible/mariadb/run.sh mode change 100644 => 100755 benchmark/compatible/monetdb/benchmark.sh mode change 100644 => 100755 benchmark/compatible/mysql/benchmark.sh mode change 100644 => 100755 benchmark/compatible/mysql/run.sh mode change 100644 => 100755 benchmark/compatible/postgresql/benchmark.sh mode change 100644 => 100755 benchmark/compatible/postgresql/run.sh mode change 100644 => 100755 benchmark/compatible/questdb/benchmark.sh mode change 100644 => 100755 benchmark/compatible/singlestore/benchmark.sh mode change 100644 => 100755 benchmark/compatible/singlestore/run.sh mode change 100644 => 100755 benchmark/compatible/timescaledb/benchmark.sh mode change 100644 => 100755 benchmark/compatible/timescaledb/run.sh diff --git a/benchmark/compatible/duckdb/benchmark.sh b/benchmark/compatible/duckdb/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/heavyai/benchmark.sh b/benchmark/compatible/heavyai/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/heavyai/run.sh b/benchmark/compatible/heavyai/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/mariadb-columnstore/benchmark.sh b/benchmark/compatible/mariadb-columnstore/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/mariadb/benchmark.sh b/benchmark/compatible/mariadb/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/mariadb/run.sh b/benchmark/compatible/mariadb/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/monetdb/benchmark.sh b/benchmark/compatible/monetdb/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/mysql/benchmark.sh b/benchmark/compatible/mysql/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/mysql/run.sh b/benchmark/compatible/mysql/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/postgresql/benchmark.sh b/benchmark/compatible/postgresql/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/postgresql/run.sh b/benchmark/compatible/postgresql/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/questdb/benchmark.sh b/benchmark/compatible/questdb/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/singlestore/benchmark.sh b/benchmark/compatible/singlestore/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/singlestore/run.sh b/benchmark/compatible/singlestore/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/timescaledb/benchmark.sh b/benchmark/compatible/timescaledb/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/compatible/timescaledb/run.sh b/benchmark/compatible/timescaledb/run.sh old mode 100644 new mode 100755 From 8357e84381c1a4c20ae0c1e05b96039917ebe759 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 09:30:59 +0200 Subject: [PATCH 053/627] Compatible benchmark --- benchmark/compatible/heavyai/benchmark.sh | 5 ++ .../heavyai/results/c6a.4xlarge.txt | 46 +++++++++++++++++++ benchmark/compatible/heavyai/run.sh | 2 +- benchmark/compatible/questdb/benchmark.sh | 4 +- benchmark/compatible/questdb/queries.sql | 43 +++++++++++++++++ benchmark/compatible/questdb/run.sh | 14 ++++++ 6 files changed, 112 insertions(+), 2 deletions(-) create mode 100644 benchmark/compatible/heavyai/results/c6a.4xlarge.txt create mode 100644 benchmark/compatible/questdb/queries.sql create mode 100644 benchmark/compatible/questdb/run.sh diff --git a/benchmark/compatible/heavyai/benchmark.sh b/benchmark/compatible/heavyai/benchmark.sh index a1b53a998e8..6e1f29a4ecc 100755 --- a/benchmark/compatible/heavyai/benchmark.sh +++ b/benchmark/compatible/heavyai/benchmark.sh @@ -42,3 +42,8 @@ time /opt/heavyai/bin/heavysql -t -p HyperInteractive <<< "COPY hits FROM '$(pwd # Loaded: 99997497 recs, Rejected: 0 recs in 572.633000 secs ./run.sh 2>&1 | tee log.txt + +du -bcs /var/lib/heavyai/ + +cat log.txt | grep -P 'Total time|null' | sed -r -e 's/^.*Total time: ([0-9]+) ms$/\1/' | awk '{ if ($1 == "null") { print } else { print $1 / 1000 } }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/heavyai/results/c6a.4xlarge.txt b/benchmark/compatible/heavyai/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..931a720c82c --- /dev/null +++ b/benchmark/compatible/heavyai/results/c6a.4xlarge.txt @@ -0,0 +1,46 @@ +Loading time: 572.633 sec +Data size: 50 887 437 386 bytes + +[6.525,0.022,0.029], +[0.301,0.042,0.04], +[0.287,0.095,0.093], +[2.572,0.039,0.04], +[null,null,null], +[null,null,null], +[7.327,0.093,0.097], +[0.244,0.043,0.038], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[2.939,0.295,0.294], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[4.716,3.91,3.955], +[null,null,null], +[0.154,0.083,0.106], +[14.426,0.07,0.071], +[null,null,null], +[null,null,null], +[null,null,null], +[2.276,0.258,0.272], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[1.832,1.64,1.602], +[null,null,null], +[null,null,null], +[null,null,null], +[14.811,0.494,0.497], +[null,null,null], +[null,null,null], +[1.941,0.255,0.255], +[5.457,0.172,0.283], +[0.476,0.269,0.256], +[14.239,0.179,0.178], +[3.992,0.112,0.112], +[1.031,0.116,0.116], +[1.365,0.089,0.088] diff --git a/benchmark/compatible/heavyai/run.sh b/benchmark/compatible/heavyai/run.sh index 2688e576e15..97387bee42b 100755 --- a/benchmark/compatible/heavyai/run.sh +++ b/benchmark/compatible/heavyai/run.sh @@ -8,6 +8,6 @@ cat queries.sql | while read query; do echo "$query"; for i in $(seq 1 $TRIES); do - /opt/heavyai/bin/heavysql -t -p HyperInteractive <<< "${query}" + /opt/heavyai/bin/heavysql -t -p HyperInteractive <<< "${query}" | grep 'Total time' || echo 'null' done; done; diff --git a/benchmark/compatible/questdb/benchmark.sh b/benchmark/compatible/questdb/benchmark.sh index b4433122ddd..86184942c6c 100755 --- a/benchmark/compatible/questdb/benchmark.sh +++ b/benchmark/compatible/questdb/benchmark.sh @@ -14,4 +14,6 @@ gzip -d hits.csv.gz curl -G --data-urlencode "query=$(cat create.sql)" 'http://localhost:9000/exec?timings=true' time curl -F data=@hits.csv 'http://localhost:9000/imp?name=hits' -# +# 27m 47.546s + +./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/questdb/queries.sql b/benchmark/compatible/questdb/queries.sql new file mode 100644 index 00000000000..0292fc3f8ea --- /dev/null +++ b/benchmark/compatible/questdb/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT count_distinct(CAST(UserID AS string)) FROM hits; +SELECT count_distinct(SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) AS c FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; +SELECT RegionID, count_distinct(CAST(UserID AS string)) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), count_distinct(CAST(UserID AS string)) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) AS c FROM hits GROUP BY UserID ORDER BY c DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) AS c FROM hits GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) AS c FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, count_distinct(CAST(UserID AS string)) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +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 SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000, 10; +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 EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000, 10; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100, 10; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 100000, 10; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-14T00:00:00Z' AND EventTime <= '2013-07-15T23:59:59Z' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 1000, 10; diff --git a/benchmark/compatible/questdb/run.sh b/benchmark/compatible/questdb/run.sh new file mode 100644 index 00000000000..b3b050449e5 --- /dev/null +++ b/benchmark/compatible/questdb/run.sh @@ -0,0 +1,14 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + echo "$query"; + for i in $(seq 1 $TRIES); do + curl -s -G --data-urlencode "query=${query}" 'http://localhost:9000/exec?timings=true' + echo + done; +done; From f62aa0a8f6454f2dd0c75802d3a2897464077837 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 10:14:00 +0200 Subject: [PATCH 054/627] Compatible benchmark --- benchmark/compatible/greenplum/benchmark.sh | 14 ++++++ benchmark/compatible/heavyai/benchmark.sh | 3 +- benchmark/compatible/questdb/benchmark.sh | 6 +++ benchmark/compatible/questdb/queries.sql | 2 +- .../compatible/questdb/result/c6a.4xlarge.txt | 46 +++++++++++++++++++ 5 files changed, 69 insertions(+), 2 deletions(-) create mode 100644 benchmark/compatible/greenplum/benchmark.sh create mode 100644 benchmark/compatible/questdb/result/c6a.4xlarge.txt diff --git a/benchmark/compatible/greenplum/benchmark.sh b/benchmark/compatible/greenplum/benchmark.sh new file mode 100644 index 00000000000..9c68c980560 --- /dev/null +++ b/benchmark/compatible/greenplum/benchmark.sh @@ -0,0 +1,14 @@ +#!/bin/bash + +sudo apt-get update +sudo apt-get install docker.io +sudo docker run -it --rm --volume=$(pwd):/workspace ubuntu:18.04 + +cd workspace +apt update +wget 'https://github.com/greenplum-db/gpdb/releases/download/6.21.0/greenplum-db-6.21.0-ubuntu18.04-amd64.deb' + +apt install ./greenplum-db-6.21.0-ubuntu18.04-amd64.deb +useradd gpadmin +chown -R gpadmin:gpadmin /usr/local/greenplum* +chgrp -R gpadmin /usr/local/greenplum* diff --git a/benchmark/compatible/heavyai/benchmark.sh b/benchmark/compatible/heavyai/benchmark.sh index 6e1f29a4ecc..46b956b0eab 100755 --- a/benchmark/compatible/heavyai/benchmark.sh +++ b/benchmark/compatible/heavyai/benchmark.sh @@ -45,5 +45,6 @@ time /opt/heavyai/bin/heavysql -t -p HyperInteractive <<< "COPY hits FROM '$(pwd du -bcs /var/lib/heavyai/ -cat log.txt | grep -P 'Total time|null' | sed -r -e 's/^.*Total time: ([0-9]+) ms$/\1/' | awk '{ if ($1 == "null") { print } else { print $1 / 1000 } }' | +cat log.txt | grep -P 'Total time|null' | sed -r -e 's/^.*Total time: ([0-9]+) ms$/\1/' | + awk '{ if ($1 == "null") { print } else { print $1 / 1000 } }' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/questdb/benchmark.sh b/benchmark/compatible/questdb/benchmark.sh index 86184942c6c..4cf6bb31ef1 100755 --- a/benchmark/compatible/questdb/benchmark.sh +++ b/benchmark/compatible/questdb/benchmark.sh @@ -17,3 +17,9 @@ time curl -F data=@hits.csv 'http://localhost:9000/imp?name=hits' # 27m 47.546s ./run.sh 2>&1 | tee log.txt + +du -bcs .questdb/db/hits + +cat log.txt | grep -P '"timings"|"error"|^$' | sed -r -e 's/^.*"error".*$|^$/null/; s/^.*"compiler":([0-9]*),"execute":([0-9]*),.*$/\1 \2/' | + awk '{ if ($1 == "null") { print } else { print ($1 + $2) / 1000000000 } }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/questdb/queries.sql b/benchmark/compatible/questdb/queries.sql index 0292fc3f8ea..cc8a98037ee 100644 --- a/benchmark/compatible/questdb/queries.sql +++ b/benchmark/compatible/questdb/queries.sql @@ -40,4 +40,4 @@ SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime > 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 EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000, 10; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100, 10; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 100000, 10; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-14T00:00:00Z' AND EventTime <= '2013-07-15T23:59:59Z' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 1000, 10; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-14T00:00:00Z' AND EventTime <= '2013-07-15T23:59:59Z' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY M LIMIT 1000, 10; diff --git a/benchmark/compatible/questdb/result/c6a.4xlarge.txt b/benchmark/compatible/questdb/result/c6a.4xlarge.txt new file mode 100644 index 00000000000..4e191c6066b --- /dev/null +++ b/benchmark/compatible/questdb/result/c6a.4xlarge.txt @@ -0,0 +1,46 @@ +Loading time: 27m 47.546s +Dataset size: 126 680 518 680 + +[0.0111375,0.000223265,0.000214715], +[0.624997,0.0434011,0.038081], +[5.98909,5.5704,5.71502], +[5.37973,3.08399,3.08446], +[51.5751,50.6261,47.4734], +[null,null,null], +[5.62592,4.02034,4.0187], +[0.168657,0.118581,0.124539], +[60.5636,null,53.7281], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null] From 40fd7d9a5af2f4ec55bd1da5e7593e8f343ccafd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 19:42:50 +0200 Subject: [PATCH 055/627] Compatible benchmark --- benchmark/compatible/questdb/benchmark.sh | 4 ++++ benchmark/compatible/questdb/run.sh | 3 ++- benchmark/compatible/singlestore/benchmark.sh | 4 ++-- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/benchmark/compatible/questdb/benchmark.sh b/benchmark/compatible/questdb/benchmark.sh index 4cf6bb31ef1..1e79dcd5310 100755 --- a/benchmark/compatible/questdb/benchmark.sh +++ b/benchmark/compatible/questdb/benchmark.sh @@ -16,6 +16,10 @@ time curl -F data=@hits.csv 'http://localhost:9000/imp?name=hits' # 27m 47.546s +sed -i 's/query.timeout.sec=60/query.timeout.sec=6000/' .questdb/conf/server.conf +questdb-6.4.1-rt-linux-amd64/bin/questdb.sh stop +questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start + ./run.sh 2>&1 | tee log.txt du -bcs .questdb/db/hits diff --git a/benchmark/compatible/questdb/run.sh b/benchmark/compatible/questdb/run.sh index b3b050449e5..e1b7b37cb54 100644 --- a/benchmark/compatible/questdb/run.sh +++ b/benchmark/compatible/questdb/run.sh @@ -8,7 +8,8 @@ cat queries.sql | while read query; do echo "$query"; for i in $(seq 1 $TRIES); do - curl -s -G --data-urlencode "query=${query}" 'http://localhost:9000/exec?timings=true' + curl -sS -G --data-urlencode "query=${query}" 'http://localhost:9000/exec?timings=true' || + (questdb-6.4.1-rt-linux-amd64/bin/questdb.sh stop && questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start && sleep 5) echo done; done; diff --git a/benchmark/compatible/singlestore/benchmark.sh b/benchmark/compatible/singlestore/benchmark.sh index c6a10866373..72e5e5aa361 100755 --- a/benchmark/compatible/singlestore/benchmark.sh +++ b/benchmark/compatible/singlestore/benchmark.sh @@ -5,8 +5,8 @@ sudo apt-get update sudo apt-get install docker.io -export LICENSE_KEY="BDA4OGMxMGFlNDRkYTQ0MmU4N2NkYjk4Y2MwYTUxMTQ5AAAAAAAAAAAEAAAAAAAAACgwNAIYTJwt51SEitrak4T9P7TyYzWzIRstlokzAhgy+cgwXnsXTU9gzedJ/ztTg1TPdc4jrlQAAA==" -export ROOT_PASSWORD="H@I}xfqKsw}[wfG,oLpH" +export LICENSE_KEY="..." +export ROOT_PASSWORD="..." sudo docker run -i --init \ --name memsql-ciab \ From d2cb450fd96b47b8c215e7d9c3b2b09830fc77d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 20:14:34 +0200 Subject: [PATCH 056/627] Compatible benchmark --- benchmark/compatible/postgresql/results/c6a.4xlarge.txt | 0 benchmark/compatible/timescaledb/results/c6a.4xlarge.txt | 0 2 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 benchmark/compatible/postgresql/results/c6a.4xlarge.txt create mode 100644 benchmark/compatible/timescaledb/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/postgresql/results/c6a.4xlarge.txt b/benchmark/compatible/postgresql/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..e69de29bb2d diff --git a/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt b/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..e69de29bb2d From 0c51a9c1e0110db2405dc051b025b3e766e90cac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 20:14:43 +0200 Subject: [PATCH 057/627] Compatible benchmark --- benchmark/compatible/postgresql/benchmark.sh | 5 ++ .../postgresql/results/c6a.4xlarge.txt | 46 +++++++++++++++++++ benchmark/compatible/questdb/run.sh | 2 +- benchmark/compatible/timescaledb/benchmark.sh | 7 ++- .../timescaledb/results/c6a.4xlarge.txt | 46 +++++++++++++++++++ 5 files changed, 104 insertions(+), 2 deletions(-) diff --git a/benchmark/compatible/postgresql/benchmark.sh b/benchmark/compatible/postgresql/benchmark.sh index ab7e7bf422c..53fd0ae659c 100755 --- a/benchmark/compatible/postgresql/benchmark.sh +++ b/benchmark/compatible/postgresql/benchmark.sh @@ -16,3 +16,8 @@ sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" # Time: 2341543.463 ms (39:01.543) ./run.sh 2>&1 | tee log.txt + +sudo du -bcs /var/lib/postgresql/14/main/ + +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/postgresql/results/c6a.4xlarge.txt b/benchmark/compatible/postgresql/results/c6a.4xlarge.txt index e69de29bb2d..76cec0f1d5d 100644 --- a/benchmark/compatible/postgresql/results/c6a.4xlarge.txt +++ b/benchmark/compatible/postgresql/results/c6a.4xlarge.txt @@ -0,0 +1,46 @@ +Load time: 2342 sec +Data size: 77 797 067 741 bytes + +[439753.348,309785.749,282017.708], +[317874.293,254238.464,254941.769], +[262883.138,263072.659,263090.967], +[32421.649,5310.077,5060.922], +[57134.664,42648.458,42334.061], +[358423.366,356315.487,358342.644], +[31524.566,5350.729,4994.452], +[263145.548,263193.332,263165.819], +[323659.951,322858.223,321918.464], +[327395.332,326170.524,326231.049], +[265983.201,265681.029,265912.645], +[269984.881,265336.483,265379.996], +[284096.661,284560.754,282234.726], +[277250.434,279455.446,280035.179], +[285660.969,286200.471,283611.116], +[66605.451,32023.025,38282.816], +[312452.824,304431.252,305391.155], +[289209.107,290449.044,287578.843], +[331706.364,327485.227,334428.263], +[24646.119,2543.388,2263.010], +[267561.278,267496.242,267524.846], +[267729.041,267690.012,268184.240], +[263074.820,263120.536,267040.337], +[267602.971,267488.950,267494.446], +[263141.314,263859.779,263137.581], +[262923.219,263102.864,263113.550], +[262885.041,263088.582,263114.552], +[267864.506,269127.149,268204.402], +[303376.276,306925.881,308664.073], +[263221.039,263119.663,263148.959], +[270814.146,270575.238,270294.943], +[278342.802,275925.336,276224.959], +[584599.924,576932.765,591502.540], +[462576.063,446962.684,439779.116], +[429930.882,417696.097,416704.016], +[296875.751,297283.343,295140.467], +[3461.727,842.168,794.721], +[2179.836,564.875,558.321], +[2258.891,566.078,416.122], +[2805.199,1311.129,1317.189], +[2936.932,820.554,615.959], +[2197.875,736.053,535.611], +[1983.240,320.960,312.554] diff --git a/benchmark/compatible/questdb/run.sh b/benchmark/compatible/questdb/run.sh index e1b7b37cb54..80ed2c0a918 100644 --- a/benchmark/compatible/questdb/run.sh +++ b/benchmark/compatible/questdb/run.sh @@ -8,7 +8,7 @@ cat queries.sql | while read query; do echo "$query"; for i in $(seq 1 $TRIES); do - curl -sS -G --data-urlencode "query=${query}" 'http://localhost:9000/exec?timings=true' || + curl -sS --max-time 6000 -G --data-urlencode "query=${query}" 'http://localhost:9000/exec?timings=true' || (questdb-6.4.1-rt-linux-amd64/bin/questdb.sh stop && questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start && sleep 5) echo done; diff --git a/benchmark/compatible/timescaledb/benchmark.sh b/benchmark/compatible/timescaledb/benchmark.sh index 89e748f0a3a..2a63c0b1da5 100755 --- a/benchmark/compatible/timescaledb/benchmark.sh +++ b/benchmark/compatible/timescaledb/benchmark.sh @@ -29,6 +29,11 @@ sudo -u postgres psql test -c "SELECT add_compression_policy('hits', INTERVAL '1 sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" -# 1619875.288 seconds (26:59.875) +# 1619875.288 ms (26:59.875) ./run.sh 2>&1 | tee log.txt + +sudo du -bcs /var/lib/postgresql/14/main/ + +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt b/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt index e69de29bb2d..c3d2fd3094c 100644 --- a/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt +++ b/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt @@ -0,0 +1,46 @@ +Load time: 1620 seconds +Data size: 72 882 392 030 bytes + +[437700.229,215793.314,176420.897], +[327026.232,259568.339,244578.780], +[262978.382,263090.178,263083.484], +[262807.910,263046.124,266847.716], +[337497.985,334964.363,330852.141], +[355689.862,356801.410,362894.372], +[262762.474,263012.925,262968.088], +[263055.362,263016.424,263028.660], +[319928.080,319388.513,320704.735], +[323584.015,322224.697,322488.413], +[265979.137,265465.789,265375.740], +[266019.708,265543.152,265462.654], +[277018.786,276300.778,276595.150], +[280352.481,279251.213,279572.251], +[279915.935,279896.775,279674.285], +[296377.147,298506.713,297659.537], +[314448.530,314605.256,312570.831], +[302668.006,302672.507,303039.646], +[325810.527,324061.457,324376.415], +[262447.995,262698.364,262704.067], +[267581.759,267467.643,267482.050], +[268085.555,267466.797,267696.901], +[263391.829,263097.590,263126.579], +[38291.424,435.353,335.331], +[127.768,5.054,5.307], +[263138.877,263100.888,263092.929], +[889.696,341.106,339.118], +[267586.222,267498.547,267491.245], +[289086.248,290012.052,290093.278], +[263220.195,263071.707,263109.600], +[274780.156,273995.447,273998.676], +[282217.820,281390.700,281470.632], +[429273.892,426588.599,439431.586], +[448808.430,418724.274,418207.674], +[455196.314,422750.955,423142.946], +[299263.589,296937.462,297261.160], +[18693.445,1552.021,1481.741], +[18125.858,945.707,937.442], +[18528.375,1062.610,902.568], +[18268.028,1779.357,1770.984], +[19615.320,1965.590,1966.551], +[18970.537,1435.093,1430.706], +[18330.728,1153.613,952.997] From 236448be09187e0a89e8522ff355ba1e2bfe0e3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 23:04:19 +0200 Subject: [PATCH 058/627] Compatible benchmark --- benchmark/compatible/citus/benchmark.sh | 27 +++++ benchmark/compatible/citus/create.sql | 109 ++++++++++++++++++ benchmark/compatible/citus/queries.sql | 43 +++++++ benchmark/compatible/citus/run.sh | 13 +++ benchmark/compatible/clickhouse/benchmark.sh | 2 +- benchmark/compatible/clickhouse/queries.sql | 2 +- benchmark/compatible/duckdb/benchmark.sh | 4 +- benchmark/compatible/duckdb/create.sql | 52 ++++----- benchmark/compatible/duckdb/queries.sql | 2 +- benchmark/compatible/duckdb/run.py | 29 +++++ benchmark/compatible/greenplum/benchmark.sh | 2 +- benchmark/compatible/heavyai/benchmark.sh | 2 +- benchmark/compatible/heavyai/queries.sql | 2 +- .../mariadb-columnstore/benchmark.sh | 2 +- benchmark/compatible/mariadb/benchmark.sh | 2 +- benchmark/compatible/mariadb/queries.sql | 2 +- benchmark/compatible/mysql/benchmark.sh | 10 +- benchmark/compatible/mysql/queries.sql | 4 +- .../compatible/mysql/results/c6a.4xlarge.txt | 46 ++++++++ benchmark/compatible/postgresql/benchmark.sh | 2 +- benchmark/compatible/postgresql/queries.sql | 2 +- benchmark/compatible/questdb/benchmark.sh | 6 +- benchmark/compatible/questdb/queries.sql | 2 +- .../questdb/result/c6a.4xlarge_2.txt | 88 ++++++++++++++ benchmark/compatible/questdb/run.sh | 8 +- benchmark/compatible/singlestore/benchmark.sh | 4 +- benchmark/compatible/singlestore/queries.sql | 2 +- benchmark/compatible/timescaledb/benchmark.sh | 2 +- benchmark/compatible/timescaledb/queries.sql | 2 +- 29 files changed, 420 insertions(+), 53 deletions(-) create mode 100755 benchmark/compatible/citus/benchmark.sh create mode 100644 benchmark/compatible/citus/create.sql create mode 100644 benchmark/compatible/citus/queries.sql create mode 100755 benchmark/compatible/citus/run.sh create mode 100644 benchmark/compatible/mysql/results/c6a.4xlarge.txt create mode 100644 benchmark/compatible/questdb/result/c6a.4xlarge_2.txt diff --git a/benchmark/compatible/citus/benchmark.sh b/benchmark/compatible/citus/benchmark.sh new file mode 100755 index 00000000000..ded0063d5d5 --- /dev/null +++ b/benchmark/compatible/citus/benchmark.sh @@ -0,0 +1,27 @@ +#!/bin/bash + +sudo apt-get update +sudo apt-get install docker.io +sudo apt-get install postgresql-client + +sudo docker run -d --name citus -p 5432:5432 -e POSTGRES_PASSWORD=mypass citusdata/citus:11.0 + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +echo "*:*:*:*:mypass" > .pgpass +chmod 400 .pgpass + +psql -U postgres -h localhost --no-password -t -c 'CREATE DATABASE test' +psql -U postgres -h localhost --no-password test -t < create.sql +psql -U postgres -h localhost --no-password test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" + +# COPY 99997497 +# Time: 2341543.463 ms (39:01.543) + +./run.sh 2>&1 | tee log.txt + +#sudo du -bcs /var/lib/postgresql/14/main/ + +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/citus/create.sql b/benchmark/compatible/citus/create.sql new file mode 100644 index 00000000000..1850bffedce --- /dev/null +++ b/benchmark/compatible/citus/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/citus/queries.sql b/benchmark/compatible/citus/queries.sql new file mode 100644 index 00000000000..2f986cda146 --- /dev/null +++ b/benchmark/compatible/citus/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/citus/run.sh b/benchmark/compatible/citus/run.sh new file mode 100755 index 00000000000..a5802c5c4c7 --- /dev/null +++ b/benchmark/compatible/citus/run.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + echo "$query"; + for i in $(seq 1 $TRIES); do + psql -U postgres -h localhost --no-password -t -c '\timing' -c "$query" | grep 'Time' + done; +done; diff --git a/benchmark/compatible/clickhouse/benchmark.sh b/benchmark/compatible/clickhouse/benchmark.sh index 347cc264abf..5d601d87612 100755 --- a/benchmark/compatible/clickhouse/benchmark.sh +++ b/benchmark/compatible/clickhouse/benchmark.sh @@ -10,7 +10,7 @@ sudo clickhouse start clickhouse-client < create.sql -wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz clickhouse-client --time --query "INSERT INTO hits FORMAT TSV" < hits.tsv diff --git a/benchmark/compatible/clickhouse/queries.sql b/benchmark/compatible/clickhouse/queries.sql index 439f893a06b..2f986cda146 100644 --- a/benchmark/compatible/clickhouse/queries.sql +++ b/benchmark/compatible/clickhouse/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; diff --git a/benchmark/compatible/duckdb/benchmark.sh b/benchmark/compatible/duckdb/benchmark.sh index 9b7242e2428..838f4384c13 100755 --- a/benchmark/compatible/duckdb/benchmark.sh +++ b/benchmark/compatible/duckdb/benchmark.sh @@ -4,11 +4,11 @@ sudo apt-get update sudo apt-get install python3-pip -pip install duckdb +pip install duckdb psutil # Load the data -wget 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' gzip -d hits.csv.gz # Run the queries diff --git a/benchmark/compatible/duckdb/create.sql b/benchmark/compatible/duckdb/create.sql index 1850bffedce..744d595ecac 100644 --- a/benchmark/compatible/duckdb/create.sql +++ b/benchmark/compatible/duckdb/create.sql @@ -2,7 +2,7 @@ CREATE TABLE hits ( WatchID BIGINT NOT NULL, JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, + Title TEXT, GoodEvent SMALLINT NOT NULL, EventTime TIMESTAMP NOT NULL, EventDate Date NOT NULL, @@ -13,8 +13,8 @@ CREATE TABLE hits CounterClass SMALLINT NOT NULL, OS SMALLINT NOT NULL, UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, + URL TEXT, + Referer TEXT, IsRefresh SMALLINT NOT NULL, RefererCategoryID SMALLINT NOT NULL, RefererRegionID INTEGER NOT NULL, @@ -25,7 +25,7 @@ CREATE TABLE hits ResolutionDepth SMALLINT NOT NULL, FlashMajor SMALLINT NOT NULL, FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, + FlashMinor2 TEXT, NetMajor SMALLINT NOT NULL, NetMinor SMALLINT NOT NULL, UserAgentMajor SMALLINT NOT NULL, @@ -34,12 +34,12 @@ CREATE TABLE hits JavascriptEnable SMALLINT NOT NULL, IsMobile SMALLINT NOT NULL, MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, + MobilePhoneModel TEXT, + Params TEXT, IPNetworkID INTEGER NOT NULL, TraficSourceID SMALLINT NOT NULL, SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, + SearchPhrase TEXT, AdvEngineID SMALLINT NOT NULL, IsArtifical SMALLINT NOT NULL, WindowClientWidth SMALLINT NOT NULL, @@ -50,13 +50,13 @@ CREATE TABLE hits SilverlightVersion2 SMALLINT NOT NULL, SilverlightVersion3 INTEGER NOT NULL, SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, + PageCharset TEXT, CodeVersion INTEGER NOT NULL, IsLink SMALLINT NOT NULL, IsDownload SMALLINT NOT NULL, IsNotBounce SMALLINT NOT NULL, FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, + OriginalURL TEXT, HID INTEGER NOT NULL, IsOldCounter SMALLINT NOT NULL, IsEvent SMALLINT NOT NULL, @@ -74,10 +74,10 @@ CREATE TABLE hits WindowName INTEGER NOT NULL, OpenerName INTEGER NOT NULL, HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, + BrowserLanguage TEXT, + BrowserCountry TEXT, + SocialNetwork TEXT, + SocialAction TEXT, HTTPError SMALLINT NOT NULL, SendTiming INTEGER NOT NULL, DNSTiming INTEGER NOT NULL, @@ -86,21 +86,21 @@ CREATE TABLE hits ResponseEndTiming INTEGER NOT NULL, FetchTiming INTEGER NOT NULL, SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, + SocialSourcePage TEXT, ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, + ParamOrderID TEXT, + ParamCurrency TEXT, ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, + OpenstatServiceName TEXT, + OpenstatCampaignID TEXT, + OpenstatAdID TEXT, + OpenstatSourceID TEXT, + UTMSource TEXT, + UTMMedium TEXT, + UTMCampaign TEXT, + UTMContent TEXT, + UTMTerm TEXT, + FromTag TEXT, HasGCLID SMALLINT NOT NULL, RefererHash BIGINT NOT NULL, URLHash BIGINT NOT NULL, diff --git a/benchmark/compatible/duckdb/queries.sql b/benchmark/compatible/duckdb/queries.sql index 439f893a06b..2f986cda146 100644 --- a/benchmark/compatible/duckdb/queries.sql +++ b/benchmark/compatible/duckdb/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; diff --git a/benchmark/compatible/duckdb/run.py b/benchmark/compatible/duckdb/run.py index e69de29bb2d..c0b3d8592f5 100644 --- a/benchmark/compatible/duckdb/run.py +++ b/benchmark/compatible/duckdb/run.py @@ -0,0 +1,29 @@ +#!/usr/bin/env python3 + +import duckdb +import timeit +import psutil + +con = duckdb.connect(database='my-db.duckdb', read_only=False) +# See https://github.com/duckdb/duckdb/issues/3969 +con.execute("PRAGMA memory_limit='{}b'".format(psutil.virtual_memory().total / 4)) +con.execute("PRAGMA threads={}".format(psutil.cpu_count(logical=False))) + +queries_file = open('queries.sql', 'r') +queries = queries_file.readlines() + +print("Will load the data") + +start = timeit.default_timer() +con.execute(open('create.sql').read()) +con.execute("INSERT INTO hits SELECT * FROM read_csv_auto('hits.csv')") +end = timeit.default_timer() +print(end - start) + +for query in queries: + print(query) + for try_num in range(3): + start = timeit.default_timer() + con.execute(query) + end = timeit.default_timer() + print(end - start) diff --git a/benchmark/compatible/greenplum/benchmark.sh b/benchmark/compatible/greenplum/benchmark.sh index 9c68c980560..59554f8547e 100644 --- a/benchmark/compatible/greenplum/benchmark.sh +++ b/benchmark/compatible/greenplum/benchmark.sh @@ -6,7 +6,7 @@ sudo docker run -it --rm --volume=$(pwd):/workspace ubuntu:18.04 cd workspace apt update -wget 'https://github.com/greenplum-db/gpdb/releases/download/6.21.0/greenplum-db-6.21.0-ubuntu18.04-amd64.deb' +wget --continue 'https://github.com/greenplum-db/gpdb/releases/download/6.21.0/greenplum-db-6.21.0-ubuntu18.04-amd64.deb' apt install ./greenplum-db-6.21.0-ubuntu18.04-amd64.deb useradd gpadmin diff --git a/benchmark/compatible/heavyai/benchmark.sh b/benchmark/compatible/heavyai/benchmark.sh index 46b956b0eab..366d19bac34 100755 --- a/benchmark/compatible/heavyai/benchmark.sh +++ b/benchmark/compatible/heavyai/benchmark.sh @@ -27,7 +27,7 @@ sudo systemctl enable heavydb # Load the data -wget 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' gzip -d hits.csv.gz chmod 777 ~ hits.csv diff --git a/benchmark/compatible/heavyai/queries.sql b/benchmark/compatible/heavyai/queries.sql index 439f893a06b..2f986cda146 100644 --- a/benchmark/compatible/heavyai/queries.sql +++ b/benchmark/compatible/heavyai/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; diff --git a/benchmark/compatible/mariadb-columnstore/benchmark.sh b/benchmark/compatible/mariadb-columnstore/benchmark.sh index 98a7a6ccc69..e49626a8314 100755 --- a/benchmark/compatible/mariadb-columnstore/benchmark.sh +++ b/benchmark/compatible/mariadb-columnstore/benchmark.sh @@ -16,7 +16,7 @@ mysql --password="${PASSWORD}" --host 127.0.0.1 test < create.sql # Load the data -wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz time mysql --password="${PASSWORD}" --host 127.0.0.1 test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" diff --git a/benchmark/compatible/mariadb/benchmark.sh b/benchmark/compatible/mariadb/benchmark.sh index 6dbcb4b11d9..1e0f148cadc 100755 --- a/benchmark/compatible/mariadb/benchmark.sh +++ b/benchmark/compatible/mariadb/benchmark.sh @@ -9,7 +9,7 @@ sudo service mariadb restart # Load the data -wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz sudo mysql -e "CREATE DATABASE test" diff --git a/benchmark/compatible/mariadb/queries.sql b/benchmark/compatible/mariadb/queries.sql index 439f893a06b..2f986cda146 100644 --- a/benchmark/compatible/mariadb/queries.sql +++ b/benchmark/compatible/mariadb/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; diff --git a/benchmark/compatible/mysql/benchmark.sh b/benchmark/compatible/mysql/benchmark.sh index 1a58b801f55..0ecdbed42e8 100755 --- a/benchmark/compatible/mysql/benchmark.sh +++ b/benchmark/compatible/mysql/benchmark.sh @@ -9,7 +9,7 @@ sudo service mysql restart # Load the data -wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz sudo mysql -e "CREATE DATABASE test" @@ -19,3 +19,11 @@ time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" # 2:37:52 elapsed ./run.sh 2>&1 | tee log.txt + +sudo du -bcs /var/lib/mysql + +cat log.txt | + grep -P 'rows? in set|^ERROR' | + sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | + awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/mysql/queries.sql b/benchmark/compatible/mysql/queries.sql index 439f893a06b..1ff8a9daad8 100644 --- a/benchmark/compatible/mysql/queries.sql +++ b/benchmark/compatible/mysql/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; @@ -40,4 +40,4 @@ SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate > 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 IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; +SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/mysql/results/c6a.4xlarge.txt b/benchmark/compatible/mysql/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..a7b339b71eb --- /dev/null +++ b/benchmark/compatible/mysql/results/c6a.4xlarge.txt @@ -0,0 +1,46 @@ +Load time: 2:37:52 +Data size: 171 953 585 825 bytes + +[339.77,339.88,339.77], +[364.91,371.86,367.55], +[366.2,368.91,389.66], +[364.39,377.53,571.45], +[377.69,390.02,384.86], +[569.48,576.51,574.68], +[367.4,368.23,370.41], +[371.29,384.02,613.22], +[478.85,683.22,495.68], +[489.9,635.96,662.43], +[386.07,396.49,640.15], +[389.13,412.55,444.12], +[447.97,455.54,448.06], +[423.22,845.44,813.6], +[452.48,460.07,453.98], +[577.54,623.21,586.49], +[852.07,856.36,862.66], +[838.09,848.92,851.12], +[1006.37,1011.16,1023.17], +[369.76,375.61,415.28], +[412.45,419.9,456.62], +[411.65,432.88,482.2], +[412.73,420.73,429.5], +[551.16,577.62,545.45], +[382.89,394.76,386.37], +[380.9,391.4,385.05], +[385.3,394.67,460.32], +[388.95,394.7,387.21], +[null,null,null], +[706.03,745.27,718.9], +[450.9,489.59,530.97], +[625.5,651.93,647.32], +[2721.13,2792.12,2819.26], +[945.9,954.94,957.54], +[945.42,953.78,965.16], +[684.36,716.29,708.75], +[10.01,3.79,3.77], +[7.48,3.32,3.27], +[5.09,0.98,0.96], +[8.70,4.77,4.68], +[4.82,0.76,0.74], +[4.46,0.77,0.75], +[7.04,1.17,1.13] diff --git a/benchmark/compatible/postgresql/benchmark.sh b/benchmark/compatible/postgresql/benchmark.sh index 53fd0ae659c..89c0378086a 100755 --- a/benchmark/compatible/postgresql/benchmark.sh +++ b/benchmark/compatible/postgresql/benchmark.sh @@ -4,7 +4,7 @@ sudo apt-get update sudo apt-get install postgresql-common sudo apt-get install postgresql-14 -wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz chmod 777 ~ hits.tsv diff --git a/benchmark/compatible/postgresql/queries.sql b/benchmark/compatible/postgresql/queries.sql index 439f893a06b..2f986cda146 100644 --- a/benchmark/compatible/postgresql/queries.sql +++ b/benchmark/compatible/postgresql/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; diff --git a/benchmark/compatible/questdb/benchmark.sh b/benchmark/compatible/questdb/benchmark.sh index 1e79dcd5310..ebec8ef9cf9 100755 --- a/benchmark/compatible/questdb/benchmark.sh +++ b/benchmark/compatible/questdb/benchmark.sh @@ -8,7 +8,7 @@ questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start # Import the data -wget 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' gzip -d hits.csv.gz curl -G --data-urlencode "query=$(cat create.sql)" 'http://localhost:9000/exec?timings=true' @@ -24,6 +24,6 @@ questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start du -bcs .questdb/db/hits -cat log.txt | grep -P '"timings"|"error"|^$' | sed -r -e 's/^.*"error".*$|^$/null/; s/^.*"compiler":([0-9]*),"execute":([0-9]*),.*$/\1 \2/' | - awk '{ if ($1 == "null") { print } else { print ($1 + $2) / 1000000000 } }' | +cat log2.txt | grep -P '"timings"|"error"|Failed to connect|Operation timed out' | sed -r -e 's/^.*"error".*$|Failed to connect|Operation timed out/null/; s/^.*"compiler":([0-9]*),"execute":([0-9]*),.*$/\1 \2/' | + awk '{ print ($1 + $2) / 1000000000 }' | sed -r -e 's/^0$/null/' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/questdb/queries.sql b/benchmark/compatible/questdb/queries.sql index cc8a98037ee..546606439f7 100644 --- a/benchmark/compatible/questdb/queries.sql +++ b/benchmark/compatible/questdb/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; diff --git a/benchmark/compatible/questdb/result/c6a.4xlarge_2.txt b/benchmark/compatible/questdb/result/c6a.4xlarge_2.txt new file mode 100644 index 00000000000..925085d3ad9 --- /dev/null +++ b/benchmark/compatible/questdb/result/c6a.4xlarge_2.txt @@ -0,0 +1,88 @@ +[null,null,null], +[null,null,null], +[0.0105163,0.000226535,0.641283], +[0.0486715,0.0437709,5.92925], +[6.29189,5.78622,5.37717], +[3.10767,3.97005,39.8604], +[34.6356,38.2921,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,5.64056], +[3.70324,4.25853,0.708885], +[0.174593,0.112991,53.8225], +[null,null,null], +[null,null,null], +[42.9022,56.0935,55.8935], +[null,null,null], +[null,null,null], +[39.1034,46.7616,47.6015], +[null,null,null], +[null,null,null], +[40.2136,45.1588,15.7707], +[8.65482,10.818,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,56.5586], +[16.5303,12.6013,12.1614], +[18.5537,10.7626,12.3802], +[25.726,15.8504,18.1685], +[23.4373,16.8201,16.8349], +[39.8485,24.8796,25.9404], +[0.0489035,0.0272907,0.027413], +[null,null,null], +[null,null,null], +[33.8977,13.2292,null], +[null,null,null], +[null,null,20.9485], +[15.0937,13.0797,11.161], +[4.00018,2.1012,3.05835], +[3.28689,3.09242,2.88045], +[2.80586,2.8738,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null] diff --git a/benchmark/compatible/questdb/run.sh b/benchmark/compatible/questdb/run.sh index 80ed2c0a918..a9c27c952c6 100644 --- a/benchmark/compatible/questdb/run.sh +++ b/benchmark/compatible/questdb/run.sh @@ -2,14 +2,18 @@ TRIES=3 +questdb-6.4.1-rt-linux-amd64/bin/questdb.sh stop +questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start +sleep 5 + cat queries.sql | while read query; do sync echo 3 | sudo tee /proc/sys/vm/drop_caches echo "$query"; for i in $(seq 1 $TRIES); do - curl -sS --max-time 6000 -G --data-urlencode "query=${query}" 'http://localhost:9000/exec?timings=true' || - (questdb-6.4.1-rt-linux-amd64/bin/questdb.sh stop && questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start && sleep 5) + curl -sS --max-time 6000 -G --data-urlencode "query=${query}" 'http://localhost:9000/exec?timings=true' 2>&1 | grep '"timings"' || + (questdb-6.4.1-rt-linux-amd64/bin/questdb.sh stop >/dev/null 2>&1; questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start >/dev/null 2>&1; sleep 5; echo 'null') echo done; done; diff --git a/benchmark/compatible/singlestore/benchmark.sh b/benchmark/compatible/singlestore/benchmark.sh index 72e5e5aa361..b088042930d 100755 --- a/benchmark/compatible/singlestore/benchmark.sh +++ b/benchmark/compatible/singlestore/benchmark.sh @@ -21,7 +21,7 @@ sudo docker exec -it memsql-ciab memsql -p"${ROOT_PASSWORD}" # Load the data -wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz sudo docker cp hits.tsv memsql-ciab:/ @@ -38,7 +38,7 @@ sudo docker exec memsql-ciab du -bcs /var/lib/memsql # 29836263469 bytes cat log.txt | - grep -P 'rows? in set|^ERROR' result.txt | + grep -P 'rows? in set|^ERROR' | sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/singlestore/queries.sql b/benchmark/compatible/singlestore/queries.sql index 439f893a06b..2f986cda146 100644 --- a/benchmark/compatible/singlestore/queries.sql +++ b/benchmark/compatible/singlestore/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; diff --git a/benchmark/compatible/timescaledb/benchmark.sh b/benchmark/compatible/timescaledb/benchmark.sh index 2a63c0b1da5..97e30fb894a 100755 --- a/benchmark/compatible/timescaledb/benchmark.sh +++ b/benchmark/compatible/timescaledb/benchmark.sh @@ -17,7 +17,7 @@ sudo -u postgres psql test -c "CREATE EXTENSION IF NOT EXISTS timescaledb" # Import the data -wget 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz chmod 777 ~ hits.tsv diff --git a/benchmark/compatible/timescaledb/queries.sql b/benchmark/compatible/timescaledb/queries.sql index 439f893a06b..2f986cda146 100644 --- a/benchmark/compatible/timescaledb/queries.sql +++ b/benchmark/compatible/timescaledb/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; From 48b320396a1d0c394ab2214e569c1ae96eeb8ead Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Jun 2022 23:25:39 +0200 Subject: [PATCH 059/627] Compatible benchmark --- .../compatible/mysql/results/c6a.4xlarge.txt | 2 +- benchmark/compatible/questdb/benchmark.sh | 2 +- .../compatible/questdb/result/c6a.4xlarge.txt | 72 +++++++-------- .../questdb/result/c6a.4xlarge_2.txt | 88 ------------------- 4 files changed, 38 insertions(+), 126 deletions(-) delete mode 100644 benchmark/compatible/questdb/result/c6a.4xlarge_2.txt diff --git a/benchmark/compatible/mysql/results/c6a.4xlarge.txt b/benchmark/compatible/mysql/results/c6a.4xlarge.txt index a7b339b71eb..5b9052bc910 100644 --- a/benchmark/compatible/mysql/results/c6a.4xlarge.txt +++ b/benchmark/compatible/mysql/results/c6a.4xlarge.txt @@ -29,7 +29,7 @@ Data size: 171 953 585 825 bytes [380.9,391.4,385.05], [385.3,394.67,460.32], [388.95,394.7,387.21], -[null,null,null], +[800.33,null,null], [706.03,745.27,718.9], [450.9,489.59,530.97], [625.5,651.93,647.32], diff --git a/benchmark/compatible/questdb/benchmark.sh b/benchmark/compatible/questdb/benchmark.sh index ebec8ef9cf9..dce8569e927 100755 --- a/benchmark/compatible/questdb/benchmark.sh +++ b/benchmark/compatible/questdb/benchmark.sh @@ -24,6 +24,6 @@ questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start du -bcs .questdb/db/hits -cat log2.txt | grep -P '"timings"|"error"|Failed to connect|Operation timed out' | sed -r -e 's/^.*"error".*$|Failed to connect|Operation timed out/null/; s/^.*"compiler":([0-9]*),"execute":([0-9]*),.*$/\1 \2/' | +cat log.txt | grep -P '"timings"|"error"|null' | sed -r -e 's/^.*"error".*$/null/; s/^.*"compiler":([0-9]*),"execute":([0-9]*),.*$/\1 \2/' | awk '{ print ($1 + $2) / 1000000000 }' | sed -r -e 's/^0$/null/' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/questdb/result/c6a.4xlarge.txt b/benchmark/compatible/questdb/result/c6a.4xlarge.txt index 4e191c6066b..7372937a321 100644 --- a/benchmark/compatible/questdb/result/c6a.4xlarge.txt +++ b/benchmark/compatible/questdb/result/c6a.4xlarge.txt @@ -1,46 +1,46 @@ Loading time: 27m 47.546s Dataset size: 126 680 518 680 -[0.0111375,0.000223265,0.000214715], -[0.624997,0.0434011,0.038081], -[5.98909,5.5704,5.71502], -[5.37973,3.08399,3.08446], -[51.5751,50.6261,47.4734], -[null,null,null], -[5.62592,4.02034,4.0187], -[0.168657,0.118581,0.124539], -[60.5636,null,53.7281], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], +[0.0155878,0.000283657,0.000328587], +[0.586734,0.0486348,0.0485346], +[6.41881,5.57954,6.25638], +[6.8554,3.11997,3.11813], +[47.8723,54.5425,53.1198], [null,null,null], +[5.63439,3.70619,3.92043], +[0.699764,0.130462,0.126272], +[55.7179,62.8433,62.0045], +[null,59.6416,64.7753], +[63.7134,null,39.8928], +[46.759,46.8544,null], +[16.7181,10.9064,10.0735], +[79.9885,null,52.5617], +[16.1015,12.1835,13.1469], +[17.0225,11.7342,10.6194], +[24.1167,15.6749,16.8193], +[23.0848,15.3149,17.375], +[42.0965,26.2421,25.7593], +[0.0356335,0.027459,0.0289404], +[70.7679,71.5825,71.4573], [null,null,null], [null,null,null], +[77.0625,13.2257,12.7578], +[11.0775,2.90421,2.33398], +[3.31611,3.38837,3.35419], +[3.13233,5.2785,3.07075], [null,null,null], [null,null,null], +[186.032,185.627,180.963], +[26.7279,16.4799,18.8758], +[63.8785,32.1097,32.1561], +[64.4635,31.4538,35.654], +[118.897,null,119.015], [null,null,null], +[19.6853,17.4427,16.7998], +[1.50985,0.877967,0.885536], +[0.805639,0.577352,0.534731], +[0.509284,0.448942,0.467679], +[1.37609,0.966942,0.912858], +[0.567887,0.394619,0.438952], +[0.508977,0.441015,0.40528], [null,null,null] diff --git a/benchmark/compatible/questdb/result/c6a.4xlarge_2.txt b/benchmark/compatible/questdb/result/c6a.4xlarge_2.txt deleted file mode 100644 index 925085d3ad9..00000000000 --- a/benchmark/compatible/questdb/result/c6a.4xlarge_2.txt +++ /dev/null @@ -1,88 +0,0 @@ -[null,null,null], -[null,null,null], -[0.0105163,0.000226535,0.641283], -[0.0486715,0.0437709,5.92925], -[6.29189,5.78622,5.37717], -[3.10767,3.97005,39.8604], -[34.6356,38.2921,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,5.64056], -[3.70324,4.25853,0.708885], -[0.174593,0.112991,53.8225], -[null,null,null], -[null,null,null], -[42.9022,56.0935,55.8935], -[null,null,null], -[null,null,null], -[39.1034,46.7616,47.6015], -[null,null,null], -[null,null,null], -[40.2136,45.1588,15.7707], -[8.65482,10.818,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,56.5586], -[16.5303,12.6013,12.1614], -[18.5537,10.7626,12.3802], -[25.726,15.8504,18.1685], -[23.4373,16.8201,16.8349], -[39.8485,24.8796,25.9404], -[0.0489035,0.0272907,0.027413], -[null,null,null], -[null,null,null], -[33.8977,13.2292,null], -[null,null,null], -[null,null,20.9485], -[15.0937,13.0797,11.161], -[4.00018,2.1012,3.05835], -[3.28689,3.09242,2.88045], -[2.80586,2.8738,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null] From 6a98943c33588e212ea1cfe3a34710b290362a09 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 00:00:00 +0200 Subject: [PATCH 060/627] Compatible benchmark --- benchmark/compatible/citus/benchmark.sh | 2 +- benchmark/compatible/citus/run.sh | 2 +- benchmark/compatible/greenplum/benchmark.sh | 35 ++++++++++++++----- .../compatible/mysql/results/c6a.4xlarge.txt | 2 +- 4 files changed, 29 insertions(+), 12 deletions(-) diff --git a/benchmark/compatible/citus/benchmark.sh b/benchmark/compatible/citus/benchmark.sh index ded0063d5d5..43b651e11ea 100755 --- a/benchmark/compatible/citus/benchmark.sh +++ b/benchmark/compatible/citus/benchmark.sh @@ -17,7 +17,7 @@ psql -U postgres -h localhost --no-password test -t < create.sql psql -U postgres -h localhost --no-password test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" # COPY 99997497 -# Time: 2341543.463 ms (39:01.543) +# Time: 2182125.360 ms (36:22.125) ./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/citus/run.sh b/benchmark/compatible/citus/run.sh index a5802c5c4c7..4602870aae4 100755 --- a/benchmark/compatible/citus/run.sh +++ b/benchmark/compatible/citus/run.sh @@ -8,6 +8,6 @@ cat queries.sql | while read query; do echo "$query"; for i in $(seq 1 $TRIES); do - psql -U postgres -h localhost --no-password -t -c '\timing' -c "$query" | grep 'Time' + psql -U postgres -h localhost --no-password test -t -c '\timing' -c "$query" | grep 'Time' done; done; diff --git a/benchmark/compatible/greenplum/benchmark.sh b/benchmark/compatible/greenplum/benchmark.sh index 59554f8547e..b83aebe13e6 100644 --- a/benchmark/compatible/greenplum/benchmark.sh +++ b/benchmark/compatible/greenplum/benchmark.sh @@ -1,14 +1,31 @@ #!/bin/bash -sudo apt-get update sudo apt-get install docker.io -sudo docker run -it --rm --volume=$(pwd):/workspace ubuntu:18.04 +sudo docker run --network host -it --volume=$(pwd):/workspace ubuntu:18.04 + +# https://greenplum.org/install-greenplum-oss-on-ubuntu/ +apt-get update +apt install software-properties-common +add-apt-repository ppa:greenplum/db +apt-get update +apt install greenplum-db-6 + +echo 'en_US.UTF-8 UTF-8' > /etc/locale.gen +dpkg-reconfigure --frontend=noninteractive locales + +useradd ubuntu +mkdir /home/ubuntu +chown ubuntu /home/ubuntu +su ubuntu +bash +cd ~ + +export GPHOME=/opt/greenplum-db-* +cp $GPHOME/docs/cli_help/gpconfigs/gpinitsystem_singlenode . + +source /opt/greenplum-db-*/greenplum_path.sh +echo $(hostname) > ./hostlist_singlenode + + cd workspace -apt update -wget --continue 'https://github.com/greenplum-db/gpdb/releases/download/6.21.0/greenplum-db-6.21.0-ubuntu18.04-amd64.deb' - -apt install ./greenplum-db-6.21.0-ubuntu18.04-amd64.deb -useradd gpadmin -chown -R gpadmin:gpadmin /usr/local/greenplum* -chgrp -R gpadmin /usr/local/greenplum* diff --git a/benchmark/compatible/mysql/results/c6a.4xlarge.txt b/benchmark/compatible/mysql/results/c6a.4xlarge.txt index 5b9052bc910..dc48d66a8a8 100644 --- a/benchmark/compatible/mysql/results/c6a.4xlarge.txt +++ b/benchmark/compatible/mysql/results/c6a.4xlarge.txt @@ -29,7 +29,7 @@ Data size: 171 953 585 825 bytes [380.9,391.4,385.05], [385.3,394.67,460.32], [388.95,394.7,387.21], -[800.33,null,null], +[800.33,807.90,807.11], [706.03,745.27,718.9], [450.9,489.59,530.97], [625.5,651.93,647.32], From 57ad9b7a2c1bcb94668204369af0a6881fbc02b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 00:24:11 +0200 Subject: [PATCH 061/627] Compatible benchmark --- benchmark/compatible/citus/benchmark.sh | 4 +- benchmark/compatible/duckdb/benchmark.sh | 7 +- .../compatible/duckdb/{run.py => load.py} | 11 -- benchmark/compatible/duckdb/query.py | 19 +++ benchmark/compatible/duckdb/run.sh | 7 ++ benchmark/compatible/greenplum/benchmark.sh | 31 +---- .../mariadb-columnstore/benchmark.sh | 2 +- benchmark/compatible/mariadb/benchmark.sh | 2 +- benchmark/compatible/monetdb/benchmark.sh | 2 +- .../compatible/mysql-myisam/benchmark.sh | 29 +++++ benchmark/compatible/mysql-myisam/create.sql | 110 ++++++++++++++++++ benchmark/compatible/mysql-myisam/queries.sql | 43 +++++++ .../mysql-myisam/results/c6a.4xlarge.txt | 2 + benchmark/compatible/mysql-myisam/run.sh | 12 ++ benchmark/compatible/mysql/benchmark.sh | 2 +- benchmark/compatible/postgresql/benchmark.sh | 4 +- benchmark/compatible/singlestore/benchmark.sh | 2 +- 17 files changed, 238 insertions(+), 51 deletions(-) rename benchmark/compatible/duckdb/{run.py => load.py} (66%) mode change 100644 => 100755 create mode 100755 benchmark/compatible/duckdb/query.py create mode 100755 benchmark/compatible/duckdb/run.sh create mode 100755 benchmark/compatible/mysql-myisam/benchmark.sh create mode 100644 benchmark/compatible/mysql-myisam/create.sql create mode 100644 benchmark/compatible/mysql-myisam/queries.sql create mode 100644 benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt create mode 100755 benchmark/compatible/mysql-myisam/run.sh diff --git a/benchmark/compatible/citus/benchmark.sh b/benchmark/compatible/citus/benchmark.sh index 43b651e11ea..5c60d153438 100755 --- a/benchmark/compatible/citus/benchmark.sh +++ b/benchmark/compatible/citus/benchmark.sh @@ -1,8 +1,8 @@ #!/bin/bash sudo apt-get update -sudo apt-get install docker.io -sudo apt-get install postgresql-client +sudo apt-get install -y docker.io +sudo apt-get install -y postgresql-client sudo docker run -d --name citus -p 5432:5432 -e POSTGRES_PASSWORD=mypass citusdata/citus:11.0 diff --git a/benchmark/compatible/duckdb/benchmark.sh b/benchmark/compatible/duckdb/benchmark.sh index 838f4384c13..fc52266fa96 100755 --- a/benchmark/compatible/duckdb/benchmark.sh +++ b/benchmark/compatible/duckdb/benchmark.sh @@ -3,7 +3,7 @@ # Install sudo apt-get update -sudo apt-get install python3-pip +sudo apt-get install -y python3-pip pip install duckdb psutil # Load the data @@ -11,8 +11,11 @@ pip install duckdb psutil wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' gzip -d hits.csv.gz +./load.py +# 4216.5390389899985 seconds + # Run the queries -./run.expect +./run.sh | tee log.txt wc -c my-db.duckdb diff --git a/benchmark/compatible/duckdb/run.py b/benchmark/compatible/duckdb/load.py old mode 100644 new mode 100755 similarity index 66% rename from benchmark/compatible/duckdb/run.py rename to benchmark/compatible/duckdb/load.py index c0b3d8592f5..d900e8be61b --- a/benchmark/compatible/duckdb/run.py +++ b/benchmark/compatible/duckdb/load.py @@ -9,9 +9,6 @@ con = duckdb.connect(database='my-db.duckdb', read_only=False) con.execute("PRAGMA memory_limit='{}b'".format(psutil.virtual_memory().total / 4)) con.execute("PRAGMA threads={}".format(psutil.cpu_count(logical=False))) -queries_file = open('queries.sql', 'r') -queries = queries_file.readlines() - print("Will load the data") start = timeit.default_timer() @@ -19,11 +16,3 @@ con.execute(open('create.sql').read()) con.execute("INSERT INTO hits SELECT * FROM read_csv_auto('hits.csv')") end = timeit.default_timer() print(end - start) - -for query in queries: - print(query) - for try_num in range(3): - start = timeit.default_timer() - con.execute(query) - end = timeit.default_timer() - print(end - start) diff --git a/benchmark/compatible/duckdb/query.py b/benchmark/compatible/duckdb/query.py new file mode 100755 index 00000000000..6255d44a626 --- /dev/null +++ b/benchmark/compatible/duckdb/query.py @@ -0,0 +1,19 @@ +#!/usr/bin/env python3 + +import duckdb +import timeit +import psutil +import sys + +con = duckdb.connect(database='my-db.duckdb', read_only=False) +# See https://github.com/duckdb/duckdb/issues/3969 +con.execute("PRAGMA memory_limit='{}b'".format(psutil.virtual_memory().total / 4)) +con.execute("PRAGMA threads={}".format(psutil.cpu_count(logical=False))) + +query = sys.stdin.read() +print(query) +for try_num in range(3): + start = timeit.default_timer() + con.execute(query) + end = timeit.default_timer() + print(end - start) diff --git a/benchmark/compatible/duckdb/run.sh b/benchmark/compatible/duckdb/run.sh new file mode 100755 index 00000000000..0197c5baa73 --- /dev/null +++ b/benchmark/compatible/duckdb/run.sh @@ -0,0 +1,7 @@ +#!/bin/bash + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + ./query.py <<< "$(query)" +done diff --git a/benchmark/compatible/greenplum/benchmark.sh b/benchmark/compatible/greenplum/benchmark.sh index b83aebe13e6..23fa17714a3 100644 --- a/benchmark/compatible/greenplum/benchmark.sh +++ b/benchmark/compatible/greenplum/benchmark.sh @@ -1,31 +1,4 @@ #!/bin/bash -sudo apt-get install docker.io -sudo docker run --network host -it --volume=$(pwd):/workspace ubuntu:18.04 - -# https://greenplum.org/install-greenplum-oss-on-ubuntu/ -apt-get update -apt install software-properties-common -add-apt-repository ppa:greenplum/db -apt-get update -apt install greenplum-db-6 - -echo 'en_US.UTF-8 UTF-8' > /etc/locale.gen -dpkg-reconfigure --frontend=noninteractive locales - -useradd ubuntu -mkdir /home/ubuntu -chown ubuntu /home/ubuntu -su ubuntu -bash -cd ~ - -export GPHOME=/opt/greenplum-db-* -cp $GPHOME/docs/cli_help/gpconfigs/gpinitsystem_singlenode . - -source /opt/greenplum-db-*/greenplum_path.sh -echo $(hostname) > ./hostlist_singlenode - - - -cd workspace +sudo apt-get install -y docker.io +sudo docker run --name greenplum --volume=$(pwd):/workspace gptext/gpdb6 diff --git a/benchmark/compatible/mariadb-columnstore/benchmark.sh b/benchmark/compatible/mariadb-columnstore/benchmark.sh index e49626a8314..7fbfe341aff 100755 --- a/benchmark/compatible/mariadb-columnstore/benchmark.sh +++ b/benchmark/compatible/mariadb-columnstore/benchmark.sh @@ -3,7 +3,7 @@ # Install sudo apt-get update -sudo apt-get install docker.io +sudo apt-get install -y docker.io sudo docker run -d -p 3306:3306 -e ANALYTICS_ONLY=1 --name mcs_container mariadb/columnstore export PASSWORD="tsFgm457%3cj" diff --git a/benchmark/compatible/mariadb/benchmark.sh b/benchmark/compatible/mariadb/benchmark.sh index 1e0f148cadc..57a2387b530 100755 --- a/benchmark/compatible/mariadb/benchmark.sh +++ b/benchmark/compatible/mariadb/benchmark.sh @@ -3,7 +3,7 @@ # Install sudo apt-get update -sudo apt-get install mariadb-server +sudo apt-get install -y mariadb-server sudo bash -c "echo -e '[mysql]\nlocal-infile=1\n\n[mysqld]\nlocal-infile=1\n' > /etc/mysql/conf.d/local_infile.cnf" sudo service mariadb restart diff --git a/benchmark/compatible/monetdb/benchmark.sh b/benchmark/compatible/monetdb/benchmark.sh index b662bb5f59f..369596a4b24 100755 --- a/benchmark/compatible/monetdb/benchmark.sh +++ b/benchmark/compatible/monetdb/benchmark.sh @@ -6,7 +6,7 @@ echo "deb https://dev.monetdb.org/downloads/deb/ $(lsb_release -cs) monetdb" | s sudo wget --output-document=/etc/apt/trusted.gpg.d/monetdb.gpg https://www.monetdb.org/downloads/MonetDB-GPG-KEY.gpg sudo apt-get update -sudo apt-get install monetdb5-sql monetdb-client +sudo apt-get install -y monetdb5-sql monetdb-client sudo monetdbd create /var/lib/monetdb sudo monetdbd start /var/lib/monetdb diff --git a/benchmark/compatible/mysql-myisam/benchmark.sh b/benchmark/compatible/mysql-myisam/benchmark.sh new file mode 100755 index 00000000000..c70ff807b1c --- /dev/null +++ b/benchmark/compatible/mysql-myisam/benchmark.sh @@ -0,0 +1,29 @@ +#!/bin/bash + +# Install + +sudo apt-get update +sudo apt-get install -y mysql-server-8.0 +sudo bash -c "echo -e '[mysql]\nlocal-infile=1\n\n[mysqld]\nlocal-infile=1\n' > /etc/mysql/conf.d/local_infile.cnf" +sudo service mysql restart + +# Load the data + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +sudo mysql -e "CREATE DATABASE test" +sudo mysql test < create.sql +time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" + +# 2:37:52 elapsed + +./run.sh 2>&1 | tee log.txt + +sudo du -bcs /var/lib/mysql + +cat log.txt | + grep -P 'rows? in set|^ERROR' | + sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | + awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/mysql-myisam/create.sql b/benchmark/compatible/mysql-myisam/create.sql new file mode 100644 index 00000000000..961a0fea486 --- /dev/null +++ b/benchmark/compatible/mysql-myisam/create.sql @@ -0,0 +1,110 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +) +ENGINE = MyISAM; diff --git a/benchmark/compatible/mysql-myisam/queries.sql b/benchmark/compatible/mysql-myisam/queries.sql new file mode 100644 index 00000000000..1ff8a9daad8 --- /dev/null +++ b/benchmark/compatible/mysql-myisam/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt b/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..831235dc486 --- /dev/null +++ b/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt @@ -0,0 +1,2 @@ +Load time: +Data size: bytes diff --git a/benchmark/compatible/mysql-myisam/run.sh b/benchmark/compatible/mysql-myisam/run.sh new file mode 100755 index 00000000000..31f31682f32 --- /dev/null +++ b/benchmark/compatible/mysql-myisam/run.sh @@ -0,0 +1,12 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + for i in $(seq 1 $TRIES); do + sudo mysql test -vvv -e "${query}" + done; +done; diff --git a/benchmark/compatible/mysql/benchmark.sh b/benchmark/compatible/mysql/benchmark.sh index 0ecdbed42e8..c70ff807b1c 100755 --- a/benchmark/compatible/mysql/benchmark.sh +++ b/benchmark/compatible/mysql/benchmark.sh @@ -3,7 +3,7 @@ # Install sudo apt-get update -sudo apt-get install mysql-server-8.0 +sudo apt-get install -y mysql-server-8.0 sudo bash -c "echo -e '[mysql]\nlocal-infile=1\n\n[mysqld]\nlocal-infile=1\n' > /etc/mysql/conf.d/local_infile.cnf" sudo service mysql restart diff --git a/benchmark/compatible/postgresql/benchmark.sh b/benchmark/compatible/postgresql/benchmark.sh index 89c0378086a..1942fa4efaa 100755 --- a/benchmark/compatible/postgresql/benchmark.sh +++ b/benchmark/compatible/postgresql/benchmark.sh @@ -1,8 +1,8 @@ #!/bin/bash sudo apt-get update -sudo apt-get install postgresql-common -sudo apt-get install postgresql-14 +sudo apt-get install -y postgresql-common +sudo apt-get install -y postgresql-14 wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz diff --git a/benchmark/compatible/singlestore/benchmark.sh b/benchmark/compatible/singlestore/benchmark.sh index b088042930d..07fa597dedf 100755 --- a/benchmark/compatible/singlestore/benchmark.sh +++ b/benchmark/compatible/singlestore/benchmark.sh @@ -3,7 +3,7 @@ # Install sudo apt-get update -sudo apt-get install docker.io +sudo apt-get install -y docker.io export LICENSE_KEY="..." export ROOT_PASSWORD="..." From 424765f5968292a71f488e885f1b7d29d66876b7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 00:25:50 +0200 Subject: [PATCH 062/627] Compatible benchmark --- benchmark/compatible/duckdb/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmark/compatible/duckdb/run.sh b/benchmark/compatible/duckdb/run.sh index 0197c5baa73..5287c2e8d47 100755 --- a/benchmark/compatible/duckdb/run.sh +++ b/benchmark/compatible/duckdb/run.sh @@ -3,5 +3,5 @@ cat queries.sql | while read query; do sync echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - ./query.py <<< "$(query)" + ./query.py <<< "${query}" done From c953edbe3813ff074029d671036dba6a28585fff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 00:39:23 +0200 Subject: [PATCH 063/627] Compatible benchmark --- benchmark/compatible/duckdb/run.sh | 3 +++ benchmark/compatible/mysql-myisam/benchmark.sh | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/benchmark/compatible/duckdb/run.sh b/benchmark/compatible/duckdb/run.sh index 5287c2e8d47..68cea914794 100755 --- a/benchmark/compatible/duckdb/run.sh +++ b/benchmark/compatible/duckdb/run.sh @@ -3,5 +3,8 @@ cat queries.sql | while read query; do sync echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + + # We need to restart and reinitialize DuckDB after every query, + # because it often fails with Segmentation fault (core dumped) ./query.py <<< "${query}" done diff --git a/benchmark/compatible/mysql-myisam/benchmark.sh b/benchmark/compatible/mysql-myisam/benchmark.sh index c70ff807b1c..2119f547bc0 100755 --- a/benchmark/compatible/mysql-myisam/benchmark.sh +++ b/benchmark/compatible/mysql-myisam/benchmark.sh @@ -16,7 +16,7 @@ sudo mysql -e "CREATE DATABASE test" sudo mysql test < create.sql time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" -# 2:37:52 elapsed +# ./run.sh 2>&1 | tee log.txt From f117a86072122dd17105d5cca803909c5e068440 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 00:51:34 +0200 Subject: [PATCH 064/627] Compatible benchmark --- benchmark/compatible/citus/benchmark.sh | 6 +++--- benchmark/compatible/citus/create.sql | 3 ++- benchmark/compatible/citus/run.sh | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/benchmark/compatible/citus/benchmark.sh b/benchmark/compatible/citus/benchmark.sh index 5c60d153438..a1a23073e41 100755 --- a/benchmark/compatible/citus/benchmark.sh +++ b/benchmark/compatible/citus/benchmark.sh @@ -12,9 +12,9 @@ gzip -d hits.tsv.gz echo "*:*:*:*:mypass" > .pgpass chmod 400 .pgpass -psql -U postgres -h localhost --no-password -t -c 'CREATE DATABASE test' -psql -U postgres -h localhost --no-password test -t < create.sql -psql -U postgres -h localhost --no-password test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" +psql -U postgres -h localhost -d postgres --no-password -t -c 'CREATE DATABASE test' +psql -U postgres -h localhost -d postgres --no-password test -t < create.sql +psql -U postgres -h localhost -d postgres --no-password test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" # COPY 99997497 # Time: 2182125.360 ms (36:22.125) diff --git a/benchmark/compatible/citus/create.sql b/benchmark/compatible/citus/create.sql index 1850bffedce..ab013c71fd8 100644 --- a/benchmark/compatible/citus/create.sql +++ b/benchmark/compatible/citus/create.sql @@ -106,4 +106,5 @@ CREATE TABLE hits URLHash BIGINT NOT NULL, CLID INTEGER NOT NULL, PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); +) +USING COLUMNAR; diff --git a/benchmark/compatible/citus/run.sh b/benchmark/compatible/citus/run.sh index 4602870aae4..51f7c928764 100755 --- a/benchmark/compatible/citus/run.sh +++ b/benchmark/compatible/citus/run.sh @@ -8,6 +8,6 @@ cat queries.sql | while read query; do echo "$query"; for i in $(seq 1 $TRIES); do - psql -U postgres -h localhost --no-password test -t -c '\timing' -c "$query" | grep 'Time' + psql -U postgres -h localhost -d postgres --no-password test -t -c '\timing' -c "$query" | grep 'Time' done; done; From 602739d487d55a043fd2809508f9e70157981c6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 02:53:55 +0200 Subject: [PATCH 065/627] Compatible benchmark --- benchmark/compatible/citus/benchmark.sh | 2 +- benchmark/compatible/citus/run.sh | 2 +- benchmark/compatible/cratedb/benchmark.sh | 25 ++++ benchmark/compatible/cratedb/create.sql | 109 ++++++++++++++++++ benchmark/compatible/cratedb/queries.sql | 43 +++++++ .../cratedb/results/c6a.4xlarge.txt | 2 + benchmark/compatible/cratedb/run.sh | 13 +++ benchmark/compatible/duckdb/benchmark.sh | 5 +- benchmark/compatible/duckdb/query.py | 5 +- .../compatible/duckdb/results/c6a.4xlarge.txt | 46 ++++++++ .../compatible/mysql-myisam/benchmark.sh | 2 +- .../mysql-myisam/results/c6a.4xlarge.txt | 4 +- 12 files changed, 250 insertions(+), 8 deletions(-) create mode 100755 benchmark/compatible/cratedb/benchmark.sh create mode 100644 benchmark/compatible/cratedb/create.sql create mode 100644 benchmark/compatible/cratedb/queries.sql create mode 100644 benchmark/compatible/cratedb/results/c6a.4xlarge.txt create mode 100755 benchmark/compatible/cratedb/run.sh create mode 100644 benchmark/compatible/duckdb/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/citus/benchmark.sh b/benchmark/compatible/citus/benchmark.sh index a1a23073e41..5c023a0291b 100755 --- a/benchmark/compatible/citus/benchmark.sh +++ b/benchmark/compatible/citus/benchmark.sh @@ -17,7 +17,7 @@ psql -U postgres -h localhost -d postgres --no-password test -t < create.sql psql -U postgres -h localhost -d postgres --no-password test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" # COPY 99997497 -# Time: 2182125.360 ms (36:22.125) +# Time: 1579203.482 ms (26:19.203) ./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/citus/run.sh b/benchmark/compatible/citus/run.sh index 51f7c928764..7adee7c460d 100755 --- a/benchmark/compatible/citus/run.sh +++ b/benchmark/compatible/citus/run.sh @@ -8,6 +8,6 @@ cat queries.sql | while read query; do echo "$query"; for i in $(seq 1 $TRIES); do - psql -U postgres -h localhost -d postgres --no-password test -t -c '\timing' -c "$query" | grep 'Time' + psql -U postgres -h localhost -d postgres --no-password -t -c '\timing' -c "$query" | grep 'Time' done; done; diff --git a/benchmark/compatible/cratedb/benchmark.sh b/benchmark/compatible/cratedb/benchmark.sh new file mode 100755 index 00000000000..3fd729de9f8 --- /dev/null +++ b/benchmark/compatible/cratedb/benchmark.sh @@ -0,0 +1,25 @@ +#!/bin/bash + +bash -c "$(curl -L https://try.crate.io/)" > crate.log 2>&1 & + +sudo apt-get update +sudo apt-get install -y postgresql-client + +psql -U crate -h localhost --no-password -t -c 'SELECT 1' + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +psql -U crate -h localhost --no-password -t -c 'CREATE DATABASE test' +psql -U crate -h localhost --no-password test -t < create.sql +psql -U crate -h localhost --no-password test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" + +# +# + +./run.sh 2>&1 | tee log.txt + +du -bcs crate-* + +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/cratedb/create.sql b/benchmark/compatible/cratedb/create.sql new file mode 100644 index 00000000000..1850bffedce --- /dev/null +++ b/benchmark/compatible/cratedb/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/cratedb/queries.sql b/benchmark/compatible/cratedb/queries.sql new file mode 100644 index 00000000000..2f986cda146 --- /dev/null +++ b/benchmark/compatible/cratedb/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/cratedb/results/c6a.4xlarge.txt b/benchmark/compatible/cratedb/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..b4d66d3a983 --- /dev/null +++ b/benchmark/compatible/cratedb/results/c6a.4xlarge.txt @@ -0,0 +1,2 @@ +Load time: +Data size: diff --git a/benchmark/compatible/cratedb/run.sh b/benchmark/compatible/cratedb/run.sh new file mode 100755 index 00000000000..eef0153c260 --- /dev/null +++ b/benchmark/compatible/cratedb/run.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + echo "$query"; + for i in $(seq 1 $TRIES); do + psql -U crate -h localhost --no-password test -t -c '\timing' -c "$query" | grep 'Time' + done; +done; diff --git a/benchmark/compatible/duckdb/benchmark.sh b/benchmark/compatible/duckdb/benchmark.sh index fc52266fa96..392f084c560 100755 --- a/benchmark/compatible/duckdb/benchmark.sh +++ b/benchmark/compatible/duckdb/benchmark.sh @@ -16,6 +16,9 @@ gzip -d hits.csv.gz # Run the queries -./run.sh | tee log.txt +./run.sh 2>&1 | tee log.txt wc -c my-db.duckdb + +cat log.txt | grep -P '^\d|Killed|Segmentation' | sed -r -e 's/^.*(Killed|Segmentation).*$/null\nnull\nnull/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/duckdb/query.py b/benchmark/compatible/duckdb/query.py index 6255d44a626..c1b07890c22 100755 --- a/benchmark/compatible/duckdb/query.py +++ b/benchmark/compatible/duckdb/query.py @@ -5,13 +5,14 @@ import timeit import psutil import sys +query = sys.stdin.read() +print(query) + con = duckdb.connect(database='my-db.duckdb', read_only=False) # See https://github.com/duckdb/duckdb/issues/3969 con.execute("PRAGMA memory_limit='{}b'".format(psutil.virtual_memory().total / 4)) con.execute("PRAGMA threads={}".format(psutil.cpu_count(logical=False))) -query = sys.stdin.read() -print(query) for try_num in range(3): start = timeit.default_timer() con.execute(query) diff --git a/benchmark/compatible/duckdb/results/c6a.4xlarge.txt b/benchmark/compatible/duckdb/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..3423d11bd70 --- /dev/null +++ b/benchmark/compatible/duckdb/results/c6a.4xlarge.txt @@ -0,0 +1,46 @@ +Load time: 4217 seconds +Data size: 27 241 492 480 bytes + +[0.005694353996659629,0.003944558004150167,0.003837226002360694], +[0.16991353100456763,0.03919722700084094,0.03835860399703961], +[0.44898432699847035,0.04947217500011902,0.04852217998995911], +[0.07586832098604646,0.07051395199960098,0.07007493599667214], +[9.554053236002801,8.153356187991449,8.73448242500308], +[7.66042533799191,6.931124911992811,7.103380946995458], +[0.030703739990713075,0.027668555994750932,0.027583695002249442], +[0.1778664360026596,0.03942437999648973,0.03882004099432379], +[8.53439180701389,8.869582625004114,9.020313234999776], +[10.40215514000738,11.125320470004226,8.941559945000336], +[1.1747649609897053,1.04221136700653,1.004799570000614], +[1.2380354650085792,1.1211603130068397,2.4278587239969056], +[3.1751541379926493,0.9360461989999749,0.8868292279948946], +[6.855684430003748,7.300301584007684,5.712960822012974], +[3.70588762400439,1.0249276379909134,0.9473389159975341], +[2.1037107890006155,1.6215517020027619,1.5671920729946578], +[null,null,null], +[null,null,null], +[null,null,null], +[0.0002772739971987903,0.00016792300448287278,0.0001574420020915568], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[2.9310110910009826,0.19020285899750888,0.1736805049877148], +[2.939304119994631,0.18754731099761557,0.18073286200524308], +[2.8706370779982535,0.18822155400994234,0.17905898999015335], +[null,null,null], +[null,null,null], +[0.884408778991201,0.714329167996766,0.7135983259940986], +[5.3762675570033025,0.8803737630078103,0.8728962720051641], +[7.249190265996731,2.9648747390019707,2.866687831003219], +[null,null,null], +[null,null,null], +[null,null,null], +[4.515183198003797,4.030519469000865,4.014251719010645], +[0.11604027298744768,0.040539135996368714,0.04280066800129134], +[0.0457908230018802,0.021069509006338194,0.019683108999743126], +[0.0680370800109813,0.011889394998434,0.01056639499438461], +[0.22029169600864407,0.08547276000899728,0.09095505000732373], +[0.03759863799496088,0.008373684002435766,0.007633563989656977], +[0.025631797994719818,0.008081699008471332,0.007858585988287814], +[0.034359957004198804,0.025543516996549442,0.02533275399764534] diff --git a/benchmark/compatible/mysql-myisam/benchmark.sh b/benchmark/compatible/mysql-myisam/benchmark.sh index 2119f547bc0..5b83fa30c42 100755 --- a/benchmark/compatible/mysql-myisam/benchmark.sh +++ b/benchmark/compatible/mysql-myisam/benchmark.sh @@ -16,7 +16,7 @@ sudo mysql -e "CREATE DATABASE test" sudo mysql test < create.sql time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" -# +# 41m8.979s ./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt b/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt index 831235dc486..4723e639dda 100644 --- a/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt +++ b/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt @@ -1,2 +1,2 @@ -Load time: -Data size: bytes +Load time: 41:09 +Data size: 121 588 958 061 bytes From 486cc06880bd9548dc7a9721d99cdffddc3e2fb8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 03:48:57 +0200 Subject: [PATCH 066/627] Compatible benchmark --- benchmark/compatible/citus/benchmark.sh | 4 +- .../compatible/citus/results/c6a.4xlarge.txt | 46 ++++++++ benchmark/compatible/cratedb/benchmark.sh | 12 +- benchmark/compatible/cratedb/create.sql | 2 +- benchmark/compatible/cratedb/run.sh | 2 +- benchmark/compatible/postgresql/benchmark.sh | 2 +- .../postgresql/results/c6a.4xlarge.txt | 86 +++++++------- benchmark/compatible/timescaledb/benchmark.sh | 2 +- .../timescaledb/results/c6a.4xlarge.txt | 86 +++++++------- benchmark/compatible/vertica/README.md | 5 + benchmark/compatible/vertica/benchmark.sh | 11 ++ benchmark/compatible/vertica/create.sql | 109 ++++++++++++++++++ 12 files changed, 269 insertions(+), 98 deletions(-) create mode 100644 benchmark/compatible/citus/results/c6a.4xlarge.txt create mode 100644 benchmark/compatible/vertica/README.md create mode 100644 benchmark/compatible/vertica/benchmark.sh create mode 100644 benchmark/compatible/vertica/create.sql diff --git a/benchmark/compatible/citus/benchmark.sh b/benchmark/compatible/citus/benchmark.sh index 5c023a0291b..c7b71b367c4 100755 --- a/benchmark/compatible/citus/benchmark.sh +++ b/benchmark/compatible/citus/benchmark.sh @@ -21,7 +21,7 @@ psql -U postgres -h localhost -d postgres --no-password test -t -c '\timing' -c ./run.sh 2>&1 | tee log.txt -#sudo du -bcs /var/lib/postgresql/14/main/ +sudo docker exec -it citus du -bcs /var/lib/postgresql/data cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/citus/results/c6a.4xlarge.txt b/benchmark/compatible/citus/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..5a98ce455bd --- /dev/null +++ b/benchmark/compatible/citus/results/c6a.4xlarge.txt @@ -0,0 +1,46 @@ +Load time: 1579 sec +Data size: 18 980 918 899 bytes + +[7.58503,6.70447,6.52499], +[6.33941,5.06063,5.00238], +[11.7488,9.86417,9.93223], +[12.6306,9.36305,9.17061], +[40.6101,39.0803,38.1187], +[117.654,113.912,113.441], +[10.3404,8.08936,7.70732], +[6.31542,4.72821,4.72989], +[82.5425,77.2124,76.9219], +[91.1776,83.4492,82.4727], +[14.5474,10.0815,10.3873], +[15.4899,11.2922,11.1877], +[19.9794,15.5002,17.4492], +[76.9216,72.5172,72.7915], +[21.5446,17.5691,18.561], +[56.9438,54.6387,53.5745], +[75.0977,69.7842,70.0259], +[31.3299,27.0267,26.3216], +[129.417,122.956,121.182], +[3.73386,2.14148,2.12737], +[34.6021,27.9727,28.6878], +[37.152,29.6193,29.2966], +[52.2157,37.8589,37.6994], +[181.955,149.08,148.471], +[15.4687,11.3138,10.3856], +[10.2779,8.46868,8.8324], +[14.4687,10.4076,11.4263], +[47.009,40.2969,39.6888], +[749.946,742.979,744.461], +[69.4383,67.5636,67.2128], +[27.0317,21.4008,20.9524], +[36.6675,25.6347,26.4408], +[140.424,130.546,129.738], +[106.959,92.033,90.1609], +[110.98,94.4787,96.2656], +[64.4474,60.1853,60.6816], +[6.17549,6.25376,5.87004], +[1.99153,1.81776,1.80596], +[1.00141,0.800271,0.801975], +[7.91778,7.70928,8.33299], +[0.929845,0.642076,0.638478], +[0.866536,0.683567,0.680218], +[0.937823,0.784747,0.765929] diff --git a/benchmark/compatible/cratedb/benchmark.sh b/benchmark/compatible/cratedb/benchmark.sh index 3fd729de9f8..d05772f2181 100755 --- a/benchmark/compatible/cratedb/benchmark.sh +++ b/benchmark/compatible/cratedb/benchmark.sh @@ -10,16 +10,16 @@ psql -U crate -h localhost --no-password -t -c 'SELECT 1' wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz -psql -U crate -h localhost --no-password -t -c 'CREATE DATABASE test' -psql -U crate -h localhost --no-password test -t < create.sql -psql -U crate -h localhost --no-password test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" +psql -U crate -h localhost --no-password -t < create.sql +psql -U crate -h localhost --no-password -t -c '\timing' -c "COPY hits FROM 'file:///$(pwd)/hits.tsv'" -# -# +# It failed to load the data. +# COPY 0 +# Time: 1004421.355 ms (16:44.421) ./run.sh 2>&1 | tee log.txt du -bcs crate-* cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/cratedb/create.sql b/benchmark/compatible/cratedb/create.sql index 1850bffedce..c9d65a98bb9 100644 --- a/benchmark/compatible/cratedb/create.sql +++ b/benchmark/compatible/cratedb/create.sql @@ -5,7 +5,7 @@ CREATE TABLE hits Title TEXT NOT NULL, GoodEvent SMALLINT NOT NULL, EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, + EventDate TIMESTAMP NOT NULL, CounterID INTEGER NOT NULL, ClientIP INTEGER NOT NULL, RegionID INTEGER NOT NULL, diff --git a/benchmark/compatible/cratedb/run.sh b/benchmark/compatible/cratedb/run.sh index eef0153c260..477f463289b 100755 --- a/benchmark/compatible/cratedb/run.sh +++ b/benchmark/compatible/cratedb/run.sh @@ -8,6 +8,6 @@ cat queries.sql | while read query; do echo "$query"; for i in $(seq 1 $TRIES); do - psql -U crate -h localhost --no-password test -t -c '\timing' -c "$query" | grep 'Time' + psql -U crate -h localhost --no-password -t -c '\timing' -c "$query" | grep 'Time' done; done; diff --git a/benchmark/compatible/postgresql/benchmark.sh b/benchmark/compatible/postgresql/benchmark.sh index 1942fa4efaa..13459abfe8a 100755 --- a/benchmark/compatible/postgresql/benchmark.sh +++ b/benchmark/compatible/postgresql/benchmark.sh @@ -20,4 +20,4 @@ sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" sudo du -bcs /var/lib/postgresql/14/main/ cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/postgresql/results/c6a.4xlarge.txt b/benchmark/compatible/postgresql/results/c6a.4xlarge.txt index 76cec0f1d5d..f08de68b864 100644 --- a/benchmark/compatible/postgresql/results/c6a.4xlarge.txt +++ b/benchmark/compatible/postgresql/results/c6a.4xlarge.txt @@ -1,46 +1,46 @@ Load time: 2342 sec Data size: 77 797 067 741 bytes -[439753.348,309785.749,282017.708], -[317874.293,254238.464,254941.769], -[262883.138,263072.659,263090.967], -[32421.649,5310.077,5060.922], -[57134.664,42648.458,42334.061], -[358423.366,356315.487,358342.644], -[31524.566,5350.729,4994.452], -[263145.548,263193.332,263165.819], -[323659.951,322858.223,321918.464], -[327395.332,326170.524,326231.049], -[265983.201,265681.029,265912.645], -[269984.881,265336.483,265379.996], -[284096.661,284560.754,282234.726], -[277250.434,279455.446,280035.179], -[285660.969,286200.471,283611.116], -[66605.451,32023.025,38282.816], -[312452.824,304431.252,305391.155], -[289209.107,290449.044,287578.843], -[331706.364,327485.227,334428.263], -[24646.119,2543.388,2263.010], -[267561.278,267496.242,267524.846], -[267729.041,267690.012,268184.240], -[263074.820,263120.536,267040.337], -[267602.971,267488.950,267494.446], -[263141.314,263859.779,263137.581], -[262923.219,263102.864,263113.550], -[262885.041,263088.582,263114.552], -[267864.506,269127.149,268204.402], -[303376.276,306925.881,308664.073], -[263221.039,263119.663,263148.959], -[270814.146,270575.238,270294.943], -[278342.802,275925.336,276224.959], -[584599.924,576932.765,591502.540], -[462576.063,446962.684,439779.116], -[429930.882,417696.097,416704.016], -[296875.751,297283.343,295140.467], -[3461.727,842.168,794.721], -[2179.836,564.875,558.321], -[2258.891,566.078,416.122], -[2805.199,1311.129,1317.189], -[2936.932,820.554,615.959], -[2197.875,736.053,535.611], -[1983.240,320.960,312.554] +[439.753,309.785,282.017], +[317.874,254.238,254.941], +[262.883,263.072,263.090], +[32.421,5.310,5.060], +[57.134,42.648,42.334], +[358.423,356.315,358.342], +[31.524,5.350,4.994], +[263.145,263.193,263.165], +[323.659,322.858,321.918], +[327.395,326.170,326.231], +[265.983,265.681,265.912], +[269.984,265.336,265.379], +[284.096,284.560,282.234], +[277.250,279.455,280.035], +[285.660,286.200,283.611], +[66.605,32.023,38.282], +[312.452,304.431,305.391], +[289.209,290.449,287.578], +[331.706,327.485,334.428], +[24.646,2.543,2.263], +[267.561,267.496,267.524], +[267.729,267.690,268.184], +[263.074,263.120,267.040], +[267.602,267.488,267.494], +[263.141,263.859,263.137], +[262.923,263.102,263.113], +[262.885,263.088,263.114], +[267.864,269.127,268.204], +[303.376,306.925,308.664], +[263.221,263.119,263.148], +[270.814,270.575,270.294], +[278.342,275.925,276.224], +[584.599,576.932,591.502], +[462.576,446.962,439.779], +[429.930,417.696,416.704], +[296.875,297.283,295.140], +[3.461,.842,.794], +[2.179,.564,.558], +[2.258,.566,.416], +[2.805,1.311,1.317], +[2.936,.820,.615], +[2.197,.736,.535], +[1.983,.320,.312] diff --git a/benchmark/compatible/timescaledb/benchmark.sh b/benchmark/compatible/timescaledb/benchmark.sh index 97e30fb894a..a36f351e76c 100755 --- a/benchmark/compatible/timescaledb/benchmark.sh +++ b/benchmark/compatible/timescaledb/benchmark.sh @@ -36,4 +36,4 @@ sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" sudo du -bcs /var/lib/postgresql/14/main/ cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt b/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt index c3d2fd3094c..7c64aa755ba 100644 --- a/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt +++ b/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt @@ -1,46 +1,46 @@ Load time: 1620 seconds Data size: 72 882 392 030 bytes -[437700.229,215793.314,176420.897], -[327026.232,259568.339,244578.780], -[262978.382,263090.178,263083.484], -[262807.910,263046.124,266847.716], -[337497.985,334964.363,330852.141], -[355689.862,356801.410,362894.372], -[262762.474,263012.925,262968.088], -[263055.362,263016.424,263028.660], -[319928.080,319388.513,320704.735], -[323584.015,322224.697,322488.413], -[265979.137,265465.789,265375.740], -[266019.708,265543.152,265462.654], -[277018.786,276300.778,276595.150], -[280352.481,279251.213,279572.251], -[279915.935,279896.775,279674.285], -[296377.147,298506.713,297659.537], -[314448.530,314605.256,312570.831], -[302668.006,302672.507,303039.646], -[325810.527,324061.457,324376.415], -[262447.995,262698.364,262704.067], -[267581.759,267467.643,267482.050], -[268085.555,267466.797,267696.901], -[263391.829,263097.590,263126.579], -[38291.424,435.353,335.331], -[127.768,5.054,5.307], -[263138.877,263100.888,263092.929], -[889.696,341.106,339.118], -[267586.222,267498.547,267491.245], -[289086.248,290012.052,290093.278], -[263220.195,263071.707,263109.600], -[274780.156,273995.447,273998.676], -[282217.820,281390.700,281470.632], -[429273.892,426588.599,439431.586], -[448808.430,418724.274,418207.674], -[455196.314,422750.955,423142.946], -[299263.589,296937.462,297261.160], -[18693.445,1552.021,1481.741], -[18125.858,945.707,937.442], -[18528.375,1062.610,902.568], -[18268.028,1779.357,1770.984], -[19615.320,1965.590,1966.551], -[18970.537,1435.093,1430.706], -[18330.728,1153.613,952.997] +[437.700,215.793,176.420], +[327.026,259.568,244.578], +[262.978,263.090,263.083], +[262.807,263.046,266.847], +[337.497,334.964,330.852], +[355.689,356.801,362.894], +[262.762,263.012,262.968], +[263.055,263.016,263.028], +[319.928,319.388,320.704], +[323.584,322.224,322.488], +[265.979,265.465,265.375], +[266.019,265.543,265.462], +[277.018,276.300,276.595], +[280.352,279.251,279.572], +[279.915,279.896,279.674], +[296.377,298.506,297.659], +[314.448,314.605,312.570], +[302.668,302.672,303.039], +[325.810,324.061,324.376], +[262.447,262.698,262.704], +[267.581,267.467,267.482], +[268.085,267.466,267.696], +[263.391,263.097,263.126], +[38.291,0.435,0.335], +[0.127,0.005,0.005], +[263.138,263.100,263.092], +[0.889,0.341,0.339], +[267.586,267.498,267.491], +[289.086,290.012,290.093], +[263.220,263.071,263.109], +[274.780,273.995,273.998], +[282.217,281.390,281.470], +[429.273,426.588,439.431], +[448.808,418.724,418.207], +[455.196,422.750,423.142], +[299.263,296.937,297.261], +[18.693,1.552,1.481], +[18.125,0.945,0.937], +[18.528,1.062,0.902], +[18.268,1.779,1.770], +[19.615,1.965,1.966], +[18.970,1.435,1.430], +[18.330,1.153,0.952] diff --git a/benchmark/compatible/vertica/README.md b/benchmark/compatible/vertica/README.md new file mode 100644 index 00000000000..0adab4f41f9 --- /dev/null +++ b/benchmark/compatible/vertica/README.md @@ -0,0 +1,5 @@ +Although Vertica EULA does not prevent doing benchmarks, it restricts from disclosing the results: + +> You may not disclose to any third-party performance information or analysis (including, without limitation, benchmarks and performance tests) from any source relating to the Software. + +https://www.vertica.com/end-user-license-agreement-ce-version/ diff --git a/benchmark/compatible/vertica/benchmark.sh b/benchmark/compatible/vertica/benchmark.sh new file mode 100644 index 00000000000..9ece8ab3428 --- /dev/null +++ b/benchmark/compatible/vertica/benchmark.sh @@ -0,0 +1,11 @@ +#!/bin/bash + +sudo apt-get update +sudo apt-get install -y docker.io + +sudo docker run --name vertica -p 5433:5433 -p 5444:5444 --mount type=volume,source=vertica-data,target=/data --name vertica_ce vertica/vertica-ce + +sudo docker exec vertica /opt/vertica/bin/vsql -U dbadmin -c "$(cat create.sql)" + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz diff --git a/benchmark/compatible/vertica/create.sql b/benchmark/compatible/vertica/create.sql new file mode 100644 index 00000000000..a6316c5e227 --- /dev/null +++ b/benchmark/compatible/vertica/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title VARCHAR(65000) NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL VARCHAR(65000) NOT NULL, + Referer VARCHAR(65000) NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 VARCHAR(65000) NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel VARCHAR(65000) NOT NULL, + Params VARCHAR(65000) NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase VARCHAR(65000) NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset VARCHAR(65000) NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL VARCHAR(65000) NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage VARCHAR(65000) NOT NULL, + BrowserCountry VARCHAR(65000) NOT NULL, + SocialNetwork VARCHAR(65000) NOT NULL, + SocialAction VARCHAR(65000) NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage VARCHAR(65000) NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID VARCHAR(65000) NOT NULL, + ParamCurrency VARCHAR(65000) NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName VARCHAR(65000) NOT NULL, + OpenstatCampaignID VARCHAR(65000) NOT NULL, + OpenstatAdID VARCHAR(65000) NOT NULL, + OpenstatSourceID VARCHAR(65000) NOT NULL, + UTMSource VARCHAR(65000) NOT NULL, + UTMMedium VARCHAR(65000) NOT NULL, + UTMCampaign VARCHAR(65000) NOT NULL, + UTMContent VARCHAR(65000) NOT NULL, + UTMTerm VARCHAR(65000) NOT NULL, + FromTag VARCHAR(65000) NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +) +ORDER BY CounterID, EventDate, UserID, EventTime, WatchID; From a091a5655c9cc2c2d2f015d02223f4a49106595f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 04:04:53 +0200 Subject: [PATCH 067/627] Compatible benchmark --- benchmark/compatible/cratedb/benchmark.sh | 6 +++--- benchmark/compatible/vertica/benchmark.sh | 4 ++++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/benchmark/compatible/cratedb/benchmark.sh b/benchmark/compatible/cratedb/benchmark.sh index d05772f2181..033ed0dfe01 100755 --- a/benchmark/compatible/cratedb/benchmark.sh +++ b/benchmark/compatible/cratedb/benchmark.sh @@ -7,11 +7,11 @@ sudo apt-get install -y postgresql-client psql -U crate -h localhost --no-password -t -c 'SELECT 1' -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +gzip -d hits.csv.gz psql -U crate -h localhost --no-password -t < create.sql -psql -U crate -h localhost --no-password -t -c '\timing' -c "COPY hits FROM 'file:///$(pwd)/hits.tsv'" +psql -U crate -h localhost --no-password -t -c '\timing' -c "COPY hits FROM 'file:///$(pwd)/hits.csv'" # It failed to load the data. # COPY 0 diff --git a/benchmark/compatible/vertica/benchmark.sh b/benchmark/compatible/vertica/benchmark.sh index 9ece8ab3428..160f7dfc3be 100644 --- a/benchmark/compatible/vertica/benchmark.sh +++ b/benchmark/compatible/vertica/benchmark.sh @@ -9,3 +9,7 @@ sudo docker exec vertica /opt/vertica/bin/vsql -U dbadmin -c "$(cat create.sql)" wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz + +sudo docker cp hits.tsv vertica:/hits.tsv + +time sudo docker exec vertica /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM '/hits.tsv' DELIMITER E'\\t'" From f6d5a00d15da38e9b5c8f4fa183fb2b012140388 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 05:31:19 +0200 Subject: [PATCH 068/627] Compatible benchmark --- benchmark/compatible/citus/queries.sql | 38 +++--- benchmark/compatible/clickhouse/queries.sql | 38 +++--- benchmark/compatible/cratedb/queries.sql | 38 +++--- benchmark/compatible/duckdb/queries.sql | 38 +++--- benchmark/compatible/greenplum/benchmark.sh | 17 ++- benchmark/compatible/heavyai/queries.sql | 38 +++--- benchmark/compatible/mariadb/queries.sql | 38 +++--- benchmark/compatible/monetdb/benchmark.sh | 19 +++ benchmark/compatible/monetdb/create.sql | 108 ++++++++++++++++++ benchmark/compatible/monetdb/queries.sql | 43 +++++++ benchmark/compatible/monetdb/query.expect | 10 ++ benchmark/compatible/monetdb/run.sh | 12 ++ benchmark/compatible/mysql-myisam/queries.sql | 38 +++--- benchmark/compatible/mysql/queries.sql | 38 +++--- benchmark/compatible/postgresql/queries.sql | 38 +++--- benchmark/compatible/questdb/queries.sql | 38 +++--- benchmark/compatible/singlestore/queries.sql | 38 +++--- benchmark/compatible/timescaledb/queries.sql | 38 +++--- benchmark/compatible/vertica/benchmark.sh | 8 +- 19 files changed, 439 insertions(+), 234 deletions(-) create mode 100644 benchmark/compatible/monetdb/create.sql create mode 100644 benchmark/compatible/monetdb/queries.sql create mode 100644 benchmark/compatible/monetdb/query.expect create mode 100644 benchmark/compatible/monetdb/run.sh diff --git a/benchmark/compatible/citus/queries.sql b/benchmark/compatible/citus/queries.sql index 2f986cda146..31f65fc898d 100644 --- a/benchmark/compatible/citus/queries.sql +++ b/benchmark/compatible/citus/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/clickhouse/queries.sql b/benchmark/compatible/clickhouse/queries.sql index 2f986cda146..31f65fc898d 100644 --- a/benchmark/compatible/clickhouse/queries.sql +++ b/benchmark/compatible/clickhouse/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/cratedb/queries.sql b/benchmark/compatible/cratedb/queries.sql index 2f986cda146..31f65fc898d 100644 --- a/benchmark/compatible/cratedb/queries.sql +++ b/benchmark/compatible/cratedb/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/duckdb/queries.sql b/benchmark/compatible/duckdb/queries.sql index 2f986cda146..31f65fc898d 100644 --- a/benchmark/compatible/duckdb/queries.sql +++ b/benchmark/compatible/duckdb/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/greenplum/benchmark.sh b/benchmark/compatible/greenplum/benchmark.sh index 23fa17714a3..a83363fbcfa 100644 --- a/benchmark/compatible/greenplum/benchmark.sh +++ b/benchmark/compatible/greenplum/benchmark.sh @@ -1,4 +1,17 @@ #!/bin/bash -sudo apt-get install -y docker.io -sudo docker run --name greenplum --volume=$(pwd):/workspace gptext/gpdb6 +# NOTE: it requires Ubuntu 18.04 +# Greenplum does not install on any newer system. + +sudo apt update +sudo apt install -y software-properties-common +sudo add-apt-repository ppa:greenplum/db +sudo apt update +sudo apt install greenplum-db-6 +source /opt/greenplum-db-*.0/greenplum_path.sh +cp $GPHOME/docs/cli_help/gpconfigs/gpinitsystem_singlenode . +echo localhost > ./hostlist_singlenode +sed -i "s/MASTER_HOSTNAME=[a-z_]*/MASTER_HOSTNAME=$(hostname)/" gpinitsystem_singlenode +sudo mkdir /gpmaster +sudo chmod 777 /gpmaster +gpinitsystem -c gpinitsystem_singlenode diff --git a/benchmark/compatible/heavyai/queries.sql b/benchmark/compatible/heavyai/queries.sql index 2f986cda146..31f65fc898d 100644 --- a/benchmark/compatible/heavyai/queries.sql +++ b/benchmark/compatible/heavyai/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/mariadb/queries.sql b/benchmark/compatible/mariadb/queries.sql index 2f986cda146..31f65fc898d 100644 --- a/benchmark/compatible/mariadb/queries.sql +++ b/benchmark/compatible/mariadb/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/monetdb/benchmark.sh b/benchmark/compatible/monetdb/benchmark.sh index 369596a4b24..5e77fc2e5c8 100755 --- a/benchmark/compatible/monetdb/benchmark.sh +++ b/benchmark/compatible/monetdb/benchmark.sh @@ -8,8 +8,27 @@ sudo wget --output-document=/etc/apt/trusted.gpg.d/monetdb.gpg https://www.monet sudo apt-get update sudo apt-get install -y monetdb5-sql monetdb-client +sudo systemctl enable monetdbd +sudo systemctl start monetdbd + sudo monetdbd create /var/lib/monetdb sudo monetdbd start /var/lib/monetdb +sudo usermod -a -G monetdb $USER sudo monetdb create test sudo monetdb release test + +sudo apt-get install -y expect + +./query.expect "$(cat create.sql)" + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz +chmod 777 ~ hits.tsv + +./query.expect "COPY INTO hits FROM '$(pwd)/hits.tsv' USING DELIMITERS '\t'" + +# 99997497 affected rows +# clk: 15:39 min + +./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/monetdb/create.sql b/benchmark/compatible/monetdb/create.sql new file mode 100644 index 00000000000..41c961c00fc --- /dev/null +++ b/benchmark/compatible/monetdb/create.sql @@ -0,0 +1,108 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +); diff --git a/benchmark/compatible/monetdb/queries.sql b/benchmark/compatible/monetdb/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/monetdb/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/monetdb/query.expect b/benchmark/compatible/monetdb/query.expect new file mode 100644 index 00000000000..8e8e43cea98 --- /dev/null +++ b/benchmark/compatible/monetdb/query.expect @@ -0,0 +1,10 @@ +#!/usr/bin/expect + +set timeout 3600 +set query [lindex $argv 0] + +spawn mclient -u monetdb -d test; +expect "password:"; send "monetdb\r"; +expect "sql>"; send "\\t clock\r"; +expect "sql>"; send "$query;\r"; +expect "sql>"; send "\4" diff --git a/benchmark/compatible/monetdb/run.sh b/benchmark/compatible/monetdb/run.sh new file mode 100644 index 00000000000..c1eadeab22c --- /dev/null +++ b/benchmark/compatible/monetdb/run.sh @@ -0,0 +1,12 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + for i in $(seq 1 $TRIES); do + ./query.expect "$query" 2>&1 + done; +done; diff --git a/benchmark/compatible/mysql-myisam/queries.sql b/benchmark/compatible/mysql-myisam/queries.sql index 1ff8a9daad8..ea2bde47802 100644 --- a/benchmark/compatible/mysql-myisam/queries.sql +++ b/benchmark/compatible/mysql-myisam/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/mysql/queries.sql b/benchmark/compatible/mysql/queries.sql index 1ff8a9daad8..ea2bde47802 100644 --- a/benchmark/compatible/mysql/queries.sql +++ b/benchmark/compatible/mysql/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/postgresql/queries.sql b/benchmark/compatible/postgresql/queries.sql index 2f986cda146..31f65fc898d 100644 --- a/benchmark/compatible/postgresql/queries.sql +++ b/benchmark/compatible/postgresql/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/questdb/queries.sql b/benchmark/compatible/questdb/queries.sql index 546606439f7..5ddf9fa318c 100644 --- a/benchmark/compatible/questdb/queries.sql +++ b/benchmark/compatible/questdb/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT count_distinct(CAST(UserID AS string)) FROM hits; SELECT count_distinct(SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) AS c FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; +SELECT AdvEngineID, COUNT(*) AS c FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY c DESC; SELECT RegionID, count_distinct(CAST(UserID AS string)) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), count_distinct(CAST(UserID AS string)) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) AS c FROM hits GROUP BY UserID ORDER BY c DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) AS c FROM hits GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) AS c FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, count_distinct(CAST(UserID AS string)) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, count_distinct(CAST(UserID AS string)) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000, 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000, 10; 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 EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000, 10; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100, 10; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 100000, 10; diff --git a/benchmark/compatible/singlestore/queries.sql b/benchmark/compatible/singlestore/queries.sql index 2f986cda146..31f65fc898d 100644 --- a/benchmark/compatible/singlestore/queries.sql +++ b/benchmark/compatible/singlestore/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/timescaledb/queries.sql b/benchmark/compatible/timescaledb/queries.sql index 2f986cda146..31f65fc898d 100644 --- a/benchmark/compatible/timescaledb/queries.sql +++ b/benchmark/compatible/timescaledb/queries.sql @@ -1,42 +1,42 @@ SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID != 0; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer != '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; diff --git a/benchmark/compatible/vertica/benchmark.sh b/benchmark/compatible/vertica/benchmark.sh index 160f7dfc3be..9047686597f 100644 --- a/benchmark/compatible/vertica/benchmark.sh +++ b/benchmark/compatible/vertica/benchmark.sh @@ -7,9 +7,9 @@ sudo docker run --name vertica -p 5433:5433 -p 5444:5444 --mount type=volume,sou sudo docker exec vertica /opt/vertica/bin/vsql -U dbadmin -c "$(cat create.sql)" -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +gzip -d hits.csv.gz -sudo docker cp hits.tsv vertica:/hits.tsv +sudo docker cp hits.tsv vertica:/hits.csv -time sudo docker exec vertica /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM '/hits.tsv' DELIMITER E'\\t'" +time sudo docker exec 4e9120959a41 /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM '/hits.csv'" From a884e47261311748337374e1b7adc07456e1122b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 06:12:07 +0200 Subject: [PATCH 069/627] Compatible benchmark --- benchmark/compatible/monetdb/benchmark.sh | 8 +++- .../monetdb/results/c6a.4xlarge.txt | 46 +++++++++++++++++++ 2 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 benchmark/compatible/monetdb/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/monetdb/benchmark.sh b/benchmark/compatible/monetdb/benchmark.sh index 5e77fc2e5c8..f0afb400802 100755 --- a/benchmark/compatible/monetdb/benchmark.sh +++ b/benchmark/compatible/monetdb/benchmark.sh @@ -6,7 +6,7 @@ echo "deb https://dev.monetdb.org/downloads/deb/ $(lsb_release -cs) monetdb" | s sudo wget --output-document=/etc/apt/trusted.gpg.d/monetdb.gpg https://www.monetdb.org/downloads/MonetDB-GPG-KEY.gpg sudo apt-get update -sudo apt-get install -y monetdb5-sql monetdb-client +sudo apt-get install -y monetdb5-sql monetdb-client dos2unix sudo systemctl enable monetdbd sudo systemctl start monetdbd @@ -32,3 +32,9 @@ chmod 777 ~ hits.tsv # clk: 15:39 min ./run.sh 2>&1 | tee log.txt + +sudo du -bcs /var/monetdb5/ + +cat log.txt | dos2unix -f | grep -P 'clk|tuple' | + awk '/tuple/ { ok = 1 } /clk/ { if (ok) { if ($3 == "ms") { print $2 / 1000 } else { print $2 } } else { print "null" }; ok = 0 }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/monetdb/results/c6a.4xlarge.txt b/benchmark/compatible/monetdb/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..21e543a61bb --- /dev/null +++ b/benchmark/compatible/monetdb/results/c6a.4xlarge.txt @@ -0,0 +1,46 @@ +Load time: 15:39 min +Data size: 49 696 606 499 + +[0.000218,0.000157,0.000155], +[0.101903,0.019908,0.018439], +[0.282431,0.035987,0.034938], +[2.868,0.029387,0.029207], +[4.675,4.515,4.511], +[6.584,4.269,4.650], +[0.528827,0.063135,0.065742], +[0.506863,0.020966,0.021687], +[8.343,4.457,4.408], +[7.224,6.548,7.576], +[0.267003,0.233353,0.230444], +[0.347206,0.28358,0.266085], +[5.389,3.099,3.074], +[7.653,7.759,8.596], +[3.276,3.326,3.292], +[5.310,3.465,3.578], +[9.341,9.143,9.536], +[9.584,9.604,9.419], +[19.539,19.783,19.611], +[0.004509,0.000702,0.000643], +[20.801,1.570,1.603], +[2.752,0.418221,0.395884], +[14.717,0.800894,0.395477], +[14.429,1.804,1.869], +[1.386,0.159602,0.156426], +[0.189736,0.167664,0.168781], +[0.164681,0.176666,0.17126], +[3.005,3.113,3.882], +[null,null,null], +[2.751,2.846,2.676], +[7.937,2.579,2.447], +[5.120,3.492,3.467], +[22.862,22.567,23.211], +[33.437,18.889,19.043], +[18.898,19.583,19.047], +[14.774,12.984,13.803], +[3.865,0.322143,0.323117], +[0.192149,0.177791,0.175984], +[0.194173,0.159398,0.165201], +[0.680778,0.592252,0.560738], +[0.106465,0.10638,0.102692], +[0.154871,0.153752,0.155782], +[0.11459,0.09639,0.095594] From 8224320d979bac384f60adb7065fd2b09ef77db5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 06:33:31 +0200 Subject: [PATCH 070/627] Compatible benchmark --- benchmark/compatible/vertica/benchmark.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/benchmark/compatible/vertica/benchmark.sh b/benchmark/compatible/vertica/benchmark.sh index 9047686597f..201893788cc 100644 --- a/benchmark/compatible/vertica/benchmark.sh +++ b/benchmark/compatible/vertica/benchmark.sh @@ -7,9 +7,9 @@ sudo docker run --name vertica -p 5433:5433 -p 5444:5444 --mount type=volume,sou sudo docker exec vertica /opt/vertica/bin/vsql -U dbadmin -c "$(cat create.sql)" -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' -gzip -d hits.csv.gz +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz -sudo docker cp hits.tsv vertica:/hits.csv +sudo docker cp hits.tsv vertica:/hits.tsv -time sudo docker exec 4e9120959a41 /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM '/hits.csv'" +time sudo docker exec 4e9120959a41 /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM '/hits.tsv' DELIMITER E'\\t'" From 0d906e311d4398a4af9193b8d3896da374fdcf55 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Jun 2022 06:53:09 +0000 Subject: [PATCH 071/627] PR review fixes --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 4 ++-- src/Common/ZooKeeper/ZooKeeperIO.cpp | 10 ++++++++++ src/Common/ZooKeeper/ZooKeeperIO.h | 2 ++ src/Coordination/KeeperStorage.cpp | 4 +++- 4 files changed, 17 insertions(+), 3 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index f2b8fe59e99..41c5312bfa0 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -286,7 +286,7 @@ void ZooKeeperListRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); Coordination::write(has_watch, out); - Coordination::write(static_cast(list_request_type), out); + Coordination::write(static_cast(list_request_type), out); } void ZooKeeperListRequest::readImpl(ReadBuffer & in) @@ -296,7 +296,7 @@ void ZooKeeperListRequest::readImpl(ReadBuffer & in) if (!in.eof()) { - int32_t read_request_type; + uint8_t read_request_type; Coordination::read(read_request_type, in); list_request_type = static_cast(read_request_type); } diff --git a/src/Common/ZooKeeper/ZooKeeperIO.cpp b/src/Common/ZooKeeper/ZooKeeperIO.cpp index 066aa1a24f6..c84a8624d78 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.cpp +++ b/src/Common/ZooKeeper/ZooKeeperIO.cpp @@ -28,6 +28,11 @@ void write(int32_t x, WriteBuffer & out) writeBinary(x, out); } +void write(uint8_t x, WriteBuffer & out) +{ + writeBinary(x, out); +} + void write(OpNum x, WriteBuffer & out) { write(static_cast(x), out); @@ -91,6 +96,11 @@ void read(int64_t & x, ReadBuffer & in) x = __builtin_bswap64(x); } +void read(uint8_t & x, ReadBuffer & in) +{ + readBinary(x, in); +} + void read(int32_t & x, ReadBuffer & in) { readBinary(x, in); diff --git a/src/Common/ZooKeeper/ZooKeeperIO.h b/src/Common/ZooKeeper/ZooKeeperIO.h index c2c6149cd11..ec77b46f3d9 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.h +++ b/src/Common/ZooKeeper/ZooKeeperIO.h @@ -22,6 +22,7 @@ void write(uint64_t x, WriteBuffer & out); void write(int64_t x, WriteBuffer & out); void write(int32_t x, WriteBuffer & out); +void write(uint8_t x, WriteBuffer & out); void write(OpNum x, WriteBuffer & out); void write(bool x, WriteBuffer & out); void write(const std::string & s, WriteBuffer & out); @@ -50,6 +51,7 @@ void read(uint64_t & x, ReadBuffer & in); #endif void read(int64_t & x, ReadBuffer & in); void read(int32_t & x, ReadBuffer & in); +void read(uint8_t & x, ReadBuffer & in); void read(OpNum & x, ReadBuffer & in); void read(bool & x, ReadBuffer & in); void read(int8_t & x, ReadBuffer & in); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index f15c10038c7..f86f3ed7962 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -19,6 +19,7 @@ #include #include #include +#include namespace DB { @@ -1186,7 +1187,8 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc if (request.list_request_type == ALL) return true; - auto child_it = container.find(fmt::format("{}{}{}", request.path, (request.path.ends_with('/') ? "" : "/"), child)); + auto child_path = (std::filesystem::path(request.path) / child.toView()).generic_string(); + auto child_it = container.find(child_path); if (child_it == container.end()) onStorageInconsistency(); From 585be7431d7e9bc1274b165adf5c2a61ca4b3c3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 09:33:42 +0200 Subject: [PATCH 072/627] Compatible benchmark --- benchmark/compatible/infobright/benchmark.sh | 24 ++++ benchmark/compatible/infobright/create.sql | 108 ++++++++++++++++++ benchmark/compatible/infobright/queries.sql | 43 +++++++ benchmark/compatible/infobright/run.sh | 12 ++ .../mariadb-columnstore/benchmark.sh | 18 ++- .../compatible/mariadb-columnstore/create.sql | 52 ++++----- .../mariadb-columnstore/queries.sql | 43 +++++++ .../results/c6a.4xlarge.txt | 45 ++++++++ .../compatible/mariadb-columnstore/run.sh | 12 ++ benchmark/compatible/mariadb/benchmark.sh | 6 +- benchmark/compatible/mariadb/run.sh | 2 +- benchmark/compatible/vertica/benchmark.sh | 8 +- 12 files changed, 336 insertions(+), 37 deletions(-) create mode 100755 benchmark/compatible/infobright/benchmark.sh create mode 100644 benchmark/compatible/infobright/create.sql create mode 100644 benchmark/compatible/infobright/queries.sql create mode 100755 benchmark/compatible/infobright/run.sh create mode 100644 benchmark/compatible/mariadb-columnstore/queries.sql create mode 100644 benchmark/compatible/mariadb-columnstore/results/c6a.4xlarge.txt create mode 100644 benchmark/compatible/mariadb-columnstore/run.sh diff --git a/benchmark/compatible/infobright/benchmark.sh b/benchmark/compatible/infobright/benchmark.sh new file mode 100755 index 00000000000..6ebf3861a78 --- /dev/null +++ b/benchmark/compatible/infobright/benchmark.sh @@ -0,0 +1,24 @@ +#!/bin/bash + +# Install + +sudo apt-get update +sudo apt-get install -y docker.io + +mkdir infobright +sudo docker run --name mysql_ib -e MYSQL_ROOT_PASSWORD=mypass -v $(pwd)/infobright:/mnt/mysql_data -p 5029:5029 -p 5555 -d flolas/infobright + +sudo docker run -it --rm --network host mysql:5 mysql --host 127.0.0.1 --port 5029 --user=root --password=mypass -e "CREATE DATABASE test" +sudo docker run -it --rm --network host mysql:5 mysql --host 127.0.0.1 --port 5029 --user=root --password=mypass --database=test -e "$(cat create.sql)" + +# Load the data + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +# ERROR 2 (HY000) at line 1: Wrong data or column definition. Row: 93557187, field: 100. +head -n 90000000 hits.tsv > hits90m.tsv + +time sudo docker run -it --rm --volume $(pwd):/workdir --network host mysql:5 mysql --host 127.0.0.1 --port 5029 --user=root --password=mypass --database=test -e " + LOAD DATA LOCAL INFILE '/workdir/hits90m.tsv' INTO TABLE test.hits + FIELDS TERMINATED BY '\\t' ENCLOSED BY '' ESCAPED BY '\\\\' LINES TERMINATED BY '\\n' STARTING BY ''" diff --git a/benchmark/compatible/infobright/create.sql b/benchmark/compatible/infobright/create.sql new file mode 100644 index 00000000000..4d23eaac635 --- /dev/null +++ b/benchmark/compatible/infobright/create.sql @@ -0,0 +1,108 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT, + Referer TEXT, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT, + Params TEXT, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT, + BrowserCountry TEXT, + SocialNetwork TEXT, + SocialAction TEXT, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT, + ParamCurrency TEXT, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT, + OpenstatCampaignID TEXT, + OpenstatAdID TEXT, + OpenstatSourceID TEXT, + UTMSource TEXT, + UTMMedium TEXT, + UTMCampaign TEXT, + UTMContent TEXT, + UTMTerm TEXT, + FromTag TEXT, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +); diff --git a/benchmark/compatible/infobright/queries.sql b/benchmark/compatible/infobright/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/infobright/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/infobright/run.sh b/benchmark/compatible/infobright/run.sh new file mode 100755 index 00000000000..16cc104e712 --- /dev/null +++ b/benchmark/compatible/infobright/run.sh @@ -0,0 +1,12 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + for i in $(seq 1 $TRIES); do + sudo docker exec memsql-ciab memsql -vvv -p"${ROOT_PASSWORD}" --database=test -e "USE test; ${query}" + done; +done; diff --git a/benchmark/compatible/mariadb-columnstore/benchmark.sh b/benchmark/compatible/mariadb-columnstore/benchmark.sh index 7fbfe341aff..884a624490d 100755 --- a/benchmark/compatible/mariadb-columnstore/benchmark.sh +++ b/benchmark/compatible/mariadb-columnstore/benchmark.sh @@ -9,7 +9,7 @@ sudo docker run -d -p 3306:3306 -e ANALYTICS_ONLY=1 --name mcs_container mariadb export PASSWORD="tsFgm457%3cj" sudo docker exec mcs_container mariadb -e "GRANT ALL PRIVILEGES ON *.* TO 'ubuntu'@'%' IDENTIFIED BY '${PASSWORD}';" -sudo apt install mariadb-client +sudo apt-get install -y mariadb-client mysql --password="${PASSWORD}" --host 127.0.0.1 -e "CREATE DATABASE test" mysql --password="${PASSWORD}" --host 127.0.0.1 test < create.sql @@ -19,4 +19,18 @@ mysql --password="${PASSWORD}" --host 127.0.0.1 test < create.sql wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz -time mysql --password="${PASSWORD}" --host 127.0.0.1 test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" +time mysql --password="${PASSWORD}" --host 127.0.0.1 test -e " + LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits + FIELDS TERMINATED BY '\\t' ENCLOSED BY '' ESCAPED BY '\\\\' LINES TERMINATED BY '\\n' STARTING BY ''" + +# 41m47.856s + +./run.sh 2>&1 | tee log.txt + +sudo docker exec mcs_container du -bcs /var/lib/columnstore + +cat log.txt | + grep -P 'rows? in set|^ERROR' | + sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | + awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/mariadb-columnstore/create.sql b/benchmark/compatible/mariadb-columnstore/create.sql index 3de1aa0a510..565596f04c9 100644 --- a/benchmark/compatible/mariadb-columnstore/create.sql +++ b/benchmark/compatible/mariadb-columnstore/create.sql @@ -2,7 +2,7 @@ CREATE TABLE hits ( WatchID BIGINT NOT NULL, JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, + Title TEXT, GoodEvent SMALLINT NOT NULL, EventTime TIMESTAMP NOT NULL, EventDate Date NOT NULL, @@ -13,8 +13,8 @@ CREATE TABLE hits CounterClass SMALLINT NOT NULL, OS SMALLINT NOT NULL, UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, + URL TEXT, + Referer TEXT, IsRefresh SMALLINT NOT NULL, RefererCategoryID SMALLINT NOT NULL, RefererRegionID INTEGER NOT NULL, @@ -25,7 +25,7 @@ CREATE TABLE hits ResolutionDepth SMALLINT NOT NULL, FlashMajor SMALLINT NOT NULL, FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, + FlashMinor2 TEXT, NetMajor SMALLINT NOT NULL, NetMinor SMALLINT NOT NULL, UserAgentMajor SMALLINT NOT NULL, @@ -34,12 +34,12 @@ CREATE TABLE hits JavascriptEnable SMALLINT NOT NULL, IsMobile SMALLINT NOT NULL, MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, + MobilePhoneModel TEXT, + Params TEXT, IPNetworkID INTEGER NOT NULL, TraficSourceID SMALLINT NOT NULL, SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, + SearchPhrase TEXT, AdvEngineID SMALLINT NOT NULL, IsArtifical SMALLINT NOT NULL, WindowClientWidth SMALLINT NOT NULL, @@ -50,13 +50,13 @@ CREATE TABLE hits SilverlightVersion2 SMALLINT NOT NULL, SilverlightVersion3 INTEGER NOT NULL, SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, + PageCharset TEXT, CodeVersion INTEGER NOT NULL, IsLink SMALLINT NOT NULL, IsDownload SMALLINT NOT NULL, IsNotBounce SMALLINT NOT NULL, FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, + OriginalURL TEXT, HID INTEGER NOT NULL, IsOldCounter SMALLINT NOT NULL, IsEvent SMALLINT NOT NULL, @@ -74,10 +74,10 @@ CREATE TABLE hits WindowName INTEGER NOT NULL, OpenerName INTEGER NOT NULL, HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, + BrowserLanguage TEXT, + BrowserCountry TEXT, + SocialNetwork TEXT, + SocialAction TEXT, HTTPError SMALLINT NOT NULL, SendTiming INTEGER NOT NULL, DNSTiming INTEGER NOT NULL, @@ -86,21 +86,21 @@ CREATE TABLE hits ResponseEndTiming INTEGER NOT NULL, FetchTiming INTEGER NOT NULL, SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, + SocialSourcePage TEXT, ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, + ParamOrderID TEXT, + ParamCurrency TEXT, ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, + OpenstatServiceName TEXT, + OpenstatCampaignID TEXT, + OpenstatAdID TEXT, + OpenstatSourceID TEXT, + UTMSource TEXT, + UTMMedium TEXT, + UTMCampaign TEXT, + UTMContent TEXT, + UTMTerm TEXT, + FromTag TEXT, HasGCLID SMALLINT NOT NULL, RefererHash BIGINT NOT NULL, URLHash BIGINT NOT NULL, diff --git a/benchmark/compatible/mariadb-columnstore/queries.sql b/benchmark/compatible/mariadb-columnstore/queries.sql new file mode 100644 index 00000000000..ea2bde47802 --- /dev/null +++ b/benchmark/compatible/mariadb-columnstore/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/mariadb-columnstore/results/c6a.4xlarge.txt b/benchmark/compatible/mariadb-columnstore/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..cfaa356717c --- /dev/null +++ b/benchmark/compatible/mariadb-columnstore/results/c6a.4xlarge.txt @@ -0,0 +1,45 @@ +Load time: 41m47.856s +Data size: 19 712 857 022 + +[0.151,0.158,0.148], +[0.100,0.101,0.106], +[1.221,1.233,1.226], +[0.739,0.736,0.741], +[2.025,2.046,2.004], +[3.725,4.801,3.755], +[0.871,0.749,0.736], +[0.118,0.108,0.103], +[2.108,2.029,2.029], +[4.225,4.271,4.288], +[1.711,1.402,1.407], +[1.526,1.435,1.420], +[5.339,4.172,3.610], +[4.692,4.729,4.960], +[4.013,3.860,3.918], +[3.236,2.680,2.629], +[null,null,null], +[null,null,null], +[null,null,null], +[0.137,0.109,0.147], +[12.331,6.069,8.619], +[2.162,2.178,2.192], +[16.849,30.463,26.639], +[92,90.208,92.814], +[3.042,1.763,1.791], +[1.779,1.772,1.749], +[1.793,1.821,1.888], +[13.036,10.747,9.590], +[null,null,null], +[78.224,79.141,77.806], +[2.837,2.654,2.675], +[5.833,4.552,3.678], +[null,null,null], +[null,null,null], +[null,null,null], +[3.626,3.546,3.709], +[1.719,1.787,1.876], +[1.345,0.906,0.910], +[0.117,0.091,0.093], +[1.217,1.133,1.133], +[0.114,0.063,0.062], +[0.100,0.062,0.061] diff --git a/benchmark/compatible/mariadb-columnstore/run.sh b/benchmark/compatible/mariadb-columnstore/run.sh new file mode 100644 index 00000000000..b917a815d2c --- /dev/null +++ b/benchmark/compatible/mariadb-columnstore/run.sh @@ -0,0 +1,12 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + for i in $(seq 1 $TRIES); do + mysql --password="${PASSWORD}" --host 127.0.0.1 -vvv test -e "${query}" + done; +done; diff --git a/benchmark/compatible/mariadb/benchmark.sh b/benchmark/compatible/mariadb/benchmark.sh index 57a2387b530..612a6ac39ab 100755 --- a/benchmark/compatible/mariadb/benchmark.sh +++ b/benchmark/compatible/mariadb/benchmark.sh @@ -12,9 +12,9 @@ sudo service mariadb restart wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz -sudo mysql -e "CREATE DATABASE test" -sudo mysql test < create.sql -time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" +sudo mariadb -e "CREATE DATABASE test" +sudo mariadb test < create.sql +time sudo mariadb test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" # 2:23:45 elapsed diff --git a/benchmark/compatible/mariadb/run.sh b/benchmark/compatible/mariadb/run.sh index 31f31682f32..5f80a539bbe 100755 --- a/benchmark/compatible/mariadb/run.sh +++ b/benchmark/compatible/mariadb/run.sh @@ -7,6 +7,6 @@ cat queries.sql | while read query; do echo 3 | sudo tee /proc/sys/vm/drop_caches for i in $(seq 1 $TRIES); do - sudo mysql test -vvv -e "${query}" + sudo mariadb test -vvv -e "${query}" done; done; diff --git a/benchmark/compatible/vertica/benchmark.sh b/benchmark/compatible/vertica/benchmark.sh index 201893788cc..0054d36b0f9 100644 --- a/benchmark/compatible/vertica/benchmark.sh +++ b/benchmark/compatible/vertica/benchmark.sh @@ -3,13 +3,11 @@ sudo apt-get update sudo apt-get install -y docker.io -sudo docker run --name vertica -p 5433:5433 -p 5444:5444 --mount type=volume,source=vertica-data,target=/data --name vertica_ce vertica/vertica-ce +sudo docker run -p 5433:5433 -p 5444:5444 --volume $(pwd):/workdir --mount type=volume,source=vertica-data,target=/data --name vertica_ce vertica/vertica-ce -sudo docker exec vertica /opt/vertica/bin/vsql -U dbadmin -c "$(cat create.sql)" +sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c "$(cat create.sql)" wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz -sudo docker cp hits.tsv vertica:/hits.tsv - -time sudo docker exec 4e9120959a41 /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM '/hits.tsv' DELIMITER E'\\t'" +time sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM LOCAL '/workdir/hits.tsv' DELIMITER E'\\t' NULL E'\\001' DIRECT" From 89ace4e9e0aab7ce2b71ce58cd9d9d7173244c81 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 10:18:14 +0200 Subject: [PATCH 073/627] Compatible benchmark --- benchmark/compatible/redshift/README.md | 55 +++++++++ benchmark/compatible/redshift/create.sql | 109 ++++++++++++++++++ benchmark/compatible/redshift/queries.sql | 43 +++++++ .../redshift/result/4x.ra3.xplus.txt | 46 ++++++++ benchmark/compatible/redshift/run.sh | 10 ++ 5 files changed, 263 insertions(+) create mode 100644 benchmark/compatible/redshift/README.md create mode 100644 benchmark/compatible/redshift/create.sql create mode 100644 benchmark/compatible/redshift/queries.sql create mode 100644 benchmark/compatible/redshift/result/4x.ra3.xplus.txt create mode 100644 benchmark/compatible/redshift/run.sh diff --git a/benchmark/compatible/redshift/README.md b/benchmark/compatible/redshift/README.md new file mode 100644 index 00000000000..4ea34b677f8 --- /dev/null +++ b/benchmark/compatible/redshift/README.md @@ -0,0 +1,55 @@ +This benchmark is not automated. + +Go to AWS Redshift service. +Create a cluster. Note: this is a classic Redshift, not "serverless". + +Choose the node type and cluster size. +I've selected 4 nodes of ra3.xplus 4vCPU to get 16vCPU in total. + +Set up some password for the admin user. +The cluster will take a few minutes to start. + +We need to perform two modifications: +1. Allow inbound access. Go to VPC and edit the security group. Modify inbound rules. Allow connections from any IPv4 to port 5439. +2. Add IAM role. Just create something by default. + +To create a table, you can go to the Query Editor v2. +Open the "dev" database. +Run the CREATE TABLE statement you find in `create.sql`. + +Note: Redshift prefers VARCHAR(MAX) instead of TEXT. + +Then press on the "Load data". +This will generate a statement: + +``` +COPY dev.public.hits FROM 's3://clickhouse-public-datasets/hits_compatible/hits.csv.gz' GZIP +IAM_ROLE 'arn:aws:iam::...:role/service-role/AmazonRedshift-CommandsAccessRole-...' +FORMAT AS CSV DELIMITER ',' QUOTE '"' +REGION AS 'eu-central-1' +``` + +> Elapsed time: 35m 35.9s + +We will run the queries from another server with `psql` client. + +``` +sudo apt-get install -y postgresql-client + +echo "*:*:*:*:your_password" > .pgpass +chmod 400 .pgpass + +psql -h redshift-cluster-1.chedgchbam32.eu-central-1.redshift.amazonaws.com -U awsuser -d dev -p 5439 +``` + +Then run the benchmark: +``` +export HOST=... +./run.sh 2>&1 | tee log.txt + + +``` + +`SELECT sum(used * 1048576) FROM stv_node_storage_capacity` + +> 30 794 579 968 diff --git a/benchmark/compatible/redshift/create.sql b/benchmark/compatible/redshift/create.sql new file mode 100644 index 00000000000..94a8fb0958a --- /dev/null +++ b/benchmark/compatible/redshift/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title VARCHAR(MAX) NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL VARCHAR(MAX) NOT NULL, + Referer VARCHAR(MAX) NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 VARCHAR(MAX) NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(MAX) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel VARCHAR(MAX) NOT NULL, + Params VARCHAR(MAX) NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase VARCHAR(MAX) NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset VARCHAR(MAX) NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL VARCHAR(MAX) NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor VARCHAR(MAX) NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage VARCHAR(MAX) NOT NULL, + BrowserCountry VARCHAR(MAX) NOT NULL, + SocialNetwork VARCHAR(MAX) NOT NULL, + SocialAction VARCHAR(MAX) NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage VARCHAR(MAX) NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID VARCHAR(MAX) NOT NULL, + ParamCurrency VARCHAR(MAX) NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName VARCHAR(MAX) NOT NULL, + OpenstatCampaignID VARCHAR(MAX) NOT NULL, + OpenstatAdID VARCHAR(MAX) NOT NULL, + OpenstatSourceID VARCHAR(MAX) NOT NULL, + UTMSource VARCHAR(MAX) NOT NULL, + UTMMedium VARCHAR(MAX) NOT NULL, + UTMCampaign VARCHAR(MAX) NOT NULL, + UTMContent VARCHAR(MAX) NOT NULL, + UTMTerm VARCHAR(MAX) NOT NULL, + FromTag VARCHAR(MAX) NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/redshift/queries.sql b/benchmark/compatible/redshift/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/redshift/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/redshift/result/4x.ra3.xplus.txt b/benchmark/compatible/redshift/result/4x.ra3.xplus.txt new file mode 100644 index 00000000000..92e4f1f3dcf --- /dev/null +++ b/benchmark/compatible/redshift/result/4x.ra3.xplus.txt @@ -0,0 +1,46 @@ +Load time: 35m 35.9s +Data size: 30 794 579 968 + +[0.081437,0.022376,0.022491], +[2.54868,0.024112,0.024072], +[2.62053,0.049948,0.049768], +[2.57591,0.020689,0.020651], +[0.62714,0.447655,0.43262], +[1.14153,1.09479,1.09441], +[0.153399,0.053941,0.048224], +[3.76153,0.032235,0.030949], +[4.43439,0.746808,0.723719], +[6.13424,2.20458,2.20031], +[4.35338,0.220122,0.228684], +[5.15139,0.231856,0.230993], +[5.00006,1.10841,1.12871], +[5.80125,1.86531,1.90209], +[5.16246,1.21239,1.14848], +[0.579428,0.535859,0.546178], +[6.05764,2.17455,2.18286], +[5.86612,2.52634,2.48672], +[7.80075,3.35512,3.41153], +[2.54112,0.036378,0.035944], +[6.47189,2.26909,2.2673], +[6.95344,2.61929,2.65637], +[12.9508,6.85457,6.99], +[25.3022,11.8857,11.8493], +[4.37592,0.452737,0.452867], +[4.17199,0.469457,0.476302], +[6.24746,0.470935,0.482502], +[6.00065,2.08332,2.08059], +[0.695955,0.078488,0.077928], +[22.2567,18.5376,18.3441], +[5.37492,0.868068,0.849486], +[7.29067,1.06155,1.11209], +[7.7832,4.07132,4.00384], +[8.95385,5.15488,5.21863], +[9.2232,5.32052,5.73207], +[4.99205,0.664347,0.618918], +[6.82279,1.89738,1.89398], +[7.09077,2.19008,2.26612], +[5.29731,0.19626,0.204603], +[6.07138,0.276315,0.267161], +[4.03108,0.054134,0.058568], +[4.35647,0.061157,0.053367], +[5.23605,0.037217,0.036335] diff --git a/benchmark/compatible/redshift/run.sh b/benchmark/compatible/redshift/run.sh new file mode 100644 index 00000000000..07a47eda1ab --- /dev/null +++ b/benchmark/compatible/redshift/run.sh @@ -0,0 +1,10 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + echo "$query"; + for i in $(seq 1 $TRIES); do + psql -h "${HOST}" -U awsuser -d dev -p 5439 -t -c 'SET enable_result_cache_for_session = off' -c '\timing' -c "$query" | grep 'Time' + done; +done; From b727e1073d21bb7bcd3ac3d9898edb3b56d1fa46 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 10:28:13 +0200 Subject: [PATCH 074/627] Compatible benchmark --- benchmark/compatible/README.md | 54 +++++++++++++++++++ benchmark/compatible/infobright/benchmark.sh | 2 + benchmark/compatible/infobright/queries.sql | 2 +- .../infobright/results/c6a.4xlarge.txt | 2 + benchmark/compatible/infobright/run.sh | 2 +- 5 files changed, 60 insertions(+), 2 deletions(-) create mode 100644 benchmark/compatible/README.md create mode 100644 benchmark/compatible/infobright/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md new file mode 100644 index 00000000000..491c2984e65 --- /dev/null +++ b/benchmark/compatible/README.md @@ -0,0 +1,54 @@ +- [x] ClickHouse +- [x] MySQL InnoDB +- [ ] MySQL MyISAM +- [ ] MariaDB +- [x] MariaDB ColumnStore +- [x] MemSQL/SingleStore +- [x] PostgreSQL +- [ ] Greenplum +- [x] TimescaleDB +- [x] Citus +- [ ] Vertica (without publishing) +- [x] QuestDB +- [x] DuckDB +- [x] MonetDB +- [x] mapD/Omnisci/HeavyAI +- [ ] Databend +- [ ] Doris +- [ ] Druid +- [ ] Pinot +- [ ] CrateDB +- [ ] Spark SQL +- [ ] Starrocks +- [ ] ShitholeSQL +- [ ] Hive +- [ ] Impala +- [ ] Hyper +- [ ] Umbra +- [ ] SQLite +- [x] Redshift +- [ ] Redshift Serverless +- [ ] Presto/Trino +- [ ] Athena +- [ ] Bigquery (without publishing) +- [ ] Snowflake +- [ ] Rockset +- [ ] CockroachDB +- [ ] CockroachDB Serverless +- [ ] Databricks +- [ ] Planetscale (without publishing) +- [ ] TiDB (TiFlash) +- [ ] Amazon RDS +- [ ] InfluxDB +- [ ] TDEngine +- [ ] MongoDB +- [ ] Elasticsearch +- [ ] Apache Ignite +- [ ] Infobright +- [ ] Actian Vector +- [ ] Manticore Search +- [ ] Vertica (without publishing) +- [ ] Azure Synapse +- [ ] Starbirst Galaxy +- [ ] MS SQL Server with Column Store Index (without publishing) +- [ ] Dremio (without publishing) diff --git a/benchmark/compatible/infobright/benchmark.sh b/benchmark/compatible/infobright/benchmark.sh index 6ebf3861a78..7b169359564 100755 --- a/benchmark/compatible/infobright/benchmark.sh +++ b/benchmark/compatible/infobright/benchmark.sh @@ -22,3 +22,5 @@ head -n 90000000 hits.tsv > hits90m.tsv time sudo docker run -it --rm --volume $(pwd):/workdir --network host mysql:5 mysql --host 127.0.0.1 --port 5029 --user=root --password=mypass --database=test -e " LOAD DATA LOCAL INFILE '/workdir/hits90m.tsv' INTO TABLE test.hits FIELDS TERMINATED BY '\\t' ENCLOSED BY '' ESCAPED BY '\\\\' LINES TERMINATED BY '\\n' STARTING BY ''" + +# 38m37.466s diff --git a/benchmark/compatible/infobright/queries.sql b/benchmark/compatible/infobright/queries.sql index 31f65fc898d..ea2bde47802 100644 --- a/benchmark/compatible/infobright/queries.sql +++ b/benchmark/compatible/infobright/queries.sql @@ -40,4 +40,4 @@ SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate > 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 IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; +SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/infobright/results/c6a.4xlarge.txt b/benchmark/compatible/infobright/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..7346cb63653 --- /dev/null +++ b/benchmark/compatible/infobright/results/c6a.4xlarge.txt @@ -0,0 +1,2 @@ +Load time: 2317 sec. +Data size: diff --git a/benchmark/compatible/infobright/run.sh b/benchmark/compatible/infobright/run.sh index 16cc104e712..36e5fa167fb 100755 --- a/benchmark/compatible/infobright/run.sh +++ b/benchmark/compatible/infobright/run.sh @@ -7,6 +7,6 @@ cat queries.sql | while read query; do echo 3 | sudo tee /proc/sys/vm/drop_caches for i in $(seq 1 $TRIES); do - sudo docker exec memsql-ciab memsql -vvv -p"${ROOT_PASSWORD}" --database=test -e "USE test; ${query}" + sudo docker run --rm --network host mysql:5 mysql --host 127.0.0.1 --port 5029 --user=root --password=mypass --database=test -vvv -e "${query}" done; done; From 312371fe938ac9983ad21060d91c11a161dfa794 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 10:30:31 +0200 Subject: [PATCH 075/627] Compatible benchmark --- benchmark/compatible/README.md | 2 +- benchmark/compatible/infobright/results/c6a.4xlarge.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index 491c2984e65..ceb42f9ba7a 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -49,6 +49,6 @@ - [ ] Manticore Search - [ ] Vertica (without publishing) - [ ] Azure Synapse -- [ ] Starbirst Galaxy +- [ ] Starburst Galaxy - [ ] MS SQL Server with Column Store Index (without publishing) - [ ] Dremio (without publishing) diff --git a/benchmark/compatible/infobright/results/c6a.4xlarge.txt b/benchmark/compatible/infobright/results/c6a.4xlarge.txt index 7346cb63653..7dce1056bd6 100644 --- a/benchmark/compatible/infobright/results/c6a.4xlarge.txt +++ b/benchmark/compatible/infobright/results/c6a.4xlarge.txt @@ -1,2 +1,2 @@ Load time: 2317 sec. -Data size: +Data size: From 2c5aeaaa1a1ed7db8772107772e35754ae296354 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Jun 2022 23:44:59 +0200 Subject: [PATCH 076/627] Add auto close for postgres connection --- src/Core/PostgreSQL/Connection.cpp | 1 + src/Core/PostgreSQL/Connection.h | 5 +- src/Core/PostgreSQL/ConnectionHolder.h | 15 +++++- src/Core/PostgreSQL/PoolWithFailover.cpp | 27 +++++++---- src/Core/PostgreSQL/PoolWithFailover.h | 23 +++++---- src/Core/Settings.h | 1 + src/Databases/DatabaseFactory.cpp | 10 ++-- .../PostgreSQLDictionarySource.cpp | 9 ++-- src/Storages/StorageExternalDistributed.cpp | 7 ++- src/Storages/StoragePostgreSQL.cpp | 7 ++- .../TableFunctionPostgreSQL.cpp | 10 ++-- .../configs/settings.xml | 8 ++++ .../test_storage_postgresql/test.py | 48 ++++++++++++++++++- 13 files changed, 136 insertions(+), 35 deletions(-) create mode 100644 tests/integration/test_storage_postgresql/configs/settings.xml diff --git a/src/Core/PostgreSQL/Connection.cpp b/src/Core/PostgreSQL/Connection.cpp index dfdf14d3506..5a589a80d02 100644 --- a/src/Core/PostgreSQL/Connection.cpp +++ b/src/Core/PostgreSQL/Connection.cpp @@ -73,6 +73,7 @@ void Connection::connect() if (!connection || !connection->is_open()) updateConnection(); } + } #endif diff --git a/src/Core/PostgreSQL/Connection.h b/src/Core/PostgreSQL/Connection.h index 8c5609dc66b..d39659a9953 100644 --- a/src/Core/PostgreSQL/Connection.h +++ b/src/Core/PostgreSQL/Connection.h @@ -32,7 +32,10 @@ struct ConnectionInfo class Connection : private boost::noncopyable { public: - Connection(const ConnectionInfo & connection_info_, bool replication_ = false, size_t num_tries = 3); + explicit Connection( + const ConnectionInfo & connection_info_, + bool replication_ = false, + size_t num_tries = 3); void execWithRetry(const std::function & exec); diff --git a/src/Core/PostgreSQL/ConnectionHolder.h b/src/Core/PostgreSQL/ConnectionHolder.h index 38e321e222c..d608f705559 100644 --- a/src/Core/PostgreSQL/ConnectionHolder.h +++ b/src/Core/PostgreSQL/ConnectionHolder.h @@ -7,6 +7,7 @@ #include #include #include +#include #include "Connection.h" @@ -20,11 +21,20 @@ class ConnectionHolder { public: - ConnectionHolder(PoolPtr pool_, ConnectionPtr connection_) : pool(pool_), connection(std::move(connection_)) {} + ConnectionHolder(PoolPtr pool_, ConnectionPtr connection_, bool auto_close_) + : pool(pool_) + , connection(std::move(connection_)) + , auto_close(auto_close_) + {} ConnectionHolder(const ConnectionHolder & other) = delete; - ~ConnectionHolder() { pool->returnObject(std::move(connection)); } + ~ConnectionHolder() + { + if (auto_close) + connection.reset(); + pool->returnObject(std::move(connection)); + } pqxx::connection & get() { @@ -39,6 +49,7 @@ public: private: PoolPtr pool; ConnectionPtr connection; + bool auto_close; }; using ConnectionHolderPtr = std::unique_ptr; diff --git a/src/Core/PostgreSQL/PoolWithFailover.cpp b/src/Core/PostgreSQL/PoolWithFailover.cpp index 844c60087e0..b5e2903ef2b 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.cpp +++ b/src/Core/PostgreSQL/PoolWithFailover.cpp @@ -5,6 +5,7 @@ #include "Utils.h" #include #include +#include #include #include @@ -20,10 +21,14 @@ namespace postgres { PoolWithFailover::PoolWithFailover( - const DB::ExternalDataSourcesConfigurationByPriority & configurations_by_priority, - size_t pool_size, size_t pool_wait_timeout_, size_t max_tries_) - : pool_wait_timeout(pool_wait_timeout_) - , max_tries(max_tries_) + const DB::ExternalDataSourcesConfigurationByPriority & configurations_by_priority, + size_t pool_size, + size_t pool_wait_timeout_, + size_t max_tries_, + bool auto_close_connection_) + : pool_wait_timeout(pool_wait_timeout_) + , max_tries(max_tries_) + , auto_close_connection(auto_close_connection_) { LOG_TRACE(&Poco::Logger::get("PostgreSQLConnectionPool"), "PostgreSQL connection pool size: {}, connection wait timeout: {}, max failover tries: {}", pool_size, pool_wait_timeout, max_tries_); @@ -40,10 +45,14 @@ PoolWithFailover::PoolWithFailover( } PoolWithFailover::PoolWithFailover( - const DB::StoragePostgreSQLConfiguration & configuration, - size_t pool_size, size_t pool_wait_timeout_, size_t max_tries_) + const DB::StoragePostgreSQLConfiguration & configuration, + size_t pool_size, + size_t pool_wait_timeout_, + size_t max_tries_, + bool auto_close_connection_) : pool_wait_timeout(pool_wait_timeout_) , max_tries(max_tries_) + , auto_close_connection(auto_close_connection_) { LOG_TRACE(&Poco::Logger::get("PostgreSQLConnectionPool"), "PostgreSQL connection pool size: {}, connection wait timeout: {}, max failover tries: {}", pool_size, pool_wait_timeout, max_tries_); @@ -94,7 +103,9 @@ ConnectionHolderPtr PoolWithFailover::get() catch (const pqxx::broken_connection & pqxx_error) { LOG_ERROR(log, "Connection error: {}", pqxx_error.what()); - error_message << "Try " << try_idx + 1 << ". Connection to `" << replica.connection_info.host_port << "` failed: " << pqxx_error.what() << "\n"; + error_message << "Try " << try_idx + 1 << ". " + << "Connection to " << DB::backQuote(replica.connection_info.host_port) + << " failed with error: " << pqxx_error.what() << "\n"; replica.pool->returnObject(std::move(connection)); continue; @@ -105,7 +116,7 @@ ConnectionHolderPtr PoolWithFailover::get() throw; } - auto connection_holder = std::make_unique(replica.pool, std::move(connection)); + auto connection_holder = std::make_unique(replica.pool, std::move(connection), auto_close_connection); /// Move all traversed replicas to the end. if (replicas.size() > 1) diff --git a/src/Core/PostgreSQL/PoolWithFailover.h b/src/Core/PostgreSQL/PoolWithFailover.h index 600e12fb53a..fef22168a92 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.h +++ b/src/Core/PostgreSQL/PoolWithFailover.h @@ -12,6 +12,10 @@ #include +static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SIZE = 16; +static constexpr inline auto POSTGRESQL_POOL_WAIT_TIMEOUT = 5000; +static constexpr inline auto POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES = 5; + namespace postgres { @@ -21,21 +25,19 @@ class PoolWithFailover using RemoteDescription = std::vector>; public: - static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SIZE = 16; - static constexpr inline auto POSTGRESQL_POOL_WAIT_TIMEOUT = 5000; - static constexpr inline auto POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES = 5; - PoolWithFailover( const DB::ExternalDataSourcesConfigurationByPriority & configurations_by_priority, - size_t pool_size = POSTGRESQL_POOL_DEFAULT_SIZE, - size_t pool_wait_timeout = POSTGRESQL_POOL_WAIT_TIMEOUT, - size_t max_tries_ = POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES); + size_t pool_size, + size_t pool_wait_timeout, + size_t max_tries_, + bool auto_close_connection_); PoolWithFailover( const DB::StoragePostgreSQLConfiguration & configuration, - size_t pool_size = POSTGRESQL_POOL_DEFAULT_SIZE, - size_t pool_wait_timeout = POSTGRESQL_POOL_WAIT_TIMEOUT, - size_t max_tries_ = POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES); + size_t pool_size, + size_t pool_wait_timeout, + size_t max_tries_, + bool auto_close_connection_); PoolWithFailover(const PoolWithFailover & other) = delete; @@ -58,6 +60,7 @@ private: ReplicasWithPriority replicas_with_priority; size_t pool_wait_timeout; size_t max_tries; + bool auto_close_connection; std::mutex mutex; Poco::Logger * log = &Poco::Logger::get("PostgreSQLConnectionPool"); }; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1fd9d20f00..70f79def8ba 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -428,6 +428,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(UInt64, postgresql_connection_pool_size, 16, "Connection pool size for PostgreSQL table engine and database engine.", 0) \ M(UInt64, postgresql_connection_pool_wait_timeout, 5000, "Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.", 0) \ + M(Bool, postgresql_connection_pool_auto_close_connection, false, "Close connection before returning connection to the pool.", 0) \ M(UInt64, glob_expansion_max_elements, 1000, "Maximum number of allowed addresses (For external storages, table functions, etc).", 0) \ M(UInt64, odbc_bridge_connection_pool_size, 16, "Connection pool size for each connection settings string in ODBC bridge.", 0) \ M(Bool, odbc_bridge_use_connection_pooling, true, "Use connection pooling in ODBC bridge. If set to false, a new connection is created every time", 0) \ diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 5cc334eaad4..e238fa8ba3c 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -341,9 +341,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine_args.size() >= 6) use_table_cache = safeGetLiteralValue(engine_args[5], engine_name); - auto pool = std::make_shared(configuration, - context->getSettingsRef().postgresql_connection_pool_size, - context->getSettingsRef().postgresql_connection_pool_wait_timeout); + const auto & settings = context->getSettingsRef(); + auto pool = std::make_shared( + configuration, + settings.postgresql_connection_pool_size, + settings.postgresql_connection_pool_wait_timeout, + POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, + settings.postgresql_connection_pool_auto_close_connection); return std::make_shared( context, metadata_path, engine_define, database_name, configuration, pool, use_table_cache); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index eb1a4caf2fc..42884278e7d 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -191,10 +191,13 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) const auto settings_config_prefix = config_prefix + ".postgresql"; auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key) || key.starts_with("replica"); }; auto configuration = getExternalDataSourceConfigurationByPriority(config, settings_config_prefix, context, has_config_key); + const auto & settings = context->getSettingsRef(); auto pool = std::make_shared( - configuration.replicas_configurations, - context->getSettingsRef().postgresql_connection_pool_size, - context->getSettingsRef().postgresql_connection_pool_wait_timeout); + configuration.replicas_configurations, + settings.postgresql_connection_pool_size, + settings.postgresql_connection_pool_wait_timeout, + POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, + settings.postgresql_connection_pool_auto_close_connection); PostgreSQLDictionarySource::Configuration dictionary_configuration { diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 181cf0ca183..bcdbb950a41 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -95,10 +95,13 @@ StorageExternalDistributed::StorageExternalDistributed( postgres_conf.set(configuration); postgres_conf.addresses = addresses; + const auto & settings = context->getSettingsRef(); auto pool = std::make_shared( postgres_conf, - context->getSettingsRef().postgresql_connection_pool_size, - context->getSettingsRef().postgresql_connection_pool_wait_timeout); + settings.postgresql_connection_pool_size, + settings.postgresql_connection_pool_wait_timeout, + POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, + settings.postgresql_connection_pool_auto_close_connection); shard = std::make_shared(table_id_, std::move(pool), configuration.table, columns_, constraints_, String{}); break; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 5b57384c1dd..d6065ada1b1 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -449,9 +449,12 @@ void registerStoragePostgreSQL(StorageFactory & factory) factory.registerStorage("PostgreSQL", [](const StorageFactory::Arguments & args) { auto configuration = StoragePostgreSQL::getConfiguration(args.engine_args, args.getLocalContext()); + const auto & settings = args.getContext()->getSettingsRef(); auto pool = std::make_shared(configuration, - args.getContext()->getSettingsRef().postgresql_connection_pool_size, - args.getContext()->getSettingsRef().postgresql_connection_pool_wait_timeout); + settings.postgresql_connection_pool_size, + settings.postgresql_connection_pool_wait_timeout, + POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, + settings.postgresql_connection_pool_auto_close_connection); return std::make_shared( args.table_id, diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index 7e7424be38f..d61140e1a07 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -62,9 +62,13 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, Contex throw Exception("Table function 'PostgreSQL' must have arguments.", ErrorCodes::BAD_ARGUMENTS); configuration.emplace(StoragePostgreSQL::getConfiguration(func_args.arguments->children, context)); - connection_pool = std::make_shared(*configuration, - context->getSettingsRef().postgresql_connection_pool_size, - context->getSettingsRef().postgresql_connection_pool_wait_timeout); + const auto & settings = context->getSettingsRef(); + connection_pool = std::make_shared( + *configuration, + settings.postgresql_connection_pool_size, + settings.postgresql_connection_pool_wait_timeout, + POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, + settings.postgresql_connection_pool_auto_close_connection); } diff --git a/tests/integration/test_storage_postgresql/configs/settings.xml b/tests/integration/test_storage_postgresql/configs/settings.xml new file mode 100644 index 00000000000..7054c274771 --- /dev/null +++ b/tests/integration/test_storage_postgresql/configs/settings.xml @@ -0,0 +1,8 @@ + + + + + 1 + + + diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 8366ca5dc25..05a8c72019f 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -10,7 +10,10 @@ node1 = cluster.add_instance( "node1", main_configs=["configs/named_collections.xml"], with_postgres=True ) node2 = cluster.add_instance( - "node2", main_configs=["configs/named_collections.xml"], with_postgres_cluster=True + "node2", + main_configs=["configs/named_collections.xml"], + user_configs=["configs/settings.xml"], + with_postgres_cluster=True, ) @@ -19,6 +22,7 @@ def started_cluster(): try: cluster.start() node1.query("CREATE DATABASE test") + node2.query("CREATE DATABASE test") yield cluster finally: @@ -625,6 +629,48 @@ def test_uuid(started_cluster): assert result.strip() == "Nullable(UUID)" +def test_auto_close_connection(started_cluster): + conn = get_postgres_conn( + started_cluster.postgres_ip, started_cluster.postgres_port, database=False + ) + cursor = conn.cursor() + database_name = "auto_close_connection_test" + + cursor.execute(f"DROP DATABASE IF EXISTS {database_name}") + cursor.execute(f"CREATE DATABASE {database_name}") + conn = get_postgres_conn( + started_cluster.postgres_ip, + started_cluster.postgres_port, + database=True, + database_name=database_name, + ) + cursor = conn.cursor() + cursor.execute("CREATE TABLE test_table (key integer, value integer)") + + node2.query( + f""" + CREATE TABLE test.test_table (key UInt32, value UInt32) + ENGINE = PostgreSQL(postgres1, database='{database_name}', table='test_table') + """ + ) + + result = node2.query( + "INSERT INTO test.test_table SELECT number, number FROM numbers(1000)", + user="default", + ) + + result = node2.query("SELECT * FROM test.test_table LIMIT 100", user="default") + + count = int( + node2.query( + f"SELECT numbackends FROM postrgesql(postgres1, database='{database_name}', table='pg_stat_database') WHERE datname = '{database_name}'" + ) + ) + + # Connection from python + assert count == 1 + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From d1243c1e01e30917353f42a949d5370e9e8951f2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 27 Jun 2022 15:35:35 +0200 Subject: [PATCH 077/627] Update test.py --- tests/integration/test_storage_postgresql/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 05a8c72019f..2b0173a4a1f 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -663,7 +663,7 @@ def test_auto_close_connection(started_cluster): count = int( node2.query( - f"SELECT numbackends FROM postrgesql(postgres1, database='{database_name}', table='pg_stat_database') WHERE datname = '{database_name}'" + f"SELECT numbackends FROM postgresql(postgres1, database='{database_name}', table='pg_stat_database') WHERE datname = '{database_name}'" ) ) From 344ddc6e7dd709d7ab2b67a949764adba35ff1d3 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Jun 2022 13:55:51 +0000 Subject: [PATCH 078/627] Fix tests --- src/TableFunctions/TableFunctionSQLite.cpp | 2 +- tests/queries/0_stateless/02245_s3_schema_desc.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/TableFunctions/TableFunctionSQLite.cpp b/src/TableFunctions/TableFunctionSQLite.cpp index 453b135303a..64ff93494db 100644 --- a/src/TableFunctions/TableFunctionSQLite.cpp +++ b/src/TableFunctions/TableFunctionSQLite.cpp @@ -74,7 +74,7 @@ void TableFunctionSQLite::parseArguments(const ASTPtr & ast_function, ContextPtr arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); database_path = checkAndGetLiteralArgument(args[0], "database_path"); - remote_table_name = checkAndGetLiteralArgument(args[0], "table_name"); + remote_table_name = checkAndGetLiteralArgument(args[1], "table_name"); sqlite_db = openSQLiteDB(database_path, context); } diff --git a/tests/queries/0_stateless/02245_s3_schema_desc.sql b/tests/queries/0_stateless/02245_s3_schema_desc.sql index 2cd362ff233..8c12d196800 100644 --- a/tests/queries/0_stateless/02245_s3_schema_desc.sql +++ b/tests/queries/0_stateless/02245_s3_schema_desc.sql @@ -11,4 +11,4 @@ desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); -SELECT * FROM s3(decodeURLComponent(NULL), [NULL]); --{serverError 170} +SELECT * FROM s3(decodeURLComponent(NULL), [NULL]); --{serverError BAD_ARGUMENTS} From 12448f94007ac64575113f0b6121d6035d92b8f1 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Jun 2022 13:56:43 +0000 Subject: [PATCH 079/627] Fix style --- src/Storages/checkAndGetLiteralArgument.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/checkAndGetLiteralArgument.h b/src/Storages/checkAndGetLiteralArgument.h index 6247dc66455..086deca5121 100644 --- a/src/Storages/checkAndGetLiteralArgument.h +++ b/src/Storages/checkAndGetLiteralArgument.h @@ -11,6 +11,4 @@ T checkAndGetLiteralArgument(const ASTPtr & arg, const String & arg_name); template T checkAndGetLiteralArgument(const ASTLiteral & arg, const String & arg_name); - - } From f44250b3f1c352350e9a59fbc9de5458cea66514 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 17:56:11 +0200 Subject: [PATCH 080/627] Compatible benchmark --- benchmark/compatible/README.md | 7 ++- benchmark/compatible/infobright/benchmark.sh | 6 +++ .../mysql-myisam/results/c6a.4xlarge.txt | 43 +++++++++++++++++++ benchmark/compatible/vertica/benchmark.sh | 10 +++++ benchmark/compatible/vertica/queries.sql | 43 +++++++++++++++++++ benchmark/compatible/vertica/run.sh | 13 ++++++ 6 files changed, 120 insertions(+), 2 deletions(-) create mode 100644 benchmark/compatible/vertica/queries.sql create mode 100644 benchmark/compatible/vertica/run.sh diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index ceb42f9ba7a..d04f92ec3fe 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -1,3 +1,5 @@ +Run all tests on c6a.4xlarge, 500 GB gp2. + - [x] ClickHouse - [x] MySQL InnoDB - [ ] MySQL MyISAM @@ -8,7 +10,7 @@ - [ ] Greenplum - [x] TimescaleDB - [x] Citus -- [ ] Vertica (without publishing) +- [x] Vertica (without publishing) - [x] QuestDB - [x] DuckDB - [x] MonetDB @@ -20,7 +22,7 @@ - [ ] CrateDB - [ ] Spark SQL - [ ] Starrocks -- [ ] ShitholeSQL +- [ ] ShitholeDB - [ ] Hive - [ ] Impala - [ ] Hyper @@ -52,3 +54,4 @@ - [ ] Starburst Galaxy - [ ] MS SQL Server with Column Store Index (without publishing) - [ ] Dremio (without publishing) +- [ ] Exasol diff --git a/benchmark/compatible/infobright/benchmark.sh b/benchmark/compatible/infobright/benchmark.sh index 7b169359564..97ae6284d82 100755 --- a/benchmark/compatible/infobright/benchmark.sh +++ b/benchmark/compatible/infobright/benchmark.sh @@ -24,3 +24,9 @@ time sudo docker run -it --rm --volume $(pwd):/workdir --network host mysql:5 my FIELDS TERMINATED BY '\\t' ENCLOSED BY '' ESCAPED BY '\\\\' LINES TERMINATED BY '\\n' STARTING BY ''" # 38m37.466s + +sudo docker exec mysql_ib du -bcs /mnt/mysql_data/ /usr/local/infobright-4.0.7-x86_64/cache + +# 13 760 341 294 + +./run.sh 2>&1 | log diff --git a/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt b/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt index 4723e639dda..5dc7c642eb6 100644 --- a/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt +++ b/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt @@ -1,2 +1,45 @@ Load time: 41:09 Data size: 121 588 958 061 bytes + +[0.00,0.00,0.00], +[283.32,276.83,274.52], +[276.93,278.29,283.27], +[28.83,23.63,21.55], +[46.41,40.81,40.93], +[467.04,467.39,469.08], +[31.02,25.89,24.20], +[277.89,275.3,277.3], +[329.34,325.8,325.35], +[342.86,338.43,336.95], +[282.03,279.87,281.22], +[277.74,282.68,282], +[335.66,334.83,336.44], +[305.24,310.39,307.3], +[337.41,338.52,342.94], +[308.66,307.34,306.27], +[738.38,748.44,740.75], +[738.75,734.01,738.25], +[867.01,872.92,868.84], +[25.65,20.61,18.46], +[312.39,313.67,306.66], +[301.66,305.12,308.01], +[298.12,298.44,312.4], +[311.34,309.9,311.85], +[281.87,278.5,275], +[277.46,277.46,277.46], +[280.75,278.04,281.76], +[263.9,417.39,406.88], +[707.21,711.96,705], +[668.1,668.33,665.96], +[330.31,333.36,331.94], +[506.57,506.18,500.53], +[2604.49,2681.96,2703.12], +[830.65,832.88,831.14], +[831.98,830.46,833.41], +[608.49,608.51,613.68], +[4.56,4.13,4.16], +[3.80,3.80,3.70], +[1.65,1.45,1.46], +[6.33,5.14,6.15], +[1.60,1.41,1.41], +[1.56,1.42,1.39] diff --git a/benchmark/compatible/vertica/benchmark.sh b/benchmark/compatible/vertica/benchmark.sh index 0054d36b0f9..fdae7578d1f 100644 --- a/benchmark/compatible/vertica/benchmark.sh +++ b/benchmark/compatible/vertica/benchmark.sh @@ -11,3 +11,13 @@ wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz time sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM LOCAL '/workdir/hits.tsv' DELIMITER E'\\t' NULL E'\\001' DIRECT" + +sudo docker exec vertica_ce du -bcs /data/vertica/VMart + +./run.sh 2>&1 | tee log.txt + +# If you run the script on your own, you may get numbers like this: +# 200m00.000s +# 25000000000 + +# Note: the real numbers cannot be published. diff --git a/benchmark/compatible/vertica/queries.sql b/benchmark/compatible/vertica/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/vertica/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/vertica/run.sh b/benchmark/compatible/vertica/run.sh new file mode 100644 index 00000000000..7638dbb0299 --- /dev/null +++ b/benchmark/compatible/vertica/run.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + echo "$query"; + for i in $(seq 1 $TRIES); do + sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c '\timing' -c "$query" + done; +done; From 69cef6ebc6626b196d8ca3c94044ea387a62014f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 18:02:46 +0200 Subject: [PATCH 081/627] Compatible benchmark --- benchmark/compatible/redshift/README.md | 3 ++- benchmark/compatible/vertica/.gitignore | 1 + benchmark/compatible/vertica/benchmark.sh | 3 +++ 3 files changed, 6 insertions(+), 1 deletion(-) create mode 100644 benchmark/compatible/vertica/.gitignore diff --git a/benchmark/compatible/redshift/README.md b/benchmark/compatible/redshift/README.md index 4ea34b677f8..325c19f3fe1 100644 --- a/benchmark/compatible/redshift/README.md +++ b/benchmark/compatible/redshift/README.md @@ -47,7 +47,8 @@ Then run the benchmark: export HOST=... ./run.sh 2>&1 | tee log.txt - +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' ``` `SELECT sum(used * 1048576) FROM stv_node_storage_capacity` diff --git a/benchmark/compatible/vertica/.gitignore b/benchmark/compatible/vertica/.gitignore new file mode 100644 index 00000000000..1a06816d838 --- /dev/null +++ b/benchmark/compatible/vertica/.gitignore @@ -0,0 +1 @@ +results diff --git a/benchmark/compatible/vertica/benchmark.sh b/benchmark/compatible/vertica/benchmark.sh index fdae7578d1f..86312a3a438 100644 --- a/benchmark/compatible/vertica/benchmark.sh +++ b/benchmark/compatible/vertica/benchmark.sh @@ -21,3 +21,6 @@ sudo docker exec vertica_ce du -bcs /data/vertica/VMart # 25000000000 # Note: the real numbers cannot be published. + +grep -F 'All rows formatted' logs.txt | sed -r -e 's/^.* ([0-9.]+) ms$/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' From aa3db815ad13748c34ae196d39b6455c0f31eeef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Jun 2022 18:03:18 +0200 Subject: [PATCH 082/627] Compatible benchmark --- benchmark/compatible/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index d04f92ec3fe..248454affba 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -2,7 +2,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [x] ClickHouse - [x] MySQL InnoDB -- [ ] MySQL MyISAM +- [x] MySQL MyISAM - [ ] MariaDB - [x] MariaDB ColumnStore - [x] MemSQL/SingleStore From 6a5819a4055cec168718b34c63b3d6111c82c859 Mon Sep 17 00:00:00 2001 From: HeenaBansal2009 Date: Mon, 27 Jun 2022 22:51:31 -0700 Subject: [PATCH 083/627] Fixed bug in parsing{..} regex --- src/Common/parseGlobs.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index 75539512b6d..c7e008a3d71 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -51,7 +51,8 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob ReadBufferFromString buf_range(buffer); buf_range >> range_begin >> point >> point >> range_end; bool leading_zeros = buffer[0] == '0'; - size_t num_len = std::to_string(range_end).size(); + //Append '0' for values starting with {00..99} and not {0..10} + size_t num_len = buffer.size() - 2 - std::to_string(range_end).size(); if (leading_zeros) oss_for_replacing << std::setfill('0') << std::setw(num_len); oss_for_replacing << range_begin; From 4c99f608b3bd5f1f8d48fe574341b66be7a7304c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Jun 2022 02:07:16 +0200 Subject: [PATCH 084/627] Fix --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 36 ++++++++++++---- .../integration/test_storage_rabbitmq/test.py | 41 +++++++++++++++++++ 2 files changed, 70 insertions(+), 7 deletions(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 58f08c48c68..6a94b1a28dc 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -93,18 +93,40 @@ StorageRabbitMQ::StorageRabbitMQ( , milliseconds_to_wait(RESCHEDULE_MS) , is_attach(is_attach_) { - auto parsed_address = parseAddress(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_host_port), 5672); - context_->getRemoteHostFilter().checkHostAndPort(parsed_address.first, toString(parsed_address.second)); + const auto & config = getContext()->getConfigRef(); + + std::pair parsed_address; + auto setting_rabbitmq_username = rabbitmq_settings->rabbitmq_username.value; + auto setting_rabbitmq_password = rabbitmq_settings->rabbitmq_password.value; + String username, password; + + if (rabbitmq_settings->rabbitmq_host_port.changed) + { + username = setting_rabbitmq_username.empty() ? config.getString("rabbitmq.username", "") : setting_rabbitmq_username; + password = setting_rabbitmq_password.empty() ? config.getString("rabbitmq.password", "") : setting_rabbitmq_password; + if (username.empty() || password.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "No username or password. They can be specified either in config or in storage settings"); + + parsed_address = parseAddress(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_host_port), 5672); + if (parsed_address.first.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Host or port is incorrect (host: {}, port: {})", parsed_address.first, parsed_address.second); + + context_->getRemoteHostFilter().checkHostAndPort(parsed_address.first, toString(parsed_address.second)); + } + else if (!rabbitmq_settings->rabbitmq_address.changed) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "RabbitMQ requires either `rabbitmq_host_port` or `rabbitmq_address` setting"); - auto rabbitmq_username = rabbitmq_settings->rabbitmq_username.value; - auto rabbitmq_password = rabbitmq_settings->rabbitmq_password.value; configuration = { .host = parsed_address.first, .port = parsed_address.second, - .username = rabbitmq_username.empty() ? getContext()->getConfigRef().getString("rabbitmq.username") : rabbitmq_username, - .password = rabbitmq_password.empty() ? getContext()->getConfigRef().getString("rabbitmq.password") : rabbitmq_password, - .vhost = getContext()->getConfigRef().getString("rabbitmq.vhost", getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_vhost)), + .username = username, + .password = password, + .vhost = config.getString("rabbitmq.vhost", getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_vhost)), .secure = rabbitmq_settings->rabbitmq_secure.value, .connection_string = getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_address) }; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index c1bd136126f..f497a5bb2c0 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -30,6 +30,11 @@ instance = cluster.add_instance( stay_alive=True, ) +instance2 = cluster.add_instance( + "instance2", + user_configs=["configs/users.xml"], + with_rabbitmq=True, +) # Helpers @@ -2745,6 +2750,42 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): break +def test_rabbitmq_address(rabbitmq_cluster): + + instance2.query(""" + drop table if exists rabbit_in; + drop table if exists rabbit_out; + create table + rabbit_in (val String) + engine=RabbitMQ + SETTINGS rabbitmq_exchange_name = 'rxhep', + rabbitmq_format = 'CSV', + rabbitmq_num_consumers = 1, + rabbitmq_address='amqp://root:clickhouse@rabbitmq1:5672/'; + create table + rabbit_out (val String) engine=RabbitMQ + SETTINGS rabbitmq_exchange_name = 'rxhep', + rabbitmq_format = 'CSV', + rabbitmq_num_consumers = 1, + rabbitmq_address='amqp://root:clickhouse@rabbitmq1:5672/'; + set stream_like_engine_allow_direct_select=1; + insert into rabbit_out select 'kek'; + """) + + result = "" + try_no = 0 + while True: + result = instance2.query("select * from rabbit_in;") + if result.strip() == "kek": + break + else: + try_no = try_no + 1 + if try_no == 20: + break + time.sleep(1) + assert result.strip() == "kek" + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From a28a7a363ec1de7e9731713d492bf3c5c7f5fd2c Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Tue, 28 Jun 2022 19:44:31 +0000 Subject: [PATCH 085/627] Add Apache Druid results --- benchmark/compatible/druid/benchmark.sh | 32 + benchmark/compatible/druid/ingest.json | 573 ++++++++++++++++++ benchmark/compatible/druid/queries.sql | 43 ++ .../compatible/druid/results/c6a.4xlarge.txt | 43 ++ benchmark/compatible/druid/run.sh | 21 + 5 files changed, 712 insertions(+) create mode 100755 benchmark/compatible/druid/benchmark.sh create mode 100644 benchmark/compatible/druid/ingest.json create mode 100644 benchmark/compatible/druid/queries.sql create mode 100644 benchmark/compatible/druid/results/c6a.4xlarge.txt create mode 100755 benchmark/compatible/druid/run.sh diff --git a/benchmark/compatible/druid/benchmark.sh b/benchmark/compatible/druid/benchmark.sh new file mode 100755 index 00000000000..b147eef7c1c --- /dev/null +++ b/benchmark/compatible/druid/benchmark.sh @@ -0,0 +1,32 @@ +#!/bin/bash +# If you have no java installed: +# sudo apt-get update +# sudo apt install openjdk-8-jdk -y +# sudo update-alternatives --config java + +VERSION=0.23.0 +# Install + +wget -O"apache-druid-${VERSION}-bin.tar.gz" "https://dlcdn.apache.org/druid/${VERSION}/apache-druid-${VERSION}-bin.tar.gz" +tar xf apache-druid-${VERSION}-bin.tar.gz +./apache-druid-${VERSION}/bin/verify-java +# Have to increase indexer memory limit +sed -i 's MaxDirectMemorySize=1g MaxDirectMemorySize=5g g' apache-druid-$VERSION/conf/druid/single-server/medium/middleManager/runtime.properties +# Druid launcher does not start Druid as a daemon. Run it in background +./apache-druid-${VERSION}/bin/start-single-server-medium & + +# Load the data + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz +split --additional-suffix .tsv --verbose -n l/10 hits.tsv input +# Running 10 tasks one by one to make it work in parallel +./apache-druid-${VERSION}/bin/post-index-task --file ingest.json --url http://localhost:8081 +# Run the queries + +./run.sh + +# stop Druid services +kill %1 + +du -bcs ./apache-druid-${VERSION}/var diff --git a/benchmark/compatible/druid/ingest.json b/benchmark/compatible/druid/ingest.json new file mode 100644 index 00000000000..45bb40482af --- /dev/null +++ b/benchmark/compatible/druid/ingest.json @@ -0,0 +1,573 @@ +{ + "type": "index_parallel", + "spec": { + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "local", + "baseDir": "../", + "filter": "inputa?.tsv" + }, + "inputFormat": { + "type": "tsv", + "findColumnsFromHeader": false, + "columns": [ + "WatchID", + "JavaEnable", + "Title", + "GoodEvent", + "EventTime", + "EventDate", + "CounterID", + "ClientIP", + "RegionID", + "UserID", + "CounterClass", + "OS", + "UserAgent", + "URL", + "Referer", + "IsRefresh", + "RefererCategoryID", + "RefererRegionID", + "URLCategoryID", + "URLRegionID", + "ResolutionWidth", + "ResolutionHeight", + "ResolutionDepth", + "FlashMajor", + "FlashMinor", + "FlashMinor2", + "NetMajor", + "NetMinor", + "UserAgentMajor", + "UserAgentMinor", + "CookieEnable", + "JavascriptEnable", + "IsMobile", + "MobilePhone", + "MobilePhoneModel", + "Params", + "IPNetworkID", + "TraficSourceID", + "SearchEngineID", + "SearchPhrase", + "AdvEngineID", + "IsArtifical", + "WindowClientWidth", + "WindowClientHeight", + "ClientTimeZone", + "ClientEventTime", + "SilverlightVersion1", + "SilverlightVersion2", + "SilverlightVersion3", + "SilverlightVersion4", + "PageCharset", + "CodeVersion", + "IsLink", + "IsDownload", + "IsNotBounce", + "FUniqID", + "OriginalURL", + "HID", + "IsOldCounter", + "IsEvent", + "IsParameter", + "DontCountHits", + "WithHash", + "HitColor", + "LocalEventTime", + "Age", + "Sex", + "Income", + "Interests", + "Robotness", + "RemoteIP", + "WindowName", + "OpenerName", + "HistoryLength", + "BrowserLanguage", + "BrowserCountry", + "SocialNetwork", + "SocialAction", + "HTTPError", + "SendTiming", + "DNSTiming", + "ConnectTiming", + "ResponseStartTiming", + "ResponseEndTiming", + "FetchTiming", + "SocialSourceNetworkID", + "SocialSourcePage", + "ParamPrice", + "ParamOrderID", + "ParamCurrency", + "ParamCurrencyID", + "OpenstatServiceName", + "OpenstatCampaignID", + "OpenstatAdID", + "OpenstatSourceID", + "UTMSource", + "UTMMedium", + "UTMCampaign", + "UTMContent", + "UTMTerm", + "FromTag", + "HasGCLID", + "RefererHash", + "URLHash", + "CLID" + ] + } + }, + "tuningConfig": { + "type": "index_parallel", + "partitionsSpec": { + "type": "hashed", + "partitionDimensions": [ + "CounterID", + "EventDate", + "UserID", + "EventTime", + "WatchID" + ] + }, + "forceGuaranteedRollup": true, + "logParseExceptions": true, + "maxParseExceptions": 1, + "maxNumConcurrentSubTasks": 10 + }, + "dataSchema": { + "dataSource": "hits", + "timestampSpec": { + "column": "EventTime", + "format": "yyyy-MM-dd HH:mm:ss" + }, + "dimensionsSpec": { + "dimensions": [ + { + "name": "WatchID", + "type": "long" + }, + { + "name": "JavaEnable", + "type": "long" + }, + { + "name": "Title", + "type": "string" + }, + { + "name": "GoodEvent", + "type": "long" + }, + { + "name": "EventDate", + "type": "string" + }, + { + "name": "CounterID", + "type": "long" + }, + { + "name": "ClientIP", + "type": "long" + }, + { + "name": "RegionID", + "type": "long" + }, + { + "name": "UserID", + "type": "long" + }, + { + "name": "CounterClass", + "type": "long" + }, + { + "name": "OS", + "type": "long" + }, + { + "name": "UserAgent", + "type": "long" + }, + { + "name": "URL", + "type": "string" + }, + { + "name": "Referer", + "type": "string" + }, + { + "name": "IsRefresh", + "type": "long" + }, + { + "name": "RefererCategoryID", + "type": "long" + }, + { + "name": "RefererRegionID", + "type": "long" + }, + { + "name": "URLCategoryID", + "type": "long" + }, + { + "name": "URLRegionID", + "type": "long" + }, + { + "name": "ResolutionWidth", + "type": "long" + }, + { + "name": "ResolutionHeight", + "type": "long" + }, + { + "name": "ResolutionDepth", + "type": "long" + }, + { + "name": "FlashMajor", + "type": "long" + }, + { + "name": "FlashMinor", + "type": "long" + }, + { + "name": "FlashMinor2", + "type": "string" + }, + { + "name": "NetMajor", + "type": "long" + }, + { + "name": "NetMinor", + "type": "long" + }, + { + "name": "UserAgentMajor", + "type": "long" + }, + { + "name": "UserAgentMinor", + "type": "string" + }, + { + "name": "CookieEnable", + "type": "long" + }, + { + "name": "JavascriptEnable", + "type": "long" + }, + { + "name": "IsMobile", + "type": "long" + }, + { + "name": "MobilePhone", + "type": "long" + }, + { + "name": "MobilePhoneModel", + "type": "string" + }, + { + "name": "Params", + "type": "string" + }, + { + "name": "IPNetworkID", + "type": "long" + }, + { + "name": "TraficSourceID", + "type": "long" + }, + { + "name": "SearchEngineID", + "type": "long" + }, + { + "name": "SearchPhrase", + "type": "string" + }, + { + "name": "AdvEngineID", + "type": "long" + }, + { + "name": "IsArtifical", + "type": "long" + }, + { + "name": "WindowClientWidth", + "type": "long" + }, + { + "name": "WindowClientHeight", + "type": "long" + }, + { + "name": "ClientTimeZone", + "type": "long" + }, + { + "name": "ClientEventTime", + "type": "string" + }, + { + "name": "SilverlightVersion1", + "type": "long" + }, + { + "name": "SilverlightVersion2", + "type": "long" + }, + { + "name": "SilverlightVersion3", + "type": "long" + }, + { + "name": "SilverlightVersion4", + "type": "long" + }, + { + "name": "PageCharset", + "type": "string" + }, + { + "name": "CodeVersion", + "type": "long" + }, + { + "name": "IsLink", + "type": "long" + }, + { + "name": "IsDownload", + "type": "long" + }, + { + "name": "IsNotBounce", + "type": "long" + }, + { + "name": "FUniqID", + "type": "long" + }, + { + "name": "OriginalURL", + "type": "string" + }, + { + "name": "HID", + "type": "long" + }, + { + "name": "IsOldCounter", + "type": "long" + }, + { + "name": "IsEvent", + "type": "long" + }, + { + "name": "IsParameter", + "type": "long" + }, + { + "name": "DontCountHits", + "type": "long" + }, + { + "name": "WithHash", + "type": "long" + }, + { + "name": "HitColor", + "type": "string" + }, + { + "name": "LocalEventTime", + "type": "string" + }, + { + "name": "Age", + "type": "long" + }, + { + "name": "Sex", + "type": "long" + }, + { + "name": "Income", + "type": "long" + }, + { + "name": "Interests", + "type": "long" + }, + { + "name": "Robotness", + "type": "long" + }, + { + "name": "RemoteIP", + "type": "long" + }, + { + "name": "WindowName", + "type": "long" + }, + { + "name": "OpenerName", + "type": "long" + }, + { + "name": "HistoryLength", + "type": "long" + }, + { + "name": "BrowserLanguage", + "type": "string" + }, + { + "name": "BrowserCountry", + "type": "string" + }, + { + "name": "SocialNetwork", + "type": "string" + }, + { + "name": "SocialAction", + "type": "string" + }, + { + "name": "HTTPError", + "type": "long" + }, + { + "name": "SendTiming", + "type": "long" + }, + { + "name": "DNSTiming", + "type": "long" + }, + { + "name": "ConnectTiming", + "type": "long" + }, + { + "name": "ResponseStartTiming", + "type": "long" + }, + { + "name": "ResponseEndTiming", + "type": "long" + }, + { + "name": "FetchTiming", + "type": "long" + }, + { + "name": "SocialSourceNetworkID", + "type": "long" + }, + { + "name": "SocialSourcePage", + "type": "string" + }, + { + "name": "ParamPrice", + "type": "long" + }, + { + "name": "ParamOrderID", + "type": "string" + }, + { + "name": "ParamCurrency", + "type": "string" + }, + { + "name": "ParamCurrencyID", + "type": "long" + }, + { + "name": "OpenstatServiceName", + "type": "string" + }, + { + "name": "OpenstatCampaignID", + "type": "string" + }, + { + "name": "OpenstatAdID", + "type": "string" + }, + { + "name": "OpenstatSourceID", + "type": "string" + }, + { + "name": "UTMSource", + "type": "string" + }, + { + "name": "UTMMedium", + "type": "string" + }, + { + "name": "UTMCampaign", + "type": "string" + }, + { + "name": "UTMContent", + "type": "string" + }, + { + "name": "UTMTerm", + "type": "string" + }, + { + "name": "FromTag", + "type": "string" + }, + { + "name": "HasGCLID", + "type": "long" + }, + { + "name": "RefererHash", + "type": "long" + }, + { + "name": "URLHash", + "type": "long" + }, + { + "name": "CLID", + "type": "long" + } + ] + }, + "granularitySpec": { + "queryGranularity": "none", + "rollup": false, + "segmentGranularity": "day" + } + } + } +} diff --git a/benchmark/compatible/druid/queries.sql b/benchmark/compatible/druid/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/druid/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/druid/results/c6a.4xlarge.txt b/benchmark/compatible/druid/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..dd185c349f9 --- /dev/null +++ b/benchmark/compatible/druid/results/c6a.4xlarge.txt @@ -0,0 +1,43 @@ +[0.045539, 0.028086, 0.024169], +[0.287153, 0.026538, 0.024661], +[0.171896, 0.023620, 0.023566], +[1.553365, 0.021595, 0.021120], +[1.432360, 0.023391, 0.025546], +[3.418428, 0.020131, 0.020552], +[0.051194, 0.018222, 0.011529], +[0.268591, 0.041415, 0.034791], +[3.194689, 0.370084, 0.325961], +[2.973067, 0.316630, 0.300042], +[0.362751, 0.033894, 0.034276], +[0.352240, 0.067931, 0.069787], +[0.829385, 0.077289, 0.053920], +[3.075808, 0.235023, 0.281975], +[3.595905, 1.883551, 1.893956], +[6.186890, 0.039829, 0.032672], +[4.399702, 4.404082, 3.892671], +[3.744337, 4.524327, 5.312231], +[0.024576, 0.003828, 0.003150], +[0.943878, 0.899737, 0.895774], +[20.332790, 0.017449, 0.015121], +[0.081357, 0.025420, 0.014019], +[0.032782, 0.018635, 0.018162], +[8.747672, 10.071709, 6.743290], +[0.137468, 0.018731, 0.016640], +[0.014924, 0.011045, 0.011923], +[0.012629, 0.010551, 0.010522], +[300.245240, 300.337436, 300.220133], +[0.010773, 0.004759, 0.003960], +[1.618904, 0.114704, 0.095059], +[2.409857, 1.982224, 1.915120], +[4.036227, 1.908413, 1.817666], +[8.306265, 7.861117, 8.076500], +[17.398126, 0.054853, 0.039628], +[300.044544, 300.081014, 300.144849], +[4.389508, 3.980315, 4.060862], +[0.690731, 0.041153, 0.038142], +[0.574323, 0.034436, 0.036309], +[0.501737, 0.145582, 0.154863], +[0.005135, 0.004237, 0.003943], +[0.472923, 0.339387, 0.286213], +[0.349483, 0.142984, 0.095204], +[0.245475, 0.057260, 0.060106] diff --git a/benchmark/compatible/druid/run.sh b/benchmark/compatible/druid/run.sh new file mode 100755 index 00000000000..edf9c199d95 --- /dev/null +++ b/benchmark/compatible/druid/run.sh @@ -0,0 +1,21 @@ +#!/bin/bash + +TRIES=3 +QUERY_NUM=1 +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + + echo -n "[" + for i in $(seq 1 $TRIES); do + echo "{\"query\":\"$query\"}"| sed -e 's EventTime __time g' | tr -d ';' > query.json + RES=$(curl -o /dev/null -s -w '%{time_total}\n' -XPOST -H'Content-Type: application/json' http://localhost:8888/druid/v2/sql/ -d @query.json) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + + echo "${QUERY_NUM},${i},${RES}" >> result.csv + done + echo "]," + + QUERY_NUM=$((QUERY_NUM + 1)) +done From 608bfb8453cf4c6792ab963ca9383c35d00053ad Mon Sep 17 00:00:00 2001 From: HeenaBansal2009 Date: Tue, 28 Jun 2022 12:49:31 -0700 Subject: [PATCH 086/627] Fixed {..} regex parsing as shell does --- src/Common/parseGlobs.cpp | 12 ++++++++++-- .../tests/gtest_makeRegexpPatternFromGlobs.cpp | 11 +++++++---- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index c7e008a3d71..9e18a9c0780 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -51,8 +51,16 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob ReadBufferFromString buf_range(buffer); buf_range >> range_begin >> point >> point >> range_end; bool leading_zeros = buffer[0] == '0'; - //Append '0' for values starting with {00..99} and not {0..10} - size_t num_len = buffer.size() - 2 - std::to_string(range_end).size(); + + size_t range_begin_width = buffer.find('.'); + size_t range_end_width = buffer.size() - buffer.find_last_of('.') - 1; + //Scenarios {0..10} {0..999} + size_t num_len = 0; + if (range_begin_width == 1 && leading_zeros) + num_len = 1; + //Scenarios {00..99} {00..099} + else + num_len = range_begin_width < range_end_width ? range_end_width : range_begin_width; if (leading_zeros) oss_for_replacing << std::setfill('0') << std::setw(num_len); oss_for_replacing << range_begin; diff --git a/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp b/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp index 42777d0bbba..833a0a6ba73 100644 --- a/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp +++ b/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp @@ -13,13 +13,16 @@ TEST(Common, makeRegexpPatternFromGlobs) EXPECT_EQ(makeRegexpPatternFromGlobs("/?"), "/[^/]"); EXPECT_EQ(makeRegexpPatternFromGlobs("/*"), "/[^/]*"); EXPECT_EQ(makeRegexpPatternFromGlobs("*_{{a,b,c,d}}/?.csv"), "[^/]*_\\{(a|b|c|d)\\}/[^/]\\.csv"); - EXPECT_EQ(makeRegexpPatternFromGlobs("f{01..09}"), "f(1|2|3|4|5|6|7|8|9)"); - EXPECT_EQ(makeRegexpPatternFromGlobs("f{01..9}"), "f(1|2|3|4|5|6|7|8|9)"); - EXPECT_EQ(makeRegexpPatternFromGlobs("f{0001..0000009}"), "f(1|2|3|4|5|6|7|8|9)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{01..9}"), "f(01|02|03|04|05|06|07|08|09)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{001..0009}"), "f(0001|0002|0003|0004|0005|0006|0007|0008|0009)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{0..10}"), "f(0|1|2|3|4|5|6|7|8|9|10)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{00..10}"), "f(00|01|02|03|04|05|06|07|08|09|10)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{000..9}"), "f(000|001|002|003|004|005|006|007|008|009)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{0001..0009}"), "f(0001|0002|0003|0004|0005|0006|0007|0008|0009)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..2}{1..2}"), "f(1|2)(1|2)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..1}{1..1}"), "f(1)(1)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{0..0}{0..0}"), "f(0)(0)"); EXPECT_EQ(makeRegexpPatternFromGlobs("file{1..5}"),"file(1|2|3|4|5)"); EXPECT_EQ(makeRegexpPatternFromGlobs("file{1,2,3}"),"file(1|2|3)"); EXPECT_EQ(makeRegexpPatternFromGlobs("{1,2,3}blabla{a.x,b.x,c.x}smth[]_else{aa,bb}?*"), "(1|2|3)blabla(a\\.x|b\\.x|c\\.x)smth\\[\\]_else(aa|bb)[^/][^/]*"); -} +} \ No newline at end of file From 71e9363bafc35a62e55ba19abcff45645cdb0cf8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 29 Jun 2022 07:32:27 +0200 Subject: [PATCH 087/627] Fix black --- tests/integration/test_storage_rabbitmq/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index f497a5bb2c0..a8dcfe3a7d6 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2752,7 +2752,8 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): def test_rabbitmq_address(rabbitmq_cluster): - instance2.query(""" + instance2.query( + """ drop table if exists rabbit_in; drop table if exists rabbit_out; create table @@ -2770,7 +2771,8 @@ def test_rabbitmq_address(rabbitmq_cluster): rabbitmq_address='amqp://root:clickhouse@rabbitmq1:5672/'; set stream_like_engine_allow_direct_select=1; insert into rabbit_out select 'kek'; - """) + """ + ) result = "" try_no = 0 From e32a0838d1b42beca1538f741f0d38ec09631ab4 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 29 Jun 2022 15:52:08 +0800 Subject: [PATCH 088/627] fix bug for mapUpdate --- src/Functions/map.cpp | 23 +++++++++++++++---- .../0_stateless/02169_map_functions.reference | 10 ++++++++ .../0_stateless/02169_map_functions.sql | 11 +++++++++ 3 files changed, 40 insertions(+), 4 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 8c891fdec81..036e242cfc2 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -561,15 +561,23 @@ public: { const ColumnMap * col_map_left = typeid_cast(arguments[0].column.get()); const auto * col_const_map_left = checkAndGetColumnConst(arguments[0].column.get()); + bool col_const_map_left_flag = false; if (col_const_map_left) + { + col_const_map_left_flag = true; col_map_left = typeid_cast(&col_const_map_left->getDataColumn()); + } if (!col_map_left) return nullptr; const ColumnMap * col_map_right = typeid_cast(arguments[1].column.get()); const auto * col_const_map_right = checkAndGetColumnConst(arguments[1].column.get()); + bool col_const_map_right_flag = false; if (col_const_map_right) + { + col_const_map_right_flag = true; col_map_right = typeid_cast(&col_const_map_right->getDataColumn()); + } if (!col_map_right) return nullptr; @@ -591,13 +599,18 @@ public: MutableColumnPtr offsets = DataTypeNumber().createColumn(); IColumn::Offset current_offset = 0; - for (size_t idx = 0; idx < input_rows_count; ++idx) + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) { - for (size_t i = offsets_left[idx - 1]; i < offsets_left[idx]; ++i) + size_t left_it_begin = col_const_map_left_flag ? 0 : offsets_left[row_idx - 1]; + size_t left_it_end = col_const_map_left_flag ? offsets_left.size() : offsets_left[row_idx]; + size_t right_it_begin = col_const_map_right_flag ? 0 : offsets_right[row_idx - 1]; + size_t right_it_end = col_const_map_right_flag ? offsets_right.size() : offsets_right[row_idx]; + + for (size_t i = left_it_begin; i < left_it_end; ++i) { bool matched = false; auto key = keys_data_left.getDataAt(i); - for (size_t j = offsets_right[idx - 1]; j < offsets_right[idx]; ++j) + for (size_t j = right_it_begin; j < right_it_end; ++j) { if (keys_data_right.getDataAt(j).toString() == key.toString()) { @@ -612,12 +625,14 @@ public: ++current_offset; } } - for (size_t j = offsets_right[idx - 1]; j < offsets_right[idx]; ++j) + + for (size_t j = right_it_begin; j < right_it_end; ++j) { keys_data->insertFrom(keys_data_right, j); values_data->insertFrom(values_data_right, j); ++current_offset; } + offsets->insert(current_offset); } diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 160aebbc852..16fda0bb840 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -31,3 +31,13 @@ {1:2,2:3} {'x':'y','x':'y'} {'x':'y','x':'y'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 4cccaa56722..9e93412739f 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -37,3 +37,14 @@ SELECT mapUpdate(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } DROP TABLE table_map; + +DROP TABLE IF EXISTS map_test; +CREATE TABLE map_test(`tags` Map(String, String)) ENGINE = MergeTree PRIMARY KEY tags ORDER BY tags SETTINGS index_granularity = 8192; +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +SELECT mapUpdate(mapFilter((k, v) -> (k in ('fruit')), tags), map('season', 'autumn')) FROM map_test; +SELECT mapUpdate(map('season','autumn'), mapFilter((k, v) -> (k in ('fruit')), tags)) FROM map_test; +DROP TABLE map_test; From 1d26446f8413ec9821e9e97961786e04570755c3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 29 Jun 2022 08:30:39 +0000 Subject: [PATCH 089/627] Define new list request type --- src/Common/ZooKeeper/IKeeper.h | 1 - src/Common/ZooKeeper/TestKeeper.cpp | 13 +++++-- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 34 ++++++++++++++----- src/Common/ZooKeeper/ZooKeeperCommon.h | 12 +++++++ src/Common/ZooKeeper/ZooKeeperConstants.cpp | 2 ++ src/Common/ZooKeeper/ZooKeeperConstants.h | 4 +++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- src/Coordination/KeeperStorage.cpp | 10 ++++-- src/Coordination/tests/gtest_coordination.cpp | 2 +- 9 files changed, 65 insertions(+), 15 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 9bd3ff3653c..79f9943cb57 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -291,7 +291,6 @@ enum class ListRequestType : uint8_t struct ListRequest : virtual Request { String path; - ListRequestType list_request_type{ListRequestType::ALL}; void addRootPath(const String & root_path) override; String getPath() const override { return path; } diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index d6fddb4d4b0..3af5dfcc177 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -120,12 +120,17 @@ struct TestKeeperSetRequest final : SetRequest, TestKeeperRequest } }; -struct TestKeeperListRequest final : ListRequest, TestKeeperRequest +struct TestKeeperListRequest : ListRequest, TestKeeperRequest { ResponsePtr createResponse() const override; std::pair process(TestKeeper::Container & container, int64_t zxid) const override; }; +struct TestKeeperFilteredListRequest final : TestKeeperListRequest +{ + ListRequestType list_request_type; +}; + struct TestKeeperCheckRequest final : CheckRequest, TestKeeperRequest { TestKeeperCheckRequest() = default; @@ -394,6 +399,10 @@ std::pair TestKeeperListRequest::process(TestKeeper::Containe using enum ListRequestType; if (parentPath(child_it->first) == path) { + ListRequestType list_request_type = ALL; + if (const auto * filtered_list = dynamic_cast(this)) + list_request_type = filtered_list->list_request_type; + const auto is_ephemeral = child_it->second.stat.ephemeralOwner != 0; if (list_request_type == ALL || (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY)) @@ -779,7 +788,7 @@ void TestKeeper::list( ListCallback callback, WatchCallback watch) { - TestKeeperListRequest request; + TestKeeperFilteredListRequest request; request.path = path; request.list_request_type = list_request_type; diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 41c5312bfa0..837ea5bbad8 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -286,20 +286,12 @@ void ZooKeeperListRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); Coordination::write(has_watch, out); - Coordination::write(static_cast(list_request_type), out); } void ZooKeeperListRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); Coordination::read(has_watch, in); - - if (!in.eof()) - { - uint8_t read_request_type; - Coordination::read(read_request_type, in); - list_request_type = static_cast(read_request_type); - } } std::string ZooKeeperListRequest::toStringImpl() const @@ -307,6 +299,32 @@ std::string ZooKeeperListRequest::toStringImpl() const return fmt::format("path = {}", path); } +void ZooKeeperFilteredListRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(has_watch, out); + Coordination::write(static_cast(list_request_type), out); +} + +void ZooKeeperFilteredListRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(has_watch, in); + + uint8_t read_request_type{0}; + Coordination::read(read_request_type, in); + list_request_type = static_cast(read_request_type); +} + +std::string ZooKeeperFilteredListRequest::toStringImpl() const +{ + return fmt::format( + "path = {}\n" + "list_request_type = {}", + path, + list_request_type); +} + void ZooKeeperListResponse::readImpl(ReadBuffer & in) { Coordination::read(names, in); diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index c7bfbe95b74..09f797fb47b 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -347,6 +347,18 @@ struct ZooKeeperSimpleListRequest final : ZooKeeperListRequest OpNum getOpNum() const override { return OpNum::SimpleList; } }; +struct ZooKeeperFilteredListRequest final : ZooKeeperListRequest +{ + ListRequestType list_request_type{ListRequestType::ALL}; + + OpNum getOpNum() const override { return OpNum::FilteredList; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + std::string toStringImpl() const override; + + size_t bytesSize() const override { return ZooKeeperListRequest::bytesSize() + sizeof(list_request_type); } +}; + struct ZooKeeperListResponse : ListResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.cpp b/src/Common/ZooKeeper/ZooKeeperConstants.cpp index b0a05fe6c8d..5b121ed6138 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.cpp +++ b/src/Common/ZooKeeper/ZooKeeperConstants.cpp @@ -64,6 +64,8 @@ std::string toString(OpNum op_num) return "SetACL"; case OpNum::GetACL: return "GetACL"; + case OpNum::FilteredList: + return "FilteredList"; } int32_t raw_op = static_cast(op_num); throw Exception("Operation " + std::to_string(raw_op) + " is unknown", Error::ZUNIMPLEMENTED); diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index 1ed2c442f6c..44f8437f12c 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -32,6 +32,10 @@ enum class OpNum : int32_t Check = 13, Multi = 14, Auth = 100, + + // CH Keeper specific operations + FilteredList = 500, + SessionID = 997, /// Special internal request }; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 7ab2d1960ab..8fa6f28c29c 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1172,7 +1172,7 @@ void ZooKeeper::list( ListCallback callback, WatchCallback watch) { - ZooKeeperListRequest request; + ZooKeeperFilteredListRequest request; request.path = path; request.list_request_type = list_request_type; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index f86f3ed7962..fd1fab5b6b0 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -6,6 +6,7 @@ #include #include #include +#include "Common/ZooKeeper/ZooKeeperCommon.h" #include #include #include @@ -1184,7 +1185,11 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc { using enum Coordination::ListRequestType; - if (request.list_request_type == ALL) + auto list_request_type = ALL; + if (auto * filtered_list = dynamic_cast(&request)) + list_request_type = filtered_list->list_request_type; + + if (list_request_type == ALL) return true; auto child_path = (std::filesystem::path(request.path) / child.toView()).generic_string(); @@ -1193,7 +1198,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc onStorageInconsistency(); const auto is_ephemeral = child_it->value.stat.ephemeralOwner != 0; - return (is_ephemeral && request.list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && request.list_request_type == PERSISTENT_ONLY); + return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); }; for (const auto child : children) @@ -1710,6 +1715,7 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() registerKeeperRequestProcessor(*this); registerKeeperRequestProcessor(*this); registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); registerKeeperRequestProcessor(*this); registerKeeperRequestProcessor(*this); registerKeeperRequestProcessor(*this); diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 61578b32912..bd0d329ef8d 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2001,7 +2001,7 @@ TEST_P(CoordinationTest, TestListRequestTypes) const auto get_children = [&](const auto list_request_type) { - const auto list_request = std::make_shared(); + const auto list_request = std::make_shared(); int new_zxid = ++zxid; list_request->path = parentPath(StringRef{path}).toString(); list_request->list_request_type = list_request_type; From 2de659715ba28ddf9b0ee21f474abec1516301c8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 29 Jun 2022 10:21:37 +0000 Subject: [PATCH 090/627] Don't rollback SessionID request --- src/Coordination/KeeperStateMachine.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 368b23f34d2..1b399e8cc92 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -289,6 +289,9 @@ void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data) if (!request_for_session.zxid) request_for_session.zxid = log_idx; + if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) + return; + std::lock_guard lock(storage_and_responses_lock); storage->rollbackRequest(request_for_session.zxid); } From e0acb6e33762629f30061fce72d765b17b19d337 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Wed, 29 Jun 2022 15:23:53 +0200 Subject: [PATCH 091/627] Add check for empty proccessors in AggregatingTransform::expandPipeline --- src/Processors/Transforms/AggregatingTransform.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index f8332742978..7f5896f5e97 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -501,6 +501,8 @@ void AggregatingTransform::work() Processors AggregatingTransform::expandPipeline() { + if (processors.empty()) + throw Exception("Can not expandPipeline in AggregatingTransform. This is a bug.", ErrorCodes::LOGICAL_ERROR); auto & out = processors.back()->getOutputs().front(); inputs.emplace_back(out.getHeader(), this); connect(out, inputs.back()); From 86076839376e1a2ec776c2ae66bd53ccb40ecb7c Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 29 Jun 2022 16:49:20 -0300 Subject: [PATCH 092/627] Update mysql.md --- .../table-engines/integrations/mysql.md | 40 +++++++++++++++++-- 1 file changed, 36 insertions(+), 4 deletions(-) diff --git a/docs/ru/engines/table-engines/integrations/mysql.md b/docs/ru/engines/table-engines/integrations/mysql.md index 801f2e1433a..df9743a386d 100644 --- a/docs/ru/engines/table-engines/integrations/mysql.md +++ b/docs/ru/engines/table-engines/integrations/mysql.md @@ -17,10 +17,12 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ... ) ENGINE = MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause']) SETTINGS - [connection_pool_size=16, ] - [connection_max_tries=3, ] - [connection_wait_timeout=5, ] /* 0 -- не ждать */ - [connection_auto_close=true ] + [ connection_pool_size=16, ] + [ connection_max_tries=3, ] + [ connection_wait_timeout=5, ] + [ connection_auto_close=true, ] + [ connect_timeout=10, ] + [ read_write_timeout=300 ] ; ``` @@ -144,6 +146,36 @@ SELECT * FROM mysql_table Значение по умолчанию: `16`. +### connection_wait_timeout {#connection-wait-timeout} + +Задает таймаут (в секундах) ожидания свободного подключения (в случае, если уже есть активные подключения connection_pool_size), 0 - не ждать. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `5`. + +### connect_timeout {#connect-timeout} + +Задает таймаут ожидания подключения (в секундах). + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `10`. + +### read_write_timeout {#read-write-timeout} + +Задает таймаут ожидания ввода/вывода (в секундах). + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `300`. + ## См. также {#see-also} - [Табличная функция mysql](../../../engines/table-engines/integrations/mysql.md) From 394a15a792137d65552d9e2da3e4b413f2e76ec8 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 29 Jun 2022 16:52:32 -0300 Subject: [PATCH 093/627] Update mysql.md --- .../table-engines/integrations/mysql.md | 40 +++++++++++++++++-- 1 file changed, 36 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/mysql.md b/docs/en/engines/table-engines/integrations/mysql.md index 54d71daa88a..5cd43d8a6ba 100644 --- a/docs/en/engines/table-engines/integrations/mysql.md +++ b/docs/en/engines/table-engines/integrations/mysql.md @@ -17,10 +17,12 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ... ) ENGINE = MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause']) SETTINGS - [connection_pool_size=16, ] - [connection_max_tries=3, ] - [connection_wait_timeout=5, ] /* 0 -- do not wait */ - [connection_auto_close=true ] + [ connection_pool_size=16, ] + [ connection_max_tries=3, ] + [ connection_wait_timeout=5, ] + [ connection_auto_close=true, ] + [ connect_timeout=10, ] + [ read_write_timeout=300 ] ; ``` @@ -144,6 +146,36 @@ Possible values: Default value: `16`. +### connection_wait_timeout {#connection-wait-timeout} + +Timeout (in seconds) for waiting for free connection (in case of there is already connection_pool_size active connections), 0 - do not wait. + +Possible values: + +- Positive integer. + +Default value: `5`. + +### connect_timeout {#connect-timeout} + +Connect timeout (in seconds). + +Possible values: + +- Positive integer. + +Default value: `10`. + +### read_write_timeout {#read-write-timeout} + +Read/write timeout (in seconds). + +Possible values: + +- Positive integer. + +Default value: `300`. + ## See Also {#see-also} - [The mysql table function](../../../sql-reference/table-functions/mysql.md) From 47f98cabcfc7c0ffb8b6a63f66608420ee21bfd3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 29 Jun 2022 17:31:27 -0300 Subject: [PATCH 094/627] Update mysql.md --- docs/ru/engines/table-engines/integrations/mysql.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/engines/table-engines/integrations/mysql.md b/docs/ru/engines/table-engines/integrations/mysql.md index df9743a386d..7380d926f90 100644 --- a/docs/ru/engines/table-engines/integrations/mysql.md +++ b/docs/ru/engines/table-engines/integrations/mysql.md @@ -178,5 +178,5 @@ SELECT * FROM mysql_table ## См. также {#see-also} -- [Табличная функция mysql](../../../engines/table-engines/integrations/mysql.md) -- [Использование MySQL в качестве источника для внешнего словаря](../../../engines/table-engines/integrations/mysql.md#dicts-external_dicts_dict_sources-mysql) +- [Табличная функция mysql](../../../sql-reference/table-functions/mysql.md) +- [Использование MySQL в качестве источника для внешнего словаря](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-mysql) From 3949480ed38bbeb4e98178248753cc3e74ac5881 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 30 Jun 2022 11:38:05 +0200 Subject: [PATCH 095/627] Fix exception messages in clickhouse su Fix exception messages in --- programs/su/clickhouse-su.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/su/clickhouse-su.cpp b/programs/su/clickhouse-su.cpp index 9aa41085094..71027ddd385 100644 --- a/programs/su/clickhouse-su.cpp +++ b/programs/su/clickhouse-su.cpp @@ -59,7 +59,7 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid) throwFromErrno(fmt::format("Cannot do 'getgrnam_r' to obtain gid from group name ({})", arg_gid), ErrorCodes::SYSTEM_ERROR); if (!result) - throw Exception("Group {} is not found in the system", ErrorCodes::BAD_ARGUMENTS); + throw Exception(fmt::format("Group {} is not found in the system", arg_gid), ErrorCodes::BAD_ARGUMENTS); gid = entry.gr_gid; } @@ -84,7 +84,7 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid) throwFromErrno(fmt::format("Cannot do 'getpwnam_r' to obtain uid from user name ({})", arg_uid), ErrorCodes::SYSTEM_ERROR); if (!result) - throw Exception("User {} is not found in the system", ErrorCodes::BAD_ARGUMENTS); + throw Exception(fmt::format("User {} is not found in the system", arg_uid), ErrorCodes::BAD_ARGUMENTS); uid = entry.pw_uid; } From c221e982c87fac52a179054b637f09c823e3bcc6 Mon Sep 17 00:00:00 2001 From: Dmitry Pavlov Date: Thu, 30 Jun 2022 12:05:14 +0200 Subject: [PATCH 096/627] Added Greenplum benchmark --- benchmark/compatible/greenplum/benchmark.sh | 62 ++++- benchmark/compatible/greenplum/create.sql | 116 ++++++++++ benchmark/compatible/greenplum/log.txt | 215 ++++++++++++++++++ benchmark/compatible/greenplum/queries.sql | 43 ++++ .../greenplum/results/c6a.4xlarge.txt | 46 ++++ benchmark/compatible/greenplum/run.sh | 15 ++ 6 files changed, 494 insertions(+), 3 deletions(-) create mode 100644 benchmark/compatible/greenplum/create.sql create mode 100644 benchmark/compatible/greenplum/log.txt create mode 100644 benchmark/compatible/greenplum/queries.sql create mode 100644 benchmark/compatible/greenplum/results/c6a.4xlarge.txt create mode 100644 benchmark/compatible/greenplum/run.sh diff --git a/benchmark/compatible/greenplum/benchmark.sh b/benchmark/compatible/greenplum/benchmark.sh index a83363fbcfa..a1802ee223b 100644 --- a/benchmark/compatible/greenplum/benchmark.sh +++ b/benchmark/compatible/greenplum/benchmark.sh @@ -3,15 +3,71 @@ # NOTE: it requires Ubuntu 18.04 # Greenplum does not install on any newer system. +echo "This script must be run from gpadmin user. Press enter to continue." +read sudo apt update sudo apt install -y software-properties-common sudo add-apt-repository ppa:greenplum/db sudo apt update sudo apt install greenplum-db-6 +sudo rm -rf /gpmaster /gpdata* +ssh-keygen -t rsa -b 4096 +cat /home/gpadmin/.ssh/id_rsa.pub >> /home/gpadmin/.ssh/authorized_keys +mod 600 ~/.ssh/authorized_keys +sudo echo "# kernel.shmall = _PHYS_PAGES / 2 # See Shared Memory Pages +kernel.shmall = 197951838 +# kernel.shmmax = kernel.shmall * PAGE_SIZE +kernel.shmmax = 810810728448 +kernel.shmmni = 4096 +vm.overcommit_memory = 2 # See Segment Host Memory +vm.overcommit_ratio = 95 # See Segment Host Memory + +net.ipv4.ip_local_port_range = 10000 65535 # See Port Settings +kernel.sem = 500 2048000 200 4096 +kernel.sysrq = 1 +kernel.core_uses_pid = 1 +kernel.msgmnb = 65536 +kernel.msgmax = 65536 +kernel.msgmni = 2048 +net.ipv4.tcp_syncookies = 1 +net.ipv4.conf.default.accept_source_route = 0 +net.ipv4.tcp_max_syn_backlog = 4096 +net.ipv4.conf.all.arp_filter = 1 +net.core.netdev_max_backlog = 10000 +net.core.rmem_max = 2097152 +net.core.wmem_max = 2097152 +vm.swappiness = 10 +vm.zone_reclaim_mode = 0 +vm.dirty_expire_centisecs = 500 +vm.dirty_writeback_centisecs = 100 +vm.dirty_background_ratio = 0 # See System Memory +vm.dirty_ratio = 0 +vm.dirty_background_bytes = 1610612736 +vm.dirty_bytes = 4294967296" |sudo tee -a /etc/sysctl.conf +sudo sysctl -p + +echo "* soft nofile 524288 +* hard nofile 524288 +* soft nproc 131072 +* hard nproc 131072" |sudo tee -a /etc/security/limits.conf +echo "RemoveIPC=no" |sudo tee -a /etc/systemd/logind.conf +echo "Now you need to reboot the machine. Press Enter if you already rebooted, or reboot now and run the script once again" +read source /opt/greenplum-db-*.0/greenplum_path.sh cp $GPHOME/docs/cli_help/gpconfigs/gpinitsystem_singlenode . echo localhost > ./hostlist_singlenode sed -i "s/MASTER_HOSTNAME=[a-z_]*/MASTER_HOSTNAME=$(hostname)/" gpinitsystem_singlenode -sudo mkdir /gpmaster -sudo chmod 777 /gpmaster -gpinitsystem -c gpinitsystem_singlenode +sed -i "s@declare -a DATA_DIRECTORY=(/gpdata1 /gpdata2)@declare -a DATA_DIRECTORY=(/gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14)@" gpinitsystem_singlenode +sudo mkdir /gpmaster /gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14 +sudo chmod 777 /gpmaster /gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14 +gpinitsystem -ac gpinitsystem_singlenode +export MASTER_DATA_DIRECTORY=/gpmaster/gpsne-1/ +#wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +#gzip -d hits.tsv.gz +chmod 777 ~ hits.tsv +psql -d postgres -f create.sql +nohup gpfdist & +time psql -d postgres -t -c '\timing' -c "insert into hits select * from hits_ext;" +du -sh /gpdata* +./run.sh 2>&1 | tee log.txt +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' |awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/greenplum/create.sql b/benchmark/compatible/greenplum/create.sql new file mode 100644 index 00000000000..ddaf587d0f9 --- /dev/null +++ b/benchmark/compatible/greenplum/create.sql @@ -0,0 +1,116 @@ +drop table if exists hits; +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +) +with (appendoptimized=true,orientation=column,compresstype=zstd) +DISTRIBUTED RANDOMLY; +CREATE INDEX hits_idx on hits using btree (CounterID, EventDate, UserID, EventTime, WatchID); +drop external table if exists hits_ext; +CREATE EXTERNAL TABLE hits_ext (like hits) +LOCATION ('gpfdist://localhost:8080/hits.tsv') +FORMAT 'TEXT'; diff --git a/benchmark/compatible/greenplum/log.txt b/benchmark/compatible/greenplum/log.txt new file mode 100644 index 00000000000..91eeeb8447e --- /dev/null +++ b/benchmark/compatible/greenplum/log.txt @@ -0,0 +1,215 @@ +3 +SELECT COUNT(*) FROM hits; +Time: 2200.088 ms +Time: 1458.828 ms +Time: 1135.728 ms +3 +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +Time: 1286.610 ms +Time: 1078.970 ms +Time: 1018.564 ms +3 +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +Time: 2258.198 ms +Time: 2421.929 ms +Time: 2596.925 ms +3 +SELECT AVG(UserID) FROM hits; +Time: 1893.709 ms +Time: 1924.597 ms +Time: 1738.663 ms +3 +SELECT COUNT(DISTINCT UserID) FROM hits; +Time: 9544.705 ms +Time: 9719.561 ms +Time: 10035.660 ms +3 +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +Time: 4559.571 ms +Time: 4101.119 ms +Time: 4704.585 ms +3 +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +Time: 1651.671 ms +Time: 1897.668 ms +Time: 1970.511 ms +3 +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +Time: 1025.366 ms +Time: 999.323 ms +Time: 1005.235 ms +3 +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +Time: 7897.922 ms +Time: 7757.179 ms +Time: 8012.193 ms +3 +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +Time: 11377.889 ms +Time: 11600.291 ms +Time: 11255.236 ms +3 +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +Time: 2119.548 ms +Time: 2050.025 ms +Time: 1955.304 ms +3 +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +Time: 2019.971 ms +Time: 1979.324 ms +Time: 1982.835 ms +3 +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +Time: 2886.663 ms +Time: 2928.279 ms +Time: 2968.219 ms +3 +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +Time: 4062.722 ms +Time: 4017.708 ms +Time: 4030.113 ms +3 +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +Time: 3463.033 ms +Time: 3247.791 ms +Time: 3064.752 ms +3 +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +Time: 8437.169 ms +Time: 8186.413 ms +Time: 7992.017 ms +3 +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +Time: 9021.759 ms +Time: 8915.718 ms +Time: 8927.481 ms +3 +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +Time: 8669.821 ms +Time: 8393.315 ms +Time: 8624.970 ms +3 +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +Time: 15132.099 ms +Time: 14950.178 ms +Time: 15234.422 ms +3 +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +Time: 1008.775 ms +Time: 934.589 ms +Time: 975.342 ms +3 +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +Time: 11058.183 ms +Time: 3258.070 ms +Time: 3176.875 ms +3 +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +Time: 12263.435 ms +Time: 3518.802 ms +Time: 3937.050 ms +3 +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +Time: 24152.214 ms +Time: 4863.278 ms +Time: 4887.644 ms +3 +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +Time: 67151.698 ms +Time: 20400.002 ms +Time: 20261.041 ms +3 +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +Time: 2579.895 ms +Time: 1661.144 ms +Time: 1904.318 ms +3 +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +Time: 1728.308 ms +Time: 1895.359 ms +Time: 1251.501 ms +3 +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +Time: 2441.544 ms +Time: 1465.950 ms +Time: 1882.380 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; +Time: 11117.452 ms +Time: 6095.514 ms +Time: 6022.919 ms +3 +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www.)?([^/]+)/.*$', '1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +Time: 83040.984 ms +Time: 82978.396 ms +Time: 82867.763 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; +Time: 82544.095 ms +Time: 82542.673 ms +Time: 83983.646 ms +3 +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +Time: 3096.198 ms +Time: 3222.251 ms +Time: 3194.314 ms +3 +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +Time: 5775.759 ms +Time: 4059.735 ms +Time: 4003.655 ms +3 +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +Time: 43682.613 ms +Time: 40437.523 ms +Time: 40107.640 ms +3 +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +Time: 18097.819 ms +Time: 13274.130 ms +Time: 12889.385 ms +3 +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +Time: 17318.672 ms +Time: 13541.070 ms +Time: 13592.715 ms +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; +Time: 9724.053 ms +Time: 9900.294 ms +Time: 10017.686 ms +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 IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +Time: 294.344 ms +Time: 169.606 ms +Time: 173.804 ms +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 IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +Time: 162.524 ms +Time: 117.489 ms +Time: 115.532 ms +3 +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +Time: 145.205 ms +Time: 98.342 ms +Time: 97.275 ms +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 IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +Time: 334.809 ms +Time: 275.365 ms +Time: 265.053 ms +3 +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +Time: 154.522 ms +Time: 107.654 ms +Time: 105.290 ms +3 +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +Time: 158.957 ms +Time: 117.284 ms +Time: 119.068 ms +3 +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; +Time: 193.756 ms +Time: 144.787 ms +Time: 145.485 ms diff --git a/benchmark/compatible/greenplum/queries.sql b/benchmark/compatible/greenplum/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/greenplum/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/greenplum/results/c6a.4xlarge.txt b/benchmark/compatible/greenplum/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..5fca032be36 --- /dev/null +++ b/benchmark/compatible/greenplum/results/c6a.4xlarge.txt @@ -0,0 +1,46 @@ +Load time: 1080 sec +Data size: 42 Gb + +[2.20009,1.45883,1.13573], +[1.28661,1.07897,1.01856], +[2.2582,2.42193,2.59693], +[1.89371,1.9246,1.73866], +[9.54471,9.71956,10.0357], +[4.55957,4.10112,4.70458], +[1.65167,1.89767,1.97051], +[1.02537,0.999323,1.00524], +[7.89792,7.75718,8.01219], +[11.3779,11.6003,11.2552], +[2.11955,2.05003,1.9553], +[2.01997,1.97932,1.98284], +[2.88666,2.92828,2.96822], +[4.06272,4.01771,4.03011], +[3.46303,3.24779,3.06475], +[8.43717,8.18641,7.99202], +[9.02176,8.91572,8.92748], +[8.66982,8.39332,8.62497], +[15.1321,14.9502,15.2344], +[1.00877,0.934589,0.975342], +[11.0582,3.25807,3.17687], +[12.2634,3.5188,3.93705], +[24.1522,4.86328,4.88764], +[67.1517,20.4,20.261], +[2.5799,1.66114,1.90432], +[1.72831,1.89536,1.2515], +[2.44154,1.46595,1.88238], +[11.1175,6.09551,6.02292], +[83.041,82.9784,82.8678], +[82.5441,82.5427,83.9836], +[3.0962,3.22225,3.19431], +[5.77576,4.05973,4.00366], +[43.6826,40.4375,40.1076], +[18.0978,13.2741,12.8894], +[17.3187,13.5411,13.5927], +[9.72405,9.90029,10.0177], +[0.294344,0.169606,0.173804], +[0.162524,0.117489,0.115532], +[0.145205,0.098342,0.097275], +[0.334809,0.275365,0.265053], +[0.154522,0.107654,0.10529], +[0.158957,0.117284,0.119068], +[0.193756,0.144787,0.145485], diff --git a/benchmark/compatible/greenplum/run.sh b/benchmark/compatible/greenplum/run.sh new file mode 100644 index 00000000000..1eb622de977 --- /dev/null +++ b/benchmark/compatible/greenplum/run.sh @@ -0,0 +1,15 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + echo "$query"; + for i in $(seq 1 $TRIES); do + echo '\timing' > /tmp/query_temp.sql + echo "$query" >> /tmp/query_temp.sql + psql -d postgres -t -f /tmp/query_temp.sql | grep 'Time' + done; +done; From b2cc6787ee9caa9641031545c7eac17c6cf3a9ec Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 30 Jun 2022 12:44:30 +0200 Subject: [PATCH 097/627] Update programs/su/clickhouse-su.cpp --- programs/su/clickhouse-su.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/su/clickhouse-su.cpp b/programs/su/clickhouse-su.cpp index 71027ddd385..87896ea249f 100644 --- a/programs/su/clickhouse-su.cpp +++ b/programs/su/clickhouse-su.cpp @@ -59,7 +59,7 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid) throwFromErrno(fmt::format("Cannot do 'getgrnam_r' to obtain gid from group name ({})", arg_gid), ErrorCodes::SYSTEM_ERROR); if (!result) - throw Exception(fmt::format("Group {} is not found in the system", arg_gid), ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Group {} is not found in the system", arg_gid); gid = entry.gr_gid; } From e66114984c8eda4963ad576defc70548c3c60995 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 30 Jun 2022 12:44:59 +0200 Subject: [PATCH 098/627] Update programs/su/clickhouse-su.cpp --- programs/su/clickhouse-su.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/su/clickhouse-su.cpp b/programs/su/clickhouse-su.cpp index 87896ea249f..0979abf353d 100644 --- a/programs/su/clickhouse-su.cpp +++ b/programs/su/clickhouse-su.cpp @@ -84,7 +84,7 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid) throwFromErrno(fmt::format("Cannot do 'getpwnam_r' to obtain uid from user name ({})", arg_uid), ErrorCodes::SYSTEM_ERROR); if (!result) - throw Exception(fmt::format("User {} is not found in the system", arg_uid), ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "User {} is not found in the system", arg_uid); uid = entry.pw_uid; } From 69b0508d4765e7becf56dc9590fd2c5876f8aeec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 1 Jul 2022 00:15:17 +0200 Subject: [PATCH 099/627] Adjustments --- benchmark/compatible/druid/benchmark.sh | 16 ++++++++++------ benchmark/compatible/druid/run.sh | 7 +------ 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/benchmark/compatible/druid/benchmark.sh b/benchmark/compatible/druid/benchmark.sh index b147eef7c1c..54ac6d76137 100755 --- a/benchmark/compatible/druid/benchmark.sh +++ b/benchmark/compatible/druid/benchmark.sh @@ -1,17 +1,20 @@ #!/bin/bash -# If you have no java installed: -# sudo apt-get update -# sudo apt install openjdk-8-jdk -y -# sudo update-alternatives --config java + +sudo apt-get update +sudo apt install openjdk-8-jdk -y +sudo update-alternatives --config java + +# Install VERSION=0.23.0 -# Install wget -O"apache-druid-${VERSION}-bin.tar.gz" "https://dlcdn.apache.org/druid/${VERSION}/apache-druid-${VERSION}-bin.tar.gz" tar xf apache-druid-${VERSION}-bin.tar.gz ./apache-druid-${VERSION}/bin/verify-java + # Have to increase indexer memory limit sed -i 's MaxDirectMemorySize=1g MaxDirectMemorySize=5g g' apache-druid-$VERSION/conf/druid/single-server/medium/middleManager/runtime.properties + # Druid launcher does not start Druid as a daemon. Run it in background ./apache-druid-${VERSION}/bin/start-single-server-medium & @@ -20,10 +23,11 @@ sed -i 's MaxDirectMemorySize=1g MaxDirectMemorySize=5g g' apache-druid-$VERSION wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz split --additional-suffix .tsv --verbose -n l/10 hits.tsv input + # Running 10 tasks one by one to make it work in parallel ./apache-druid-${VERSION}/bin/post-index-task --file ingest.json --url http://localhost:8081 -# Run the queries +# Run the queries ./run.sh # stop Druid services diff --git a/benchmark/compatible/druid/run.sh b/benchmark/compatible/druid/run.sh index edf9c199d95..3df4d91aa99 100755 --- a/benchmark/compatible/druid/run.sh +++ b/benchmark/compatible/druid/run.sh @@ -1,21 +1,16 @@ #!/bin/bash TRIES=3 -QUERY_NUM=1 cat queries.sql | while read query; do sync echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null echo -n "[" for i in $(seq 1 $TRIES); do - echo "{\"query\":\"$query\"}"| sed -e 's EventTime __time g' | tr -d ';' > query.json + echo "{\"query\":\"$query\"}"| sed -e 's EventTime __time g' | tr -d ';' > query.json RES=$(curl -o /dev/null -s -w '%{time_total}\n' -XPOST -H'Content-Type: application/json' http://localhost:8888/druid/v2/sql/ -d @query.json) [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" [[ "$i" != $TRIES ]] && echo -n ", " - - echo "${QUERY_NUM},${i},${RES}" >> result.csv done echo "]," - - QUERY_NUM=$((QUERY_NUM + 1)) done From 03a5ce542da70f2cd07b81efd4759caa6f708292 Mon Sep 17 00:00:00 2001 From: lingpeng0314 Date: Fri, 1 Jul 2022 11:13:33 +0800 Subject: [PATCH 100/627] Remove Exception in getKeeperDispatcher - to handle the nil case by caller --- programs/keeper/TinyContext.cpp | 7 ++----- src/Interpreters/AsynchronousMetrics.cpp | 19 ++++++++----------- src/Interpreters/Context.cpp | 6 +----- 3 files changed, 11 insertions(+), 21 deletions(-) diff --git a/programs/keeper/TinyContext.cpp b/programs/keeper/TinyContext.cpp index 20b24893ebd..5e1de81c7ca 100644 --- a/programs/keeper/TinyContext.cpp +++ b/programs/keeper/TinyContext.cpp @@ -8,7 +8,7 @@ namespace DB namespace ErrorCodes { - extern const int KEEPER_EXCEPTION; + extern const int LOGICAL_ERROR; } void TinyContext::setConfig(const ConfigurationPtr & config_) @@ -31,7 +31,7 @@ void TinyContext::initializeKeeperDispatcher([[maybe_unused]] bool start_async) std::lock_guard lock(keeper_dispatcher_mutex); if (keeper_dispatcher) - throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Trying to initialize Keeper multiple times"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); if (config_ref.has("keeper_server")) { @@ -43,9 +43,6 @@ void TinyContext::initializeKeeperDispatcher([[maybe_unused]] bool start_async) std::shared_ptr TinyContext::getKeeperDispatcher() const { std::lock_guard lock(keeper_dispatcher_mutex); - if (!keeper_dispatcher) - throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Keeper must be initialized before requests"); - return keeper_dispatcher; } diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 4b3c37129c2..9a41ae8dd0b 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1470,8 +1470,9 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } #if USE_NURAFT { - try { - auto keeper_dispatcher = getContext()->getKeeperDispatcher(); + auto keeper_dispatcher = getContext()->getKeeperDispatcher(); + if (keeper_dispatcher) + { size_t is_leader = 0; size_t is_follower = 0; size_t is_observer = 0; @@ -1492,13 +1493,13 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti size_t snapshot_dir_size = 0; size_t log_dir_size = 0; - if (keeper_dispatcher && keeper_dispatcher->isServerActive()) + if (keeper_dispatcher->isServerActive()) { auto keeper_4LW_info = keeper_dispatcher -> getKeeper4LWInfo(); - is_standalone = static_cast(keeper_4LW_info.is_standalone; - is_leader = static_cast(keeper_4LW_info.is_leader; - is_observer = static_cast(keeper_4LW_info.is_observer; - is_follower = static_cast(keeper_4LW_info.is_follower; + is_standalone = static_cast(keeper_4LW_info.is_standalone); + is_leader = static_cast(keeper_4LW_info.is_leader); + is_observer = static_cast(keeper_4LW_info.is_observer); + is_follower = static_cast(keeper_4LW_info.is_follower); zxid = keeper_4LW_info.last_zxid; const auto & state_machine = keeper_dispatcher->getStateMachine(); @@ -1549,10 +1550,6 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti new_values["KeeperSnapshotDirSize"] = snapshot_dir_size; new_values["KeeperLogDirSize"] = log_dir_size; } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } } #endif diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8b468c47b5c..7658b350c6c 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -135,7 +135,6 @@ namespace ErrorCodes extern const int INVALID_SETTING_VALUE; extern const int UNKNOWN_READ_METHOD; extern const int NOT_IMPLEMENTED; - extern const int KEEPER_EXCEPTION; } @@ -2069,7 +2068,7 @@ void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) cons std::lock_guard lock(shared->keeper_dispatcher_mutex); if (shared->keeper_dispatcher) - throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Trying to initialize Keeper multiple times"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); const auto & config = getConfigRef(); if (config.has("keeper_server")) @@ -2097,9 +2096,6 @@ void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) cons std::shared_ptr & Context::getKeeperDispatcher() const { std::lock_guard lock(shared->keeper_dispatcher_mutex); - if (!shared->keeper_dispatcher) - throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Keeper must be initialized before requests"); - return shared->keeper_dispatcher; } #endif From d73f9758492e9ab2d9090f6d392a14f653302d7d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 12 May 2022 21:57:12 +0800 Subject: [PATCH 101/627] Fix intermediate header in aggregate projection --- src/Storages/MergeTree/MergeTreeData.cpp | 15 ++------------- .../01710_minmax_count_projection.reference | 1 + .../0_stateless/01710_minmax_count_projection.sql | 3 +++ 3 files changed, 6 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2c4dcfa05ee..e7f87c59d19 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5438,7 +5438,7 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg if (analysis_result.before_where) { candidate.where_column_name = analysis_result.where_column_name; - candidate.remove_where_filter = analysis_result.remove_where_filter; + candidate.remove_where_filter = !required_columns.contains(analysis_result.where_column_name); candidate.before_where = analysis_result.before_where->clone(); auto new_required_columns = candidate.before_where->foldActionsByProjection( @@ -5571,20 +5571,9 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg candidate.before_aggregation->reorderAggregationKeysForProjection(key_name_pos_map); candidate.before_aggregation->addAggregatesViaProjection(aggregates); - // minmax_count_projections only have aggregation actions - if (minmax_count_projection) - candidate.required_columns = {required_columns.begin(), required_columns.end()}; - if (rewrite_before_where(candidate, projection, required_columns, sample_block_for_keys, aggregates)) { - if (minmax_count_projection) - { - candidate.before_where = nullptr; - candidate.prewhere_info = nullptr; - } - else - candidate.required_columns = {required_columns.begin(), required_columns.end()}; - + candidate.required_columns = {required_columns.begin(), required_columns.end()}; for (const auto & aggregate : aggregates) candidate.required_columns.push_back(aggregate.name); candidates.push_back(std::move(candidate)); diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index 259d320a38a..3e24cac4f6d 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -3,6 +3,7 @@ 1 9999 5000 0 9998 5000 1 +0 9998 5000 0 1 0 9999 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index a6c04725583..4aea3cfd29e 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -15,6 +15,9 @@ select min(i), max(i), count() from d where _partition_value.1 = 10 group by _pa -- fuzz crash select min(i) from d where 1 = _partition_value.1; +-- fuzz crash https://github.com/ClickHouse/ClickHouse/issues/37151 +SELECT min(i), max(i), count() FROM d WHERE (_partition_value.1) = 0 GROUP BY ignore(bitTest(ignore(NULL), 65535), NULL, (_partition_value.1) = 7, '10.25', bitTest(NULL, -9223372036854775808), NULL, ignore(ignore(-2147483647, NULL)), 1024), _partition_id ORDER BY _partition_id ASC NULLS FIRST; + drop table d; drop table if exists has_final_mark; From 84a407f381b1846ee7ae5c7f961a1a4131914652 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 1 Jul 2022 18:24:24 +0800 Subject: [PATCH 102/627] Fix toHour monotonicity --- .../FunctionDateOrDateTimeToSomething.h | 18 ++++++++++-------- .../02346_to_hour_monotonicity_fix.reference | 1 + .../02346_to_hour_monotonicity_fix.sql | 8 ++++++++ 3 files changed, 19 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02346_to_hour_monotonicity_fix.reference create mode 100644 tests/queries/0_stateless/02346_to_hour_monotonicity_fix.sql diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index 5269eecea37..d734c7f87c1 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -148,8 +148,10 @@ public: const IFunction::Monotonicity is_monotonic = { .is_monotonic = true }; const IFunction::Monotonicity is_not_monotonic; - /// This method is called only if the function has one argument. Therefore, we do not care about the non-local time zone. - const DateLUTImpl & date_lut = DateLUT::instance(); + const DateLUTImpl * date_lut = &DateLUT::instance(); + if (const auto * timezone = dynamic_cast(&type)) + date_lut = &timezone->getTimeZone(); + if (left.isNull() || right.isNull()) return is_not_monotonic; @@ -157,20 +159,20 @@ public: if (checkAndGetDataType(&type)) { - return Transform::FactorTransform::execute(UInt16(left.get()), date_lut) - == Transform::FactorTransform::execute(UInt16(right.get()), date_lut) + return Transform::FactorTransform::execute(UInt16(left.get()), *date_lut) + == Transform::FactorTransform::execute(UInt16(right.get()), *date_lut) ? is_monotonic : is_not_monotonic; } else if (checkAndGetDataType(&type)) { - return Transform::FactorTransform::execute(Int32(left.get()), date_lut) - == Transform::FactorTransform::execute(Int32(right.get()), date_lut) + return Transform::FactorTransform::execute(Int32(left.get()), *date_lut) + == Transform::FactorTransform::execute(Int32(right.get()), *date_lut) ? is_monotonic : is_not_monotonic; } else { - return Transform::FactorTransform::execute(UInt32(left.get()), date_lut) - == Transform::FactorTransform::execute(UInt32(right.get()), date_lut) + return Transform::FactorTransform::execute(UInt32(left.get()), *date_lut) + == Transform::FactorTransform::execute(UInt32(right.get()), *date_lut) ? is_monotonic : is_not_monotonic; } } diff --git a/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.reference b/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.reference new file mode 100644 index 00000000000..5d49984b71b --- /dev/null +++ b/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.reference @@ -0,0 +1 @@ +5 8 42 diff --git a/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.sql b/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.sql new file mode 100644 index 00000000000..cad7df63eb5 --- /dev/null +++ b/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.sql @@ -0,0 +1,8 @@ +drop table if exists test_tz_hour; + +create table test_tz_hour(t DateTime, x String) engine MergeTree partition by toYYYYMMDD(t) order by x; +insert into test_tz_hour select toDateTime('2021-06-01 00:00:00') + number * 600, 'x' from numbers(1e3); + +select toHour(toTimeZone(t, 'UTC')) as toHour_UTC, toHour(toTimeZone(t, 'Asia/Jerusalem')) as toHour_Israel, count() from test_tz_hour where toHour_Israel = 8 group by toHour_UTC, toHour_Israel; + +drop table test_tz_hour; From 202008e80bebaed8cb425d804705246e14b35081 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 20 Jun 2022 09:53:35 +0000 Subject: [PATCH 103/627] Add asserts --- src/Interpreters/OpenTelemetrySpanLog.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/OpenTelemetrySpanLog.cpp b/src/Interpreters/OpenTelemetrySpanLog.cpp index fe204b34876..e1199fa03b1 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.cpp +++ b/src/Interpreters/OpenTelemetrySpanLog.cpp @@ -160,6 +160,7 @@ void OpenTelemetrySpanHolder::addAttribute(const std::string& name, UInt64 value this->attribute_names.push_back(name); this->attribute_values.push_back(std::to_string(value)); + assert(this->attribute_names.size() == this->attribute_values.size()); } void OpenTelemetrySpanHolder::addAttribute(const std::string& name, const std::string& value) @@ -169,6 +170,7 @@ void OpenTelemetrySpanHolder::addAttribute(const std::string& name, const std::s this->attribute_names.push_back(name); this->attribute_values.push_back(value); + assert(this->attribute_names.size() == this->attribute_values.size()); } void OpenTelemetrySpanHolder::addAttribute(const Exception & e) @@ -178,6 +180,7 @@ void OpenTelemetrySpanHolder::addAttribute(const Exception & e) this->attribute_names.push_back("clickhouse.exception"); this->attribute_values.push_back(getExceptionMessage(e, false)); + assert(this->attribute_names.size() == this->attribute_values.size()); } void OpenTelemetrySpanHolder::addAttribute(std::exception_ptr e) @@ -187,6 +190,7 @@ void OpenTelemetrySpanHolder::addAttribute(std::exception_ptr e) this->attribute_names.push_back("clickhouse.exception"); this->attribute_values.push_back(getExceptionMessage(e, false)); + assert(this->attribute_names.size() == this->attribute_values.size()); } bool OpenTelemetryTraceContext::parseTraceparentHeader(const std::string & traceparent, From 3b5332d15e5c9200f23a49c84e119e7130270088 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 1 Jul 2022 18:59:07 +0500 Subject: [PATCH 104/627] Revert "Revert "Non Negative Derivative window function"" This reverts commit dea3b5bfcecfa8ce778bd61fab86e448ad648fec. --- src/Common/IntervalKind.cpp | 27 ++++- src/Common/IntervalKind.h | 5 +- src/Processors/Transforms/WindowTransform.cpp | 112 ++++++++++++++++++ .../02232_non_negative_derivative.reference | 64 ++++++++++ .../02232_non_negative_derivative.sql | 63 ++++++++++ 5 files changed, 266 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02232_non_negative_derivative.reference create mode 100644 tests/queries/0_stateless/02232_non_negative_derivative.sql diff --git a/src/Common/IntervalKind.cpp b/src/Common/IntervalKind.cpp index d3cd4eeff8f..75c2a83e9fb 100644 --- a/src/Common/IntervalKind.cpp +++ b/src/Common/IntervalKind.cpp @@ -9,13 +9,13 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -Int32 IntervalKind::toAvgSeconds() const +Float64 IntervalKind::toAvgSeconds() const { switch (kind) { - case IntervalKind::Nanosecond: - case IntervalKind::Microsecond: - case IntervalKind::Millisecond: return 0; /// fractional parts of seconds have 0 seconds + case IntervalKind::Nanosecond: return 0.000000001; + case IntervalKind::Microsecond: return 0.000001; + case IntervalKind::Millisecond: return 0.001; case IntervalKind::Second: return 1; case IntervalKind::Minute: return 60; case IntervalKind::Hour: return 3600; @@ -28,6 +28,25 @@ Int32 IntervalKind::toAvgSeconds() const __builtin_unreachable(); } +bool IntervalKind::isFixedLength() const +{ + switch (kind) + { + case IntervalKind::Nanosecond: + case IntervalKind::Microsecond: + case IntervalKind::Millisecond: + case IntervalKind::Second: + case IntervalKind::Minute: + case IntervalKind::Hour: + case IntervalKind::Day: + case IntervalKind::Week: return true; + case IntervalKind::Month: + case IntervalKind::Quarter: + case IntervalKind::Year: return false; + } + __builtin_unreachable(); +} + IntervalKind IntervalKind::fromAvgSeconds(Int64 num_seconds) { if (num_seconds) diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index d5f2b5672cd..65c14515e34 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -31,12 +31,15 @@ struct IntervalKind /// Returns number of seconds in one interval. /// For `Month`, `Quarter` and `Year` the function returns an average number of seconds. - Int32 toAvgSeconds() const; + Float64 toAvgSeconds() const; /// Chooses an interval kind based on number of seconds. /// For example, `IntervalKind::fromAvgSeconds(3600)` returns `IntervalKind::Hour`. static IntervalKind fromAvgSeconds(Int64 num_seconds); + /// Returns whether IntervalKind has a fixed number of seconds (e.g. Day) or non-fixed(e.g. Month) + bool isFixedLength() const; + /// Returns an uppercased version of what `toString()` returns. const char * toKeyword() const; diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 3eb0f62cb01..09805696472 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -27,6 +28,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int NOT_IMPLEMENTED; extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } // Interface for true window functions. It's not much of an interface, they just @@ -2200,6 +2202,109 @@ struct WindowFunctionNthValue final : public WindowFunction } }; +struct NonNegativeDerivativeState +{ + Float64 previous_metric = 0; + Float64 previous_timestamp = 0; +}; + +// nonNegativeDerivative(metric_column, timestamp_column[, INTERVAL 1 SECOND]) +struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction +{ + static constexpr size_t ARGUMENT_METRIC = 0; + static constexpr size_t ARGUMENT_TIMESTAMP = 1; + static constexpr size_t ARGUMENT_INTERVAL = 2; + + WindowFunctionNonNegativeDerivative(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) + : StatefulWindowFunction(name_, argument_types_, parameters_) + { + if (!parameters.empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Function {} cannot be parameterized", name_); + } + + if (argument_types.size() != 2 && argument_types.size() != 3) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Function {} takes 2 or 3 arguments", name_); + } + + if (!isNumber(argument_types[ARGUMENT_METRIC])) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument {} must be a number, '{}' given", + ARGUMENT_METRIC, + argument_types[ARGUMENT_METRIC]->getName()); + } + + if (!isDateTime(argument_types[ARGUMENT_TIMESTAMP]) && !isDateTime64(argument_types[ARGUMENT_TIMESTAMP])) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument {} must be DateTime or DateTime64, '{}' given", + ARGUMENT_TIMESTAMP, + argument_types[ARGUMENT_TIMESTAMP]->getName()); + } + + if (argument_types.size() == 3) + { + const DataTypeInterval * interval_datatype = checkAndGetDataType(argument_types[ARGUMENT_INTERVAL].get()); + if (!interval_datatype) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Argument {} must be an INTERVAL, '{}' given", + ARGUMENT_INTERVAL, + argument_types[ARGUMENT_INTERVAL]->getName()); + } + if (!interval_datatype->getKind().isFixedLength()) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The INTERVAL must be a week or shorter, '{}' given", + argument_types[ARGUMENT_INTERVAL]->getName()); + } + interval_length = interval_datatype->getKind().toAvgSeconds(); + interval_specified = true; + } + } + + + DataTypePtr getReturnType() const override { return argument_types[0]; } + + bool allocatesMemoryInArena() const override { return false; } + + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) override + { + const auto & current_block = transform->blockAt(transform->current_row); + const auto & workspace = transform->workspaces[function_index]; + auto & state = getState(workspace); + + auto interval_duration = interval_specified ? interval_length * + (*current_block.input_columns[workspace.argument_column_indices[ARGUMENT_INTERVAL]]).getFloat64(0) : 1; + + Float64 last_metric = state.previous_metric; + Float64 last_timestamp = state.previous_timestamp; + + Float64 curr_metric = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_METRIC, transform->current_row); + Float64 curr_timestamp = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIMESTAMP, transform->current_row); + + Float64 time_elapsed = curr_timestamp - last_timestamp; + Float64 metric_diff = curr_metric - last_metric; + Float64 result = (time_elapsed != 0) ? (metric_diff / time_elapsed * interval_duration) : 0; + + state.previous_metric = curr_metric; + state.previous_timestamp = curr_timestamp; + + WindowFunctionHelpers::setValueToOutputColumn(transform, function_index, result >= 0 ? result : 0); + } +private: + Float64 interval_length = 1; + bool interval_specified = false; +}; + void registerWindowFunctions(AggregateFunctionFactory & factory) { @@ -2299,6 +2404,13 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) return std::make_shared( name, argument_types, parameters); }, properties}); + + factory.registerFunction("nonNegativeDerivative", {[](const std::string & name, + const DataTypes & argument_types, const Array & parameters, const Settings *) + { + return std::make_shared( + name, argument_types, parameters); + }, properties}); } } diff --git a/tests/queries/0_stateless/02232_non_negative_derivative.reference b/tests/queries/0_stateless/02232_non_negative_derivative.reference new file mode 100644 index 00000000000..7559f527c7a --- /dev/null +++ b/tests/queries/0_stateless/02232_non_negative_derivative.reference @@ -0,0 +1,64 @@ +1 +1979-12-12 21:21:21.123 1.1 3.5045052519931732e-9 +1979-12-12 21:21:22.000 1.3345 0.26738883339230357 +1979-12-12 21:21:23.000 1.54 0.20550000000000002 +1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.129 2.1 0 +1979-12-12 21:21:21.124 2.34 0 +1979-12-12 21:21:21.127 3.7 453.33916989529325 +1979-12-12 21:21:21.123 1.1 1.0513515755979521e-17 +1979-12-12 21:21:22.000 1.3345 8.021665001769108e-10 +1979-12-12 21:21:23.000 1.54 6.165000000000001e-10 +1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.129 2.1 0 +1979-12-12 21:21:21.124 2.34 0 +1979-12-12 21:21:21.127 3.7 0.0000013600175096858798 +1979-12-12 21:21:21.123 1.1 1.4018021007972692e-14 +1979-12-12 21:21:22.000 1.3345 0.0000010695553335692141 +1979-12-12 21:21:23.000 1.54 8.22e-7 +1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.129 2.1 0 +1979-12-12 21:21:21.124 2.34 0 +1979-12-12 21:21:21.127 3.7 0.001813356679581173 +1979-12-12 21:21:21.123 1.1 1.7522526259965866e-11 +1979-12-12 21:21:22.000 1.3345 0.0013369441669615178 +1979-12-12 21:21:23.000 1.54 0.0010275000000000002 +1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.129 2.1 0 +1979-12-12 21:21:21.124 2.34 0 +1979-12-12 21:21:21.127 3.7 2.2666958494764664 +1979-12-12 21:21:21.123 1.1 2.102703151195904e-8 +1979-12-12 21:21:22.000 1.3345 1.6043330003538214 +1979-12-12 21:21:23.000 1.54 1.233 +1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.129 2.1 0 +1979-12-12 21:21:21.124 2.34 0 +1979-12-12 21:21:21.127 3.7 2720.0350193717595 +1979-12-12 21:21:21.123 1.1 0.0000014718922058371327 +1979-12-12 21:21:22.000 1.3345 112.3033100247675 +1979-12-12 21:21:23.000 1.54 86.31 +1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.129 2.1 0 +1979-12-12 21:21:21.124 2.34 0 +1979-12-12 21:21:21.127 3.7 190402.45135602317 +1979-12-12 21:21:21.123 1.1 0.0001009297512574034 +1979-12-12 21:21:21.124 2.34 35712459.78375156 +1979-12-12 21:21:21.127 3.7 13056168.092984445 +1979-12-12 21:21:21.129 2.1 0 +1979-12-12 21:21:22.000 1.3345 0 +1979-12-12 21:21:23.000 1.54 5918.400000000001 +1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.123 1.1 0.0027251032839498914 +1979-12-12 21:21:21.124 2.34 964236414.1612921 +1979-12-12 21:21:21.127 3.7 352516538.51058006 +1979-12-12 21:21:21.129 2.1 0 +1979-12-12 21:21:22.000 1.3345 0 +1979-12-12 21:21:23.000 1.54 159796.80000000002 +1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.123 1.1 0.021195247764054712 +1979-12-12 21:21:21.124 2.34 7499616554.587828 +1979-12-12 21:21:21.127 3.7 2741795299.5267334 +1979-12-12 21:21:21.129 2.1 0 +1979-12-12 21:21:22.000 1.3345 0 +1979-12-12 21:21:23.000 1.54 1242864 +1979-12-12 21:21:23.000 1.54 0 diff --git a/tests/queries/0_stateless/02232_non_negative_derivative.sql b/tests/queries/0_stateless/02232_non_negative_derivative.sql new file mode 100644 index 00000000000..c4cbadb68a8 --- /dev/null +++ b/tests/queries/0_stateless/02232_non_negative_derivative.sql @@ -0,0 +1,63 @@ +DROP TABLE IF EXISTS nnd; + +CREATE TABLE nnd +( + id Int8, ts DateTime64(3, 'UTC'), metric Float64 +) +ENGINE=MergeTree() +ORDER BY id; + +INSERT INTO nnd VALUES (1, toDateTime64('1979-12-12 21:21:21.123', 3, 'UTC'), 1.1), (2, toDateTime64('1979-12-12 21:21:21.124', 3, 'UTC'), 2.34), (3, toDateTime64('1979-12-12 21:21:21.127', 3, 'UTC'), 3.7); +INSERT INTO nnd VALUES (4, toDateTime64('1979-12-12 21:21:21.129', 3, 'UTC'), 2.1), (5, toDateTime('1979-12-12 21:21:22', 'UTC'), 1.3345), (6, toDateTime('1979-12-12 21:21:23', 'UTC'), 1.54), (7, toDateTime('1979-12-12 21:21:23', 'UTC'), 1.54); + +-- shall work for precise intervals +-- INTERVAL 1 SECOND shall be default +SELECT ( + SELECT + ts, + metric, + nonNegativeDerivative(metric, ts) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv + FROM nnd + LIMIT 5, 1 + ) = ( + SELECT + ts, + metric, + nonNegativeDerivative(metric, ts, toIntervalSecond(1)) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv + FROM nnd + LIMIT 5, 1 + ); +SELECT ts, metric, nonNegativeDerivative(metric, ts) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +-- Nanosecond +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +-- Microsecond +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 4 MICROSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +-- Millisecond +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 5 MILLISECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +-- Second +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 6 SECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +-- Minute +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 7 MINUTE) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +-- Hour +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 8 HOUR) OVER (ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +-- Day +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 9 DAY) OVER (ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +-- Week +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 10 WEEK) OVER (ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; + +-- shall not work for month, quarter, year (intervals with floating number of seconds) +-- Month +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 11 MONTH) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Quarter +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 12 QUARTER) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Year +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 13 YEAR) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +-- test against wrong arguments/types +SELECT ts, metric, nonNegativeDerivative(metric, 1, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS } +SELECT ts, metric, nonNegativeDerivative('string not datetime', ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS } +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND, id) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS } +SELECT ts, metric, nonNegativeDerivative(metric) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS } + +-- cleanup +DROP TABLE IF EXISTS nnd; From 53f47127e968518418e91ccbdd48c86c7dff6f42 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 1 Jul 2022 16:46:58 +0800 Subject: [PATCH 105/627] Fix only_merge header --- src/Interpreters/Aggregator.cpp | 10 +++++++++- .../01710_minmax_count_projection.reference | 2 ++ .../0_stateless/01710_minmax_count_projection.sql | 4 ++++ 3 files changed, 15 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 511e5c9e031..a99ecee43bf 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -374,7 +374,15 @@ Block Aggregator::Params::getHeader( if (only_merge) { - res = header.cloneEmpty(); + NameSet needed_columns(keys.begin(), keys.end()); + for (const auto & aggregate : aggregates) + needed_columns.emplace(aggregate.column_name); + + for (const auto & column : header) + { + if (needed_columns.contains(column.name)) + res.insert(column.cloneEmpty()); + } if (final) { diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index 3e24cac4f6d..bbcec98fb74 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -2,6 +2,7 @@ 0 9998 5000 1 9999 5000 0 9998 5000 +0 9998 5000 1 0 9998 5000 0 @@ -15,6 +16,7 @@ 1 1 1 +2021-10-25 10:00:00 3 2021-10-27 10:00:00 3 \N 2021-10-27 10:00:00 3 0 2021-10-24 10:00:00 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 4aea3cfd29e..7ceff6a2662 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -10,6 +10,7 @@ set max_rows_to_read = 2, allow_experimental_projection_optimization = 1; select min(i), max(i), count() from d; select min(i), max(i), count() from d group by _partition_id order by _partition_id; select min(i), max(i), count() from d where _partition_value.1 = 0 group by _partition_id order by _partition_id; +select min(i), max(i), count() from d where moduloLegacy(i, 2) = 0 group by _partition_id order by _partition_id; select min(i), max(i), count() from d where _partition_value.1 = 10 group by _partition_id order by _partition_id; -- fuzz crash @@ -57,6 +58,9 @@ select min(dt), max(dt), count() from d where toDate(dt) >= '2021-10-25'; select min(dt), max(dt), count(toDate(dt) >= '2021-10-25') from d where toDate(dt) >= '2021-10-25'; select count() from d group by toDate(dt); +-- fuzz crash +SELECT min(dt), count(ignore(ignore(ignore(tupleElement(_partition_value, NULL) = NULL), NULL, NULL, NULL), 0, '10485.76', NULL)), max(dt), count(toDate(dt) >= '2021-10-25') FROM d WHERE toDate(dt) >= '2021-10-25'; + -- fuzz crash SELECT pointInEllipses(min(j), NULL), max(dt), count('0.0000000007') FROM d WHERE toDate(dt) >= '2021-10-25'; SELECT min(j) FROM d PREWHERE ceil(j) <= 0; From d77351ebe73a3488e710f7dfd86be37ab90f4482 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 2 Jul 2022 03:10:11 +0200 Subject: [PATCH 106/627] Compatible benchmark --- benchmark/compatible/README.md | 4 ++-- benchmark/compatible/cratedb/benchmark.sh | 18 ++++++++++++++---- benchmark/compatible/cratedb/create.sql | 2 +- 3 files changed, 17 insertions(+), 7 deletions(-) diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index 248454affba..70940b8b375 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -7,7 +7,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [x] MariaDB ColumnStore - [x] MemSQL/SingleStore - [x] PostgreSQL -- [ ] Greenplum +- [x] Greenplum - [x] TimescaleDB - [x] Citus - [x] Vertica (without publishing) @@ -17,7 +17,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [x] mapD/Omnisci/HeavyAI - [ ] Databend - [ ] Doris -- [ ] Druid +- [x] Druid - [ ] Pinot - [ ] CrateDB - [ ] Spark SQL diff --git a/benchmark/compatible/cratedb/benchmark.sh b/benchmark/compatible/cratedb/benchmark.sh index 033ed0dfe01..2f672630836 100755 --- a/benchmark/compatible/cratedb/benchmark.sh +++ b/benchmark/compatible/cratedb/benchmark.sh @@ -7,13 +7,23 @@ sudo apt-get install -y postgresql-client psql -U crate -h localhost --no-password -t -c 'SELECT 1' -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' -gzip -d hits.csv.gz +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz psql -U crate -h localhost --no-password -t < create.sql -psql -U crate -h localhost --no-password -t -c '\timing' -c "COPY hits FROM 'file:///$(pwd)/hits.csv'" -# It failed to load the data. +psql -U crate -h localhost --no-password -t -c '\timing' -c " + COPY hits + FROM 'file://$(pwd)/hits.tsv' + WITH + ( + "delimiter"=e'\t', + "format"='csv', + "header"=false, + "empty_string_as_null"=false + ) + RETURN SUMMARY;" + # COPY 0 # Time: 1004421.355 ms (16:44.421) diff --git a/benchmark/compatible/cratedb/create.sql b/benchmark/compatible/cratedb/create.sql index c9d65a98bb9..3bf1815ceba 100644 --- a/benchmark/compatible/cratedb/create.sql +++ b/benchmark/compatible/cratedb/create.sql @@ -63,7 +63,7 @@ CREATE TABLE hits IsParameter SMALLINT NOT NULL, DontCountHits SMALLINT NOT NULL, WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, + HitColor TEXT NOT NULL, LocalEventTime TIMESTAMP NOT NULL, Age SMALLINT NOT NULL, Sex SMALLINT NOT NULL, From 9d5f08f4f95aa6a61adc21f10bd8a8e4c6afae1a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 2 Jul 2022 03:52:45 +0200 Subject: [PATCH 107/627] Update README --- benchmark/compatible/README.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index 70940b8b375..58c3902fd1e 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -42,6 +42,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] TiDB (TiFlash) - [ ] Amazon RDS - [ ] InfluxDB +- [ ] VictoriaMetrics - [ ] TDEngine - [ ] MongoDB - [ ] Elasticsearch @@ -55,3 +56,5 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] MS SQL Server with Column Store Index (without publishing) - [ ] Dremio (without publishing) - [ ] Exasol +- [ ] LocustDB +- [ ] EventQL From e8f5cd3c681f872b2be4a783e4ecac3276a8aa62 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 29 Jun 2022 19:21:28 +0300 Subject: [PATCH 108/627] Add separate option to omit symbols from heavy contrib Sometimes it is useful to build contrib with debug symbols for further debugging. With everything turned ON (i.e. debug build) I got 3.3GB vs 3.0GB w/o this patch, 9% bloat, thoughts about this is this OK or not for you, if not STRIP_DEBUG_SYMBOLS_HEAVY_CONTRIB can be OFF by default (regardless of build type). P.S. aws debug symbols adds just 1.7%. v2: rename STRIP_HEAVY_DEBUG_SYMBOLS v3: OMIT_HEAVY_DEBUG_SYMBOLS v4: documentation had been removed Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 16 +++++++++++++++- contrib/arrow-cmake/CMakeLists.txt | 6 ++++-- contrib/aws-s3-cmake/CMakeLists.txt | 6 ++++-- contrib/curl-cmake/CMakeLists.txt | 4 +++- contrib/vectorscan-cmake/CMakeLists.txt | 5 ++++- src/Dictionaries/CMakeLists.txt | 5 ++--- src/Functions/CMakeLists.txt | 19 +------------------ src/Functions/GatherUtils/CMakeLists.txt | 2 +- src/Functions/JSONPath/CMakeLists.txt | 2 +- src/Functions/URL/CMakeLists.txt | 2 +- src/Functions/array/CMakeLists.txt | 2 +- 11 files changed, 37 insertions(+), 32 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4c7b732c68c..4180ae9cce7 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -223,11 +223,25 @@ if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") endif () endif() +if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" + OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" + OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") + set (OMIT_HEAVY_DEBUG_SYMBOLS_DEFAULT ON) +else() + set (OMIT_HEAVY_DEBUG_SYMBOLS_DEFAULT OFF) +endif() +# Provides faster linking and lower binary size. +# Tradeoff is the inability to debug some source files with e.g. gdb +# (empty stack frames and no local variables)." +option(OMIT_HEAVY_DEBUG_SYMBOLS + "Do not generate debugger info for heavy modules (ClickHouse functions and dictionaries, some contrib)" + ${OMIT_HEAVY_DEBUG_SYMBOLS_DEFAULT}) + if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") set(USE_DEBUG_HELPERS ON) endif() - option(USE_DEBUG_HELPERS "Enable debug helpers" ${USE_DEBUG_HELPERS}) + option(BUILD_STANDALONE_KEEPER "Build keeper as small standalone binary" OFF) if (NOT BUILD_STANDALONE_KEEPER) option(CREATE_KEEPER_SYMLINK "Create symlink for clickhouse-keeper to main server binary" ON) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 74bbb300fa5..c289c88ef7b 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -462,5 +462,7 @@ foreach (TOOL ${PARQUET_TOOLS}) endforeach () # The library is large - avoid bloat. -target_compile_options (_arrow PRIVATE -g0) -target_compile_options (_parquet PRIVATE -g0) +if (OMIT_HEAVY_DEBUG_SYMBOLS) + target_compile_options (_arrow PRIVATE -g0) + target_compile_options (_parquet PRIVATE -g0) +endif() diff --git a/contrib/aws-s3-cmake/CMakeLists.txt b/contrib/aws-s3-cmake/CMakeLists.txt index de6486e58fd..eabed601722 100644 --- a/contrib/aws-s3-cmake/CMakeLists.txt +++ b/contrib/aws-s3-cmake/CMakeLists.txt @@ -114,7 +114,9 @@ endif() target_link_libraries(_aws_s3 PRIVATE _aws_s3_checksums) # The library is large - avoid bloat. -target_compile_options (_aws_s3 PRIVATE -g0) -target_compile_options (_aws_s3_checksums PRIVATE -g0) +if (OMIT_HEAVY_DEBUG_SYMBOLS) + target_compile_options (_aws_s3 PRIVATE -g0) + target_compile_options (_aws_s3_checksums PRIVATE -g0) +endif() add_library(ch_contrib::aws_s3 ALIAS _aws_s3) diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index 761ee036e66..207b7c66371 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -171,6 +171,8 @@ target_include_directories (_curl SYSTEM PUBLIC target_link_libraries (_curl PRIVATE OpenSSL::SSL) # The library is large - avoid bloat (XXX: is it?) -target_compile_options (_curl PRIVATE -g0) +if (OMIT_HEAVY_DEBUG_SYMBOLS) + target_compile_options (_curl PRIVATE -g0) +endif() add_library (ch_contrib::curl ALIAS _curl) diff --git a/contrib/vectorscan-cmake/CMakeLists.txt b/contrib/vectorscan-cmake/CMakeLists.txt index 480ab3a384f..bc17105be99 100644 --- a/contrib/vectorscan-cmake/CMakeLists.txt +++ b/contrib/vectorscan-cmake/CMakeLists.txt @@ -268,10 +268,13 @@ endif() add_library (_vectorscan ${SRCS}) target_compile_options (_vectorscan PRIVATE - -g0 # library has too much debug information -fno-sanitize=undefined # assume the library takes care of itself -O2 -fno-strict-aliasing -fno-omit-frame-pointer -fvisibility=hidden # options from original build system ) +# library has too much debug information +if (OMIT_HEAVY_DEBUG_SYMBOLS) + target_compile_options (_vectorscan PRIVATE -g0) +endif() # Include version header manually generated by running the original build system target_include_directories (_vectorscan SYSTEM PRIVATE common) diff --git a/src/Dictionaries/CMakeLists.txt b/src/Dictionaries/CMakeLists.txt index 19e82c45cc2..964e44efffe 100644 --- a/src/Dictionaries/CMakeLists.txt +++ b/src/Dictionaries/CMakeLists.txt @@ -4,8 +4,7 @@ add_headers_and_sources(clickhouse_dictionaries .) add_headers_and_sources(clickhouse_dictionaries "${CMAKE_CURRENT_BINARY_DIR}/generated/") -if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") - +if (OMIT_HEAVY_DEBUG_SYMBOLS) # Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. set_source_files_properties( FlatDictionary.cpp @@ -15,7 +14,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELW RangeHashedDictionary.cpp DirectDictionary.cpp PROPERTIES COMPILE_FLAGS -g0) -endif () +endif() list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp DictionarySourceFactory.cpp DictionaryStructure.cpp getDictionaryConfigurationFromAST.cpp) list(REMOVE_ITEM clickhouse_dictionaries_headers DictionaryFactory.h DictionarySourceFactory.h DictionaryStructure.h getDictionaryConfigurationFromAST.h) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 1f728489b73..7d7aba6955f 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -35,25 +35,8 @@ if (TARGET OpenSSL::Crypto) target_link_libraries(clickhouse_functions PUBLIC OpenSSL::Crypto) endif() -if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" - OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" - OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") - set (STRIP_DSF_DEFAULT ON) -else() - set (STRIP_DSF_DEFAULT OFF) -endif() - - -# Provides faster linking and lower binary size. -# Tradeoff is the inability to debug some source files with e.g. gdb -# (empty stack frames and no local variables)." -option(STRIP_DEBUG_SYMBOLS_FUNCTIONS "Do not generate debugger info for ClickHouse functions" ${STRIP_DSF_DEFAULT}) - -if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) - message(INFO "Not generating debugger info for ClickHouse functions") +if (OMIT_HEAVY_DEBUG_SYMBOLS) target_compile_options(clickhouse_functions PRIVATE "-g0") -else() - message(STATUS "Generating debugger info for ClickHouse functions") endif() if (TARGET ch_contrib::icu) diff --git a/src/Functions/GatherUtils/CMakeLists.txt b/src/Functions/GatherUtils/CMakeLists.txt index 460b02326a1..fe600f86d07 100644 --- a/src/Functions/GatherUtils/CMakeLists.txt +++ b/src/Functions/GatherUtils/CMakeLists.txt @@ -12,7 +12,7 @@ if (HAS_SUGGEST_DESTRUCTOR_OVERRIDE) target_compile_definitions(clickhouse_functions_gatherutils PUBLIC HAS_SUGGEST_DESTRUCTOR_OVERRIDE) endif() -if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) +if (OMIT_HEAVY_DEBUG_SYMBOLS) target_compile_options(clickhouse_functions_gatherutils PRIVATE "-g0") endif() diff --git a/src/Functions/JSONPath/CMakeLists.txt b/src/Functions/JSONPath/CMakeLists.txt index 2531a0f84e6..bdc826008eb 100644 --- a/src/Functions/JSONPath/CMakeLists.txt +++ b/src/Functions/JSONPath/CMakeLists.txt @@ -8,6 +8,6 @@ target_link_libraries(clickhouse_functions_jsonpath PRIVATE dbms) target_link_libraries(clickhouse_functions_jsonpath PRIVATE clickhouse_parsers) target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_jsonpath) -if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) +if (OMIT_HEAVY_DEBUG_SYMBOLS) target_compile_options(clickhouse_functions_jsonpath PRIVATE "-g0") endif() diff --git a/src/Functions/URL/CMakeLists.txt b/src/Functions/URL/CMakeLists.txt index a1af30e6f81..28a72651315 100644 --- a/src/Functions/URL/CMakeLists.txt +++ b/src/Functions/URL/CMakeLists.txt @@ -3,7 +3,7 @@ add_headers_and_sources(clickhouse_functions_url .) add_library(clickhouse_functions_url ${clickhouse_functions_url_sources} ${clickhouse_functions_url_headers}) target_link_libraries(clickhouse_functions_url PRIVATE dbms) -if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) +if (OMIT_HEAVY_DEBUG_SYMBOLS) target_compile_options(clickhouse_functions_url PRIVATE "-g0") endif() diff --git a/src/Functions/array/CMakeLists.txt b/src/Functions/array/CMakeLists.txt index 9762674d6e9..6b4a8122d16 100644 --- a/src/Functions/array/CMakeLists.txt +++ b/src/Functions/array/CMakeLists.txt @@ -3,6 +3,6 @@ add_headers_and_sources(clickhouse_functions_array .) add_library(clickhouse_functions_array ${clickhouse_functions_array_sources} ${clickhouse_functions_array_headers}) target_link_libraries(clickhouse_functions_array PRIVATE dbms clickhouse_functions_gatherutils) -if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) +if (OMIT_HEAVY_DEBUG_SYMBOLS) target_compile_options(clickhouse_functions_array PRIVATE "-g0") endif() From 070349feb6b4685fb7c974befd479f035b15f35f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 2 Jul 2022 12:17:00 +0200 Subject: [PATCH 109/627] Add CrateDB --- benchmark/compatible/README.md | 3 +- benchmark/compatible/cratedb/benchmark.sh | 12 +++-- .../cratedb/results/c6a.4xlarge.txt | 48 ++++++++++++++++++- 3 files changed, 56 insertions(+), 7 deletions(-) diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index 58c3902fd1e..6562eea6852 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -19,7 +19,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] Doris - [x] Druid - [ ] Pinot -- [ ] CrateDB +- [x] CrateDB - [ ] Spark SQL - [ ] Starrocks - [ ] ShitholeDB @@ -58,3 +58,4 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] Exasol - [ ] LocustDB - [ ] EventQL +- [ ] Apache Drill diff --git a/benchmark/compatible/cratedb/benchmark.sh b/benchmark/compatible/cratedb/benchmark.sh index 2f672630836..b45826ee861 100755 --- a/benchmark/compatible/cratedb/benchmark.sh +++ b/benchmark/compatible/cratedb/benchmark.sh @@ -24,12 +24,16 @@ psql -U crate -h localhost --no-password -t -c '\timing' -c " ) RETURN SUMMARY;" -# COPY 0 -# Time: 1004421.355 ms (16:44.421) +# One record did not load: +# 99997496 +# {"Missing closing quote for value\n at [Source: UNKNOWN; line: 1, column: 1069]":{"count":1,"line_numbers":[93557187]}} +# Time: 10687056.069 ms (02:58:07.056) ./run.sh 2>&1 | tee log.txt +# For some queries it gives "Data too large". + du -bcs crate-* -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' +cat log.txt | grep -oP 'Time: \d+\.\d+ ms|ERROR' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if ($1 == "ERROR") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : ($1 / 1000); if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' diff --git a/benchmark/compatible/cratedb/results/c6a.4xlarge.txt b/benchmark/compatible/cratedb/results/c6a.4xlarge.txt index b4d66d3a983..bc0413fcf6a 100644 --- a/benchmark/compatible/cratedb/results/c6a.4xlarge.txt +++ b/benchmark/compatible/cratedb/results/c6a.4xlarge.txt @@ -1,2 +1,46 @@ -Load time: -Data size: +Load time: 10 687 sec. +Data size: 109 636 633 416 bytes. + +[0.008162,0.005118,0.002553], +[0.350014,0.39977,0.133775], +[2.58426,2.47192,2.59779], +[2.12939,0.532981,0.507246], +[null,null,null], +[null,null,null], +[1.18488,1.06603,1.07219], +[0.209264,0.073284,0.067912], +[null,null,null], +[null,null,null], +[1.68892,1.2866,1.47428], +[1.62976,1.43073,1.26904], +[12.7517,13.0334,13.2685], +[18.8587,null,18.6951], +[11.2982,11.2108,11.577], +[20.2964,20.4035,19.1076], +[null,null,null], +[null,null,null], +[null,null,null], +[0.202044,0.010009,0.005566], +[9.22964,4.54606,0.774149], +[1.41673,1.09885,0.789775], +[12.3933,8.06911,1.69671], +[1.45018,0.969528,0.979718], +[0.357589,0.14887,0.153326], +[0.189282,0.133963,0.130279], +[0.153222,0.140756,0.139861], +[27.5195,19.6862,20.1825], +[72.7575,68.2,67.1238], +[144.533,146.579,152.144], +[8.76866,9.00563,8.46917], +[17.6652,16.6755,16.0558], +[null,null,null], +[null,null,null], +[null,null,null], +[42.2967,44.9621,44.4386], +[0.786911,0.4904,0.508416], +[0.602075,0.226261,0.182399], +[0.131407,0.058958,0.054518], +[0.954736,1.1361,1.14233], +[0.23764,0.139109,0.134472], +[0.110253,0.057695,0.056073], +[0.124285,0.150479,0.066226] From 8e99ea84a8f1287ddfbd2519ab83c3847b6ae046 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 2 Jul 2022 14:09:51 +0300 Subject: [PATCH 110/627] fix LOGICAL_ERROR --- src/Processors/Transforms/WindowTransform.cpp | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 09805696472..4de2e4d8604 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2271,7 +2271,7 @@ struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction } - DataTypePtr getReturnType() const override { return argument_types[0]; } + DataTypePtr getReturnType() const override { return std::make_shared(); } bool allocatesMemoryInArena() const override { return false; } @@ -2285,15 +2285,12 @@ struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction auto interval_duration = interval_specified ? interval_length * (*current_block.input_columns[workspace.argument_column_indices[ARGUMENT_INTERVAL]]).getFloat64(0) : 1; - Float64 last_metric = state.previous_metric; - Float64 last_timestamp = state.previous_timestamp; - Float64 curr_metric = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_METRIC, transform->current_row); Float64 curr_timestamp = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIMESTAMP, transform->current_row); - Float64 time_elapsed = curr_timestamp - last_timestamp; - Float64 metric_diff = curr_metric - last_metric; - Float64 result = (time_elapsed != 0) ? (metric_diff / time_elapsed * interval_duration) : 0; + Float64 time_elapsed = curr_timestamp - state.previous_timestamp; + Float64 metric_diff = curr_metric - state.previous_metric; + Float64 result = (time_elapsed > 0) ? (metric_diff / time_elapsed * interval_duration) : 0; state.previous_metric = curr_metric; state.previous_timestamp = curr_timestamp; From 8047e11c09bf1c96736bb8d954e022787505f04a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 2 Jul 2022 16:56:33 -0300 Subject: [PATCH 111/627] Update index.md --- .../sql-reference/window-functions/index.md | 106 +++++++++++++++++- 1 file changed, 103 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 0f1165abf71..767a4a3b279 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -70,13 +70,29 @@ https://dev.mysql.com/doc/refman/8.0/en/window-functions-frames.html ```text aggregate_function (column_name) - OVER ([PARTITION BY groupping_column] [ORDER BY sorting_column] - [ROWS or RANGE expression_to_bounds_of_frame]) + OVER ([[PARTITION BY groupping_column] [ORDER BY sorting_column] + [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) +FROM table_name +WINDOW window_name as ([[PARTITION BY groupping_column] [ORDER BY sorting_column]) ``` - `PARTITION BY` - defines how to break a resultset into groups. - `ORDER BY` - defines how to order rows inside the group during calculation aggregate_function. - `ROWS or RANGE` - defines bounds of a frame, aggregate_function is calculated within a frame. +- `WINDOW` - allows to reuse a window definition with multiple exressions. + +### Functions + +These functions can be used only as a window function. + +`row_number()` - Number the current row within its partition starting from 1. +`first_value(x)` - Return a value evaluated against the first row within its ordered frame. +`last_value(x)` - Return a value evaluated against the last row within its ordered frame. +`nth_value(x, offset)` - Return a value evaluated against the nth row (offset) in an ordered frame. +`rank()` - Rank the current row within its partition with gaps. +`dense_rank()` - Rank the current row within its partition without gaps. +`lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. +`leadInFrame(x)` - Return a value evaluated at the row that is offset rows after the current row within the ordered frame. ```text PARTITION @@ -101,7 +117,8 @@ aggregate_function (column_name) CREATE TABLE wf_partition ( `part_key` UInt64, - `value` UInt64 + `value` UInt64, + `order` UInt64 ) ENGINE = Memory; @@ -271,6 +288,89 @@ ORDER BY │ 1 │ 4 │ 4 │ [3,4,5] │ │ 1 │ 5 │ 5 │ [4,5] │ └──────────┴───────┴───────┴──────────────┘ + +-- row_number does not respect the frame, so rn_1 = rn_2 = rn_3 != rn_4 +SELECT + part_key, + value, + order, + groupArray(value) OVER w1 AS frame_values, + row_number() OVER w1 AS rn_1, + sum(1) OVER w1 AS rn_2, + row_number() OVER w2 AS rn_3, + sum(1) OVER w2 AS rn_4 +FROM wf_frame +WINDOW + w1 AS (PARTITION BY part_key ORDER BY order DESC), + w2 AS (PARTITION BY part_key ORDER BY order DESC + Rows BETWEEN 1 PRECEDING AND CURRENT ROW) +ORDER BY + part_key ASC, + value ASC; +┌─part_key─┬─value─┬─order─┬─frame_values─┬─rn_1─┬─rn_2─┬─rn_3─┬─rn_4─┐ +│ 1 │ 1 │ 1 │ [5,4,3,2,1] │ 5 │ 5 │ 5 │ 2 │ +│ 1 │ 2 │ 2 │ [5,4,3,2] │ 4 │ 4 │ 4 │ 2 │ +│ 1 │ 3 │ 3 │ [5,4,3] │ 3 │ 3 │ 3 │ 2 │ +│ 1 │ 4 │ 4 │ [5,4] │ 2 │ 2 │ 2 │ 2 │ +│ 1 │ 5 │ 5 │ [5] │ 1 │ 1 │ 1 │ 1 │ +└──────────┴───────┴───────┴──────────────┴──────┴──────┴──────┴──────┘ + +-- first_value and last_value respect the frame +SELECT + groupArray(value) OVER w1 AS frame_values_1, + first_value(value) OVER w1 AS first_value_1, + last_value(value) OVER w1 AS last_value_1, + groupArray(value) OVER w2 AS frame_values_2, + first_value(value) OVER w2 AS first_value_2, + last_value(value) OVER w2 AS last_value_2 +FROM wf_frame +WINDOW + w1 AS (PARTITION BY part_key ORDER BY order ASC), + w2 AS (PARTITION BY part_key ORDER BY order ASC Rows BETWEEN 1 PRECEDING AND CURRENT ROW) +ORDER BY + part_key ASC, + value ASC; +┌─frame_values_1─┬─first_value_1─┬─last_value_1─┬─frame_values_2─┬─first_value_2─┬─last_value_2─┐ +│ [1] │ 1 │ 1 │ [1] │ 1 │ 1 │ +│ [1,2] │ 1 │ 2 │ [1,2] │ 1 │ 2 │ +│ [1,2,3] │ 1 │ 3 │ [2,3] │ 2 │ 3 │ +│ [1,2,3,4] │ 1 │ 4 │ [3,4] │ 3 │ 4 │ +│ [1,2,3,4,5] │ 1 │ 5 │ [4,5] │ 4 │ 5 │ +└────────────────┴───────────────┴──────────────┴────────────────┴───────────────┴──────────────┘ + +-- second value within the frame +SELECT + groupArray(value) OVER w1 AS frame_values_1, + nth_value(value, 2) OVER w1 AS second_value +FROM wf_frame +WINDOW w1 AS (PARTITION BY part_key ORDER BY order ASC Rows BETWEEN 3 PRECEDING AND CURRENT ROW) +ORDER BY + part_key ASC, + value ASC +┌─frame_values_1─┬─second_value─┐ +│ [1] │ 0 │ +│ [1,2] │ 2 │ +│ [1,2,3] │ 2 │ +│ [1,2,3,4] │ 2 │ +│ [2,3,4,5] │ 3 │ +└────────────────┴──────────────┘ + +-- second value within the frame + Null for missing values +SELECT + groupArray(value) OVER w1 AS frame_values_1, + nth_value(toNullable(value), 2) OVER w1 AS second_value +FROM wf_frame +WINDOW w1 AS (PARTITION BY part_key ORDER BY order ASC Rows BETWEEN 3 PRECEDING AND CURRENT ROW) +ORDER BY + part_key ASC, + value ASC +┌─frame_values_1─┬─second_value─┐ +│ [1] │ ᴺᵁᴸᴸ │ +│ [1,2] │ 2 │ +│ [1,2,3] │ 2 │ +│ [1,2,3,4] │ 2 │ +│ [2,3,4,5] │ 3 │ +└────────────────┴──────────────┘ ``` ## Real world examples From e266ef14a825d67c1bb6959dfc048aa0b93dd714 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 2 Jul 2022 17:34:43 -0300 Subject: [PATCH 112/627] Update index.md --- docs/en/sql-reference/window-functions/index.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 767a4a3b279..6d0c10ad22a 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -86,9 +86,9 @@ WINDOW window_name as ([[PARTITION BY groupping_column] [ORDER BY sorting_column These functions can be used only as a window function. `row_number()` - Number the current row within its partition starting from 1. -`first_value(x)` - Return a value evaluated against the first row within its ordered frame. -`last_value(x)` - Return a value evaluated against the last row within its ordered frame. -`nth_value(x, offset)` - Return a value evaluated against the nth row (offset) in an ordered frame. +`first_value(x)` - Return the first non-NULL value evaluated within its ordered frame. +`last_value(x)` - Return the last non-NULL value evaluated within its ordered frame. +`nth_value(x, offset)` - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. `rank()` - Rank the current row within its partition with gaps. `dense_rank()` - Rank the current row within its partition without gaps. `lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. From 59d1bd3e0022d129e7db17960a809d9df0b8bdbe Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 2 Jul 2022 18:46:49 -0300 Subject: [PATCH 113/627] Update index.md --- docs/en/sql-reference/window-functions/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 6d0c10ad22a..d42828b74e2 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -70,10 +70,10 @@ https://dev.mysql.com/doc/refman/8.0/en/window-functions-frames.html ```text aggregate_function (column_name) - OVER ([[PARTITION BY groupping_column] [ORDER BY sorting_column] + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name -WINDOW window_name as ([[PARTITION BY groupping_column] [ORDER BY sorting_column]) +WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) ``` - `PARTITION BY` - defines how to break a resultset into groups. From aefe64742d682081378dd7c42f1252c47baaa8a9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Jul 2022 00:25:00 +0200 Subject: [PATCH 114/627] Add Redshift serverless --- benchmark/compatible/databend/README.md | 1 + benchmark/compatible/databend/benchmark.sh | 90 +++++++++++++++ benchmark/compatible/databend/create.sql | 109 ++++++++++++++++++ benchmark/compatible/databend/queries.sql | 43 +++++++ benchmark/compatible/databend/run.sh | 20 ++++ benchmark/compatible/monetdb/README.md | 1 - .../compatible/redshift-serverless/README.md | 55 +++++++++ .../compatible/redshift-serverless/create.sql | 109 ++++++++++++++++++ .../redshift-serverless/queries.sql | 43 +++++++ .../redshift-serverless/result/serverless.txt | 46 ++++++++ .../compatible/redshift-serverless/run.sh | 10 ++ benchmark/compatible/redshift/README.md | 4 +- .../redshift/result/4x.ra3.xplus.txt | 4 +- benchmark/compatible/starrocks/README.md | 3 + benchmark/compatible/starrocks/benchmark.sh | 3 + 15 files changed, 536 insertions(+), 5 deletions(-) create mode 100644 benchmark/compatible/databend/README.md create mode 100644 benchmark/compatible/databend/benchmark.sh create mode 100644 benchmark/compatible/databend/create.sql create mode 100644 benchmark/compatible/databend/queries.sql create mode 100755 benchmark/compatible/databend/run.sh delete mode 100644 benchmark/compatible/monetdb/README.md create mode 100644 benchmark/compatible/redshift-serverless/README.md create mode 100644 benchmark/compatible/redshift-serverless/create.sql create mode 100644 benchmark/compatible/redshift-serverless/queries.sql create mode 100644 benchmark/compatible/redshift-serverless/result/serverless.txt create mode 100644 benchmark/compatible/redshift-serverless/run.sh create mode 100644 benchmark/compatible/starrocks/README.md create mode 100644 benchmark/compatible/starrocks/benchmark.sh diff --git a/benchmark/compatible/databend/README.md b/benchmark/compatible/databend/README.md new file mode 100644 index 00000000000..ca25528a956 --- /dev/null +++ b/benchmark/compatible/databend/README.md @@ -0,0 +1 @@ +It failed to load the data. diff --git a/benchmark/compatible/databend/benchmark.sh b/benchmark/compatible/databend/benchmark.sh new file mode 100644 index 00000000000..ff9f141abba --- /dev/null +++ b/benchmark/compatible/databend/benchmark.sh @@ -0,0 +1,90 @@ +#!/bin/bash + +mkdir databend && cd databend +curl -LJO 'https://github.com/datafuselabs/databend/releases/download/v0.7.113-nightly/databend-v0.7.113-nightly-x86_64-unknown-linux-musl.tar.gz' +tar xzvf 'databend-v0.7.113-nightly-x86_64-unknown-linux-musl.tar.gz' + +echo 'dir = "metadata/_logs" +admin_api_address = "127.0.0.1:8101" +grpc_api_address = "127.0.0.1:9101" + +[raft_config] +id = 1 +single = true +raft_dir = "metadata/datas"' > databend-meta.toml + +./bin/databend-meta -c ./databend-meta.toml > meta.log 2>&1 & +curl -I 'http://127.0.0.1:8101/v1/health' + +echo '[log] +level = "INFO" +dir = "benddata/_logs" + +[query] +# For admin RESET API. +admin_api_address = "127.0.0.1:8001" + +# Metrics. +metric_api_address = "127.0.0.1:7071" + +# Cluster flight RPC. +flight_api_address = "127.0.0.1:9091" + +# Query MySQL Handler. +mysql_handler_host = "127.0.0.1" +mysql_handler_port = 3307 + +# Query ClickHouse Handler. +clickhouse_handler_host = "127.0.0.1" +clickhouse_handler_port = 9001 + +# Query ClickHouse HTTP Handler. +clickhouse_http_handler_host = "127.0.0.1" +clickhouse_http_handler_port = 8125 + +# Query HTTP Handler. +http_handler_host = "127.0.0.1" +http_handler_port = 8081 + +tenant_id = "tenant1" +cluster_id = "cluster1" + +[meta] +# databend-meta grpc api address. +address = "127.0.0.1:9101" +username = "root" +password = "root" + +[storage] +# fs|s3 +type = "fs" + +[storage.fs] +data_path = "benddata/datas"' > databend-query.toml + +./bin/databend-query -c ./databend-query.toml > query.log 2>&1 & + +curl https://clickhouse.com/ | sh +sudo ./clickhouse install + +# Load the data + +curl 'http://default@localhost:8124/' --data-binary @create.sql + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +gzip -d hits.csv.gz + +# Note: if I run +# clickhouse-client --time --query "INSERT INTO hits FORMAT TSV" < hits.tsv +# it panics: +# ERROR common_tracing::panic_hook: panicked at 'called `Result::unwrap()` on an `Err` value: SendError + +# Note: if I run +# curl -XPUT 'http://root:@127.0.0.1:8000/v1/streaming_load' -H 'insert_sql: insert into hits format CSV' -H 'skip_header: 0' -H 'field_delimiter: ,' -H 'record_delimiter: \n' -F 'upload=@"./hits.csv"' +# curl: (55) Send failure: Broken pipe + +# This is not entirely correct, but starts to work: +# curl -XPUT 'http://root:@127.0.0.1:8000/v1/streaming_load' -H 'insert_sql: insert into hits format TSV' -H 'skip_header: 0' -H 'field_delimiter: \t' -H 'record_delimiter: \n' -F 'upload=@"./hits.tsv"' +# and fails after 7 minutes 38 seconds without loading any data: +# Code: 4000, displayText = invalid data (Expected to have terminated string literal.) (while in processor thread 5). +# the diagnostics is terrible. diff --git a/benchmark/compatible/databend/create.sql b/benchmark/compatible/databend/create.sql new file mode 100644 index 00000000000..b446288b409 --- /dev/null +++ b/benchmark/compatible/databend/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +) +CLUSTER BY (CounterID, EventDate, UserID, EventTime, WatchID); diff --git a/benchmark/compatible/databend/queries.sql b/benchmark/compatible/databend/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/databend/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/databend/run.sh b/benchmark/compatible/databend/run.sh new file mode 100755 index 00000000000..86bc4c03f72 --- /dev/null +++ b/benchmark/compatible/databend/run.sh @@ -0,0 +1,20 @@ +#!/bin/bash + +TRIES=3 +QUERY_NUM=1 +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + + echo -n "[" + for i in $(seq 1 $TRIES); do + RES=$(clickhouse-client --time --format=Null --query="$query" 2>&1 ||:) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + + echo "${QUERY_NUM},${i},${RES}" >> result.csv + done + echo "]," + + QUERY_NUM=$((QUERY_NUM + 1)) +done diff --git a/benchmark/compatible/monetdb/README.md b/benchmark/compatible/monetdb/README.md deleted file mode 100644 index c9c0747c8f9..00000000000 --- a/benchmark/compatible/monetdb/README.md +++ /dev/null @@ -1 +0,0 @@ -MonetDB failed to install. diff --git a/benchmark/compatible/redshift-serverless/README.md b/benchmark/compatible/redshift-serverless/README.md new file mode 100644 index 00000000000..59aec7496cd --- /dev/null +++ b/benchmark/compatible/redshift-serverless/README.md @@ -0,0 +1,55 @@ +This benchmark is not automated. + +Go to AWS Redshift service. +Try Redshift Serverless. Use the default configuration. +The cluster will take a few minutes to start. +Go to "Query Editor". Establishing a connection takes around 10 seconds. + +Run the CREATE TABLE statement you find in `create.sql`. + +Note: Redshift prefers VARCHAR(MAX) instead of TEXT. + +Then press on the "Load data". +This will generate a statement: + +``` +COPY dev.public.hits FROM 's3://clickhouse-public-datasets/hits_compatible/hits.csv.gz' GZIP +IAM_ROLE 'arn:aws:iam::...:role/service-role/AmazonRedshift-CommandsAccessRole-...' +FORMAT AS CSV DELIMITER ',' QUOTE '"' +REGION AS 'eu-central-1' +``` + +> Elapsed time: 32m 13.7s + +It also have run 2380 "queries" for this task. + +Namespace configuration, +General Information, Storage used: + +30.3 GB + +Change admin user password: +dev, fGH4{dbas7 + +It's very difficult to find how to connect to it: +https://docs.aws.amazon.com/redshift/latest/mgmt/serverless-connecting.html + +We will run the queries from another server with `psql` client. + +``` +sudo apt-get install -y postgresql-client + +echo "*:*:*:*:your_password" > .pgpass +chmod 400 .pgpass + +psql -h default.111111111111.eu-central-1.redshift-serverless.amazonaws.com -U dev -d dev -p 5439 +``` + +Then run the benchmark: +``` +export HOST=... +./run.sh 2>&1 | tee log.txt + +cat log.txt | grep -oP 'Time: \d+\.\d+ ms|ERROR' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if ($1 == "ERROR") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : ($1 / 1000); if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' +``` diff --git a/benchmark/compatible/redshift-serverless/create.sql b/benchmark/compatible/redshift-serverless/create.sql new file mode 100644 index 00000000000..94a8fb0958a --- /dev/null +++ b/benchmark/compatible/redshift-serverless/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title VARCHAR(MAX) NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL VARCHAR(MAX) NOT NULL, + Referer VARCHAR(MAX) NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 VARCHAR(MAX) NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(MAX) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel VARCHAR(MAX) NOT NULL, + Params VARCHAR(MAX) NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase VARCHAR(MAX) NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset VARCHAR(MAX) NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL VARCHAR(MAX) NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor VARCHAR(MAX) NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage VARCHAR(MAX) NOT NULL, + BrowserCountry VARCHAR(MAX) NOT NULL, + SocialNetwork VARCHAR(MAX) NOT NULL, + SocialAction VARCHAR(MAX) NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage VARCHAR(MAX) NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID VARCHAR(MAX) NOT NULL, + ParamCurrency VARCHAR(MAX) NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName VARCHAR(MAX) NOT NULL, + OpenstatCampaignID VARCHAR(MAX) NOT NULL, + OpenstatAdID VARCHAR(MAX) NOT NULL, + OpenstatSourceID VARCHAR(MAX) NOT NULL, + UTMSource VARCHAR(MAX) NOT NULL, + UTMMedium VARCHAR(MAX) NOT NULL, + UTMCampaign VARCHAR(MAX) NOT NULL, + UTMContent VARCHAR(MAX) NOT NULL, + UTMTerm VARCHAR(MAX) NOT NULL, + FromTag VARCHAR(MAX) NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/redshift-serverless/queries.sql b/benchmark/compatible/redshift-serverless/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/redshift-serverless/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/redshift-serverless/result/serverless.txt b/benchmark/compatible/redshift-serverless/result/serverless.txt new file mode 100644 index 00000000000..3a30ffbafc2 --- /dev/null +++ b/benchmark/compatible/redshift-serverless/result/serverless.txt @@ -0,0 +1,46 @@ +Load time: 32m 13.7s +Data size: 30.3 GB + +[0.436955,0.390225,0.387823], +[2.74529,0.280571,0.198074], +[7.24569,0.295612,0.386568], +[null,null,null], +[3.98825,0.488446,0.325645], +[4.24601,0.538891,0.690124], +[7.25492,0.288505,0.207515], +[5.99594,0.212732,0.281278], +[6.17534,0.681868,0.586073], +[6.37779,1.37865,1.31614], +[6.41405,0.549735,0.421345], +[6.64688,0.445678,0.477323], +[4.2294,0.361772,0.520471], +[4.73701,0.788258,0.743465], +[4.49038,0.670446,0.52727], +[6.19886,0.663381,0.583588], +[5.9867,0.679607,0.672772], +[2.76661,0.567555,0.51494], +[6.20219,0.816422,0.760568], +[2.51526,0.053,0.025066], +[2.96003,0.282737,0.226979], +[6.79648,0.57495,0.400798], +[6.37854,0.92746,1.05793], +[25.8462,0.902664,0.905365], +[5.00521,0.247895,0.308836], +[6.38373,0.308781,0.244082], +[4.25427,0.229966,0.247201], +[5.98382,0.398218,0.455249], +[null,null,null], +[6.52367,1.35877,1.30562], +[7.36935,0.536226,0.582304], +[7.05948,0.621982,0.639653], +[4.62901,0.954522,0.908651], +[4.95273,1.03062,1.10289], +[4.71404,1.06378,1.04157], +[4.8201,0.499996,0.575546], +[5.32757,0.566517,0.562058], +[5.37681,0.626458,0.658628], +[5.23137,0.470622,0.540079], +[6.09326,0.561312,0.574978], +[4.86561,0.595546,0.534209], +[4.34256,0.433804,0.414541], +[4.19814,0.288269,0.415328] diff --git a/benchmark/compatible/redshift-serverless/run.sh b/benchmark/compatible/redshift-serverless/run.sh new file mode 100644 index 00000000000..07a47eda1ab --- /dev/null +++ b/benchmark/compatible/redshift-serverless/run.sh @@ -0,0 +1,10 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + echo "$query"; + for i in $(seq 1 $TRIES); do + psql -h "${HOST}" -U awsuser -d dev -p 5439 -t -c 'SET enable_result_cache_for_session = off' -c '\timing' -c "$query" | grep 'Time' + done; +done; diff --git a/benchmark/compatible/redshift/README.md b/benchmark/compatible/redshift/README.md index 325c19f3fe1..e8d56840ae2 100644 --- a/benchmark/compatible/redshift/README.md +++ b/benchmark/compatible/redshift/README.md @@ -47,8 +47,8 @@ Then run the benchmark: export HOST=... ./run.sh 2>&1 | tee log.txt -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' +cat log.txt | grep -oP 'Time: \d+\.\d+ ms|ERROR' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if ($1 == "ERROR") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : ($1 / 1000); if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' ``` `SELECT sum(used * 1048576) FROM stv_node_storage_capacity` diff --git a/benchmark/compatible/redshift/result/4x.ra3.xplus.txt b/benchmark/compatible/redshift/result/4x.ra3.xplus.txt index 92e4f1f3dcf..03676af2cda 100644 --- a/benchmark/compatible/redshift/result/4x.ra3.xplus.txt +++ b/benchmark/compatible/redshift/result/4x.ra3.xplus.txt @@ -4,7 +4,7 @@ Data size: 30 794 579 968 [0.081437,0.022376,0.022491], [2.54868,0.024112,0.024072], [2.62053,0.049948,0.049768], -[2.57591,0.020689,0.020651], +[null,null,null], [0.62714,0.447655,0.43262], [1.14153,1.09479,1.09441], [0.153399,0.053941,0.048224], @@ -29,7 +29,7 @@ Data size: 30 794 579 968 [4.17199,0.469457,0.476302], [6.24746,0.470935,0.482502], [6.00065,2.08332,2.08059], -[0.695955,0.078488,0.077928], +[null,null,null], [22.2567,18.5376,18.3441], [5.37492,0.868068,0.849486], [7.29067,1.06155,1.11209], diff --git a/benchmark/compatible/starrocks/README.md b/benchmark/compatible/starrocks/README.md new file mode 100644 index 00000000000..a04fe13890d --- /dev/null +++ b/benchmark/compatible/starrocks/README.md @@ -0,0 +1,3 @@ +It requires providing an email for downloading. +But then the button on the website does not work and it does not download. +I cannot find the packages anywhere. diff --git a/benchmark/compatible/starrocks/benchmark.sh b/benchmark/compatible/starrocks/benchmark.sh new file mode 100644 index 00000000000..2418d847397 --- /dev/null +++ b/benchmark/compatible/starrocks/benchmark.sh @@ -0,0 +1,3 @@ +#!/bin/bash + +# Install From 7e859b5862492bca74737ddad72e8e34594999e1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Jul 2022 00:26:45 +0200 Subject: [PATCH 115/627] Update Redshift queries --- benchmark/compatible/redshift-serverless/queries.sql | 2 +- benchmark/compatible/redshift/queries.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmark/compatible/redshift-serverless/queries.sql b/benchmark/compatible/redshift-serverless/queries.sql index 31f65fc898d..3a5a4145464 100644 --- a/benchmark/compatible/redshift-serverless/queries.sql +++ b/benchmark/compatible/redshift-serverless/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT REGEXP_REPLACE(Referer, '^https?://(www\.)?([^/]+)/.*$', '\2') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; diff --git a/benchmark/compatible/redshift/queries.sql b/benchmark/compatible/redshift/queries.sql index 31f65fc898d..3a5a4145464 100644 --- a/benchmark/compatible/redshift/queries.sql +++ b/benchmark/compatible/redshift/queries.sql @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT REGEXP_REPLACE(Referer, '^https?://(www\.)?([^/]+)/.*$', '\2') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; From 83444c6a6b770583630c2de153579f9723f09059 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Jul 2022 06:33:25 +0200 Subject: [PATCH 116/627] More systems --- benchmark/compatible/README.md | 3 +- benchmark/compatible/athena/README.md | 9 ++ benchmark/compatible/athena/create.sql | 112 ++++++++++++++++++ benchmark/compatible/aurora-mysql/README.md | 62 ++++++++++ benchmark/compatible/aurora-mysql/create.sql | 109 +++++++++++++++++ benchmark/compatible/aurora-mysql/queries.sql | 43 +++++++ .../compatible/aurora-mysql/results/16acu.txt | 2 + benchmark/compatible/aurora-mysql/run.sh | 9 ++ benchmark/compatible/databend/benchmark.sh | 3 + .../compatible/elasticsearch/benchmark.sh | 27 +++++ benchmark/compatible/locustdb/benchmark.sh | 23 ++++ benchmark/compatible/sqlite/benchmark.sh | 14 +++ benchmark/compatible/sqlite/create.sql | 109 +++++++++++++++++ benchmark/compatible/sqlite/queries.sql | 43 +++++++ .../compatible/sqlite/results/c6a.4xlarge.txt | 2 + benchmark/compatible/sqlite/run.sh | 13 ++ benchmark/compatible/timescaledb/benchmark.sh | 12 +- .../results/c6a.4xlarge.compression.txt | 2 + 18 files changed, 593 insertions(+), 4 deletions(-) create mode 100644 benchmark/compatible/athena/README.md create mode 100644 benchmark/compatible/athena/create.sql create mode 100644 benchmark/compatible/aurora-mysql/README.md create mode 100644 benchmark/compatible/aurora-mysql/create.sql create mode 100644 benchmark/compatible/aurora-mysql/queries.sql create mode 100644 benchmark/compatible/aurora-mysql/results/16acu.txt create mode 100644 benchmark/compatible/aurora-mysql/run.sh create mode 100644 benchmark/compatible/elasticsearch/benchmark.sh create mode 100644 benchmark/compatible/locustdb/benchmark.sh create mode 100644 benchmark/compatible/sqlite/benchmark.sh create mode 100644 benchmark/compatible/sqlite/create.sql create mode 100644 benchmark/compatible/sqlite/queries.sql create mode 100644 benchmark/compatible/sqlite/results/c6a.4xlarge.txt create mode 100644 benchmark/compatible/sqlite/run.sh create mode 100644 benchmark/compatible/timescaledb/results/c6a.4xlarge.compression.txt diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index 6562eea6852..1f9275c16f3 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -26,10 +26,9 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] Hive - [ ] Impala - [ ] Hyper -- [ ] Umbra - [ ] SQLite - [x] Redshift -- [ ] Redshift Serverless +- [x] Redshift Serverless - [ ] Presto/Trino - [ ] Athena - [ ] Bigquery (without publishing) diff --git a/benchmark/compatible/athena/README.md b/benchmark/compatible/athena/README.md new file mode 100644 index 00000000000..f3717a44b1b --- /dev/null +++ b/benchmark/compatible/athena/README.md @@ -0,0 +1,9 @@ +Data Sources -> AWS Data Catalog -> Create Table -> Use S3 bucket data + +Note: Athena does not support files. Only directories: + +``` +aws s3 cp s3://clickhouse-public-datasets/hits_compatible/hits.parquet s3://clickhouse-public-datasets/hits_compatible/athena/hits.parquet +``` + +Go to query editor and run `create.sql`. diff --git a/benchmark/compatible/athena/create.sql b/benchmark/compatible/athena/create.sql new file mode 100644 index 00000000000..ef8e12a9915 --- /dev/null +++ b/benchmark/compatible/athena/create.sql @@ -0,0 +1,112 @@ +CREATE EXTERNAL TABLE IF NOT EXISTS `test`.`hits` ( +`watchid` bigint, +`javaenable` smallint, +`title` string, +`goodevent` smallint, +`eventtime` timestamp, +`eventdate` date, +`counterid` int, +`clientip` int, +`regionid` int, +`userid` bigint, +`counterclass` smallint, +`os` smallint, +`useragent` smallint, +`url` string, +`referer` string, +`isrefresh` smallint, +`referercategoryid` smallint, +`refererregionid` int, +`urlcategoryid` smallint, +`urlregionid` int, +`resolutionwidth` smallint, +`resolutionheight` smallint, +`resolutiondepth` smallint, +`flashmajor` smallint, +`flashminor` smallint, +`flashminor2` string, +`netmajor` smallint, +`netminor` smallint, +`useragentmajor` smallint, +`useragentminor` string, +`cookieenable` smallint, +`javascriptenable` smallint, +`ismobile` smallint, +`mobilephone` smallint, +`mobilephonemodel` string, +`params` string, +`ipnetworkid` int, +`traficsourceid` smallint, +`searchengineid` smallint, +`searchphrase` string, +`advengineid` smallint, +`isartifical` smallint, +`windowclientwidth` smallint, +`windowclientheight` smallint, +`clienttimezone` smallint, +`clienteventtime` timestamp, +`silverlightversion1` smallint, +`silverlightversion2` smallint, +`silverlightversion3` int, +`silverlightversion4` smallint, +`pagecharset` string, +`codeversion` int, +`islink` smallint, +`isdownload` smallint, +`isnotbounce` smallint, +`funiqid` bigint, +`originalurl` string, +`hid` int, +`isoldcounter` smallint, +`isevent` smallint, +`isparameter` smallint, +`dontcounthits` smallint, +`withhash` smallint, +`hitcolor` string, +`localeventtime` timestamp, +`age` smallint, +`sex` smallint, +`income` smallint, +`interests` smallint, +`robotness` smallint, +`remoteip` int, +`windowname` int, +`openername` int, +`historylength` smallint, +`browserlanguage` string, +`browsercountry` string, +`socialnetwork` string, +`socialaction` string, +`httperror` smallint, +`sendtiming` int, +`dnstiming` int, +`connecttiming` int, +`responsestarttiming` int, +`responseendtiming` int, +`fetchtiming` int, +`socialsourcenetworkid` smallint, +`socialsourcepage` string, +`paramprice` bigint, +`paramorderid` string, +`paramcurrency` string, +`paramcurrencyid` smallint, +`openstatservicename` string, +`openstatcampaignid` string, +`openstatadid` string, +`openstatsourceid` string, +`utmsource` string, +`utmmedium` string, +`utmcampaign` string, +`utmcontent` string, +`utmterm` string, +`fromtag` string, +`hasgclid` smallint, +`refererhash` bigint, +`urlhash` bigint, +`clid` int +) +ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' +WITH SERDEPROPERTIES ( +'serialization.format' = '1' +) LOCATION 's3://clickhouse-public-datasets/hits_compatible/athena' +TBLPROPERTIES ('has_encrypted_data'='false'); diff --git a/benchmark/compatible/aurora-mysql/README.md b/benchmark/compatible/aurora-mysql/README.md new file mode 100644 index 00000000000..5cc0eecc2cb --- /dev/null +++ b/benchmark/compatible/aurora-mysql/README.md @@ -0,0 +1,62 @@ +Select Aurora. +Select Aurora for MySQL. +Select the latest version 3.02.0 (compatible with MySQL 8.0.23) +Select Production template. + +Database: database-1 +User name: admin +Master password: vci43A32#1 + +Select serverless. +16 minimum and maximum ACU (32 GB RAM). +Don't create an Aurora replica. +Public access: yes. +Turn off DevOps Guru. + +Creation took around 15 seconds. +But creation of endpoints took longer. + +Find the writer instance endpoint. +Example: database-1.cluster-cnkeohbxcwr1.eu-central-1.rds.amazonaws.com + +``` +sudo apt-get update +sudo apt-get install -y mysql-client +``` + +Find "Security", click on the group in "VPC security groups". +Edit "Inbound rules". Add "Custom TCP", port 3306, from 0.0.0.0/0. + +``` +export HOST="database-1.cluster-cnkeohbxcwr1.eu-central-1.rds.amazonaws.com" +export PASSWORD="vci43A32#1" + +mysql -h "${HOST}" -u admin --password="${PASSWORD}" -e "CREATE DATABASE test" +``` + +Load the data + +``` +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +mysql -h "${HOST}" -u admin --password="${PASSWORD}" test < create.sql + +time mysql --local-infile=1 -h "${HOST}" -u admin --password="${PASSWORD}" test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" +``` + +> 128m7.318s + +Go to "Monitoring", find "\[Billed\] Volume Bytes Used". + +> 83.46 GiB + +``` +./run.sh 2>&1 | tee log.txt + +cat log.txt | + grep -P 'rows? in set|^ERROR' | + sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | + awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' +``` diff --git a/benchmark/compatible/aurora-mysql/create.sql b/benchmark/compatible/aurora-mysql/create.sql new file mode 100644 index 00000000000..1850bffedce --- /dev/null +++ b/benchmark/compatible/aurora-mysql/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/aurora-mysql/queries.sql b/benchmark/compatible/aurora-mysql/queries.sql new file mode 100644 index 00000000000..ea2bde47802 --- /dev/null +++ b/benchmark/compatible/aurora-mysql/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/aurora-mysql/results/16acu.txt b/benchmark/compatible/aurora-mysql/results/16acu.txt new file mode 100644 index 00000000000..8b2954b9d26 --- /dev/null +++ b/benchmark/compatible/aurora-mysql/results/16acu.txt @@ -0,0 +1,2 @@ +Load time: 128m7.318s +Data size: 83.46 GiB diff --git a/benchmark/compatible/aurora-mysql/run.sh b/benchmark/compatible/aurora-mysql/run.sh new file mode 100644 index 00000000000..c6e2bedd27b --- /dev/null +++ b/benchmark/compatible/aurora-mysql/run.sh @@ -0,0 +1,9 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + for i in $(seq 1 $TRIES); do + mysql -h "${HOST}" -u admin --password="${PASSWORD}" test -vvv -e "${query}" + done; +done; diff --git a/benchmark/compatible/databend/benchmark.sh b/benchmark/compatible/databend/benchmark.sh index ff9f141abba..fc189e8ef84 100644 --- a/benchmark/compatible/databend/benchmark.sh +++ b/benchmark/compatible/databend/benchmark.sh @@ -88,3 +88,6 @@ gzip -d hits.csv.gz # and fails after 7 minutes 38 seconds without loading any data: # Code: 4000, displayText = invalid data (Expected to have terminated string literal.) (while in processor thread 5). # the diagnostics is terrible. + +head -n 90000000 hits.tsv > hits90m.tsv +curl -XPUT 'http://root:@127.0.0.1:8000/v1/streaming_load' -H 'insert_sql: insert into hits format TSV' -H 'skip_header: 0' -H 'field_delimiter: \t' -H 'record_delimiter: \n' -F 'upload=@"./hits90m.tsv"' diff --git a/benchmark/compatible/elasticsearch/benchmark.sh b/benchmark/compatible/elasticsearch/benchmark.sh new file mode 100644 index 00000000000..f24f6144742 --- /dev/null +++ b/benchmark/compatible/elasticsearch/benchmark.sh @@ -0,0 +1,27 @@ +#!/bin/bash + +wget -qO - https://artifacts.elastic.co/GPG-KEY-elasticsearch | sudo gpg --dearmor -o /usr/share/keyrings/elasticsearch-keyring.gpg +sudo apt-get update && sudo apt-get install -y apt-transport-https +echo "deb [signed-by=/usr/share/keyrings/elasticsearch-keyring.gpg] https://artifacts.elastic.co/packages/8.x/apt stable main" | sudo tee /etc/apt/sources.list.d/elastic-8.x.list +sudo apt-get update && sudo apt-get install -y elasticsearch + +sudo systemctl start elasticsearch.service +sudo /usr/share/elasticsearch/bin/elasticsearch-reset-password -u elastic + +# Example: +# User: elastic +# Password: C0Qq9kNYMUunKTXMDOUZ + +export PASSWORD='...' + +curl -k -XGET 'https://localhost:9200' -u "elastic:${PASSWORD}" + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.json.gz' +gzip -d hits.json.gz + +# Prevent 'curl' from OOM. + +split -l 1000000000 hits.json hits_ +for table in hits_*; do mv ${table} ${table}.json; done + +time for table in hits_*; do curl -k -H "Transfer-Encoding: chunked" -XPOST -u "elastic:${PASSWORD}" 'https://localhost:9200/_bulk' -T ${table}; done diff --git a/benchmark/compatible/locustdb/benchmark.sh b/benchmark/compatible/locustdb/benchmark.sh new file mode 100644 index 00000000000..cf2117a4b5f --- /dev/null +++ b/benchmark/compatible/locustdb/benchmark.sh @@ -0,0 +1,23 @@ +#!/bin/bash + +# https://rustup.rs/ +curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh +source $HOME/.cargo/env + +sudo apt-get update +sudo apt-get install -y git + +git clone https://github.com/cswinter/LocustDB.git +cd LocustDB + +sudo apt-get install -y g++ capnproto libclang-14-dev + +cargo build --features "enable_rocksdb" --features "enable_lz4" --release + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +gzip -d hits.csv.gz + +target/release/repl --load hits.csv --db-path db + +# Loaded data in 920s. +# Table `default` (99997496 rows, 15.0GiB) diff --git a/benchmark/compatible/sqlite/benchmark.sh b/benchmark/compatible/sqlite/benchmark.sh new file mode 100644 index 00000000000..a2bd9f850a9 --- /dev/null +++ b/benchmark/compatible/sqlite/benchmark.sh @@ -0,0 +1,14 @@ +#!/bin/bash + +sudo apt-get update +sudo apt-get install -y sqlite3 + +sqlite3 mydb < create.sql + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +gzip -d hits.csv.gz + +time sqlite3 mydb '.import --csv hits.csv hits' +wc -c mydb + +./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/sqlite/create.sql b/benchmark/compatible/sqlite/create.sql new file mode 100644 index 00000000000..1850bffedce --- /dev/null +++ b/benchmark/compatible/sqlite/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/sqlite/queries.sql b/benchmark/compatible/sqlite/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/sqlite/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/sqlite/results/c6a.4xlarge.txt b/benchmark/compatible/sqlite/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..e7fcbe151f1 --- /dev/null +++ b/benchmark/compatible/sqlite/results/c6a.4xlarge.txt @@ -0,0 +1,2 @@ +Load time: 43m28.444s +Data size: 75 776 589 824 bytes diff --git a/benchmark/compatible/sqlite/run.sh b/benchmark/compatible/sqlite/run.sh new file mode 100644 index 00000000000..e3eccc1cc0b --- /dev/null +++ b/benchmark/compatible/sqlite/run.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + echo "$query"; + for i in $(seq 1 $TRIES); do + time sqlite3 mydb <<< "${query}" + done; +done; diff --git a/benchmark/compatible/timescaledb/benchmark.sh b/benchmark/compatible/timescaledb/benchmark.sh index a36f351e76c..2330deaf1b4 100755 --- a/benchmark/compatible/timescaledb/benchmark.sh +++ b/benchmark/compatible/timescaledb/benchmark.sh @@ -3,12 +3,12 @@ # Install sudo apt-get update -sudo apt install gnupg postgresql-common apt-transport-https lsb-release wget +sudo apt-get install -y gnupg postgresql-common apt-transport-https lsb-release wget sudo /usr/share/postgresql-common/pgdg/apt.postgresql.org.sh sudo bash -c 'echo "deb https://packagecloud.io/timescale/timescaledb/ubuntu/ $(lsb_release -c -s) main" > /etc/apt/sources.list.d/timescaledb.list' wget --quiet -O - https://packagecloud.io/timescale/timescaledb/gpgkey | sudo apt-key add - sudo apt-get update -sudo apt install timescaledb-2-postgresql-14 +sudo apt-get install -y timescaledb-2-postgresql-14 sudo bash -c "echo \"shared_preload_libraries = 'timescaledb'\" >> /etc/postgresql/14/main/postgresql.conf" sudo systemctl restart postgresql @@ -31,6 +31,14 @@ sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" # 1619875.288 ms (26:59.875) +# See https://github.com/timescale/timescaledb/issues/4473#issuecomment-1167095245 +# https://docs.timescale.com/timescaledb/latest/how-to-guides/compression/manually-compress-chunks/#compress-chunks-manually +# Omit this step to proceed without compression. + +time sudo -u postgres psql test -c "SELECT compress_chunk(i, if_not_compressed => true) FROM show_chunks('hits') i" + +# 49m45.120s + ./run.sh 2>&1 | tee log.txt sudo du -bcs /var/lib/postgresql/14/main/ diff --git a/benchmark/compatible/timescaledb/results/c6a.4xlarge.compression.txt b/benchmark/compatible/timescaledb/results/c6a.4xlarge.compression.txt new file mode 100644 index 00000000000..b83a782eb7f --- /dev/null +++ b/benchmark/compatible/timescaledb/results/c6a.4xlarge.compression.txt @@ -0,0 +1,2 @@ +Load time: 4605 seconds +Data size: 20 333 747 165 bytes From 1e5657e8ae7ef55b61c1fa8e8989197a93fa2ef7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sun, 3 Jul 2022 01:13:57 -0400 Subject: [PATCH 117/627] Add SimpleCheck SimpleCheck will pass if either both StyleCheck and FastTest passed or pull request is labeled 'allow simple fail' --- .github/workflows/pull_request.yml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 9cd8fd6f49d..8e971186ca3 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -179,6 +179,17 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + SimpleCheck: + needs: [StyleCheck, FastTest] + runs-on: ubuntu-latest + if: ${{ always() }} + steps: + - name: Allow simple fail + if: ${{ contains(github.event.pull_request.labels.*.name, 'allow simple fail') }} + run: exit 0 + - name: Simple failed + if: ${{ (needs.StyleCheck.result != 'success' || needs.FastTest.result != 'success') && !contains(github.event.pull_request.labels.*.name, 'allow simple fail') }} + run: exit 1 CompatibilityCheck: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] From 92e0ee0b6fdea7e97675ef1fa44144f60391a4ad Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 2 Jul 2022 18:26:08 +0200 Subject: [PATCH 118/627] More detailed error messages. --- src/Access/AccessBackup.cpp | 6 +- src/Access/AccessBackup.h | 3 +- src/Access/IAccessStorage.cpp | 3 + src/Access/ReplicatedAccessStorage.cpp | 3 + src/Backups/BackupEntriesCollector.cpp | 166 ++++++++------ src/Backups/BackupEntriesCollector.h | 3 + src/Backups/RestorerFromBackup.cpp | 275 +++++++++++++++-------- src/Backups/RestorerFromBackup.h | 12 +- src/Parsers/ASTBackupQuery.h | 18 +- src/Parsers/ParserBackupQuery.h | 8 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- src/Storages/StorageLog.cpp | 12 +- src/Storages/StorageMemory.cpp | 12 +- src/Storages/StorageStripeLog.cpp | 12 +- 14 files changed, 333 insertions(+), 204 deletions(-) diff --git a/src/Access/AccessBackup.cpp b/src/Access/AccessBackup.cpp index bd1344a6f14..d521c0a49be 100644 --- a/src/Access/AccessBackup.cpp +++ b/src/Access/AccessBackup.cpp @@ -321,7 +321,7 @@ AccessRestorerFromBackup::AccessRestorerFromBackup( AccessRestorerFromBackup::~AccessRestorerFromBackup() = default; -void AccessRestorerFromBackup::addDataPath(const String & data_path, const QualifiedTableName & table_name_for_logs) +void AccessRestorerFromBackup::addDataPath(const String & data_path) { if (!data_paths.emplace(data_path).second) return; @@ -334,8 +334,8 @@ void AccessRestorerFromBackup::addDataPath(const String & data_path, const Quali for (const String & filename : filenames) { if (!filename.starts_with("access") || !filename.ends_with(".txt")) - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File name {} doesn't match the wildcard \"access*.txt\"", - table_name_for_logs.getFullName(), String{data_path_in_backup_fs / filename}); + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File name {} doesn't match the wildcard \"access*.txt\"", + String{data_path_in_backup_fs / filename}); } ::sort(filenames.begin(), filenames.end()); diff --git a/src/Access/AccessBackup.h b/src/Access/AccessBackup.h index 74f889e2c00..99f22b6c6a8 100644 --- a/src/Access/AccessBackup.h +++ b/src/Access/AccessBackup.h @@ -17,7 +17,6 @@ using BackupPtr = std::shared_ptr; class IBackupEntry; using BackupEntryPtr = std::shared_ptr; struct RestoreSettings; -struct QualifiedTableName; /// Makes a backup of access entities of a specified type. @@ -36,7 +35,7 @@ public: ~AccessRestorerFromBackup(); /// Adds a data path to loads access entities from. - void addDataPath(const String & data_path, const QualifiedTableName & table_name_for_logs); + void addDataPath(const String & data_path); /// Checks that the current user can do restoring. AccessRightsElements getRequiredAccess() const; diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index fa9c78816c7..2be99dfb38b 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -531,6 +531,9 @@ void IAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, c auto entities = readAllWithIDs(type); boost::range::remove_erase_if(entities, [](const std::pair & x) { return !x.second->isBackupAllowed(); }); + if (entities.empty()) + return; + auto backup_entry = makeBackupEntryForAccess( entities, data_path_in_backup, diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index f6c8d0a7153..3c750c0ff4b 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -627,6 +627,9 @@ void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_col auto entities = readAllWithIDs(type); boost::range::remove_erase_if(entities, [](const std::pair & x) { return !x.second->isBackupAllowed(); }); + if (entities.empty()) + return; + auto backup_entry_with_path = makeBackupEntryForAccess( entities, data_path_in_backup, diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index d5ed9e0da2b..90f9486a0df 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -436,46 +436,7 @@ void BackupEntriesCollector::gatherTablesMetadata() table_infos.clear(); for (const auto & [database_name, database_info] : database_infos) { - const auto & database = database_info.database; - bool is_temporary_database = (database_name == DatabaseCatalog::TEMPORARY_DATABASE); - - auto filter_by_table_name = [database_info = &database_info](const String & table_name) - { - /// We skip inner tables of materialized views. - if (table_name.starts_with(".inner_id.")) - return false; - - if (database_info->tables.contains(table_name)) - return true; - - if (database_info->all_tables) - return !database_info->except_table_names.contains(table_name); - - return false; - }; - - auto db_tables = database->getTablesForBackup(filter_by_table_name, context); - - std::unordered_set found_table_names; - for (const auto & db_table : db_tables) - { - const auto & create_table_query = db_table.first; - const auto & create = create_table_query->as(); - found_table_names.emplace(create.getTable()); - - if (is_temporary_database && !create.temporary) - throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a non-temporary create query for {}", tableNameWithTypeToString(database_name, create.getTable(), false)); - - if (!is_temporary_database && (create.getDatabase() != database_name)) - throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a create query with unexpected database name {} for {}", backQuoteIfNeed(create.getDatabase()), tableNameWithTypeToString(database_name, create.getTable(), false)); - } - - /// Check that all tables were found. - for (const auto & [table_name, table_info] : database_info.tables) - { - if (table_info.throw_if_table_not_found && !found_table_names.contains(table_name)) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "{} not found", tableNameWithTypeToString(database_name, table_name, true)); - } + std::vector> db_tables = findTablesInDatabase(database_name); for (const auto & db_table : db_tables) { @@ -501,7 +462,7 @@ void BackupEntriesCollector::gatherTablesMetadata() /// Add information to `table_infos`. auto & res_table_info = table_infos[QualifiedTableName{database_name, table_name}]; - res_table_info.database = database; + res_table_info.database = database_info.database; res_table_info.storage = storage; res_table_info.create_table_query = create_table_query; res_table_info.metadata_path_in_backup = metadata_path_in_backup; @@ -528,6 +489,67 @@ void BackupEntriesCollector::gatherTablesMetadata() } } +std::vector> BackupEntriesCollector::findTablesInDatabase(const String & database_name) const +{ + const auto & database_info = database_infos.at(database_name); + const auto & database = database_info.database; + + auto filter_by_table_name = [database_info = &database_info](const String & table_name) + { + /// We skip inner tables of materialized views. + if (table_name.starts_with(".inner_id.")) + return false; + + if (database_info->tables.contains(table_name)) + return true; + + if (database_info->all_tables) + return !database_info->except_table_names.contains(table_name); + + return false; + }; + + std::vector> db_tables; + + try + { + db_tables = database->getTablesForBackup(filter_by_table_name, context); + } + catch (Exception & e) + { + e.addMessage("While collecting tables for backup in database {}", backQuoteIfNeed(database_name)); + throw; + } + + std::unordered_set found_table_names; + for (const auto & db_table : db_tables) + { + const auto & create_table_query = db_table.first; + const auto & create = create_table_query->as(); + found_table_names.emplace(create.getTable()); + + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + { + if (!create.temporary) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a non-temporary create query for {}", tableNameWithTypeToString(database_name, create.getTable(), false)); + } + else + { + if (create.getDatabase() != database_name) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a create query with unexpected database name {} for {}", backQuoteIfNeed(create.getDatabase()), tableNameWithTypeToString(database_name, create.getTable(), false)); + } + } + + /// Check that all tables were found. + for (const auto & [table_name, table_info] : database_info.tables) + { + if (table_info.throw_if_table_not_found && !found_table_names.contains(table_name)) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "{} was not found", tableNameWithTypeToString(database_name, table_name, true)); + } + + return db_tables; +} + void BackupEntriesCollector::lockTablesForReading() { for (auto & [table_name, table_info] : table_infos) @@ -544,7 +566,7 @@ void BackupEntriesCollector::lockTablesForReading() { if (e.code() != ErrorCodes::TABLE_IS_DROPPED) throw; - throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "{} is dropped", tableNameWithTypeToString(table_name.database, table_name.table, true)); + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "{} was dropped during scanning", tableNameWithTypeToString(table_name.database, table_name.table, true)); } } } @@ -648,7 +670,7 @@ void BackupEntriesCollector::makeBackupEntriesForDatabasesDefs() if (!database_info.create_database_query) continue; /// We store CREATE DATABASE queries only if there was BACKUP DATABASE specified. - LOG_TRACE(log, "Adding definition of database {}", backQuoteIfNeed(database_name)); + LOG_TRACE(log, "Adding the definition of database {} to backup", backQuoteIfNeed(database_name)); ASTPtr new_create_query = database_info.create_database_query; adjustCreateQueryForBackup(new_create_query, context->getGlobalContext(), nullptr); @@ -664,7 +686,7 @@ void BackupEntriesCollector::makeBackupEntriesForTablesDefs() { for (auto & [table_name, table_info] : table_infos) { - LOG_TRACE(log, "Adding definition of {}", tableNameWithTypeToString(table_name.database, table_name.table, false)); + LOG_TRACE(log, "Adding the definition of {} to backup", tableNameWithTypeToString(table_name.database, table_name.table, false)); ASTPtr new_create_query = table_info.create_table_query; adjustCreateQueryForBackup(new_create_query, context->getGlobalContext(), &table_info.replicated_table_shared_id); @@ -680,24 +702,40 @@ void BackupEntriesCollector::makeBackupEntriesForTablesData() if (backup_settings.structure_only) return; - for (const auto & [table_name, table_info] : table_infos) + for (const auto & table_name : table_infos | boost::adaptors::map_keys) + makeBackupEntriesForTableData(table_name); +} + +void BackupEntriesCollector::makeBackupEntriesForTableData(const QualifiedTableName & table_name) +{ + if (backup_settings.structure_only) + return; + + const auto & table_info = table_infos.at(table_name); + const auto & storage = table_info.storage; + const auto & data_path_in_backup = table_info.data_path_in_backup; + + if (!storage) { - const auto & storage = table_info.storage; - const auto & data_path_in_backup = table_info.data_path_in_backup; - if (storage) - { - LOG_TRACE(log, "Adding data of {}", tableNameWithTypeToString(table_name.database, table_name.table, false)); - storage->backupData(*this, data_path_in_backup, table_info.partitions); - } - else - { - /// Storage == null means this storage exists on other replicas but it has not been created on this replica yet. - /// If this table is replicated in this case we call IBackupCoordination::addReplicatedDataPath() which will cause - /// other replicas to fill the storage's data in the backup. - /// If this table is not replicated we'll do nothing leaving the storage's data empty in the backup. - if (table_info.replicated_table_shared_id) - backup_coordination->addReplicatedDataPath(*table_info.replicated_table_shared_id, data_path_in_backup); - } + /// If storage == null that means this storage exists on other replicas but it has not been created on this replica yet. + /// If this table is replicated in this case we call IBackupCoordination::addReplicatedDataPath() which will cause + /// other replicas to fill the storage's data in the backup. + /// If this table is not replicated we'll do nothing leaving the storage's data empty in the backup. + if (table_info.replicated_table_shared_id) + backup_coordination->addReplicatedDataPath(*table_info.replicated_table_shared_id, data_path_in_backup); + return; + } + + LOG_TRACE(log, "Collecting data of {} for backup", tableNameWithTypeToString(table_name.database, table_name.table, false)); + + try + { + storage->backupData(*this, data_path_in_backup, table_info.partitions); + } + catch (Exception & e) + { + e.addMessage("While collecting data of {} for backup", tableNameWithTypeToString(table_name.database, table_name.table, false)); + throw; } } @@ -716,21 +754,21 @@ void BackupEntriesCollector::addBackupEntry(const std::pair task) { if (current_status == kWritingBackupStatus) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding post tasks is not allowed"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of post tasks is not allowed"); post_tasks.push(std::move(task)); } diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index 9a653ee7e4d..b83846047c0 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -75,12 +75,15 @@ private: const std::set & except_table_names); void gatherTablesMetadata(); + std::vector> findTablesInDatabase(const String & database_name) const; void lockTablesForReading(); bool compareWithPrevious(std::optional & inconsistency_error); void makeBackupEntriesForDatabasesDefs(); void makeBackupEntriesForTablesDefs(); void makeBackupEntriesForTablesData(); + void makeBackupEntriesForTableData(const QualifiedTableName & table_name); + void runPostTasks(); Strings setStatus(const String & new_status, const String & message = ""); diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 5b211bc50a8..24fe2001f1f 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -381,11 +381,23 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name insertAtEnd(*res_table_info.partitions, *partitions); } + /// Special handling for ACL-related system tables. if (!restore_settings.structure_only && isSystemAccessTableName(table_name)) { if (!access_restorer) access_restorer = std::make_unique(backup, restore_settings); - access_restorer->addDataPath(data_path_in_backup, table_name); + + try + { + /// addDataPath() will parse access*.txt files and extract access entities from them. + /// We need to do that early because we need those access entities to check access. + access_restorer->addDataPath(data_path_in_backup); + } + catch (Exception & e) + { + e.addMessage("While parsing data of {} from backup", tableNameWithTypeToString(table_name.database, table_name.table, false)); + throw; + } } } @@ -563,34 +575,58 @@ void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const required_access = AccessRights{required_access}.getElements(); context->checkAccess(required_access); - } void RestorerFromBackup::createDatabases() { - for (const auto & [database_name, database_info] : database_infos) + for (const auto & database_name : database_infos | boost::adaptors::map_keys) { - bool need_create_database = (restore_settings.create_database != RestoreDatabaseCreationMode::kMustExist); - if (database_info.is_predefined_database) - need_create_database = false; /// Predefined databases always exist. + createDatabase(database_name); + checkDatabase(database_name); + } +} - if (need_create_database) - { - /// Execute CREATE DATABASE query. - auto create_database_query = database_info.create_database_query; - if (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists) - { - create_database_query = create_database_query->clone(); - create_database_query->as().if_not_exists = true; - } - LOG_TRACE(log, "Creating database {}: {}", backQuoteIfNeed(database_name), serializeAST(*create_database_query)); - InterpreterCreateQuery interpreter{create_database_query, context}; - interpreter.setInternal(true); - interpreter.execute(); - } +void RestorerFromBackup::createDatabase(const String & database_name) const +{ + if (restore_settings.create_database == RestoreDatabaseCreationMode::kMustExist) + return; + /// Predefined databases always exist. + const auto & database_info = database_infos.at(database_name); + if (database_info.is_predefined_database) + return; + + auto create_database_query = database_info.create_database_query; + if (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists) + { + create_database_query = create_database_query->clone(); + create_database_query->as().if_not_exists = true; + } + + LOG_TRACE(log, "Creating database {}: {}", backQuoteIfNeed(database_name), serializeAST(*create_database_query)); + + try + { + /// Execute CREATE DATABASE query. + InterpreterCreateQuery interpreter{create_database_query, context}; + interpreter.setInternal(true); + interpreter.execute(); + } + catch (Exception & e) + { + e.addMessage("While creating database {}", backQuoteIfNeed(database_name)); + throw; + } +} + +void RestorerFromBackup::checkDatabase(const String & database_name) +{ + auto & database_info = database_infos.at(database_name); + try + { DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name); - + database_info.database = database; + if (!restore_settings.allow_different_database_def && !database_info.is_predefined_database) { /// Check that the database's definition is the same as expected. @@ -601,14 +637,18 @@ void RestorerFromBackup::createDatabases() { throw Exception( ErrorCodes::CANNOT_RESTORE_DATABASE, - "The database {} has a different definition: {} " + "The database has a different definition: {} " "comparing to its definition in the backup: {}", - backQuoteIfNeed(database_name), serializeAST(*create_database_query), serializeAST(*expected_create_query)); } } } + catch (Exception & e) + { + e.addMessage("While checking database {}", backQuoteIfNeed(database_name)); + throw; + } } void RestorerFromBackup::createTables() @@ -622,82 +662,123 @@ void RestorerFromBackup::createTables() for (const auto & table_name : tables_to_create) { - auto & table_info = table_infos.at(table_name); + createTable(table_name); + checkTable(table_name); + insertDataToTable(table_name); + } + } +} - DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_name.database); +void RestorerFromBackup::createTable(const QualifiedTableName & table_name) +{ + if (restore_settings.create_table == RestoreTableCreationMode::kMustExist) + return; - bool need_create_table = (restore_settings.create_table != RestoreTableCreationMode::kMustExist); - if (table_info.is_predefined_table) - need_create_table = false; /// Predefined tables always exist. + /// Predefined tables always exist. + auto & table_info = table_infos.at(table_name); + if (table_info.is_predefined_table) + return; - if (need_create_table) + auto create_table_query = table_info.create_table_query; + if (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists) + { + create_table_query = create_table_query->clone(); + create_table_query->as().if_not_exists = true; + } + + LOG_TRACE( + log, "Creating {}: {}", tableNameWithTypeToString(table_name.database, table_name.table, false), serializeAST(*create_table_query)); + + try + { + DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_name.database); + table_info.database = database; + + /// Execute CREATE TABLE query (we call IDatabase::createTableRestoredFromBackup() to allow the database to do some + /// database-specific things). + database->createTableRestoredFromBackup( + create_table_query, + context, + restore_coordination, + std::chrono::duration_cast(create_table_timeout).count()); + } + catch (Exception & e) + { + e.addMessage("While creating {}", tableNameWithTypeToString(table_name.database, table_name.table, false)); + throw; + } +} + +void RestorerFromBackup::checkTable(const QualifiedTableName & table_name) +{ + auto & table_info = table_infos.at(table_name); + auto database = table_info.database; + + try + { + if (!database) + { + database = DatabaseCatalog::instance().getDatabase(table_name.database); + table_info.database = database; + } + + auto resolved_id = (table_name.database == DatabaseCatalog::TEMPORARY_DATABASE) + ? context->resolveStorageID(StorageID{"", table_name.table}, Context::ResolveExternal) + : context->resolveStorageID(StorageID{table_name.database, table_name.table}, Context::ResolveGlobal); + + StoragePtr storage = database->getTable(resolved_id.table_name, context); + table_info.storage = storage; + table_info.table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); + + if (!restore_settings.allow_different_table_def && !table_info.is_predefined_table) + { + ASTPtr create_table_query = database->getCreateTableQuery(resolved_id.table_name, context); + adjustCreateQueryForBackup(create_table_query, context->getGlobalContext(), nullptr); + ASTPtr expected_create_query = table_info.create_table_query; + if (serializeAST(*create_table_query) != serializeAST(*expected_create_query)) { - auto create_table_query = table_info.create_table_query; - if (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists) - { - create_table_query = create_table_query->clone(); - create_table_query->as().if_not_exists = true; - } - - LOG_TRACE( - log, - "Creating {}: {}", - tableNameWithTypeToString(table_name.database, table_name.table, false), - serializeAST(*create_table_query)); - - /// Execute CREATE TABLE query (we call IDatabase::createTableRestoredFromBackup() to allow the database to do some - /// database-specific things). - database->createTableRestoredFromBackup( - create_table_query, - context, - restore_coordination, - std::chrono::duration_cast(create_table_timeout).count()); - } - - table_info.created = true; - - auto resolved_id = (table_name.database == DatabaseCatalog::TEMPORARY_DATABASE) - ? context->resolveStorageID(StorageID{"", table_name.table}, Context::ResolveExternal) - : context->resolveStorageID(StorageID{table_name.database, table_name.table}, Context::ResolveGlobal); - - auto storage = database->getTable(resolved_id.table_name, context); - table_info.storage = storage; - table_info.table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); - - if (!restore_settings.allow_different_table_def && !table_info.is_predefined_table) - { - ASTPtr create_table_query = database->getCreateTableQuery(resolved_id.table_name, context); - adjustCreateQueryForBackup(create_table_query, context->getGlobalContext(), nullptr); - ASTPtr expected_create_query = table_info.create_table_query; - if (serializeAST(*create_table_query) != serializeAST(*expected_create_query)) - { - throw Exception( - ErrorCodes::CANNOT_RESTORE_TABLE, - "{} has a different definition: {} " - "comparing to its definition in the backup: {}", - tableNameWithTypeToString(table_name.database, table_name.table, true), - serializeAST(*create_table_query), - serializeAST(*expected_create_query)); - } - } - - if (!restore_settings.structure_only) - { - const auto & data_path_in_backup = table_info.data_path_in_backup; - const auto & partitions = table_info.partitions; - if (partitions && !storage->supportsBackupPartition()) - { - throw Exception( - ErrorCodes::CANNOT_RESTORE_TABLE, - "Table engine {} doesn't support partitions, cannot restore {}", - storage->getName(), - tableNameWithTypeToString(table_name.database, table_name.table, false)); - } - - storage->restoreDataFromBackup(*this, data_path_in_backup, partitions); + throw Exception( + ErrorCodes::CANNOT_RESTORE_TABLE, + "The table has a different definition: {} " + "comparing to its definition in the backup: {}", + serializeAST(*create_table_query), + serializeAST(*expected_create_query)); } } } + catch (Exception & e) + { + e.addMessage("While checking {}", tableNameWithTypeToString(table_name.database, table_name.table, false)); + throw; + } +} + +void RestorerFromBackup::insertDataToTable(const QualifiedTableName & table_name) +{ + if (restore_settings.structure_only) + return; + + auto & table_info = table_infos.at(table_name); + auto storage = table_info.storage; + + try + { + const auto & data_path_in_backup = table_info.data_path_in_backup; + const auto & partitions = table_info.partitions; + if (partitions && !storage->supportsBackupPartition()) + { + throw Exception( + ErrorCodes::CANNOT_RESTORE_TABLE, + "Table engine {} doesn't support partitions", + storage->getName()); + } + storage->restoreDataFromBackup(*this, data_path_in_backup, partitions); + } + catch (Exception & e) + { + e.addMessage("While restoring data of {}", tableNameWithTypeToString(table_name.database, table_name.table, false)); + throw; + } } /// Returns the list of tables without dependencies or those which dependencies have been created before. @@ -708,7 +789,7 @@ std::vector RestorerFromBackup::findTablesWithoutDependencie for (const auto & [key, table_info] : table_infos) { - if (table_info.created) + if (table_info.storage) continue; /// Found a table which is not created yet. @@ -719,7 +800,7 @@ std::vector RestorerFromBackup::findTablesWithoutDependencie for (const auto & dependency : table_info.dependencies) { auto it = table_infos.find(dependency); - if ((it != table_infos.end()) && !it->second.created) + if ((it != table_infos.end()) && !it->second.storage) { all_dependencies_met = false; break; @@ -740,7 +821,7 @@ std::vector RestorerFromBackup::findTablesWithoutDependencie std::vector tables_with_cyclic_dependencies; for (const auto & [key, table_info] : table_infos) { - if (!table_info.created) + if (!table_info.storage) tables_with_cyclic_dependencies.push_back(key); } @@ -759,14 +840,14 @@ std::vector RestorerFromBackup::findTablesWithoutDependencie void RestorerFromBackup::addDataRestoreTask(DataRestoreTask && new_task) { if (current_status == kInsertingDataToTablesStatus) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding data-restoring tasks is not allowed"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of data-restoring tasks is not allowed"); data_restore_tasks.push_back(std::move(new_task)); } void RestorerFromBackup::addDataRestoreTasks(DataRestoreTasks && new_tasks) { if (current_status == kInsertingDataToTablesStatus) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding data-restoring tasks is not allowed"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of data-restoring tasks is not allowed"); insertAtEnd(data_restore_tasks, std::move(new_tasks)); } diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index ae2f0c76832..a53477f6e6d 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -15,10 +15,13 @@ class IBackup; using BackupPtr = std::shared_ptr; class IRestoreCoordination; struct StorageID; +class IDatabase; +using DatabasePtr = std::shared_ptr; class AccessRestorerFromBackup; struct IAccessEntity; using AccessEntityPtr = std::shared_ptr; + /// Restores the definition of databases and tables and prepares tasks to restore the data of the tables. class RestorerFromBackup : private boost::noncopyable { @@ -87,7 +90,13 @@ private: void checkAccessForObjectsFoundInBackup() const; void createDatabases(); + void createDatabase(const String & database_name) const; + void checkDatabase(const String & database_name); + void createTables(); + void createTable(const QualifiedTableName & table_name); + void checkTable(const QualifiedTableName & table_name); + void insertDataToTable(const QualifiedTableName & table_name); DataRestoreTasks getDataRestoreTasks(); @@ -97,6 +106,7 @@ private: { ASTPtr create_database_query; bool is_predefined_database = false; + DatabasePtr database; }; struct TableInfo @@ -107,7 +117,7 @@ private: bool has_data = false; std::filesystem::path data_path_in_backup; std::optional partitions; - bool created = false; + DatabasePtr database; StoragePtr storage; TableLockHolder table_lock; }; diff --git a/src/Parsers/ASTBackupQuery.h b/src/Parsers/ASTBackupQuery.h index 491f192042b..dd094b77aae 100644 --- a/src/Parsers/ASTBackupQuery.h +++ b/src/Parsers/ASTBackupQuery.h @@ -14,19 +14,21 @@ using DatabaseAndTableName = std::pair; * DICTIONARY [db.]dictionary_name [AS [db.]dictionary_name_in_backup] | * DATABASE database_name [AS database_name_in_backup] [EXCEPT TABLES ...] | * TEMPORARY TABLE table_name [AS table_name_in_backup] | - * ALL DATABASES [EXCEPT ...] } [,...] + * ALL [EXCEPT {TABLES|DATABASES}...] } [,...] * [ON CLUSTER 'cluster_name'] * TO { File('path/') | - * Disk('disk_name', 'path/') - * [SETTINGS base_backup = {File(...) | Disk(...)}] + * Disk('disk_name', 'path/') } + * [SETTINGS ...] * * RESTORE { TABLE [db.]table_name_in_backup [AS [db.]table_name] [PARTITION[S] partition_expr [,...]] | - * DICTIONARY [db.]dictionary_name_in_backup [AS [db.]dictionary_name] | - * DATABASE database_name_in_backup [AS database_name] [EXCEPT TABLES ...] | - * TEMPORARY TABLE table_name_in_backup [AS table_name] | - * ALL DATABASES [EXCEPT ...] } [,...] + * DICTIONARY [db.]dictionary_name_in_backup [AS [db.]dictionary_name] | + * DATABASE database_name_in_backup [AS database_name] [EXCEPT TABLES ...] | + * TEMPORARY TABLE table_name_in_backup [AS table_name] | + * ALL [EXCEPT {TABLES|DATABASES} ...] } [,...] * [ON CLUSTER 'cluster_name'] - * FROM {File(...) | Disk(...)} + * FROM { File('path/') | + * Disk('disk_name', 'path/') } + * [SETTINGS ...] * * Notes: * RESTORE doesn't drop any data, it either creates a table or appends an existing table with restored data. diff --git a/src/Parsers/ParserBackupQuery.h b/src/Parsers/ParserBackupQuery.h index 4c74666382c..7b715ddfd75 100644 --- a/src/Parsers/ParserBackupQuery.h +++ b/src/Parsers/ParserBackupQuery.h @@ -13,8 +13,8 @@ namespace DB * ALL [EXCEPT {TABLES|DATABASES}...] } [,...] * [ON CLUSTER 'cluster_name'] * TO { File('path/') | - * Disk('disk_name', 'path/') - * [SETTINGS base_backup = {File(...) | Disk(...)}] + * Disk('disk_name', 'path/') } + * [SETTINGS ...] * * RESTORE { TABLE [db.]table_name_in_backup [AS [db.]table_name] [PARTITION[S] partition_expr [,...]] | * DICTIONARY [db.]dictionary_name_in_backup [AS [db.]dictionary_name] | @@ -22,7 +22,9 @@ namespace DB * TEMPORARY TABLE table_name_in_backup [AS table_name] | * ALL [EXCEPT {TABLES|DATABASES} ...] } [,...] * [ON CLUSTER 'cluster_name'] - * FROM {File(...) | Disk(...)} + * FROM { File('path/') | + * Disk('disk_name', 'path/') } + * [SETTINGS ...] */ class ParserBackupQuery : public IParserBase { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d4b2c8d177e..3c30bee7889 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4094,8 +4094,8 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version); if (!part_info) { - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File name {} doesn't look like the name of a part", - getStorageID().getFullTableName(), String{data_path_in_backup_fs / part_name}); + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File name {} is not the name of a part", + String{data_path_in_backup_fs / part_name}); } if (partition_ids && !partition_ids->contains(part_info->partition_id)) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index ac6ead54016..61c7be21f61 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -1023,10 +1023,8 @@ void StorageLog::restoreDataImpl(const BackupPtr & backup, const String & data_p { String file_path_in_backup = data_path_in_backup_fs / fileName(data_file.path); if (!backup->fileExists(file_path_in_backup)) - { - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", - getStorageID().getFullTableName(), file_path_in_backup); - } + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", file_path_in_backup); + auto backup_entry = backup->readFile(file_path_in_backup); auto in = backup_entry->getReadBuffer(); auto out = disk->writeFile(data_file.path, max_compress_block_size, WriteMode::Append); @@ -1039,10 +1037,8 @@ void StorageLog::restoreDataImpl(const BackupPtr & backup, const String & data_p size_t num_extra_marks = 0; String file_path_in_backup = data_path_in_backup_fs / fileName(marks_file_path); if (!backup->fileExists(file_path_in_backup)) - { - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", - getStorageID().getFullTableName(), file_path_in_backup); - } + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", file_path_in_backup); + size_t file_size = backup->getFileSize(file_path_in_backup); if (file_size % (num_data_files * sizeof(Mark)) != 0) throw Exception("Size of marks file is inconsistent", ErrorCodes::SIZES_OF_MARKS_FILES_ARE_INCONSISTENT); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 7baecaa594f..929da48dcfe 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -513,10 +513,8 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat { String index_file_path = data_path_in_backup_fs / "index.mrk"; if (!backup->fileExists(index_file_path)) - { - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", - getStorageID().getFullTableName(), index_file_path); - } + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", index_file_path); + auto backup_entry = backup->readFile(index_file_path); auto in = backup_entry->getReadBuffer(); CompressedReadBuffer compressed_in{*in}; @@ -530,10 +528,8 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat { String data_file_path = data_path_in_backup_fs / "data.bin"; if (!backup->fileExists(data_file_path)) - { - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", - getStorageID().getFullTableName(), data_file_path); - } + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", data_file_path); + auto backup_entry = backup->readFile(data_file_path); std::unique_ptr in = backup_entry->getReadBuffer(); std::optional temp_data_copy; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 2033d33a33d..e650fbf2b2b 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -623,10 +623,8 @@ void StorageStripeLog::restoreDataImpl(const BackupPtr & backup, const String & { String file_path_in_backup = data_path_in_backup_fs / fileName(data_file_path); if (!backup->fileExists(file_path_in_backup)) - { - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", - getStorageID().getFullTableName(), file_path_in_backup); - } + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", file_path_in_backup); + auto backup_entry = backup->readFile(file_path_in_backup); auto in = backup_entry->getReadBuffer(); auto out = disk->writeFile(data_file_path, max_compress_block_size, WriteMode::Append); @@ -638,10 +636,8 @@ void StorageStripeLog::restoreDataImpl(const BackupPtr & backup, const String & String index_path_in_backup = data_path_in_backup_fs / fileName(index_file_path); IndexForNativeFormat extra_indices; if (!backup->fileExists(index_path_in_backup)) - { - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", - getStorageID().getFullTableName(), index_path_in_backup); - } + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", index_path_in_backup); + auto backup_entry = backup->readFile(index_path_in_backup); auto index_in = backup_entry->getReadBuffer(); CompressedReadBuffer index_compressed_in{*index_in}; From 73e2d592940dac745413862dae2b78ce4cdc9bf0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 3 Jul 2022 14:48:36 +0200 Subject: [PATCH 119/627] Smallish updates of dev guide - Inspired by #38684, this commit removes the list of used third-party libraries + licenses from the development documentation. - The tips how to speed up the build is independent of the OS. Move it from the "Linux" section to the "Getting Started" guide. Also extend it with further useful build options to make the build faster. --- docs/en/development/build.md | 15 ---- docs/en/development/contrib.md | 85 +------------------- docs/en/development/developer-instruction.md | 18 +++++ 3 files changed, 21 insertions(+), 97 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 4f06c52a1b5..dbb90f8e537 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -164,18 +164,3 @@ ClickHouse is available in pre-built binaries and packages. Binaries are portabl They are built for stable, prestable and testing releases as long as for every commit to master and for every pull request. To find the freshest build from `master`, go to [commits page](https://github.com/ClickHouse/ClickHouse/commits/master), click on the first green check mark or red cross near commit, and click to the “Details” link right after “ClickHouse Build Check”. - -## Faster builds for development: Split build configuration {#split-build} - -Normally, ClickHouse is statically linked into a single static `clickhouse` binary with minimal dependencies. This is convenient for distribution, but it means that on every change the entire binary needs to be linked, which is slow and may be inconvenient for development. There is an alternative configuration which instead creates dynamically loaded shared libraries and separate binaries `clickhouse-server`, `clickhouse-client` etc., allowing for faster incremental builds. To use it, add the following flags to your `cmake` invocation: -``` --DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 -``` - -Note that the split build has several drawbacks: -* There is no single `clickhouse` binary, and you have to run `clickhouse-server`, `clickhouse-client`, etc. -* Risk of segfault if you run any of the programs while rebuilding the project. -* You cannot run the integration tests since they only work a single complete binary. -* You can't easily copy the binaries elsewhere. Instead of moving a single binary you'll need to copy all binaries and libraries. - -[Original article](https://clickhouse.com/docs/en/development/build/) diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 8c1f6b5fc9e..13af1be5097 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -6,93 +6,14 @@ description: A list of third-party libraries used # Third-Party Libraries Used -The list of third-party libraries: - -| Library name | License type | -|:-|:-| -| abseil-cpp | [Apache](https://github.com/ClickHouse-Extras/abseil-cpp/blob/4f3b686f86c3ebaba7e4e926e62a79cb1c659a54/LICENSE) | -| AMQP-CPP | [Apache](https://github.com/ClickHouse-Extras/AMQP-CPP/blob/1a6c51f4ac51ac56610fa95081bd2f349911375a/LICENSE) | -| arrow | [Apache](https://github.com/ClickHouse-Extras/arrow/blob/078e21bad344747b7656ef2d7a4f7410a0a303eb/LICENSE.txt) | -| avro | [Apache](https://github.com/ClickHouse-Extras/avro/blob/e43c46e87fd32eafdc09471e95344555454c5ef8/LICENSE.txt) | -| aws | [Apache](https://github.com/ClickHouse-Extras/aws-sdk-cpp/blob/7d48b2c8193679cc4516e5bd68ae4a64b94dae7d/LICENSE.txt) | -| aws-c-common | [Apache](https://github.com/ClickHouse-Extras/aws-c-common/blob/736a82d1697c108b04a277e66438a7f4e19b6857/LICENSE) | -| aws-c-event-stream | [Apache](https://github.com/ClickHouse-Extras/aws-c-event-stream/blob/3bc33662f9ccff4f4cbcf9509cc78c26e022fde0/LICENSE) | -| aws-checksums | [Apache](https://github.com/ClickHouse-Extras/aws-checksums/blob/519d6d9093819b6cf89ffff589a27ef8f83d0f65/LICENSE) | -| base58 | [MIT](https://github.com/ClickHouse/base-x/blob/3e58874643c087f57e82b0ff03825c933fab945a/LICENSE) | -| base64 | [BSD 2-clause](https://github.com/ClickHouse-Extras/Turbo-Base64/blob/af9b331f2b4f30b41c70f3a571ff904a8251c1d3/LICENSE) | -| boost | [Boost](https://github.com/ClickHouse-Extras/boost/blob/9cf09dbfd55a5c6202dedbdf40781a51b02c2675/LICENSE_1_0.txt) | -| boringssl | [BSD](https://github.com/ClickHouse-Extras/boringssl/blob/a6a2e2ab3e44d97ce98e51c558e989f211de7eb3/LICENSE) | -| brotli | [MIT](https://github.com/google/brotli/blob/63be8a99401992075c23e99f7c84de1c653e39e2/LICENSE) | -| capnproto | [MIT](https://github.com/capnproto/capnproto/blob/a00ccd91b3746ef2ab51d40fe3265829949d1ace/LICENSE) | -| cassandra | [Apache](https://github.com/ClickHouse-Extras/cpp-driver/blob/eb9b68dadbb4417a2c132ad4a1c2fa76e65e6fc1/LICENSE.txt) | -| cctz | [Apache](https://github.com/ClickHouse-Extras/cctz/blob/c0f1bcb97fd2782f7c3f972fadd5aad5affac4b8/LICENSE.txt) | -| cityhash102 | [MIT](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/cityhash102/COPYING) | -| cppkafka | [BSD 2-clause](https://github.com/mfontanini/cppkafka/blob/5a119f689f8a4d90d10a9635e7ee2bee5c127de1/LICENSE) | -| croaring | [Apache](https://github.com/RoaringBitmap/CRoaring/blob/2c867e9f9c9e2a3a7032791f94c4c7ae3013f6e0/LICENSE) | -| curl | [Apache](https://github.com/curl/curl/blob/3b8bbbbd1609c638a3d3d0acb148a33dedb67be3/docs/LICENSE-MIXING.md) | -| cyrus-sasl | [BSD 2-clause](https://github.com/ClickHouse-Extras/cyrus-sasl/blob/e6466edfd638cc5073debe941c53345b18a09512/COPYING) | -| double-conversion | [BSD 3-clause](https://github.com/google/double-conversion/blob/cf2f0f3d547dc73b4612028a155b80536902ba02/LICENSE) | -| dragonbox | [Apache](https://github.com/ClickHouse-Extras/dragonbox/blob/923705af6fd953aa948fc175f6020b15f7359838/LICENSE-Apache2-LLVM) | -| fast_float | [Apache](https://github.com/fastfloat/fast_float/blob/7eae925b51fd0f570ccd5c880c12e3e27a23b86f/LICENSE) | -| fastops | [MIT](https://github.com/ClickHouse-Extras/fastops/blob/88752a5e03cf34639a4a37a4b41d8b463fffd2b5/LICENSE) | -| flatbuffers | [Apache](https://github.com/ClickHouse-Extras/flatbuffers/blob/eb3f827948241ce0e701516f16cd67324802bce9/LICENSE.txt) | -| fmtlib | [Unknown](https://github.com/fmtlib/fmt/blob/c108ee1d590089ccf642fc85652b845924067af2/LICENSE.rst) | -| gcem | [Apache](https://github.com/kthohr/gcem/blob/8d4f1b5d76ea8f6ff12f3f4f34cda45424556b00/LICENSE) | -| googletest | [BSD 3-clause](https://github.com/google/googletest/blob/e7e591764baba0a0c3c9ad0014430e7a27331d16/LICENSE) | -| grpc | [Apache](https://github.com/ClickHouse-Extras/grpc/blob/60c986e15cae70aade721d26badabab1f822fdd6/LICENSE) | -| h3 | [Apache](https://github.com/ClickHouse-Extras/h3/blob/c7f46cfd71fb60e2fefc90e28abe81657deff735/LICENSE) | -| vectorscan | [Boost](https://github.com/ClickHouse-Extras/hyperscan/blob/73695e419c27af7fe2a099c7aa57931cc02aea5d/LICENSE) | -| icu | [Public Domain](https://github.com/unicode-org/icu/blob/a56dde820dc35665a66f2e9ee8ba58e75049b668/icu4c/LICENSE) | -| icudata | [Public Domain](https://github.com/ClickHouse-Extras/icudata/blob/72d9a4a7febc904e2b0a534ccb25ae40fac5f1e5/LICENSE) | -| jemalloc | [BSD 2-clause](https://github.com/ClickHouse-Extras/jemalloc/blob/e6891d9746143bf2cf617493d880ba5a0b9a3efd/COPYING) | -| krb5 | [MIT](https://github.com/ClickHouse-Extras/krb5/blob/5149dea4e2be0f67707383d2682b897c14631374/src/lib/gssapi/LICENSE) | -| libc-headers | [LGPL](https://github.com/ClickHouse-Extras/libc-headers/blob/a720b7105a610acbd7427eea475a5b6810c151eb/LICENSE) | -| libcpuid | [BSD 2-clause](https://github.com/ClickHouse-Extras/libcpuid/blob/8db3b8d2d32d22437f063ce692a1b9bb15e42d18/COPYING) | -| libcxx | [Apache](https://github.com/ClickHouse-Extras/libcxx/blob/2fa892f69acbaa40f8a18c6484854a6183a34482/LICENSE.TXT) | -| libcxxabi | [Apache](https://github.com/ClickHouse-Extras/libcxxabi/blob/df8f1e727dbc9e2bedf2282096fa189dc3fe0076/LICENSE.TXT) | -| libdivide | [zLib](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libdivide/LICENSE.txt) | -| libfarmhash | [MIT](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libfarmhash/COPYING) | -| libgsasl | [LGPL](https://github.com/ClickHouse-Extras/libgsasl/blob/383ee28e82f69fa16ed43b48bd9c8ee5b313ab84/LICENSE) | -| libhdfs3 | [Apache](https://github.com/ClickHouse-Extras/libhdfs3/blob/095b9d48b400abb72d967cb0539af13b1e3d90cf/LICENSE.txt) | -| libmetrohash | [Apache](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libmetrohash/LICENSE) | -| libpq | [Unknown](https://github.com/ClickHouse-Extras/libpq/blob/e071ea570f8985aa00e34f5b9d50a3cfe666327e/COPYRIGHT) | -| libpqxx | [BSD 3-clause](https://github.com/ClickHouse-Extras/libpqxx/blob/357608d11b7a1961c3fb7db2ef9a5dbb2e87da77/COPYING) | -| librdkafka | [MIT](https://github.com/ClickHouse-Extras/librdkafka/blob/b8554f1682062c85ba519eb54ef2f90e02b812cb/LICENSE.murmur2) | -| libunwind | [Apache](https://github.com/ClickHouse-Extras/libunwind/blob/6b816d2fba3991f8fd6aaec17d92f68947eab667/LICENSE.TXT) | -| libuv | [BSD](https://github.com/ClickHouse-Extras/libuv/blob/e2e9b7e9f978ce8a1367b5fe781d97d1ce9f94ab/LICENSE) | -| llvm | [Apache](https://github.com/ClickHouse-Extras/llvm/blob/e5751459412bce1391fb7a2e9bbc01e131bf72f1/llvm/LICENSE.TXT) | -| lz4 | [BSD](https://github.com/lz4/lz4/blob/f39b79fb02962a1cd880bbdecb6dffba4f754a11/LICENSE) | -| mariadb-connector-c | [LGPL](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/5f4034a3a6376416504f17186c55fe401c6d8e5e/COPYING.LIB) | -| miniselect | [Boost](https://github.com/danlark1/miniselect/blob/be0af6bd0b6eb044d1acc4f754b229972d99903a/LICENSE_1_0.txt) | -| msgpack-c | [Boost](https://github.com/msgpack/msgpack-c/blob/46684265d50b5d1b062d4c5c428ba08462844b1d/LICENSE_1_0.txt) | -| murmurhash | [Public Domain](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/murmurhash/LICENSE) | -| NuRaft | [Apache](https://github.com/ClickHouse-Extras/NuRaft/blob/7ecb16844af6a9c283ad432d85ecc2e7d1544676/LICENSE) | -| openldap | [Unknown](https://github.com/ClickHouse-Extras/openldap/blob/0208811b6043ca06fda8631a5e473df1ec515ccb/LICENSE) | -| orc | [Apache](https://github.com/ClickHouse-Extras/orc/blob/0a936f6bbdb9303308973073f8623b5a8d82eae1/LICENSE) | -| poco | [Boost](https://github.com/ClickHouse-Extras/poco/blob/7351c4691b5d401f59e3959adfc5b4fa263b32da/LICENSE) | -| protobuf | [BSD 3-clause](https://github.com/ClickHouse-Extras/protobuf/blob/75601841d172c73ae6bf4ce8121f42b875cdbabd/LICENSE) | -| rapidjson | [MIT](https://github.com/ClickHouse-Extras/rapidjson/blob/c4ef90ccdbc21d5d5a628d08316bfd301e32d6fa/bin/jsonschema/LICENSE) | -| re2 | [BSD 3-clause](https://github.com/google/re2/blob/13ebb377c6ad763ca61d12dd6f88b1126bd0b911/LICENSE) | -| replxx | [BSD 3-clause](https://github.com/ClickHouse-Extras/replxx/blob/c81be6c68b146f15f2096b7ef80e3f21fe27004c/LICENSE.md) | -| rocksdb | [BSD 3-clause](https://github.com/ClickHouse-Extras/rocksdb/blob/b6480c69bf3ab6e298e0d019a07fd4f69029b26a/LICENSE.leveldb) | -| s2geometry | [Apache](https://github.com/ClickHouse-Extras/s2geometry/blob/20ea540d81f4575a3fc0aea585aac611bcd03ede/LICENSE) | -| sentry-native | [MIT](https://github.com/ClickHouse-Extras/sentry-native/blob/94644e92f0a3ff14bd35ed902a8622a2d15f7be4/LICENSE) | -| simdjson | [Apache](https://github.com/simdjson/simdjson/blob/8df32cea3359cb30120795da6020b3b73da01d38/LICENSE) | -| snappy | [Public Domain](https://github.com/google/snappy/blob/3f194acb57e0487531c96b97af61dcbd025a78a3/COPYING) | -| sparsehash-c11 | [BSD 3-clause](https://github.com/sparsehash/sparsehash-c11/blob/cf0bffaa456f23bc4174462a789b90f8b6f5f42f/LICENSE) | -| stats | [Apache](https://github.com/kthohr/stats/blob/b6dd459c10a88c7ea04693c007e9e35820c5d9ad/LICENSE) | -| thrift | [Apache](https://github.com/apache/thrift/blob/010ccf0a0c7023fea0f6bf4e4078ebdff7e61982/LICENSE) | -| unixodbc | [LGPL](https://github.com/ClickHouse-Extras/UnixODBC/blob/b0ad30f7f6289c12b76f04bfb9d466374bb32168/COPYING) | -| xz | [Public Domain](https://github.com/xz-mirror/xz/blob/869b9d1b4edd6df07f819d360d306251f8147353/COPYING) | -| zlib-ng | [zLib](https://github.com/ClickHouse-Extras/zlib-ng/blob/6a5e93b9007782115f7f7e5235dedc81c4f1facb/LICENSE.md) | -| zstd | [BSD](https://github.com/facebook/zstd/blob/a488ba114ec17ea1054b9057c26a046fc122b3b6/LICENSE) | - -The list of third-party libraries can be obtained by the following query: +ClickHouse utilizes third-party libraries for different purposes, e.g., to connect to other databases, to decode (encode) data during load (save) from (to) disk or to implement certain specialized SQL functions. To be independent of the available libraries in the target system, each third-party library is imported as a Git submodule into ClickHouse's source tree and compiled and linked with ClickHouse. A list of third-party libraries and their licenses can be obtained by the following query: ``` sql SELECT library_name, license_type, license_path FROM system.licenses ORDER BY library_name COLLATE 'en'; ``` +(Note that the listed libraries are the ones located in the `contrib/` directory of the ClickHouse repository. Depending on the build options, some of of the libraries may have not been compiled, and as a result, their functionality may not be available at runtime. + [Example](https://play.clickhouse.com/play?user=play#U0VMRUNUIGxpYnJhcnlfbmFtZSwgbGljZW5zZV90eXBlLCBsaWNlbnNlX3BhdGggRlJPTSBzeXN0ZW0ubGljZW5zZXMgT1JERVIgQlkgbGlicmFyeV9uYW1lIENPTExBVEUgJ2VuJw==) ## Adding new third-party libraries and maintaining patches in third-party libraries {#adding-third-party-libraries} diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 38d99430193..a35f4c7eba5 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -276,3 +276,21 @@ Testing will commence as soon as ClickHouse employees label your PR with a tag The system will prepare ClickHouse binary builds for your pull request individually. To retrieve these builds click the “Details” link next to “ClickHouse build check” entry in the list of checks. There you will find direct links to the built .deb packages of ClickHouse which you can deploy even on your production servers (if you have no fear). Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page, you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways. + +## Faster builds for development: Split build configuration {#split-build} + +ClickHouse is normally statically linked into a single static `clickhouse` binary with minimal dependencies. This is convenient for distribution, but it means that for every change the entire binary needs to be re-linked, which is slow and inconvenient for development. As an alternative, you can instead build dynamically linked shared libraries and separate binaries `clickhouse-server`, `clickhouse-client` etc., allowing for faster incremental builds. To use it, add the following flags to your `cmake` invocation: +``` +-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 +``` + +Note that the split build has several drawbacks: +* There is no single `clickhouse` binary, and you have to run `clickhouse-server`, `clickhouse-client`, etc. +* Risk of segfault if you run any of the programs while rebuilding the project. +* You cannot run the integration tests since they only work a single complete binary. +* You can't easily copy the binaries elsewhere. Instead of moving a single binary you'll need to copy all binaries and libraries. + +If you are not interested in functionality provided by third-party libraries, you can speed up the build further using `cmake` options +``` +-DENABLE_LIBRARIES=0 -DENABLE_EMBEDDED_COMPILER=0 +``` From 484055984661a2ace7f6a62a1d9f9204ea7f04af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Jul 2022 15:35:10 +0200 Subject: [PATCH 120/627] More systems --- benchmark/compatible/aurora-mysql/README.md | 9 +++- benchmark/compatible/databend/README.md | 1 - benchmark/compatible/databend/benchmark.sh | 10 ++++- benchmark/compatible/locustdb/README.md | 24 ++++++++++ benchmark/compatible/locustdb/benchmark.sh | 23 ++++++++++ benchmark/compatible/sqlite/benchmark.sh | 6 +++ benchmark/compatible/sqlite/queries.sql | 4 +- .../compatible/sqlite/results/c6a.4xlarge.txt | 44 +++++++++++++++++++ .../results/c6a.4xlarge.compression.txt | 44 +++++++++++++++++++ 9 files changed, 159 insertions(+), 6 deletions(-) delete mode 100644 benchmark/compatible/databend/README.md create mode 100644 benchmark/compatible/locustdb/README.md diff --git a/benchmark/compatible/aurora-mysql/README.md b/benchmark/compatible/aurora-mysql/README.md index 5cc0eecc2cb..f526c67df9c 100644 --- a/benchmark/compatible/aurora-mysql/README.md +++ b/benchmark/compatible/aurora-mysql/README.md @@ -28,8 +28,8 @@ Find "Security", click on the group in "VPC security groups". Edit "Inbound rules". Add "Custom TCP", port 3306, from 0.0.0.0/0. ``` -export HOST="database-1.cluster-cnkeohbxcwr1.eu-central-1.rds.amazonaws.com" -export PASSWORD="vci43A32#1" +export HOST="database-1.cluster-....eu-central-1.rds.amazonaws.com" +export PASSWORD="..." mysql -h "${HOST}" -u admin --password="${PASSWORD}" -e "CREATE DATABASE test" ``` @@ -60,3 +60,8 @@ cat log.txt | awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' ``` + +You will get +> ERROR 1114 (HY000) at line 1: The table '/rdsdbdata/tmp/#sqlaff_e5_0' is full + +to some queries. diff --git a/benchmark/compatible/databend/README.md b/benchmark/compatible/databend/README.md deleted file mode 100644 index ca25528a956..00000000000 --- a/benchmark/compatible/databend/README.md +++ /dev/null @@ -1 +0,0 @@ -It failed to load the data. diff --git a/benchmark/compatible/databend/benchmark.sh b/benchmark/compatible/databend/benchmark.sh index fc189e8ef84..c40402b8edb 100644 --- a/benchmark/compatible/databend/benchmark.sh +++ b/benchmark/compatible/databend/benchmark.sh @@ -90,4 +90,12 @@ gzip -d hits.csv.gz # the diagnostics is terrible. head -n 90000000 hits.tsv > hits90m.tsv -curl -XPUT 'http://root:@127.0.0.1:8000/v1/streaming_load' -H 'insert_sql: insert into hits format TSV' -H 'skip_header: 0' -H 'field_delimiter: \t' -H 'record_delimiter: \n' -F 'upload=@"./hits90m.tsv"' +time curl -XPUT 'http://root:@127.0.0.1:8000/v1/streaming_load' -H 'insert_sql: insert into hits format TSV' -H 'skip_header: 0' -H 'field_delimiter: \t' -H 'record_delimiter: \n' -F 'upload=@"./hits90m.tsv"' + +# {"id":"08f59e6c-2924-483e-bb96-cbcb458588f5","state":"SUCCESS","stats":{"rows":90000000,"bytes":73152552024},"error":null} +# real 7m15.312s + +du -bcs _data +# 38714978944 + +./run.sh diff --git a/benchmark/compatible/locustdb/README.md b/benchmark/compatible/locustdb/README.md new file mode 100644 index 00000000000..fb40147cf72 --- /dev/null +++ b/benchmark/compatible/locustdb/README.md @@ -0,0 +1,24 @@ +This system does not work at all: + +``` +locustdb> SELECT * FROM default LIMIT 1 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +note: run with `RUST_BACKTRACE=1` environment variable to display a backtrace +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +``` + +It is memory-safe and blazing fast. diff --git a/benchmark/compatible/locustdb/benchmark.sh b/benchmark/compatible/locustdb/benchmark.sh index cf2117a4b5f..89cb950d90f 100644 --- a/benchmark/compatible/locustdb/benchmark.sh +++ b/benchmark/compatible/locustdb/benchmark.sh @@ -21,3 +21,26 @@ target/release/repl --load hits.csv --db-path db # Loaded data in 920s. # Table `default` (99997496 rows, 15.0GiB) + +# SELECT * FROM default LIMIT 1 + +# And it immediately panicked and hung: + +#locustdb> SELECT * FROM default LIMIT 1 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#note: run with `RUST_BACKTRACE=1` environment variable to display a backtrace +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 +#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 diff --git a/benchmark/compatible/sqlite/benchmark.sh b/benchmark/compatible/sqlite/benchmark.sh index a2bd9f850a9..a144acf680b 100644 --- a/benchmark/compatible/sqlite/benchmark.sh +++ b/benchmark/compatible/sqlite/benchmark.sh @@ -12,3 +12,9 @@ time sqlite3 mydb '.import --csv hits.csv hits' wc -c mydb ./run.sh 2>&1 | tee log.txt + +cat log.txt | + grep -P '^real|^Error' | + sed -r -e 's/^Error.*$/null/; s/^real\s*([0-9.]+)m([0-9.]+)s$/\1 \2/' | + awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + awk '{ if ($1 == "null") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' diff --git a/benchmark/compatible/sqlite/queries.sql b/benchmark/compatible/sqlite/queries.sql index 31f65fc898d..ed185494351 100644 --- a/benchmark/compatible/sqlite/queries.sql +++ b/benchmark/compatible/sqlite/queries.sql @@ -16,7 +16,7 @@ SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, strftime('%M', EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; @@ -40,4 +40,4 @@ SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate > 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 IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; +SELECT strftime('%M', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY strftime('%M', EventTime) ORDER BY strftime('%M', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/sqlite/results/c6a.4xlarge.txt b/benchmark/compatible/sqlite/results/c6a.4xlarge.txt index e7fcbe151f1..6d3ea9459f0 100644 --- a/benchmark/compatible/sqlite/results/c6a.4xlarge.txt +++ b/benchmark/compatible/sqlite/results/c6a.4xlarge.txt @@ -1,2 +1,46 @@ Load time: 43m28.444s Data size: 75 776 589 824 bytes + +[735.416,1.433,1.146], +[298.241,290.681,288.783], +[301.515,291.874,296.839], +[756.708,5.775,5.466], +[820.397,48.554,48.026], +[362.78,359.535,339.447], +[768.981,11.566,10.701], +[298.979,287.778,288.375], +[375.055,353.813,351.954], +[381.285,362.331,371.946], +[314.151,295.135,291.188], +[302.179,296.33,300.106], +[312.865,303.642,305.309], +[317.416,302.584,313.873], +[322.755,305.099,306.018], +[800.456,46.365,48.034], +[363.211,343.955,342.555], +[320.156,312.784,306.34], +[null,null,null], +[748.215,5.21,5.167], +[307.384,287.528,287.27], +[300.111,291.774,291.076], +[298.489,287.678,287.574], +[304.759,287.654,287.207], +[298.892,286.704,287.516], +[299.684,287.525,286.979], +[293.701,287.584,291.865], +[312.486,312.464,306.534], +[null,null,null], +[414.121,393.979,393.173], +[313.694,303.734,295.421], +[306.892,304.659,304.5], +[388.069,381.819,380.922], +[422.745,411.528,415.647], +[465.721,451.627,462.242], +[365.793,351.343,356.19], +[2.799,0.787,0.769], +[1.616,0.629,0.609], +[1.273,0.248,0.238], +[2.493,1.58,1.521], +[1.26,0.297,0.288], +[1.321,0.36,0.318], +[null,null,null] diff --git a/benchmark/compatible/timescaledb/results/c6a.4xlarge.compression.txt b/benchmark/compatible/timescaledb/results/c6a.4xlarge.compression.txt index b83a782eb7f..057266f6ca4 100644 --- a/benchmark/compatible/timescaledb/results/c6a.4xlarge.compression.txt +++ b/benchmark/compatible/timescaledb/results/c6a.4xlarge.compression.txt @@ -1,2 +1,46 @@ Load time: 4605 seconds Data size: 20 333 747 165 bytes + +[2.28686,1.63642,1.64263], +[32.6848,1.63476,1.40052], +[60.8633,3.70484,3.59342], +[36.4029,2.87091,2.80739], +[110.391,38.9688,38.0549], +[147.379,66.2513,65.6379], +[33.0294,2.92031,2.84375], +[33.0221,1.2984,1.19227], +[115.694,47.4651,47.0125], +[156.347,51.577,51.2694], +[68.3301,4.75521,4.68007], +[77.4356,5.55128,5.56577], +[49.7741,11.2911,11.3265], +[81.1014,14.9111,14.9541], +[82.9569,14.6156,14.6331], +[62.0338,26.399,26.3351], +[103.259,36.4122,36.6076], +[92.8828,26.2395,25.8991], +[144.281,63.5102,63.7661], +[7.00679,0.573073,0.536283], +[75.0203,7.86344,7.90495], +[81.2825,9.15868,9.01775], +[104.084,13.9528,13.8435], +[132.531,81.522,82.1561], +[80.6965,3.28231,3.16574], +[39.7693,2.51443,2.43849], +[80.4245,3.26941,3.13916], +[104.015,13.7044,13.5313], +[307.26,253.127,252.147], +[42.8549,22.4187,22.0325], +[137.601,14.9592,14.6804], +[136.767,22.8007,22.131], +[263.005,168.551,163.355], +[156.919,92.6308,91.702], +[160.842,96.0512,97.1773], +[62.8357,28.0336,28.7397], +[1.75869,0.561604,0.541215], +[0.46607,0.191863,0.19021], +[0.303671,0.137579,0.136615], +[2.32031,1.49223,1.52369], +[0.563764,0.14192,0.138234], +[0.372428,0.122989,0.123709], +[0.448574,0.159092,0.154687] From c71f5e35fcf09bec26ae49ec8b32c9cafa78ddb6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sun, 3 Jul 2022 11:40:08 -0400 Subject: [PATCH 121/627] Create pr-pass.yml --- .github/workflows/pr-pass.yml | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 .github/workflows/pr-pass.yml diff --git a/.github/workflows/pr-pass.yml b/.github/workflows/pr-pass.yml new file mode 100644 index 00000000000..2fbb2fe9e0d --- /dev/null +++ b/.github/workflows/pr-pass.yml @@ -0,0 +1,21 @@ +### Сomplementary to PullRequestCI (pull_request.yml) - should pass +name: PRPassCI +on: + pull_request: + types: + - synchronize + - reopened + - opened + branches: + - master + # Сomplementary paths + paths: + - 'docker/docs/**' + - 'docs/**' + - 'website/**' +jobs: + SimpleCheck: + runs-on: ubuntu-latest + steps: + - name: Pass + run: exit 0 From 3c3e39032c781615fbea2f18f2c20d97bfabbca7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Jul 2022 17:45:39 +0200 Subject: [PATCH 122/627] More systems --- benchmark/compatible/README.md | 11 +++-- benchmark/compatible/databend/benchmark.sh | 4 +- .../databend/results/c6a.4xlarge.txt | 46 +++++++++++++++++++ benchmark/compatible/databend/run.sh | 2 +- 4 files changed, 57 insertions(+), 6 deletions(-) create mode 100644 benchmark/compatible/databend/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index 1f9275c16f3..79cc395d3af 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -1,6 +1,8 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [x] ClickHouse +- [ ] ClickHouse operating like "Athena" +- [ ] clickhouse-local without data loading - [x] MySQL InnoDB - [x] MySQL MyISAM - [ ] MariaDB @@ -15,7 +17,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [x] DuckDB - [x] MonetDB - [x] mapD/Omnisci/HeavyAI -- [ ] Databend +- [x] Databend - [ ] Doris - [x] Druid - [ ] Pinot @@ -30,7 +32,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [x] Redshift - [x] Redshift Serverless - [ ] Presto/Trino -- [ ] Athena +- [ ] Amazon Athena - [ ] Bigquery (without publishing) - [ ] Snowflake - [ ] Rockset @@ -39,7 +41,8 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] Databricks - [ ] Planetscale (without publishing) - [ ] TiDB (TiFlash) -- [ ] Amazon RDS +- [ ] Amazon RDS Aurora for MySQL +- [ ] Amazon RDS Aurora for Postgres - [ ] InfluxDB - [ ] VictoriaMetrics - [ ] TDEngine @@ -49,7 +52,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] Infobright - [ ] Actian Vector - [ ] Manticore Search -- [ ] Vertica (without publishing) +- [x] Vertica (without publishing) - [ ] Azure Synapse - [ ] Starburst Galaxy - [ ] MS SQL Server with Column Store Index (without publishing) diff --git a/benchmark/compatible/databend/benchmark.sh b/benchmark/compatible/databend/benchmark.sh index c40402b8edb..916fcd3c2c6 100644 --- a/benchmark/compatible/databend/benchmark.sh +++ b/benchmark/compatible/databend/benchmark.sh @@ -98,4 +98,6 @@ time curl -XPUT 'http://root:@127.0.0.1:8000/v1/streaming_load' -H 'insert_sql: du -bcs _data # 38714978944 -./run.sh +# It does not support ClickHouse protocol well (it hangs on some queries if they are too long). + +./run.sh 2>&1 | tee log.txt diff --git a/benchmark/compatible/databend/results/c6a.4xlarge.txt b/benchmark/compatible/databend/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..50d8ac6a22d --- /dev/null +++ b/benchmark/compatible/databend/results/c6a.4xlarge.txt @@ -0,0 +1,46 @@ +Load time: 7m15.312s +Data size: 38 714 978 944 bytes + +[0.009078, 0.002665, 0.002944], +[0.115168, 0.072011, 0.068167], +[0.146149, 0.129570, 0.130286], +[0.227614, 0.195724, 0.195632], +[30.852923, 31.359742, 31.074248], +[22.761276, 23.213761, 23.058507], +[0.101236, 0.084480, 0.081802], +[0.077944, 0.070164, 0.068803], +[18.650883, 18.673470, 18.319382], +[18.737946, 18.652014, 18.626916], +[1.767940, 1.742031, 1.704442], +[1.662179, 1.610200, 1.587298], +[4.021342, 3.967163, 3.994232], +[17.952548, 17.897310, 17.868163], +[4.930716, 4.927015, 4.914544], +[4.958569, 4.962072, 4.826811], +[13.887323, 13.986540, 13.915090], +[13.415398, 13.526749, 13.517678], +[27.962697, 27.730587, 27.663602], +[0.252960, 0.198019, 0.195967], +[8.004366, 3.714228, 3.718520], +[9.346032, 4.143325, 4.114581], +[17.982515, 7.952690, 8.003425], +[53.807827, 47.207593, 35.947789], +[2.523617, 0.895315, 0.862402], +[0.688769, 0.657391, 0.651568], +[2.506183, 0.851985, 0.843016], +[8.014524, 4.876594, 4.847498], +[10.968887, 11.007059, 10.947711], +[3.021732, 3.056392, 2.987491], +[3.878450, 3.469479, 3.395255], +[8.962381, 6.324689, 6.199428], +[45.180512, 45.481613, 45.254759], +[22.022471, 19.100442, 18.909169], +[23.504267, 21.541563, 22.351808], +[6.760365, 6.566561, 6.592094], +[1.941106, 1.168432, 1.150597], +[1.939902, 1.183049, 1.198113], +[1.806348, 1.063508, 1.040051], +[null, null, null], +[0.436827, 0.378262, 0.375137], +[0.334918, 0.289861, 0.291220], +[null, null, null] diff --git a/benchmark/compatible/databend/run.sh b/benchmark/compatible/databend/run.sh index 86bc4c03f72..5a87f07b2ec 100755 --- a/benchmark/compatible/databend/run.sh +++ b/benchmark/compatible/databend/run.sh @@ -8,7 +8,7 @@ cat queries.sql | while read query; do echo -n "[" for i in $(seq 1 $TRIES); do - RES=$(clickhouse-client --time --format=Null --query="$query" 2>&1 ||:) + RES=$(curl -w 'Time: %{time_total}\n' http://default@localhost:8124/ -d "${query}" 2>&1 | grep -P '^Time: ' | sed 's/Time: //') [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" [[ "$i" != $TRIES ]] && echo -n ", " From 355f76d1f770d2664ee46885f771a4050983f6a3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Jul 2022 17:50:14 +0200 Subject: [PATCH 123/627] More systems --- benchmark/compatible/exasol/README.md | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 benchmark/compatible/exasol/README.md diff --git a/benchmark/compatible/exasol/README.md b/benchmark/compatible/exasol/README.md new file mode 100644 index 00000000000..abe9c5dfd67 --- /dev/null +++ b/benchmark/compatible/exasol/README.md @@ -0,0 +1,3 @@ +EXASOL does not allow using the community edition for more than 10 GiB data, therefore testing is not possible. I advise you not to trust the unfounded claims of performance. + +https://github.com/exasol/docker-db From aff10f6654c220ceb127e3b2db012964103acd0e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sun, 3 Jul 2022 13:45:36 -0400 Subject: [PATCH 124/627] rename label to 'skip simple check' --- .github/workflows/pull_request.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 8e971186ca3..8301faa149d 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -185,10 +185,10 @@ jobs: if: ${{ always() }} steps: - name: Allow simple fail - if: ${{ contains(github.event.pull_request.labels.*.name, 'allow simple fail') }} + if: ${{ contains(github.event.pull_request.labels.*.name, 'skip simple check') }} run: exit 0 - name: Simple failed - if: ${{ (needs.StyleCheck.result != 'success' || needs.FastTest.result != 'success') && !contains(github.event.pull_request.labels.*.name, 'allow simple fail') }} + if: ${{ (needs.StyleCheck.result != 'success' || needs.FastTest.result != 'success') && !contains(github.event.pull_request.labels.*.name, 'skip simple check') }} run: exit 1 CompatibilityCheck: needs: [BuilderDebRelease] From 15e6c441100dbad4d473682f70849e346d956a69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Jul 2022 20:57:03 +0200 Subject: [PATCH 125/627] Adjust numbers --- .../databend/results/c6a.4xlarge.txt | 88 +++++++++---------- 1 file changed, 44 insertions(+), 44 deletions(-) diff --git a/benchmark/compatible/databend/results/c6a.4xlarge.txt b/benchmark/compatible/databend/results/c6a.4xlarge.txt index 50d8ac6a22d..3c1c9edd174 100644 --- a/benchmark/compatible/databend/results/c6a.4xlarge.txt +++ b/benchmark/compatible/databend/results/c6a.4xlarge.txt @@ -1,46 +1,46 @@ -Load time: 7m15.312s -Data size: 38 714 978 944 bytes +Load time: 484 sec +Data size: 43 016 643 271 bytes -[0.009078, 0.002665, 0.002944], -[0.115168, 0.072011, 0.068167], -[0.146149, 0.129570, 0.130286], -[0.227614, 0.195724, 0.195632], -[30.852923, 31.359742, 31.074248], -[22.761276, 23.213761, 23.058507], -[0.101236, 0.084480, 0.081802], -[0.077944, 0.070164, 0.068803], -[18.650883, 18.673470, 18.319382], -[18.737946, 18.652014, 18.626916], -[1.767940, 1.742031, 1.704442], -[1.662179, 1.610200, 1.587298], -[4.021342, 3.967163, 3.994232], -[17.952548, 17.897310, 17.868163], -[4.930716, 4.927015, 4.914544], -[4.958569, 4.962072, 4.826811], -[13.887323, 13.986540, 13.915090], -[13.415398, 13.526749, 13.517678], -[27.962697, 27.730587, 27.663602], -[0.252960, 0.198019, 0.195967], -[8.004366, 3.714228, 3.718520], -[9.346032, 4.143325, 4.114581], -[17.982515, 7.952690, 8.003425], -[53.807827, 47.207593, 35.947789], -[2.523617, 0.895315, 0.862402], -[0.688769, 0.657391, 0.651568], -[2.506183, 0.851985, 0.843016], -[8.014524, 4.876594, 4.847498], -[10.968887, 11.007059, 10.947711], -[3.021732, 3.056392, 2.987491], -[3.878450, 3.469479, 3.395255], -[8.962381, 6.324689, 6.199428], -[45.180512, 45.481613, 45.254759], -[22.022471, 19.100442, 18.909169], -[23.504267, 21.541563, 22.351808], -[6.760365, 6.566561, 6.592094], -[1.941106, 1.168432, 1.150597], -[1.939902, 1.183049, 1.198113], -[1.806348, 1.063508, 1.040051], -[null, null, null], -[0.436827, 0.378262, 0.375137], -[0.334918, 0.289861, 0.291220], +[0.010087, 0.002961, 0.003271] +[0.127964, 0.080012, 0.075741] +[0.162388, 0.143967, 0.144762] +[0.252904, 0.217471, 0.217369] +[34.281026, 34.844158, 34.526942] +[25.290307, 25.793068, 25.620563] +[0.112484, 0.093867, 0.090891] +[0.086604, 0.07796, 0.076448] +[20.723203, 20.7483, 20.354869] +[20.81994, 20.72446, 20.696573] +[1.964378, 1.93559, 1.893824] +[1.846866, 1.789111, 1.763664] +[4.468158, 4.407959, 4.438036] +[19.947276, 19.8859, 19.853514] +[5.478573, 5.474461, 5.460604] +[5.509521, 5.513413, 5.363123] +[15.430359, 15.5406, 15.461211] +[14.905998, 15.029721, 15.019642] +[31.069663, 30.811763, 30.737336] +[0.281067, 0.220021, 0.217741] +[8.89374, 4.12692, 4.131689] +[10.38448, 4.603694, 4.571757] +[19.980572, 8.836322, 8.892694] +[59.786474, 52.452881, 39.941988] +[2.804019, 0.994794, 0.958224] +[0.765299, 0.730434, 0.723964] +[2.784648, 0.94665, 0.936684] +[8.905027, 5.418438, 5.386109] +[12.187652, 12.230066, 12.164123] +[3.35748, 3.395991, 3.319434] +[4.309389, 3.854977, 3.772506] +[9.958201, 7.027432, 6.888253] +[50.200569, 50.535126, 50.283066] +[24.469412, 21.222713, 21.010188] +[26.115852, 23.93507, 24.835342] +[7.511517, 7.296179, 7.324549] +[2.156784, 1.298258, 1.278441] +[2.155447, 1.314499, 1.331237] +[2.007053, 1.181676, 1.155612] +[null, null, null] +[0.485363, 0.420291, 0.416819] +[0.372131, 0.322068, 0.323578] [null, null, null] From f01314086e1c1b0835a1470b1ef8c0ca6ffd1a10 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Jul 2022 20:57:54 +0200 Subject: [PATCH 126/627] Adjust numbers --- benchmark/compatible/databend/benchmark.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/benchmark/compatible/databend/benchmark.sh b/benchmark/compatible/databend/benchmark.sh index 916fcd3c2c6..47ff4ff564c 100644 --- a/benchmark/compatible/databend/benchmark.sh +++ b/benchmark/compatible/databend/benchmark.sh @@ -101,3 +101,5 @@ du -bcs _data # It does not support ClickHouse protocol well (it hangs on some queries if they are too long). ./run.sh 2>&1 | tee log.txt + +# Note: divide every number by 0.9 as only 90% of the data was loaded successfully. From eac84351f664600e9a7945855fe1e9d6a4636c0e Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 4 Jul 2022 01:26:07 +0300 Subject: [PATCH 127/627] fix behavior --- src/Columns/ColumnDecimal.h | 2 +- src/Processors/Transforms/WindowTransform.cpp | 33 ++++++++++++++++--- ...> 02346_non_negative_derivative.reference} | 0 ....sql => 02346_non_negative_derivative.sql} | 0 4 files changed, 29 insertions(+), 6 deletions(-) rename tests/queries/0_stateless/{02232_non_negative_derivative.reference => 02346_non_negative_derivative.reference} (100%) rename tests/queries/0_stateless/{02232_non_negative_derivative.sql => 02346_non_negative_derivative.sql} (100%) diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 1383f583427..03875121637 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -104,7 +104,7 @@ public: Field operator[](size_t n) const override { return DecimalField(data[n], scale); } void get(size_t n, Field & res) const override { res = (*this)[n]; } bool getBool(size_t n) const override { return bool(data[n].value); } - Int64 getInt(size_t n) const override { return Int64(data[n].value) * scale; } + Int64 getInt(size_t n) const override { return Int64(data[n].value); } UInt64 get64(size_t n) const override; bool isDefaultAt(size_t n) const override { return data[n].value == 0; } diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 4de2e4d8604..8a692fb97a5 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -2247,6 +2248,12 @@ struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction argument_types[ARGUMENT_TIMESTAMP]->getName()); } + if (typeid_cast(argument_types[ARGUMENT_TIMESTAMP].get())) + { + const auto & datetime64_type = assert_cast(*argument_types[ARGUMENT_TIMESTAMP]); + ts_scale_multiplier = DecimalUtils::scaleMultiplier(datetime64_type.getScale()); + } + if (argument_types.size() == 3) { const DataTypeInterval * interval_datatype = checkAndGetDataType(argument_types[ARGUMENT_INTERVAL].get()); @@ -2286,20 +2293,36 @@ struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction (*current_block.input_columns[workspace.argument_column_indices[ARGUMENT_INTERVAL]]).getFloat64(0) : 1; Float64 curr_metric = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_METRIC, transform->current_row); - Float64 curr_timestamp = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIMESTAMP, transform->current_row); - - Float64 time_elapsed = curr_timestamp - state.previous_timestamp; Float64 metric_diff = curr_metric - state.previous_metric; - Float64 result = (time_elapsed > 0) ? (metric_diff / time_elapsed * interval_duration) : 0; + Float64 result; + if (ts_scale_multiplier) + { + const auto & column = transform->blockAt(transform->current_row.block).input_columns[workspace.argument_column_indices[ARGUMENT_TIMESTAMP]]; + const auto & curr_timestamp = checkAndGetColumn(column.get())->getInt(transform->current_row.row); + + Float64 time_elapsed = curr_timestamp - state.previous_timestamp; + result = (time_elapsed > 0) ? (metric_diff * ts_scale_multiplier / time_elapsed * interval_duration) : 0; + state.previous_timestamp = curr_timestamp; + } + else + { + Float64 curr_timestamp = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIMESTAMP, transform->current_row); + Float64 time_elapsed = curr_timestamp - state.previous_timestamp; + result = (time_elapsed > 0) ? (metric_diff / time_elapsed * interval_duration) : 0; + state.previous_timestamp = curr_timestamp; + } state.previous_metric = curr_metric; - state.previous_timestamp = curr_timestamp; + + if (unlikely(!transform->current_row.row)) + result = 0; WindowFunctionHelpers::setValueToOutputColumn(transform, function_index, result >= 0 ? result : 0); } private: Float64 interval_length = 1; bool interval_specified = false; + Int64 ts_scale_multiplier = 0; }; diff --git a/tests/queries/0_stateless/02232_non_negative_derivative.reference b/tests/queries/0_stateless/02346_non_negative_derivative.reference similarity index 100% rename from tests/queries/0_stateless/02232_non_negative_derivative.reference rename to tests/queries/0_stateless/02346_non_negative_derivative.reference diff --git a/tests/queries/0_stateless/02232_non_negative_derivative.sql b/tests/queries/0_stateless/02346_non_negative_derivative.sql similarity index 100% rename from tests/queries/0_stateless/02232_non_negative_derivative.sql rename to tests/queries/0_stateless/02346_non_negative_derivative.sql From 9495f0418846d66a6d7a971e69943582ed14dc51 Mon Sep 17 00:00:00 2001 From: Michael Schnerring <3743342+schnerring@users.noreply.github.com> Date: Mon, 4 Jul 2022 00:42:54 +0200 Subject: [PATCH 128/627] Revise Docker README --- docker/server/README.md | 91 ++++++++++++++++++++++------------------- 1 file changed, 49 insertions(+), 42 deletions(-) diff --git a/docker/server/README.md b/docker/server/README.md index c074a1bac00..f331b802c37 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -2,61 +2,64 @@ ## What is ClickHouse? -ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time. +ClickHouse is an open-source column-oriented database management system that allows generating of analytical data reports in real-time. -ClickHouse manages extremely large volumes of data in a stable and sustainable manner. It currently powers [Yandex.Metrica](https://metrica.yandex.com/), world’s [second largest](http://w3techs.com/technologies/overview/traffic_analysis/all) web analytics platform, with over 13 trillion database records and over 20 billion events a day, generating customized reports on-the-fly, directly from non-aggregated data. This system was successfully implemented at [CERN’s LHCb experiment](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) to store and process metadata on 10bn events with over 1000 attributes per event registered in 2011. +ClickHouse manages extremely large volumes of data stably and sustainably. It currently powers [Yandex.Metrica](https://metrica.yandex.com/), the world’s [second-largest](http://w3techs.com/technologies/overview/traffic_analysis/all) web analytics platform, with over 13 trillion database records and over 20 billion events a day, generating customized reports on-the-fly, directly from non-aggregated data. This system was successfully implemented at [CERN’s LHCb experiment](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) to store and process metadata on 10bn events with over 1000 attributes per event registered in 2011. For more information and documentation see https://clickhouse.com/. ## How to use this image ### start server instance + ```bash -$ docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server ``` -By default ClickHouse will be accessible only via docker network. See the [networking section below](#networking). +By default, ClickHouse will be accessible only via the Docker network. See the [networking section below](#networking). -By default, starting above server instance will be run as default user without password. +By default, starting above server instance will be run as the `default` user without a password. ### connect to it from a native client + ```bash -$ docker run -it --rm --link some-clickhouse-server:clickhouse-server --entrypoint clickhouse-client clickhouse/clickhouse-server --host clickhouse-server +docker run -it --rm --link some-clickhouse-server:clickhouse-server --entrypoint clickhouse-client clickhouse/clickhouse-server --host clickhouse-server # OR -$ docker exec -it some-clickhouse-server clickhouse-client +docker exec -it some-clickhouse-server clickhouse-client ``` -More information about [ClickHouse client](https://clickhouse.com/docs/en/interfaces/cli/). +More information about the [ClickHouse client](https://clickhouse.com/docs/en/interfaces/cli/). ### connect to it using curl ```bash echo "SELECT 'Hello, ClickHouse!'" | docker run -i --rm --link some-clickhouse-server:clickhouse-server curlimages/curl 'http://clickhouse-server:8123/?query=' -s --data-binary @- ``` + More information about [ClickHouse HTTP Interface](https://clickhouse.com/docs/en/interfaces/http/). -### stopping / removing the containter +### stopping / removing the container ```bash -$ docker stop some-clickhouse-server -$ docker rm some-clickhouse-server +docker stop some-clickhouse-server +docker rm some-clickhouse-server ``` ### networking -You can expose you ClickHouse running in docker by [mapping particular port](https://docs.docker.com/config/containers/container-networking/) from inside container to a host ports: +You can expose your ClickHouse running in docker by [mapping a particular port](https://docs.docker.com/config/containers/container-networking/) from inside the container using host ports: ```bash -$ docker run -d -p 18123:8123 -p19000:9000 --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server -$ echo 'SELECT version()' | curl 'http://localhost:18123/' --data-binary @- +docker run -d -p 18123:8123 -p19000:9000 --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +echo 'SELECT version()' | curl 'http://localhost:18123/' --data-binary @- 20.12.3.3 ``` -or by allowing container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows archiving better network performance): +or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows archiving better network performance): ```bash -$ docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server -$ echo 'SELECT version()' | curl 'http://localhost:8123/' --data-binary @- +docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +echo 'SELECT version()' | curl 'http://localhost:8123/' --data-binary @- 20.12.3.3 ``` @@ -65,68 +68,72 @@ $ echo 'SELECT version()' | curl 'http://localhost:8123/' --data-binary @- Typically you may want to mount the following folders inside your container to archieve persistency: * `/var/lib/clickhouse/` - main folder where ClickHouse stores the data -* `/val/log/clickhouse-server/` - logs +* `/var/log/clickhouse-server/` - logs ```bash -$ docker run -d \ - -v $(realpath ./ch_data):/var/lib/clickhouse/ \ - -v $(realpath ./ch_logs):/var/log/clickhouse-server/ \ - --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +docker run -d \ + -v $(realpath ./ch_data):/var/lib/clickhouse/ \ + -v $(realpath ./ch_logs):/var/log/clickhouse-server/ \ + --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server ``` You may also want to mount: * `/etc/clickhouse-server/config.d/*.xml` - files with server configuration adjustmenets -* `/etc/clickhouse-server/usert.d/*.xml` - files with use settings adjustmenets +* `/etc/clickhouse-server/users.d/*.xml` - files with user settings adjustmenets * `/docker-entrypoint-initdb.d/` - folder with database initialization scripts (see below). ### Linux capabilities -ClickHouse has some advanced functionality which requite enabling several [linux capabilities](https://man7.org/linux/man-pages/man7/capabilities.7.html). +ClickHouse has some advanced functionality, which requires enabling several [Linux capabilities](https://man7.org/linux/man-pages/man7/capabilities.7.html). -It is optional and can be enabled using the following [docker command line agruments](https://docs.docker.com/engine/reference/run/#runtime-privilege-and-linux-capabilities): +It is optional and can be enabled using the following [docker command-line arguments](https://docs.docker.com/engine/reference/run/#runtime-privilege-and-linux-capabilities): ```bash -$ docker run -d \ - --cap-add=SYS_NICE --cap-add=NET_ADMIN --cap-add=IPC_LOCK \ - --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +docker run -d \ + --cap-add=SYS_NICE --cap-add=NET_ADMIN --cap-add=IPC_LOCK \ + --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server ``` ## Configuration -Container exposes 8123 port for [HTTP interface](https://clickhouse.com/docs/en/interfaces/http_interface/) and 9000 port for [native client](https://clickhouse.com/docs/en/interfaces/tcp/). +The container exposes port 8123 for the [HTTP interface](https://clickhouse.com/docs/en/interfaces/http_interface/) and port 9000 for the [native client](https://clickhouse.com/docs/en/interfaces/tcp/). -ClickHouse configuration represented with a file "config.xml" ([documentation](https://clickhouse.com/docs/en/operations/configuration_files/)) +ClickHouse configuration is represented with a file "config.xml" ([documentation](https://clickhouse.com/docs/en/operations/configuration_files/)) ### Start server instance with custom configuration + ```bash -$ docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 -v /path/to/your/config.xml:/etc/clickhouse-server/config.xml clickhouse/clickhouse-server +docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 -v /path/to/your/config.xml:/etc/clickhouse-server/config.xml clickhouse/clickhouse-server ``` ### Start server as custom user -``` + +```bash # $(pwd)/data/clickhouse should exist and be owned by current user -$ docker run --rm --user ${UID}:${GID} --name some-clickhouse-server --ulimit nofile=262144:262144 -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server +docker run --rm --user ${UID}:${GID} --name some-clickhouse-server --ulimit nofile=262144:262144 -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server ``` -When you use the image with mounting local directories inside you probably would like to not mess your directory tree with files owner and permissions. Then you could use `--user` argument. In this case, you should mount every necessary directory (`/var/lib/clickhouse` and `/var/log/clickhouse-server`) inside the container. Otherwise, image will complain and not start. + +When you use the image with mounting local directories inside you probably would like to not mess your directory tree with files owner and permissions. Then you could use `--user` argument. In this case, you should mount every necessary directory (`/var/lib/clickhouse` and `/var/log/clickhouse-server`) inside the container. Otherwise, the image will complain and not start. ### Start server from root (useful in case of userns enabled) -``` -$ docker run --rm -e CLICKHOUSE_UID=0 -e CLICKHOUSE_GID=0 --name clickhouse-server-userns -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server + +```bash +docker run --rm -e CLICKHOUSE_UID=0 -e CLICKHOUSE_GID=0 --name clickhouse-server-userns -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server ``` ### How to create default database and user on starting -Sometimes you may want to create user (user named `default` is used by default) and database on image starting. You can do it using environment variables `CLICKHOUSE_DB`, `CLICKHOUSE_USER`, `CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT` and `CLICKHOUSE_PASSWORD`: +Sometimes you may want to create a user (user named `default` is used by default) and database on image start. You can do it using environment variables `CLICKHOUSE_DB`, `CLICKHOUSE_USER`, `CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT` and `CLICKHOUSE_PASSWORD`: -``` -$ docker run --rm -e CLICKHOUSE_DB=my_database -e CLICKHOUSE_USER=username -e CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT=1 -e CLICKHOUSE_PASSWORD=password -p 9000:9000/tcp clickhouse/clickhouse-server +```bash +docker run --rm -e CLICKHOUSE_DB=my_database -e CLICKHOUSE_USER=username -e CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT=1 -e CLICKHOUSE_PASSWORD=password -p 9000:9000/tcp clickhouse/clickhouse-server ``` ## How to extend this image If you would like to do additional initialization in an image derived from this one, add one or more `*.sql`, `*.sql.gz`, or `*.sh` scripts under `/docker-entrypoint-initdb.d`. After the entrypoint calls `initdb` it will run any `*.sql` files, run any executable `*.sh` scripts, and source any non-executable `*.sh` scripts found in that directory to do further initialization before starting the service. -Also you can provide environment variables `CLICKHOUSE_USER` & `CLICKHOUSE_PASSWORD` that will be used for clickhouse-client during initialization. +Also, you can provide environment variables `CLICKHOUSE_USER` & `CLICKHOUSE_PASSWORD` that will be used for clickhouse-client during initialization. For example, to add an additional user and database, add the following to `/docker-entrypoint-initdb.d/init-db.sh`: @@ -135,8 +142,8 @@ For example, to add an additional user and database, add the following to `/dock set -e clickhouse client -n <<-EOSQL - CREATE DATABASE docker; - CREATE TABLE docker.docker (x Int32) ENGINE = Log; + CREATE DATABASE docker; + CREATE TABLE docker.docker (x Int32) ENGINE = Log; EOSQL ``` From bc7487b66524d2dc6e8c13802edd630ee49c76ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 01:19:45 +0200 Subject: [PATCH 129/627] More results --- benchmark/compatible/bigquery/.gitignore | 1 + benchmark/compatible/bigquery/README.md | 38 ++++++ benchmark/compatible/bigquery/create.sql | 108 ++++++++++++++++++ benchmark/compatible/bigquery/queries.sql | 43 +++++++ benchmark/compatible/bigquery/run.sh | 10 ++ benchmark/compatible/snowflake/README.md | 0 benchmark/compatible/snowflake/create.sql | 0 .../compatible/sqlite/results/c6a.4xlarge.txt | 84 +++++++------- 8 files changed, 242 insertions(+), 42 deletions(-) create mode 100644 benchmark/compatible/bigquery/.gitignore create mode 100644 benchmark/compatible/bigquery/README.md create mode 100644 benchmark/compatible/bigquery/create.sql create mode 100644 benchmark/compatible/bigquery/queries.sql create mode 100644 benchmark/compatible/bigquery/run.sh create mode 100644 benchmark/compatible/snowflake/README.md create mode 100644 benchmark/compatible/snowflake/create.sql diff --git a/benchmark/compatible/bigquery/.gitignore b/benchmark/compatible/bigquery/.gitignore new file mode 100644 index 00000000000..1a06816d838 --- /dev/null +++ b/benchmark/compatible/bigquery/.gitignore @@ -0,0 +1 @@ +results diff --git a/benchmark/compatible/bigquery/README.md b/benchmark/compatible/bigquery/README.md new file mode 100644 index 00000000000..3563fd7a87b --- /dev/null +++ b/benchmark/compatible/bigquery/README.md @@ -0,0 +1,38 @@ +BigQuery has "DeWitt" clause that restricts from disclosing the benchmark results. +Nevertheless, it does not prevent from doing the benchmarks. + +It's very difficult to find, how to create a database. +Databases are named "datasets". You need to press on `⋮` near project. + +Create dataset `test`. +Go to the query editor and paste the contents of `create.sql`. +It will take two seconds to create a table. + +Download Google Cloud CLI: +``` +curl -O https://dl.google.com/dl/cloudsdk/channels/rapid/downloads/google-cloud-cli-392.0.0-linux-x86_64.tar.gz +tar -xf google-cloud-cli-392.0.0-linux-x86_64.tar.gz +./google-cloud-sdk/install.sh +source .bashrc +./google-cloud-sdk/bin/gcloud init +``` + +Load the data: +``` +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' +gzip -d hits.csv.gz + +time bq load --source_format CSV --allow_quoted_newlines=1 test.hits hits.csv +``` + +Run the benchmark: + +``` +./run.sh 2>&1 | tee log.txt + +cat log.txt | + grep -P '^real|^Error' | + sed -r -e 's/^Error.*$/null/; s/^real\s*([0-9.]+)m([0-9.]+)s$/\1 \2/' | + awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + awk '{ if ($1 == "null") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' +``` diff --git a/benchmark/compatible/bigquery/create.sql b/benchmark/compatible/bigquery/create.sql new file mode 100644 index 00000000000..9012df89a77 --- /dev/null +++ b/benchmark/compatible/bigquery/create.sql @@ -0,0 +1,108 @@ +CREATE TABLE test.hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title String NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL String NOT NULL, + Referer String NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 String NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor String NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel String NOT NULL, + Params String NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase String NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset String NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL String NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor String NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage String NOT NULL, + BrowserCountry String NOT NULL, + SocialNetwork String NOT NULL, + SocialAction String NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage String NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID String NOT NULL, + ParamCurrency String NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName String NOT NULL, + OpenstatCampaignID String NOT NULL, + OpenstatAdID String NOT NULL, + OpenstatSourceID String NOT NULL, + UTMSource String NOT NULL, + UTMMedium String NOT NULL, + UTMCampaign String NOT NULL, + UTMContent String NOT NULL, + UTMTerm String NOT NULL, + FromTag String NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +); diff --git a/benchmark/compatible/bigquery/queries.sql b/benchmark/compatible/bigquery/queries.sql new file mode 100644 index 00000000000..3dc8f405aaa --- /dev/null +++ b/benchmark/compatible/bigquery/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM test.hits; +SELECT COUNT(*) FROM test.hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM test.hits; +SELECT AVG(UserID) FROM test.hits; +SELECT COUNT(DISTINCT UserID) FROM test.hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM test.hits; +SELECT MIN(EventDate), MAX(EventDate) FROM test.hits; +SELECT AdvEngineID, COUNT(*) FROM test.hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM test.hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM test.hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM test.hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM test.hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM test.hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM test.hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM test.hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM test.hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM test.hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM test.hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM test.hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM test.hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM test.hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM test.hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM test.hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM test.hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM test.hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM test.hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM test.hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM test.hits WHERE URL <> '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM test.hits 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 test.hits; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM test.hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM test.hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM test.hits GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC(EventTime, MINUTE) AS M, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY M ORDER BY M LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/bigquery/run.sh b/benchmark/compatible/bigquery/run.sh new file mode 100644 index 00000000000..1a48f9a1c9b --- /dev/null +++ b/benchmark/compatible/bigquery/run.sh @@ -0,0 +1,10 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + echo "$query"; + for i in $(seq 1 $TRIES); do + time bq query --use_legacy_sql=false --use_cache=false <<< "$query" + done +done diff --git a/benchmark/compatible/snowflake/README.md b/benchmark/compatible/snowflake/README.md new file mode 100644 index 00000000000..e69de29bb2d diff --git a/benchmark/compatible/snowflake/create.sql b/benchmark/compatible/snowflake/create.sql new file mode 100644 index 00000000000..e69de29bb2d diff --git a/benchmark/compatible/sqlite/results/c6a.4xlarge.txt b/benchmark/compatible/sqlite/results/c6a.4xlarge.txt index 6d3ea9459f0..1db75b00a41 100644 --- a/benchmark/compatible/sqlite/results/c6a.4xlarge.txt +++ b/benchmark/compatible/sqlite/results/c6a.4xlarge.txt @@ -1,46 +1,46 @@ Load time: 43m28.444s Data size: 75 776 589 824 bytes -[735.416,1.433,1.146], -[298.241,290.681,288.783], -[301.515,291.874,296.839], -[756.708,5.775,5.466], -[820.397,48.554,48.026], -[362.78,359.535,339.447], -[768.981,11.566,10.701], -[298.979,287.778,288.375], -[375.055,353.813,351.954], -[381.285,362.331,371.946], -[314.151,295.135,291.188], -[302.179,296.33,300.106], -[312.865,303.642,305.309], -[317.416,302.584,313.873], -[322.755,305.099,306.018], -[800.456,46.365,48.034], -[363.211,343.955,342.555], -[320.156,312.784,306.34], +[752.739,2.003,1.2], +[304.302,291.521,286.965], +[293.964,287.619,287.219], +[758.302,5.879,5.65], +[836.393,48.593,48.452], +[362.605,344.884,356.245], +[763.993,11.602,10.795], +[296.348,286.879,287.557], +[365.816,360.339,354.126], +[374.403,365.196,362.261], +[302.989,293.888,298.432], +[303.64,291.729,295.347], +[316.824,298.18,301.006], +[320.665,301.15,305.227], +[313.593,301.021,301.626], +[794.881,47,47.225], +[355.346,344.615,342.442], +[316.499,305.971,305.007], +[398.177,380.383,385.571], +[751.82,5.082,4.913], +[295.522,286.573,287.368], +[298.58,287.182,288.303], +[296.474,288.747,288.638], +[296.579,287.127,287.361], +[304.709,286.865,287.56], +[300.391,290.633,288.587], +[294.605,286.91,287.799], +[305.986,312.111,305.626], [null,null,null], -[748.215,5.21,5.167], -[307.384,287.528,287.27], -[300.111,291.774,291.076], -[298.489,287.678,287.574], -[304.759,287.654,287.207], -[298.892,286.704,287.516], -[299.684,287.525,286.979], -[293.701,287.584,291.865], -[312.486,312.464,306.534], -[null,null,null], -[414.121,393.979,393.173], -[313.694,303.734,295.421], -[306.892,304.659,304.5], -[388.069,381.819,380.922], -[422.745,411.528,415.647], -[465.721,451.627,462.242], -[365.793,351.343,356.19], -[2.799,0.787,0.769], -[1.616,0.629,0.609], -[1.273,0.248,0.238], -[2.493,1.58,1.521], -[1.26,0.297,0.288], -[1.321,0.36,0.318], -[null,null,null] +[411.225,397.614,394.253], +[307.711,295.181,300.266], +[312.472,299.079,298.19], +[386.674,378.347,376.963], +[409.742,409.554,420.273], +[468.73,453.709,458.149], +[366.015,347.446,346.728], +[2.911,0.781,0.757], +[1.599,0.609,0.587], +[1.288,0.256,0.238], +[2.469,1.582,1.52], +[1.274,0.303,0.283], +[1.322,0.317,0.314], +[1.498,0.602,0.613] From 41237c761379685708d2f86a529fe6659cf4fd91 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 01:20:37 +0200 Subject: [PATCH 130/627] More systems --- benchmark/compatible/snowflake/create.sql | 109 ++++++++++++++++++++++ 1 file changed, 109 insertions(+) diff --git a/benchmark/compatible/snowflake/create.sql b/benchmark/compatible/snowflake/create.sql index e69de29bb2d..1850bffedce 100644 --- a/benchmark/compatible/snowflake/create.sql +++ b/benchmark/compatible/snowflake/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); From df87ca6a931bdfaad1af558e7f7e57f74128d0f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 03:55:22 +0200 Subject: [PATCH 131/627] Add Snowflake --- benchmark/compatible/README.md | 6 +- benchmark/compatible/snowflake/NOTES.md | 69 +++++++++++++++++++ benchmark/compatible/snowflake/README.md | 56 +++++++++++++++ benchmark/compatible/snowflake/create.sql | 2 +- benchmark/compatible/snowflake/queries.sql | 43 ++++++++++++ .../compatible/snowflake/results/2xl.txt | 46 +++++++++++++ .../compatible/snowflake/results/3xl.txt | 46 +++++++++++++ .../compatible/snowflake/results/4xl.txt | 46 +++++++++++++ benchmark/compatible/snowflake/results/l.txt | 46 +++++++++++++ benchmark/compatible/snowflake/results/m.txt | 46 +++++++++++++ benchmark/compatible/snowflake/results/s.txt | 46 +++++++++++++ benchmark/compatible/snowflake/results/xl.txt | 46 +++++++++++++ benchmark/compatible/snowflake/results/xs.txt | 46 +++++++++++++ benchmark/compatible/snowflake/run.sh | 3 + 14 files changed, 543 insertions(+), 4 deletions(-) create mode 100644 benchmark/compatible/snowflake/NOTES.md create mode 100644 benchmark/compatible/snowflake/queries.sql create mode 100644 benchmark/compatible/snowflake/results/2xl.txt create mode 100644 benchmark/compatible/snowflake/results/3xl.txt create mode 100644 benchmark/compatible/snowflake/results/4xl.txt create mode 100644 benchmark/compatible/snowflake/results/l.txt create mode 100644 benchmark/compatible/snowflake/results/m.txt create mode 100644 benchmark/compatible/snowflake/results/s.txt create mode 100644 benchmark/compatible/snowflake/results/xl.txt create mode 100644 benchmark/compatible/snowflake/results/xs.txt create mode 100644 benchmark/compatible/snowflake/run.sh diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index 79cc395d3af..29a5de71c1d 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -2,7 +2,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [x] ClickHouse - [ ] ClickHouse operating like "Athena" -- [ ] clickhouse-local without data loading +- [x] clickhouse-local without data loading - [x] MySQL InnoDB - [x] MySQL MyISAM - [ ] MariaDB @@ -28,12 +28,12 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] Hive - [ ] Impala - [ ] Hyper -- [ ] SQLite +- [x] SQLite - [x] Redshift - [x] Redshift Serverless - [ ] Presto/Trino - [ ] Amazon Athena -- [ ] Bigquery (without publishing) +- [x] Bigquery (without publishing) - [ ] Snowflake - [ ] Rockset - [ ] CockroachDB diff --git a/benchmark/compatible/snowflake/NOTES.md b/benchmark/compatible/snowflake/NOTES.md new file mode 100644 index 00000000000..284382fba91 --- /dev/null +++ b/benchmark/compatible/snowflake/NOTES.md @@ -0,0 +1,69 @@ +The choice of a warehouse size is unclear. Let's choose X-Large by default. +It is using "credits" for pricing. + +Storage cost: $23 USD per compressed TB per month +One credit is: $2.016/hour +X-Large: 16 credits/hour = $64/hour + +> Compute costs are $0.00056 per second for each credit consumed on Snowflake Standard Edition + +It is very expensive, so let's touch it with a ten-foot pole and run away as quickly as possible. + +Set up SnowSQL. + +``` +curl -O https://sfc-repo.snowflakecomputing.com/snowsql/bootstrap/1.2/linux_x86_64/snowsql-1.2.22-linux_x86_64.bash +bash snowsql-1.2.22-linux_x86_64.bash +source .profile +``` + +``` +snowsql -a HA12345 -u USER +``` + +It does not connect after typing the password. + +``` +250001 (08001): Failed to connect to DB. Verify the account name is correct: HA12345.snowflakecomputing.com:443. 000403: 403: HTTP 403: Forbidden +If the error message is unclear, enable logging using -o log_level=DEBUG and see the log to find out the cause. Contact support for further help. +Goodbye! +``` + +It said "Goodbye!" in active-aggressive tone. + +To know the account name, we have to go to the "classic console" and look at the URL in the browser. + +> https://{this}.eu-central-1.snowflakecomputing.com/console/login?disableDirectLogin=true + +But it does not help. + +It works if I specify the region in the command line. +Although `snowsql --help` saying that it is DEPRECATED. + +``` +snowsql -a nn12345 -u USER --region eu-central-1 --schemaname PUBLIC --dbname TEST --warehouse TEST +``` + +Notes: SnowSQL is using autocomplete using well known Python library. +Autocomplete is not context-aware. + +Upload the data: + +``` +put file:///home/ubuntu/hits.csv @test.public.%hits +``` + +The syntax is strange (all these @%#). +The query hung and did nothing. + +Actually it is not hung. The snowsql is using 100% to parse CSV in Python for hours. + +Let's try a different upload method. + +``` +COPY INTO test.public.hits2 FROM 's3://clickhouse-public-datasets/hits_compatible/hits.csv.gz' FILE_FORMAT = (TYPE = CSV, COMPRESSION = GZIP, FIELD_OPTIONALLY_ENCLOSED_BY = '"') +``` + +For some reason, it has selected X-Small warehouse, will need to change to X-Large. + +42 min 4 sec. diff --git a/benchmark/compatible/snowflake/README.md b/benchmark/compatible/snowflake/README.md index e69de29bb2d..c2de02dded7 100644 --- a/benchmark/compatible/snowflake/README.md +++ b/benchmark/compatible/snowflake/README.md @@ -0,0 +1,56 @@ +Snowflake recently removed the DeWitt Clause, so we are allowed to make benchmarks. + +> Customer may conduct benchmark tests of the Service (each a “Test”). Other than with respect to Tests involving Previews, which may not be disclosed externally, Customer may externally disclose a Test or otherwise cause the results of a Test to be externally disclosed if it includes as part of the disclosure all information necessary to replicate the Test. + +https://www.snowflake.com/legal/acceptable-use-policy/ + +Account setup took only 3 seconds. + +Data -> Databases -> + Database +Database 'test' created. +Press on "public" schema. + +Create table "standard". +Paste "create.sql". +Press on "create table" again. + +Press on "admin", "warehouses", + Warehouse +The choice of a warehouse size is unclear. Let's choose X-Large by default. +It is using "credits" for pricing. + +Set up SnowSQL. + +``` +curl -O https://sfc-repo.snowflakecomputing.com/snowsql/bootstrap/1.2/linux_x86_64/snowsql-1.2.22-linux_x86_64.bash +bash snowsql-1.2.22-linux_x86_64.bash +source .profile +``` + +Upload the data: + +``` +COPY INTO test.public.hits2 FROM 's3://clickhouse-public-datasets/hits_compatible/hits.csv.gz' FILE_FORMAT = (TYPE = CSV, COMPRESSION = GZIP, FIELD_OPTIONALLY_ENCLOSED_BY = '"') +``` + +42 min 4 sec. + +``` +export SNOWSQL_PWD='...' SNOWSQL_ACCOUNT='...' SNOWSQL_USER='myuser' + +snowsql --region eu-central-1 --schemaname PUBLIC --dbname HITS --warehouse TEST --query "SELECT 1" +``` + +Before the benchmark: +``` +ALTER USER myuser SET USE_CACHED_RESULT = false; +``` + +Run the benchmark: +``` +./run.sh 2>&1 | tee log.txt + +cat log.txt | + grep -P 'Time Elapsed|^\d+ \(\w+\):' | + sed -r -e 's/^[0-9]+ \([0-9A-Za-z]+\):.*$/null/; s/^.*Time Elapsed:\s*([0-9.]+)s$/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' +``` diff --git a/benchmark/compatible/snowflake/create.sql b/benchmark/compatible/snowflake/create.sql index 1850bffedce..62800d2e1b2 100644 --- a/benchmark/compatible/snowflake/create.sql +++ b/benchmark/compatible/snowflake/create.sql @@ -1,4 +1,4 @@ -CREATE TABLE hits +CREATE TABLE hits2 ( WatchID BIGINT NOT NULL, JavaEnable SMALLINT NOT NULL, diff --git a/benchmark/compatible/snowflake/queries.sql b/benchmark/compatible/snowflake/queries.sql new file mode 100644 index 00000000000..d188fd36e25 --- /dev/null +++ b/benchmark/compatible/snowflake/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM test.public.hits2; +SELECT COUNT(*) FROM test.public.hits2 WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM test.public.hits2; +SELECT AVG(UserID) FROM test.public.hits2; +SELECT COUNT(DISTINCT UserID) FROM test.public.hits2; +SELECT COUNT(DISTINCT SearchPhrase) FROM test.public.hits2; +SELECT MIN(EventDate), MAX(EventDate) FROM test.public.hits2; +SELECT AdvEngineID, COUNT(*) FROM test.public.hits2 WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM test.public.hits2 GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM test.public.hits2 GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM test.public.hits2 WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM test.public.hits2 WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM test.public.hits2 WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM test.public.hits2 WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM test.public.hits2 WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM test.public.hits2 GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM test.public.hits2 GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM test.public.hits2 GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM test.public.hits2 GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM test.public.hits2 WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM test.public.hits2 WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM test.public.hits2 WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM test.public.hits2 WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM test.public.hits2 WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM test.public.hits2 WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM test.public.hits2 WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM test.public.hits2 WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM test.public.hits2 WHERE URL <> '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT REGEXP_REPLACE(Referer, '^https?://(www\.)?([^/]+)/.*$', '\2') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM test.public.hits2 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 test.public.hits2; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.public.hits2 WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.public.hits2 WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.public.hits2 GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM test.public.hits2 GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM test.public.hits2 GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM test.public.hits2 GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/snowflake/results/2xl.txt b/benchmark/compatible/snowflake/results/2xl.txt new file mode 100644 index 00000000000..6095cb2f4ab --- /dev/null +++ b/benchmark/compatible/snowflake/results/2xl.txt @@ -0,0 +1,46 @@ +Load time: 42 min 4 sec. +Data size: 12.3 GB + +[0.177,0.052,0.090], +[0.903,0.324,0.782], +[0.458,2.909,0.275], +[0.881,0.316,0.166], +[0.404,0.257,0.256], +[0.481,0.325,0.339], +[0.056,0.062,0.060], +[0.183,0.324,0.280], +[0.444,0.314,0.320], +[0.408,0.426,0.417], +[0.345,0.241,0.253], +[0.406,0.245,0.235], +[0.521,1.259,0.326], +[0.466,0.493,0.526], +[0.447,0.349,0.362], +[0.327,0.322,0.302], +[0.462,0.508,0.461], +[0.489,0.481,0.455], +[0.731,0.659,0.674], +[0.151,0.156,0.160], +[0.832,0.339,0.312], +[0.289,0.292,0.340], +[0.591,0.484,0.419], +[2.661,0.716,0.696], +[0.190,0.199,0.377], +[0.181,0.182,0.194], +[0.220,0.209,0.195], +[0.368,0.330,0.347], +[0.677,0.645,0.646], +[0.877,0.886,0.871], +[0.415,0.282,0.301], +[1.265,0.404,0.550], +[0.786,0.750,0.757], +[0.905,0.835,0.841], +[0.864,0.865,0.823], +[0.352,0.360,0.364], +[0.201,0.186,0.200], +[0.143,0.137,0.276], +[0.290,0.132,0.146], +[0.310,0.301,0.304], +[0.212,0.197,0.170], +[0.223,0.174,0.177], +[0.172,0.172,0.172] diff --git a/benchmark/compatible/snowflake/results/3xl.txt b/benchmark/compatible/snowflake/results/3xl.txt new file mode 100644 index 00000000000..8c47256c17c --- /dev/null +++ b/benchmark/compatible/snowflake/results/3xl.txt @@ -0,0 +1,46 @@ +Load time: 42 min 4 sec. +Data size: 12.3 GB + +[0.165,0.061,0.049], +[1.356,1.252,1.502], +[1.287,0.470,0.325], +[0.627,0.331,0.181], +[0.265,0.265,0.483], +[0.887,0.450,0.298], +[0.054,0.066,0.062], +[0.182,0.222,0.194], +[0.408,0.438,0.319], +[0.434,0.462,0.411], +[1.357,0.247,0.412], +[0.343,0.250,0.517], +[0.273,0.289,0.600], +[0.404,0.405,0.392], +[0.379,0.304,0.455], +[0.275,0.271,0.264], +[0.418,0.386,0.388], +[0.417,0.434,0.567], +[0.753,0.759,0.506], +[0.291,0.307,0.400], +[0.954,0.423,0.280], +[0.568,0.562,0.248], +[0.568,0.477,0.496], +[1.458,0.492,0.514], +[0.179,0.300,0.184], +[0.165,0.169,0.176], +[0.197,0.186,0.190], +[0.289,0.547,0.397], +[0.513,0.544,0.632], +[0.766,0.754,0.775], +[0.389,0.374,0.383], +[0.484,0.297,0.286], +[0.505,0.505,0.734], +[0.656,0.598,0.621], +[0.634,0.646,0.609], +[0.309,0.298,0.370], +[0.192,0.219,0.212], +[0.840,0.174,0.139], +[0.172,0.163,0.151], +[0.323,0.296,0.347], +[0.200,0.154,0.144], +[0.191,0.121,0.125], +[0.137,0.233,0.148] diff --git a/benchmark/compatible/snowflake/results/4xl.txt b/benchmark/compatible/snowflake/results/4xl.txt new file mode 100644 index 00000000000..74f09314af7 --- /dev/null +++ b/benchmark/compatible/snowflake/results/4xl.txt @@ -0,0 +1,46 @@ +Load time: 42 min 4 sec. +Data size: 12.3 GB + +[0.164,0.061,0.078], +[2.471,2.436,1.927], +[1.656,0.222,0.639], +[0.336,1.244,0.206], +[0.435,0.414,0.373], +[0.520,0.495,0.326], +[0.052,0.051,0.057], +[0.244,0.515,0.358], +[0.473,0.477,0.659], +[0.706,0.523,0.499], +[0.619,0.361,0.303], +[0.463,0.367,0.290], +[0.385,0.319,0.376], +[0.661,0.436,0.452], +[0.446,0.317,0.336], +[0.504,0.283,0.395], +[0.478,1.395,0.350], +[0.555,1.041,0.993], +[0.565,0.558,1.473], +[0.310,0.684,0.278], +[0.637,1.202,0.249], +[0.467,0.294,0.436], +[0.671,0.478,0.611], +[6.262,0.449,0.425], +[0.476,0.213,0.184], +[0.552,0.241,0.201], +[0.458,0.415,0.402], +[0.339,0.357,0.322], +[0.732,0.549,0.483], +[0.914,0.757,0.743], +[0.718,0.310,0.606], +[0.605,0.363,0.425], +[0.468,0.860,0.784], +[0.868,1.787,0.713], +[0.807,0.691,0.544], +[0.485,0.338,0.672], +[0.263,0.221,0.230], +[0.147,0.145,0.139], +[0.135,0.239,0.136], +[0.322,0.378,0.348], +[0.236,0.138,0.132], +[0.193,0.124,0.139], +[0.146,0.145,0.139] diff --git a/benchmark/compatible/snowflake/results/l.txt b/benchmark/compatible/snowflake/results/l.txt new file mode 100644 index 00000000000..5327f0e48e9 --- /dev/null +++ b/benchmark/compatible/snowflake/results/l.txt @@ -0,0 +1,46 @@ +Load time: 42 min 4 sec. +Data size: 12.3 GB + +[0.067,0.054,0.051], +[1.158,1.409,0.190], +[1.470,0.566,0.715], +[0.892,0.534,0.264], +[0.474,0.499,0.489], +[0.950,0.650,0.699], +[0.069,0.055,0.110], +[0.317,0.307,0.321], +[0.949,0.593,0.654], +[0.713,1.099,0.860], +[0.622,0.370,0.404], +[0.457,0.369,0.414], +[0.610,0.566,0.653], +[0.970,1.006,0.976], +[1.517,0.636,0.603], +[0.532,0.541,0.533], +[1.018,1.001,1.022], +[0.942,0.996,0.940], +[2.246,1.596,1.560], +[0.181,0.184,0.200], +[1.135,0.788,0.609], +[0.669,0.528,0.524], +[1.164,0.827,0.882], +[3.545,2.214,2.107], +[0.559,0.431,0.426], +[0.340,0.296,0.383], +[0.695,0.314,0.368], +[0.628,0.658,0.637], +[1.511,1.385,1,440], +[1.390,1.418,1.322], +[1.107,0.687,0.537], +[1.026,0.737,0.659], +[1.712,1.681,1.728], +[2.141,2.130,2.225], +[2.163,2.157,2.110], +[0.650,0.619,0.627], +[0.204,0.195,0.225], +[0.159,0.152,0.156], +[0.146,0.136,0.150], +[0.359,0.290,0.364], +[0.196,0.129,0.227], +[0.201,0.128,0.143], +[0.176,0.129,0.146] diff --git a/benchmark/compatible/snowflake/results/m.txt b/benchmark/compatible/snowflake/results/m.txt new file mode 100644 index 00000000000..a4a927ebabb --- /dev/null +++ b/benchmark/compatible/snowflake/results/m.txt @@ -0,0 +1,46 @@ +Load time: 42 min 4 sec. +Data size: 12.3 GB + +[0.054,0.062,0.064], +[0.698,0.563,0.407], +[0.586,0.412,0.266], +[1.386,0.363,0.301], +[0.814,0.825,0.984], +[1.303,1.024,1.048], +[0.069,0.066,0.062], +[0.334,0.254,0.270], +[1.043,0.952,0.993], +[1.210,1.209,1.171], +[0.667,0.483,0.456], +[0.543,0.495,0.500], +[1.005,0.889,0.888], +[1.644,1.646,1.652], +[1.054,1.044,0.966], +[0.893,0.874,0.907], +[1.737,1.779,1.837], +[1.518,1.539,1.526], +[3.082,2.818,2.842], +[0.309,0.286,0.256], +[1.594,1.017,0.993], +[0.781,0.853,0.735], +[1.461,1.226,1.080], +[5.308,2.974,2.642], +[0.511,0.625,0.467], +[0.405,0.382,0.439], +[0.601,0.535,0.471], +[0.947,1.624,1.192], +[2.631,2.486,2.490], +[1.938,1.960,1.954], +[1.930,0.830,0.835], +[1.359,1.140,1.062], +[3.599,3.623,3.621], +[3.619,3.741,3.663], +[3.725,3.614,3.786], +[1.149,1.126,1.055], +[0.202,0.207,0.196], +[0.152,0.139,0.145], +[0.149,0.144,0.148], +[0.383,0.287,0.294], +[0.203,0.137,0.119], +[0.200,0.312,0.137], +[0.149,0.130,0.214] diff --git a/benchmark/compatible/snowflake/results/s.txt b/benchmark/compatible/snowflake/results/s.txt new file mode 100644 index 00000000000..e4cdc207b76 --- /dev/null +++ b/benchmark/compatible/snowflake/results/s.txt @@ -0,0 +1,46 @@ +Load time: 42 min 4 sec. +Data size: 12.3 GB + +[0.186,0.060,0.062], +[0.980,0.574,0.311], +[0.977,0.554,0.426], +[0.686,0.573,0.404], +[1.386,1.384,1.349], +[1.871,1.697,1.704], +[0.052,0.059,0.227], +[0.309,0.536,0.508], +[1.768,1.631,1.635], +[2.039,2.219,1.908], +[0.807,0.647,0.587], +[0.763,0.690,0.631], +[1.403,1.586,1.404], +[2.593,2.584,2.554], +[1.670,1.538,1.653], +[1.659,1.509,1.514], +[2.875,2.990,2.998], +[2.605,2.549,2.598], +[6.120,5.894,5.766], +[0.320,0.431,0.416], +[2.406,1.703,1.609], +[1.189,1.186,1.163], +[2.104,1.441,1.370], +[7.144,5.174,4.139], +[0.839,0.659,0.641], +[0.527,0.518,0.509], +[0.633,0.621,0.695], +[1.491,1.509,1.514], +[4.848,4.485,4.571], +[3.067,3.106,3.098], +[1.521,1.224,1.236], +[1.839,1.690,1.497], +[5.692,5.751,6.087], +[6.733,6.755,6.712], +[6.722,6.709,6.676], +[1.704,1.686,1.676], +[0.203,0.231,0.218], +[0.151,0.134,0.214], +[0.140,0.156,0.163], +[0.317,0.328,0.319], +[0.166,0.133,0.141], +[0.166,0.120,0.140], +[0.120,0.119,0.126] diff --git a/benchmark/compatible/snowflake/results/xl.txt b/benchmark/compatible/snowflake/results/xl.txt new file mode 100644 index 00000000000..589c229d7bc --- /dev/null +++ b/benchmark/compatible/snowflake/results/xl.txt @@ -0,0 +1,46 @@ +Load time: 42 min 4 sec. +Data size: 12.3 GB + +[0.071,0.053,0.057], +[0.998,0.610,0.240], +[0.420,1.138,1.051], +[0.653,0.264,0.178], +[0.352,0.312,0.349], +[1.126,0.431,0.420], +[0.067,0.057,0.054], +[0.225,0.217,0.200], +[0.617,0.366,0.371], +[1.006,0.541,0.498], +[0.463,0.425,0.293], +[0.431,0.360,0.339], +[0.392,0.371,0.386], +[0.588,0.581,0.590], +[0.634,0.414,0.400], +[0.368,0.410,0.388], +[0.594,0.639,0.663], +[0.616,0.581,0.569], +[1.092,0.933,0.901], +[0.493,0.213,0.160], +[0.886,0.480,0.442], +[0.448,0.337,0.399], +[0.840,0.572,0.505], +[2.251,1.230,0.959], +[0.295,0.253,0.241], +[0.214,0.239,0.278], +[0.261,0.232,0.314], +[0.422,0.429,0.403], +[0.892,0.934,0.883], +[1.041,1.017,1.009], +[0.715,0.442,0.363], +[0.845,0.413,0.461], +[1.101,1.085,1.102], +[1.294,1.272,1.339], +[1.839,1.327,1.241], +[0.439,0.399,0.393], +[0.199,0.211,0.190], +[0.157,0.143,0.140], +[0.145,0.157,0.141], +[0.331,0.291,0.333], +[0.173,0.214,0.138], +[0.189,0.150,0.159], +[0.135,0.149,0.138] diff --git a/benchmark/compatible/snowflake/results/xs.txt b/benchmark/compatible/snowflake/results/xs.txt new file mode 100644 index 00000000000..eb04e2f9576 --- /dev/null +++ b/benchmark/compatible/snowflake/results/xs.txt @@ -0,0 +1,46 @@ +Load time: 42 min 4 sec. +Data size: 12.3 GB + +[0.169,0.055,0.056], +[1.184,0.582,0.386], +[1.350,0.560,0.568], +[1.270,0.554,0.538], +[2.516,2.564,2.506], +[2.935,2.649,2.670], +[0.052,0.050,0.064], +[0.383,0.387,0.397], +[3.249,2.993,3.014], +[3.589,3.627,3.887], +[1.243,0.986,0.966], +[1.325,1.080,1.073], +[2.038,2.046,2.035], +[3.738,3.626,3.718], +[2.318,2.159,2.176], +[2.733,2.637,2.668], +[5.607,5.683,5.667], +[3.978,3.923,3.879], +[10.085,9.871,9.844], +[0.450,0.375,0.469], +[5.474,3.103,3.060], +[2.012,1.982,1.971], +[3.365,2.471,2.501], +[11.960,10.619,9.518], +[1.074,1.059,1.026], +[0.856,0.846,0.879], +[1.100,1.085,1.083], +[3.057,3.228,3.117], +[9.406,9.019,9.158], +[6.196,6.243,6.911], +[2.906,2.343,2.017], +[2.954,2.666,2.565], +[9.459,9.565,9.557], +[9.555,9.529,9.368], +[9.409,9.185,9.294], +[2.796,2.880,2.685], +[0.299,0.249,0.262], +[0.156,0.145,0.180], +[0.147,0.146,0.160], +[0.371,0.357,0.356], +[0.166,0.133,0.155], +[0.218,0.140,0.135], +[0.140,0.152,0.158] diff --git a/benchmark/compatible/snowflake/run.sh b/benchmark/compatible/snowflake/run.sh new file mode 100644 index 00000000000..35739a4d7f3 --- /dev/null +++ b/benchmark/compatible/snowflake/run.sh @@ -0,0 +1,3 @@ +#!/bin/bash + +sed -r -e 's/^(.*)$/\1 \1 \1/' queries.sql | snowsql --region eu-central-1 --schemaname PUBLIC --dbname HITS --warehouse TEST From b01b4dddc463af5ec5aab60ac1944947554e1021 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 03:56:03 +0200 Subject: [PATCH 132/627] Add Snowflake --- benchmark/compatible/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index 29a5de71c1d..ea937c69b8b 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -34,7 +34,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] Presto/Trino - [ ] Amazon Athena - [x] Bigquery (without publishing) -- [ ] Snowflake +- [x] Snowflake - [ ] Rockset - [ ] CockroachDB - [ ] CockroachDB Serverless From 6c18389150c687e096f4012639213209582157d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 06:11:13 +0200 Subject: [PATCH 133/627] Add Athena --- benchmark/compatible/athena/README.md | 22 +++- .../compatible/athena/create_partitioned.sql | 112 ++++++++++++++++++ .../athena/{create.sql => create_single.sql} | 0 benchmark/compatible/athena/queries.sql | 43 +++++++ .../compatible/athena/results/partitioned.txt | 46 +++++++ .../compatible/athena/results/single.txt | 46 +++++++ benchmark/compatible/athena/run.sh | 9 ++ benchmark/compatible/aurora-mysql/README.md | 4 +- benchmark/compatible/aurora-mysql/queries.sql | 2 +- .../compatible/aurora-postgresql/README.md | 59 +++++++++ .../compatible/aurora-postgresql/create.sql | 109 +++++++++++++++++ .../compatible/aurora-postgresql/queries.sql | 43 +++++++ .../aurora-postgresql/results/16acu.txt | 2 + benchmark/compatible/aurora-postgresql/run.sh | 10 ++ .../compatible/clickhouse-local/benchmark.sh | 14 +++ .../compatible/clickhouse-local/create.sql | 109 +++++++++++++++++ .../compatible/clickhouse-local/queries.sql | 43 +++++++ .../results/c6a.4xlarge.partitioned.txt | 48 ++++++++ .../results/c6a.4xlarge.single.txt | 48 ++++++++ benchmark/compatible/clickhouse-local/run.sh | 20 ++++ benchmark/compatible/clickhouse/benchmark.sh | 2 +- .../clickhouse/results/c6a.metal.txt | 48 ++++++++ 22 files changed, 830 insertions(+), 9 deletions(-) create mode 100644 benchmark/compatible/athena/create_partitioned.sql rename benchmark/compatible/athena/{create.sql => create_single.sql} (100%) create mode 100644 benchmark/compatible/athena/queries.sql create mode 100644 benchmark/compatible/athena/results/partitioned.txt create mode 100644 benchmark/compatible/athena/results/single.txt create mode 100644 benchmark/compatible/athena/run.sh create mode 100644 benchmark/compatible/aurora-postgresql/README.md create mode 100644 benchmark/compatible/aurora-postgresql/create.sql create mode 100644 benchmark/compatible/aurora-postgresql/queries.sql create mode 100644 benchmark/compatible/aurora-postgresql/results/16acu.txt create mode 100755 benchmark/compatible/aurora-postgresql/run.sh create mode 100755 benchmark/compatible/clickhouse-local/benchmark.sh create mode 100644 benchmark/compatible/clickhouse-local/create.sql create mode 100644 benchmark/compatible/clickhouse-local/queries.sql create mode 100644 benchmark/compatible/clickhouse-local/results/c6a.4xlarge.partitioned.txt create mode 100644 benchmark/compatible/clickhouse-local/results/c6a.4xlarge.single.txt create mode 100755 benchmark/compatible/clickhouse-local/run.sh create mode 100644 benchmark/compatible/clickhouse/results/c6a.metal.txt diff --git a/benchmark/compatible/athena/README.md b/benchmark/compatible/athena/README.md index f3717a44b1b..ff94029b429 100644 --- a/benchmark/compatible/athena/README.md +++ b/benchmark/compatible/athena/README.md @@ -1,9 +1,21 @@ Data Sources -> AWS Data Catalog -> Create Table -> Use S3 bucket data -Note: Athena does not support files. Only directories: - -``` -aws s3 cp s3://clickhouse-public-datasets/hits_compatible/hits.parquet s3://clickhouse-public-datasets/hits_compatible/athena/hits.parquet -``` +Note: Athena does not support files. Only directories. Go to query editor and run `create.sql`. + +``` +sudo apt-get install -y jq +export OUTPUT='s3://athena-experiments-milovidov/' + +./run1.sh | tee ids.txt +``` + +Wait a few minutes. Then: + +``` +cat ids.txt | xargs -I{} aws --output json athena get-query-execution --query-execution-id {} | tee log.txt + +cat log.txt | grep -P 'TotalExecutionTimeInMillis|FAILED' | grep -oP '\d+|FAILED' | + awk '{ if ($1 == "ERROR") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : ($1 / 1000); if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' +``` diff --git a/benchmark/compatible/athena/create_partitioned.sql b/benchmark/compatible/athena/create_partitioned.sql new file mode 100644 index 00000000000..b0f9e5d0be5 --- /dev/null +++ b/benchmark/compatible/athena/create_partitioned.sql @@ -0,0 +1,112 @@ +CREATE EXTERNAL TABLE IF NOT EXISTS `test`.`hits` ( +`watchid` bigint, +`javaenable` smallint, +`title` string, +`goodevent` smallint, +`eventtime` timestamp, +`eventdate` date, +`counterid` int, +`clientip` int, +`regionid` int, +`userid` bigint, +`counterclass` smallint, +`os` smallint, +`useragent` smallint, +`url` string, +`referer` string, +`isrefresh` smallint, +`referercategoryid` smallint, +`refererregionid` int, +`urlcategoryid` smallint, +`urlregionid` int, +`resolutionwidth` smallint, +`resolutionheight` smallint, +`resolutiondepth` smallint, +`flashmajor` smallint, +`flashminor` smallint, +`flashminor2` string, +`netmajor` smallint, +`netminor` smallint, +`useragentmajor` smallint, +`useragentminor` string, +`cookieenable` smallint, +`javascriptenable` smallint, +`ismobile` smallint, +`mobilephone` smallint, +`mobilephonemodel` string, +`params` string, +`ipnetworkid` int, +`traficsourceid` smallint, +`searchengineid` smallint, +`searchphrase` string, +`advengineid` smallint, +`isartifical` smallint, +`windowclientwidth` smallint, +`windowclientheight` smallint, +`clienttimezone` smallint, +`clienteventtime` timestamp, +`silverlightversion1` smallint, +`silverlightversion2` smallint, +`silverlightversion3` int, +`silverlightversion4` smallint, +`pagecharset` string, +`codeversion` int, +`islink` smallint, +`isdownload` smallint, +`isnotbounce` smallint, +`funiqid` bigint, +`originalurl` string, +`hid` int, +`isoldcounter` smallint, +`isevent` smallint, +`isparameter` smallint, +`dontcounthits` smallint, +`withhash` smallint, +`hitcolor` string, +`localeventtime` timestamp, +`age` smallint, +`sex` smallint, +`income` smallint, +`interests` smallint, +`robotness` smallint, +`remoteip` int, +`windowname` int, +`openername` int, +`historylength` smallint, +`browserlanguage` string, +`browsercountry` string, +`socialnetwork` string, +`socialaction` string, +`httperror` smallint, +`sendtiming` int, +`dnstiming` int, +`connecttiming` int, +`responsestarttiming` int, +`responseendtiming` int, +`fetchtiming` int, +`socialsourcenetworkid` smallint, +`socialsourcepage` string, +`paramprice` bigint, +`paramorderid` string, +`paramcurrency` string, +`paramcurrencyid` smallint, +`openstatservicename` string, +`openstatcampaignid` string, +`openstatadid` string, +`openstatsourceid` string, +`utmsource` string, +`utmmedium` string, +`utmcampaign` string, +`utmcontent` string, +`utmterm` string, +`fromtag` string, +`hasgclid` smallint, +`refererhash` bigint, +`urlhash` bigint, +`clid` int +) +ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' +WITH SERDEPROPERTIES ( +'serialization.format' = '1' +) LOCATION 's3://clickhouse-public-datasets/hits_compatible/athena_partitioned' +TBLPROPERTIES ('has_encrypted_data'='false'); diff --git a/benchmark/compatible/athena/create.sql b/benchmark/compatible/athena/create_single.sql similarity index 100% rename from benchmark/compatible/athena/create.sql rename to benchmark/compatible/athena/create_single.sql diff --git a/benchmark/compatible/athena/queries.sql b/benchmark/compatible/athena/queries.sql new file mode 100644 index 00000000000..ef8c727ead6 --- /dev/null +++ b/benchmark/compatible/athena/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, extract(minute FROM EventTime), SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC OFFSET 1000 LIMIT 10; +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 >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN (SearchEngineID = 0 AND AdvEngineID = 0) THEN Referer ELSE '' END, URL ORDER BY PageViews DESC OFFSET 1000 LIMIT 10; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC OFFSET 100 LIMIT 10; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC OFFSET 10000 LIMIT 10; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-14' AND EventDate <= DATE '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) OFFSET 1000 LIMIT 10; diff --git a/benchmark/compatible/athena/results/partitioned.txt b/benchmark/compatible/athena/results/partitioned.txt new file mode 100644 index 00000000000..a9a2ddde85a --- /dev/null +++ b/benchmark/compatible/athena/results/partitioned.txt @@ -0,0 +1,46 @@ +Load time: 0 +Data size: 13.8 GB + +[2.777,3.275,2.925], +[1.503,3.136,4.003], +[4.544,3.833,3.64], +[3.9,2.514,3.522], +[3.46,2.186,3.244], +[3.624,2.742,3.185], +[2.21,1.984,3.123], +[3.207,2.403,2.685], +[2.936,2.014,3.869], +[8.333,7.102,4.434], +[7.401,4.697,3.155], +[4.214,3.065,4.748], +[6.207,4.213,2.576], +[3.428,3.085,3.401], +[2.92,3.3,3.278], +[2.205,2.558,2.419], +[4.641,3.888,2.155], +[3.219,2.822,3.292], +[3.23,3.579,4.31], +[2.288,3.543,3.95], +[3.032,2.859,2.807], +[3.926,3.247,2.928], +[4.477,4.048,4.392], +[7.407,6.375,6.123], +[2.611,2.872,2.827], +[2.566,2.567,3.6], +[3.673,3.733,2.925], +[2.426,3.218,2.78], +[5.125,3.778,4.25], +[4.565,4.03,4.066], +[3.628,3.219,2.953], +[6.207,5.973,3.158], +[4.339,5.601,4.234], +[2.618,3.107,3.433], +[4.661,2.79,2.846], +[2.373,1.629,2.734], +[2.721,2.15,1.962], +[3.207,2.154,2.186], +[2.453,2.477,3.217], +[2.691,4.732,3.584], +[2.589,2.613,3.231], +[1.926,3.617,1.82], +[1.506,2.404,2.343] diff --git a/benchmark/compatible/athena/results/single.txt b/benchmark/compatible/athena/results/single.txt new file mode 100644 index 00000000000..6a1c6fdbec1 --- /dev/null +++ b/benchmark/compatible/athena/results/single.txt @@ -0,0 +1,46 @@ +Load time: 0 +Data size: 13.8 GB + +[2.268,1.327,2.137], +[3.427,2.248,3.605], +[3.254,2.548,2.316], +[3.025,2.314,3.003], +[2.264,2.876,4.213], +[3.044,2.745,2.698], +[2.732,2.199,2.659], +[2.022,3.692,3.072], +[2.746,2.477,2.785], +[3.53,2.782,4.031], +[2.709,2.047,2.853], +[2.318,1.969,3.4], +[2.635,1.935,2.707], +[3.049,3.38,3.071], +[3.661,2.387,2.476], +[2.479,2.591,2.21], +[3.093,3.698,4.351], +[3.479,3.236,2.274], +[4.36,2.97,3.457], +[2.525,2.384,3.328], +[3.34,3.174,3.409], +[3.163,2.971,3.034], +[2.999,3.539,2.906], +[6.454,7.597,7.858], +[2.754,1.951,2.645], +[2.852,3.018,2.718], +[2.513,2.678,2.417], +[3.293,2.521,2.771], +[4.392,3.863,3.981], +[3.658,4.246,4.027], +[3.028,3.87,2.337], +[2.923,3.635,3.591], +[3.142,4.105,3.15], +[3.66,3.187,4.745], +[2.652,2.695,2.742], +[2.262,2.776,1.815], +[1.881,2.212,2.053], +[1.934,2.551,1.524], +[2.069,2.26,1.805], +[2.626,2.902,2.793], +[1.791,2.082,2.481], +[3.757,2.6,1.946], +[2.608,1.994,3.967] diff --git a/benchmark/compatible/athena/run.sh b/benchmark/compatible/athena/run.sh new file mode 100644 index 00000000000..f1ce446f0a8 --- /dev/null +++ b/benchmark/compatible/athena/run.sh @@ -0,0 +1,9 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + for i in $(seq 1 $TRIES); do + aws athena --output json start-query-execution --query-execution-context 'Database=test' --result-configuration "OutputLocation=${OUTPUT}" --query-string "${query}" | jq '.QueryExecutionId' + done +done diff --git a/benchmark/compatible/aurora-mysql/README.md b/benchmark/compatible/aurora-mysql/README.md index f526c67df9c..89aca73ebd8 100644 --- a/benchmark/compatible/aurora-mysql/README.md +++ b/benchmark/compatible/aurora-mysql/README.md @@ -28,7 +28,7 @@ Find "Security", click on the group in "VPC security groups". Edit "Inbound rules". Add "Custom TCP", port 3306, from 0.0.0.0/0. ``` -export HOST="database-1.cluster-....eu-central-1.rds.amazonaws.com" +export HOST="database-1.cluster-cnkeohbxcwr1.eu-central-1.rds.amazonaws.com" export PASSWORD="..." mysql -h "${HOST}" -u admin --password="${PASSWORD}" -e "CREATE DATABASE test" @@ -47,7 +47,7 @@ time mysql --local-infile=1 -h "${HOST}" -u admin --password="${PASSWORD}" test > 128m7.318s -Go to "Monitoring", find "\[Billed\] Volume Bytes Used". +Go to "Monitoring", find "[Billed] Volume Bytes Used". > 83.46 GiB diff --git a/benchmark/compatible/aurora-mysql/queries.sql b/benchmark/compatible/aurora-mysql/queries.sql index ea2bde47802..3f04f648222 100644 --- a/benchmark/compatible/aurora-mysql/queries.sql +++ b/benchmark/compatible/aurora-mysql/queries.sql @@ -16,7 +16,7 @@ SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, extract(minute FROM EventTime), SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; diff --git a/benchmark/compatible/aurora-postgresql/README.md b/benchmark/compatible/aurora-postgresql/README.md new file mode 100644 index 00000000000..906258f65c5 --- /dev/null +++ b/benchmark/compatible/aurora-postgresql/README.md @@ -0,0 +1,59 @@ +Select Aurora. +Select Aurora for PostgreSQL. +Select the latest version PostgreSQL 14.3. +Select Production template. + +Database: database-2 +User name: postgres +Master password: vci43A32#1 + +Select serverless. +16 minimum and maximum ACU (32 GB RAM). +Don't create an Aurora replica. +Public access: yes. +Turn off DevOps Guru. + +Creation took around 15 seconds. +But creation of endpoints took longer (around 5..10 minutes). + +Find the writer instance endpoint. +Example: database-1.cluster-cnkeohbxcwr1.eu-central-1.rds.amazonaws.com + +``` +sudo apt-get update +sudo apt-get install -y postgresql-client +``` + +Find "Security", click on the group in "VPC security groups". +Edit "Inbound rules". Add "Custom TCP", port 5432, from 0.0.0.0/0. + +``` +export HOST="database-2-instance-1.cnkeohbxcwr1.eu-central-1.rds.amazonaws.com" +echo "*:*:*:*:..." > .pgpass +chmod 400 .pgpass +``` + +Load the data + +``` +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +psql -U postgres -h "${HOST}" -t -c 'CREATE DATABASE test' +psql -U postgres -h "${HOST}" test -t < create.sql +psql -U postgres -h "${HOST}" test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" +``` + +> COPY 99997497 +> Time: 2126515.516 ms (35:26.516) + +Go to "Monitoring", find "[Billed] Volume Bytes Used". + +> 48.6 GiB + +``` +./run.sh 2>&1 | tee log.txt + +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' +``` diff --git a/benchmark/compatible/aurora-postgresql/create.sql b/benchmark/compatible/aurora-postgresql/create.sql new file mode 100644 index 00000000000..1850bffedce --- /dev/null +++ b/benchmark/compatible/aurora-postgresql/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +); diff --git a/benchmark/compatible/aurora-postgresql/queries.sql b/benchmark/compatible/aurora-postgresql/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/aurora-postgresql/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/aurora-postgresql/results/16acu.txt b/benchmark/compatible/aurora-postgresql/results/16acu.txt new file mode 100644 index 00000000000..7a31464a23f --- /dev/null +++ b/benchmark/compatible/aurora-postgresql/results/16acu.txt @@ -0,0 +1,2 @@ +Load time: 2127 seconds +Data size: 48.6 GiB diff --git a/benchmark/compatible/aurora-postgresql/run.sh b/benchmark/compatible/aurora-postgresql/run.sh new file mode 100755 index 00000000000..f2c694359f1 --- /dev/null +++ b/benchmark/compatible/aurora-postgresql/run.sh @@ -0,0 +1,10 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + echo "$query"; + for i in $(seq 1 $TRIES); do + psql -U postgres -h "${HOST}" test -t -c '\timing' -c "$query" | grep 'Time' + done; +done; diff --git a/benchmark/compatible/clickhouse-local/benchmark.sh b/benchmark/compatible/clickhouse-local/benchmark.sh new file mode 100755 index 00000000000..9b660cc2cef --- /dev/null +++ b/benchmark/compatible/clickhouse-local/benchmark.sh @@ -0,0 +1,14 @@ +#!/bin/bash + +# Install + +curl https://clickhouse.com/ | sh + +# wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.parquet' +seq 0 99 | xargs -P100 -I{} bash -c 'wget --continue https://datasets.clickhouse.com/hits_compatible/athena_partitioned/hits_{}.parquet' + +# Run the queries + +./run.sh + +du -b hits.parquet diff --git a/benchmark/compatible/clickhouse-local/create.sql b/benchmark/compatible/clickhouse-local/create.sql new file mode 100644 index 00000000000..180a5806e46 --- /dev/null +++ b/benchmark/compatible/clickhouse-local/create.sql @@ -0,0 +1,109 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +) +ENGINE = File(Parquet, 'hits_*.parquet'); diff --git a/benchmark/compatible/clickhouse-local/queries.sql b/benchmark/compatible/clickhouse-local/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/compatible/clickhouse-local/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/clickhouse-local/results/c6a.4xlarge.partitioned.txt b/benchmark/compatible/clickhouse-local/results/c6a.4xlarge.partitioned.txt new file mode 100644 index 00000000000..ebddbc5a678 --- /dev/null +++ b/benchmark/compatible/clickhouse-local/results/c6a.4xlarge.partitioned.txt @@ -0,0 +1,48 @@ +Machine: c6a.4xlarge, EBS 500GB gp2, Ubuntu 22.04. + +Loading time: 0 +Dataset size: 14 737 666 736 bytes + +[0.850, 0.102, 0.114], +[1.342, 0.090, 0.099], +[2.547, 0.179, 0.189], +[1.681, 0.245, 0.252], +[2.704, 1.680, 1.648], +[2.194, 1.460, 1.489], +[0.832, 0.113, 0.102], +[1.371, 0.106, 0.101], +[2.240, 0.790, 0.825], +[4.548, 1.021, 1.026], +[3.094, 0.552, 0.552], +[3.088, 0.623, 0.630], +[2.017, 1.170, 1.165], +[4.319, 1.677, 1.708], +[2.157, 1.496, 1.500], +[1.629, 1.138, 1.139], +[5.026, 3.267, 3.241], +[4.142, 2.303, 2.319], +[8.295, 5.569, 5.629], +[1.331, 0.255, 0.252], +[10.712, 3.668, 3.786], +[13.053, 4.185, 4.202], +[24.170, 7.935, 8.008], +[55.965, 23.933, 23.071], +[4.417, 0.947, 0.974], +[1.793, 0.698, 0.690], +[4.376, 0.955, 0.956], +[11.731, 4.385, 4.321], +[11.403, 8.549, 8.288], +[2.764, 2.754, 2.735], +[5.096, 1.262, 1.273], +[9.515, 1.682, 1.688], +[10.325, 6.745, 6.608], +[11.686, 6.261, 6.242], +[11.769, 6.301, 6.364], +[1.675, 1.490, 1.495], +[14.937, 3.631, 3.604], +[14.187, 3.609, 3.631], +[14.842, 3.769, 3.741], +[22.222, 6.355, 6.263], +[7.212, 0.836, 0.838], +[7.863, 0.716, 0.718], +[5.120, 0.587, 0.574] diff --git a/benchmark/compatible/clickhouse-local/results/c6a.4xlarge.single.txt b/benchmark/compatible/clickhouse-local/results/c6a.4xlarge.single.txt new file mode 100644 index 00000000000..9674dbec715 --- /dev/null +++ b/benchmark/compatible/clickhouse-local/results/c6a.4xlarge.single.txt @@ -0,0 +1,48 @@ +Machine: c6a.4xlarge, EBS 500GB gp2, Ubuntu 22.04. + +Loading time: 0 +Dataset size: 14 779 976 446 bytes + +[1.176, 0.251, 0.249], +[1.037, 0.134, 0.111], +[1.609, 0.403, 0.369], +[1.616, 0.372, 0.370], +[3.008, 2.338, 2.266], +[9.061, 7.537, 7.535], +[1.206, 0.191, 0.187], +[0.882, 0.144, 0.135], +[4.610, 3.406, 3.256], +[6.712, 4.479, 4.469], +[4.081, 2.413, 2.394], +[3.894, 2.719, 2.691], +[7.651, 6.436, 6.243], +[10.765, 8.043, 7.894], +[9.860, 8.945, 8.235], +[7.159, 5.815, 5.814], +[20.916, 18.159, 18.013], +[20.952, 17.862, 17.850], +[37.585, 32.649, 32.487], +[1.767, 0.401, 0.393], +[23.713, 15.687, 15.755], +[28.700, 19.241, 19.198], +[50.740, 33.161, 33.011], +[152.485, 117.417, 118.178], +[7.606, 4.491, 5.326], +[4.331, 4.214, 3.587], +[6.743, 4.486, 5.357], +[22.910, 15.043, 15.183], +[43.342, 37.167, 36.842], +[11.807, 4.490, 4.546], +[9.557, 6.349, 6.263], +[13.964, 8.493, 8.464], +[38.110, 33.642, 33.996], +[41.266, 35.080, 27.073], +[34.056, 26.814, 26.902], +[8.855, 7.548, 7.475], +[22.596, 12.615, 12.669], +[23.217, 13.956, 13.831], +[22.528, 21.601, 13.207], +[37.890, 23.115, 22.955], +[6.490, 1.548, 1.522], +[6.413, 1.474, 1.416], +[3.796, 1.339, 1.316] diff --git a/benchmark/compatible/clickhouse-local/run.sh b/benchmark/compatible/clickhouse-local/run.sh new file mode 100755 index 00000000000..6c6980ec68a --- /dev/null +++ b/benchmark/compatible/clickhouse-local/run.sh @@ -0,0 +1,20 @@ +#!/bin/bash + +TRIES=3 +QUERY_NUM=1 +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + + echo -n "[" + for i in $(seq 1 $TRIES); do + RES=$(./clickhouse local --time --format Null --multiquery --query="$(cat create.sql); $query" 2>&1 | tail -n1) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + + echo "${QUERY_NUM},${i},${RES}" >> result.csv + done + echo "]," + + QUERY_NUM=$((QUERY_NUM + 1)) +done diff --git a/benchmark/compatible/clickhouse/benchmark.sh b/benchmark/compatible/clickhouse/benchmark.sh index 5d601d87612..2c06df90164 100755 --- a/benchmark/compatible/clickhouse/benchmark.sh +++ b/benchmark/compatible/clickhouse/benchmark.sh @@ -19,4 +19,4 @@ clickhouse-client --time --query "INSERT INTO hits FORMAT TSV" < hits.tsv ./run.sh -du -bcs /var/lib/clickhouse +clickhouse-client --query "SELECT total_bytes FROM system.tables WHERE name = 'hits' AND database = 'default'" diff --git a/benchmark/compatible/clickhouse/results/c6a.metal.txt b/benchmark/compatible/clickhouse/results/c6a.metal.txt new file mode 100644 index 00000000000..3505add5161 --- /dev/null +++ b/benchmark/compatible/clickhouse/results/c6a.metal.txt @@ -0,0 +1,48 @@ +Machine: c6a.metal, EBS 500GB gp2, Ubuntu 22.04. + +Loading time: 136.869 seconds +Dataset size: 14 571 706 777 bytes + +[0.011, 0.001, 0.001], +[0.040, 0.015, 0.013], +[0.045, 0.021, 0.023], +[0.090, 0.023, 0.023], +[1.922, 1.565, 1.576], +[0.961, 0.737, 0.739], +[0.040, 0.023, 0.018], +[0.032, 0.028, 0.028], +[0.321, 0.287, 0.275], +[0.632, 0.284, 0.287], +[0.166, 0.124, 0.118], +[0.235, 0.100, 0.102], +[1.006, 0.182, 0.159], +[1.637, 0.216, 0.213], +[0.871, 0.174, 0.177], +[0.258, 0.148, 0.148], +[1.804, 0.370, 0.358], +[1.235, 0.275, 0.278], +[3.143, 0.854, 0.815], +[0.071, 0.024, 0.016], +[8.816, 0.215, 0.155], +[10.239, 0.203, 0.173], +[19.179, 0.388, 0.357], +[43.152, 0.824, 0.823], +[1.821, 0.059, 0.052], +[0.992, 0.045, 0.051], +[2.539, 0.063, 0.058], +[9.258, 0.300, 0.278], +[7.923, 0.961, 0.936], +[0.445, 0.431, 0.428], +[1.367, 0.131, 0.113], +[4.819, 0.205, 0.175], +[3.808, 0.739, 0.726], +[8.935, 0.607, 0.600], +[8.988, 0.634, 0.615], +[0.242, 0.220, 0.226], +[0.075, 0.058, 0.056], +[0.038, 0.028, 0.026], +[0.043, 0.028, 0.021], +[0.172, 0.127, 0.119], +[0.028, 0.018, 0.017], +[0.027, 0.019, 0.014], +[0.018, 0.026, 0.015] From b7139777886c6192a3c615fa4a63c02a247a7840 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 06:38:11 +0200 Subject: [PATCH 134/627] Results for AWS Aurora for MySQL --- benchmark/compatible/README.md | 2 +- .../compatible/aurora-mysql/results/16acu.txt | 43 +++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index ea937c69b8b..ce5133ffc6e 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -32,7 +32,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [x] Redshift - [x] Redshift Serverless - [ ] Presto/Trino -- [ ] Amazon Athena +- [x] Amazon Athena - [x] Bigquery (without publishing) - [x] Snowflake - [ ] Rockset diff --git a/benchmark/compatible/aurora-mysql/results/16acu.txt b/benchmark/compatible/aurora-mysql/results/16acu.txt index 8b2954b9d26..386234b161b 100644 --- a/benchmark/compatible/aurora-mysql/results/16acu.txt +++ b/benchmark/compatible/aurora-mysql/results/16acu.txt @@ -1,2 +1,45 @@ Load time: 128m7.318s Data size: 83.46 GiB + +[740.42,739.91,746.65], +[828.2,835.67,832.87], +[830.08,830.98,832.38], +[829.88,832.83,830.87], +[845.99,842.4,843.21], +[869.51,870.69,869.75], +[823.77,829.08,825.54], +[827.74,832.87,829.25], +[916.26,909.46,929.17], +[946.49,939.27,932.32], +[852.37,857.69,854.74], +[857.99,864.05,825.14], +[null,null,null], +[863.37,860.2,865.62], +[null,null,null], +[891.84,895.28,893.68], +[null,null,null], +[null,null,null], +[1420.12,1419.34,1445.08], +[28.94,0.21,0.21], +[917.64,917.56,916.92], +[923.47,921.7,923.82], +[919.95,918.37,920.17], +[1002.19,1002.07,1001.2], +[902.23,902.65,901.8], +[901.17,900.02,898.3], +[900.04,898.89,903.35], +[901.78,902.71,901.28], +[null,null,null], +[1153.29,1154,1156.46], +[862.57,863.35,859.69], +[923.14,921.1,923.92], +[1370.78,1401.72,1401.44], +[1454.67,1455.55,1458.79], +[1463.31,1466.75,1461.83], +[941.03,944.07,937.23], +[7.42,2.80,2.77], +[2.57,2.52,2.59], +[1.50,1.52,1.59], +[3.62,3.57,3.61], +[0.95,0.94,0.94], +[0.90,0.92,0.91] From d0e84390c7a5f489302bcd9a21838288e825713c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 4 Jul 2022 07:17:27 +0200 Subject: [PATCH 135/627] Added note that development options are not supported --- docs/en/development/developer-instruction.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index a35f4c7eba5..77ddae6a756 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -290,7 +290,9 @@ Note that the split build has several drawbacks: * You cannot run the integration tests since they only work a single complete binary. * You can't easily copy the binaries elsewhere. Instead of moving a single binary you'll need to copy all binaries and libraries. -If you are not interested in functionality provided by third-party libraries, you can speed up the build further using `cmake` options +If you are not interested in functionality provided by third-party libraries, you can further speed up the build using `cmake` options ``` -DENABLE_LIBRARIES=0 -DENABLE_EMBEDDED_COMPILER=0 ``` + +In case of problems with any of the development options, you are on your own! From cca94f01ae2ad3c939b13d1cb1eab57147948008 Mon Sep 17 00:00:00 2001 From: lingpeng0314 Date: Mon, 4 Jul 2022 10:26:19 +0800 Subject: [PATCH 136/627] add function tryGetKeeperDispatcher for context - to make sure other case will get error if dispatcher not initialized --- programs/keeper/TinyContext.cpp | 9 +++++++++ programs/keeper/TinyContext.h | 1 + src/Interpreters/AsynchronousMetrics.cpp | 2 +- src/Interpreters/Context.cpp | 9 +++++++++ src/Interpreters/Context.h | 1 + 5 files changed, 21 insertions(+), 1 deletion(-) diff --git a/programs/keeper/TinyContext.cpp b/programs/keeper/TinyContext.cpp index 5e1de81c7ca..09174838c04 100644 --- a/programs/keeper/TinyContext.cpp +++ b/programs/keeper/TinyContext.cpp @@ -41,6 +41,15 @@ void TinyContext::initializeKeeperDispatcher([[maybe_unused]] bool start_async) } std::shared_ptr TinyContext::getKeeperDispatcher() const +{ + std::lock_guard lock(keeper_dispatcher_mutex); + if (!keeper_dispatcher) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests"); + + return keeper_dispatcher; +} + +std::shared_ptr TinyContext::tryGetKeeperDispatcher() const { std::lock_guard lock(keeper_dispatcher_mutex); return keeper_dispatcher; diff --git a/programs/keeper/TinyContext.h b/programs/keeper/TinyContext.h index a53a6d0377d..c5a4edb4e47 100644 --- a/programs/keeper/TinyContext.h +++ b/programs/keeper/TinyContext.h @@ -13,6 +13,7 @@ class TinyContext: public std::enable_shared_from_this { public: std::shared_ptr getKeeperDispatcher() const; + std::shared_ptr tryGetKeeperDispatcher() const; void initializeKeeperDispatcher(bool start_async) const; void shutdownKeeperDispatcher() const; void updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config); diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 9a41ae8dd0b..bb2257eeb02 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1470,7 +1470,7 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } #if USE_NURAFT { - auto keeper_dispatcher = getContext()->getKeeperDispatcher(); + auto keeper_dispatcher = getContext()->tryGetKeeperDispatcher(); if (keeper_dispatcher) { size_t is_leader = 0; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7658b350c6c..b83b6420548 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2094,6 +2094,15 @@ void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) cons #if USE_NURAFT std::shared_ptr & Context::getKeeperDispatcher() const +{ + std::lock_guard lock(shared->keeper_dispatcher_mutex); + if (!shared->keeper_dispatcher) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests"); + + return shared->keeper_dispatcher; +} + +std::shared_ptr & Context::tryGetKeeperDispatcher() const { std::lock_guard lock(shared->keeper_dispatcher_mutex); return shared->keeper_dispatcher; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 83193dd589b..37c6b4c9caa 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -757,6 +757,7 @@ public: #if USE_NURAFT std::shared_ptr & getKeeperDispatcher() const; + std::shared_ptr & tryGetKeeperDispatcher() const; #endif void initializeKeeperDispatcher(bool start_async) const; void shutdownKeeperDispatcher() const; From cd2d053ce59fc0cdc7df03eeeb181b3bf00d576a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 2 Jul 2022 11:24:11 +0300 Subject: [PATCH 137/627] Update poco to fix GCS via proxy tunnel usage GCS server does not handle requests with port, and simply report an error: ```xml InvalidURI Couldn't parse the specified URI.
Invalid URL: storage.googleapis.com:443/...
``` Removing the port fixes the issue. Note that there is port in the Host header anyway. Note, this is a problem only for proxy in a tunnel mode, since only it sends such requests, other sends requests directly via HTTP methods. Refs: https://github.com/ClickHouse/poco/pull/22#22 (cc @Jokser) Refs: https://github.com/ClickHouse/poco/pull/63 Refs: #38069 (cc @CurtizJ) Cc: @alesapin @kssenii Signed-off-by: Azat Khuzhin --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 0e32cb42db7..9fec8e11dbb 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 0e32cb42db76ddaa76848470219056908053b676 +Subproject commit 9fec8e11dbb6a352e1cfba8cc9e23ebd7fb77310 From 79d02065941a64ca72f02bfc627c7432c707c14d Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 4 Jul 2022 11:21:19 +0300 Subject: [PATCH 138/627] updated tests --- .../02346_non_negative_derivative.reference | 68 +++++++++---------- .../02346_non_negative_derivative.sql | 18 ++--- 2 files changed, 43 insertions(+), 43 deletions(-) diff --git a/tests/queries/0_stateless/02346_non_negative_derivative.reference b/tests/queries/0_stateless/02346_non_negative_derivative.reference index 7559f527c7a..ceb487039ce 100644 --- a/tests/queries/0_stateless/02346_non_negative_derivative.reference +++ b/tests/queries/0_stateless/02346_non_negative_derivative.reference @@ -1,63 +1,63 @@ 1 -1979-12-12 21:21:21.123 1.1 3.5045052519931732e-9 -1979-12-12 21:21:22.000 1.3345 0.26738883339230357 +1979-12-12 21:21:21.123 1.1 0 +1979-12-12 21:21:21.124 2.34 1239.9999999999998 +1979-12-12 21:21:21.127 3.7 453.3333333333334 +1979-12-12 21:21:21.129 2.1 0 +1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 0.20550000000000002 1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.123 1.1 0 +1979-12-12 21:21:21.124 2.34 0.00000372 +1979-12-12 21:21:21.127 3.7 0.0000013600000000000005 1979-12-12 21:21:21.129 2.1 0 -1979-12-12 21:21:21.124 2.34 0 -1979-12-12 21:21:21.127 3.7 453.33916989529325 -1979-12-12 21:21:21.123 1.1 1.0513515755979521e-17 -1979-12-12 21:21:22.000 1.3345 8.021665001769108e-10 +1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 6.165000000000001e-10 1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.123 1.1 0 +1979-12-12 21:21:21.124 2.34 0.004959999999999999 +1979-12-12 21:21:21.127 3.7 0.0018133333333333337 1979-12-12 21:21:21.129 2.1 0 -1979-12-12 21:21:21.124 2.34 0 -1979-12-12 21:21:21.127 3.7 0.0000013600175096858798 -1979-12-12 21:21:21.123 1.1 1.4018021007972692e-14 -1979-12-12 21:21:22.000 1.3345 0.0000010695553335692141 +1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 8.22e-7 1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.123 1.1 0 +1979-12-12 21:21:21.124 2.34 6.199999999999999 +1979-12-12 21:21:21.127 3.7 2.266666666666667 1979-12-12 21:21:21.129 2.1 0 -1979-12-12 21:21:21.124 2.34 0 -1979-12-12 21:21:21.127 3.7 0.001813356679581173 -1979-12-12 21:21:21.123 1.1 1.7522526259965866e-11 -1979-12-12 21:21:22.000 1.3345 0.0013369441669615178 +1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 0.0010275000000000002 1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.123 1.1 0 +1979-12-12 21:21:21.124 2.34 7439.999999999998 +1979-12-12 21:21:21.127 3.7 2720.0000000000005 1979-12-12 21:21:21.129 2.1 0 -1979-12-12 21:21:21.124 2.34 0 -1979-12-12 21:21:21.127 3.7 2.2666958494764664 -1979-12-12 21:21:21.123 1.1 2.102703151195904e-8 -1979-12-12 21:21:22.000 1.3345 1.6043330003538214 +1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 1.233 1979-12-12 21:21:23.000 1.54 0 +1979-12-12 21:21:21.123 1.1 0 +1979-12-12 21:21:21.124 2.34 520799.9999999999 +1979-12-12 21:21:21.127 3.7 190400.00000000003 1979-12-12 21:21:21.129 2.1 0 -1979-12-12 21:21:21.124 2.34 0 -1979-12-12 21:21:21.127 3.7 2720.0350193717595 -1979-12-12 21:21:21.123 1.1 0.0000014718922058371327 -1979-12-12 21:21:22.000 1.3345 112.3033100247675 +1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 86.31 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.129 2.1 0 -1979-12-12 21:21:21.124 2.34 0 -1979-12-12 21:21:21.127 3.7 190402.45135602317 -1979-12-12 21:21:21.123 1.1 0.0001009297512574034 -1979-12-12 21:21:21.124 2.34 35712459.78375156 -1979-12-12 21:21:21.127 3.7 13056168.092984445 +1979-12-12 21:21:21.123 1.1 0 +1979-12-12 21:21:21.124 2.34 35711999.99999999 +1979-12-12 21:21:21.127 3.7 13056000.000000002 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 5918.400000000001 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.123 1.1 0.0027251032839498914 -1979-12-12 21:21:21.124 2.34 964236414.1612921 -1979-12-12 21:21:21.127 3.7 352516538.51058006 +1979-12-12 21:21:21.123 1.1 0 +1979-12-12 21:21:21.124 2.34 964223999.9999999 +1979-12-12 21:21:21.127 3.7 352512000.00000006 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 159796.80000000002 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.123 1.1 0.021195247764054712 -1979-12-12 21:21:21.124 2.34 7499616554.587828 -1979-12-12 21:21:21.127 3.7 2741795299.5267334 +1979-12-12 21:21:21.123 1.1 0 +1979-12-12 21:21:21.124 2.34 7499519999.999999 +1979-12-12 21:21:21.127 3.7 2741760000.0000005 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 1242864 diff --git a/tests/queries/0_stateless/02346_non_negative_derivative.sql b/tests/queries/0_stateless/02346_non_negative_derivative.sql index c4cbadb68a8..91445ada373 100644 --- a/tests/queries/0_stateless/02346_non_negative_derivative.sql +++ b/tests/queries/0_stateless/02346_non_negative_derivative.sql @@ -27,23 +27,23 @@ SELECT ( FROM nnd LIMIT 5, 1 ); -SELECT ts, metric, nonNegativeDerivative(metric, ts) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +SELECT ts, metric, nonNegativeDerivative(metric, ts) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS deriv FROM nnd; -- Nanosecond -SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 2 PRECEDING AND 2 FOLLOWING) AS deriv FROM nnd; -- Microsecond -SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 4 MICROSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 4 MICROSECOND) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS deriv FROM nnd; -- Millisecond -SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 5 MILLISECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 5 MILLISECOND) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS deriv FROM nnd; -- Second -SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 6 SECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 6 SECOND) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- Minute -SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 7 MINUTE) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 7 MINUTE) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING) AS deriv FROM nnd; -- Hour -SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 8 HOUR) OVER (ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 8 HOUR) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- Day -SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 9 DAY) OVER (ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 9 DAY) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 3 PRECEDING AND 3 FOLLOWING) AS deriv FROM nnd; -- Week -SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 10 WEEK) OVER (ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; +SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 10 WEEK) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- shall not work for month, quarter, year (intervals with floating number of seconds) -- Month From 7d6b7f26124b6336a501d37f456061ae1e934b07 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 4 Jul 2022 11:23:37 +0300 Subject: [PATCH 139/627] updated tests --- .../02346_non_negative_derivative.reference | 54 +++++++++---------- .../02346_non_negative_derivative.sql | 2 +- 2 files changed, 28 insertions(+), 28 deletions(-) diff --git a/tests/queries/0_stateless/02346_non_negative_derivative.reference b/tests/queries/0_stateless/02346_non_negative_derivative.reference index ceb487039ce..51fe2359bd0 100644 --- a/tests/queries/0_stateless/02346_non_negative_derivative.reference +++ b/tests/queries/0_stateless/02346_non_negative_derivative.reference @@ -1,63 +1,63 @@ 1 -1979-12-12 21:21:21.123 1.1 0 -1979-12-12 21:21:21.124 2.34 1239.9999999999998 -1979-12-12 21:21:21.127 3.7 453.3333333333334 +1979-12-12 21:21:21.127 3.7 0 +2283-11-11 23:37:36.788 1.1 0 +2283-11-11 23:37:36.789 2.34 0 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 0.20550000000000002 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.123 1.1 0 -1979-12-12 21:21:21.124 2.34 0.00000372 -1979-12-12 21:21:21.127 3.7 0.0000013600000000000005 +1979-12-12 21:21:21.127 3.7 0 +2283-11-11 23:37:36.788 1.1 0 +2283-11-11 23:37:36.789 2.34 0 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 6.165000000000001e-10 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.123 1.1 0 -1979-12-12 21:21:21.124 2.34 0.004959999999999999 -1979-12-12 21:21:21.127 3.7 0.0018133333333333337 +1979-12-12 21:21:21.127 3.7 0 +2283-11-11 23:37:36.788 1.1 0 +2283-11-11 23:37:36.789 2.34 0 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 8.22e-7 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.123 1.1 0 -1979-12-12 21:21:21.124 2.34 6.199999999999999 -1979-12-12 21:21:21.127 3.7 2.266666666666667 +1979-12-12 21:21:21.127 3.7 0 +2283-11-11 23:37:36.788 1.1 0 +2283-11-11 23:37:36.789 2.34 0 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 0.0010275000000000002 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.123 1.1 0 -1979-12-12 21:21:21.124 2.34 7439.999999999998 -1979-12-12 21:21:21.127 3.7 2720.0000000000005 +1979-12-12 21:21:21.127 3.7 0 +2283-11-11 23:37:36.788 1.1 0 +2283-11-11 23:37:36.789 2.34 0 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 1.233 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.123 1.1 0 -1979-12-12 21:21:21.124 2.34 520799.9999999999 -1979-12-12 21:21:21.127 3.7 190400.00000000003 +1979-12-12 21:21:21.127 3.7 0 +2283-11-11 23:37:36.788 1.1 0 +2283-11-11 23:37:36.789 2.34 0 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 86.31 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.123 1.1 0 -1979-12-12 21:21:21.124 2.34 35711999.99999999 -1979-12-12 21:21:21.127 3.7 13056000.000000002 +1979-12-12 21:21:21.127 3.7 0 +2283-11-11 23:37:36.788 1.1 0 +2283-11-11 23:37:36.789 2.34 0 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 5918.400000000001 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.123 1.1 0 -1979-12-12 21:21:21.124 2.34 964223999.9999999 -1979-12-12 21:21:21.127 3.7 352512000.00000006 +1979-12-12 21:21:21.127 3.7 0 +2283-11-11 23:37:36.788 1.1 0 +2283-11-11 23:37:36.789 2.34 0 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 159796.80000000002 1979-12-12 21:21:23.000 1.54 0 -1979-12-12 21:21:21.123 1.1 0 -1979-12-12 21:21:21.124 2.34 7499519999.999999 -1979-12-12 21:21:21.127 3.7 2741760000.0000005 +1979-12-12 21:21:21.127 3.7 0 +2283-11-11 23:37:36.788 1.1 0 +2283-11-11 23:37:36.789 2.34 0 1979-12-12 21:21:21.129 2.1 0 1979-12-12 21:21:22.000 1.3345 0 1979-12-12 21:21:23.000 1.54 1242864 diff --git a/tests/queries/0_stateless/02346_non_negative_derivative.sql b/tests/queries/0_stateless/02346_non_negative_derivative.sql index 91445ada373..673106e100d 100644 --- a/tests/queries/0_stateless/02346_non_negative_derivative.sql +++ b/tests/queries/0_stateless/02346_non_negative_derivative.sql @@ -7,7 +7,7 @@ CREATE TABLE nnd ENGINE=MergeTree() ORDER BY id; -INSERT INTO nnd VALUES (1, toDateTime64('1979-12-12 21:21:21.123', 3, 'UTC'), 1.1), (2, toDateTime64('1979-12-12 21:21:21.124', 3, 'UTC'), 2.34), (3, toDateTime64('1979-12-12 21:21:21.127', 3, 'UTC'), 3.7); +INSERT INTO nnd VALUES (1, toDateTime64('1979-12-12 21:21:21.123456788', 9, 'UTC'), 1.1), (2, toDateTime64('1979-12-12 21:21:21.123456789', 9, 'UTC'), 2.34), (3, toDateTime64('1979-12-12 21:21:21.127', 3, 'UTC'), 3.7); INSERT INTO nnd VALUES (4, toDateTime64('1979-12-12 21:21:21.129', 3, 'UTC'), 2.1), (5, toDateTime('1979-12-12 21:21:22', 'UTC'), 1.3345), (6, toDateTime('1979-12-12 21:21:23', 'UTC'), 1.54), (7, toDateTime('1979-12-12 21:21:23', 'UTC'), 1.54); -- shall work for precise intervals From 1dba3d9f9c4f7ea768e6b7e1d57cc8a307938853 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 3 Jul 2022 16:32:11 +0200 Subject: [PATCH 140/627] Make temporary files in better places. --- .../MergeTree/DataPartStorageOnDisk.cpp | 6 +-- src/Storages/MergeTree/MergeTreeData.cpp | 37 ++++++++++----- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMemory.cpp | 47 +++++++++++-------- src/Storages/StorageMemory.h | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- 6 files changed, 58 insertions(+), 38 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index dc80b0aafb6..e2c553f55be 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -624,7 +624,7 @@ void DataPartStorageOnDisk::backup( auto disk = volume->getDisk(); auto temp_dir_it = temp_dirs.find(disk); if (temp_dir_it == temp_dirs.end()) - temp_dir_it = temp_dirs.emplace(disk, std::make_shared(disk, "tmp/backup/")).first; + temp_dir_it = temp_dirs.emplace(disk, std::make_shared(disk, "tmp/")).first; auto temp_dir_owner = temp_dir_it->second; fs::path temp_dir = temp_dir_owner->getPath(); fs::path temp_part_dir = temp_dir / part_path_in_backup.relative_path(); @@ -633,11 +633,11 @@ void DataPartStorageOnDisk::backup( /// For example, /// part_path_in_backup = /data/test/table/0_1_1_0 /// part_path_on_disk = store/f57/f5728353-44bb-4575-85e8-28deb893657a/0_1_1_0 - /// tmp_part_dir = tmp/backup/1aaaaaa/data/test/table/0_1_1_0 + /// tmp_part_dir = tmp/1aaaaaa/data/test/table/0_1_1_0 /// Or, for projections: /// part_path_in_backup = /data/test/table/0_1_1_0/prjmax.proj /// part_path_on_disk = store/f57/f5728353-44bb-4575-85e8-28deb893657a/0_1_1_0/prjmax.proj - /// tmp_part_dir = tmp/backup/1aaaaaa/data/test/table/0_1_1_0/prjmax.proj + /// tmp_part_dir = tmp/1aaaaaa/data/test/table/0_1_1_0/prjmax.proj for (const auto & [filepath, checksum] : checksums.files) { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3c30bee7889..2ed68cef972 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4041,14 +4041,22 @@ public: attachIfAllPartsRestored(); } - void addPart(MutableDataPartPtr part, std::shared_ptr temp_part_dir_owner) + void addPart(MutableDataPartPtr part) { std::lock_guard lock{mutex}; parts.emplace_back(part); - temp_part_dir_owners.emplace_back(temp_part_dir_owner); attachIfAllPartsRestored(); } + String getTemporaryDirectory(const DiskPtr & disk) + { + std::lock_guard lock{mutex}; + auto it = temp_dirs.find(disk); + if (it == temp_dirs.end()) + it = temp_dirs.emplace(disk, std::make_shared(disk, "tmp/")).first; + return it->second->getPath(); + } + private: void attachIfAllPartsRestored() { @@ -4063,7 +4071,7 @@ private: storage->attachRestoredParts(std::move(parts)); parts.clear(); - temp_part_dir_owners.clear(); + temp_dirs.clear(); num_parts = 0; } @@ -4071,7 +4079,7 @@ private: BackupPtr backup; size_t num_parts = 0; MutableDataPartsVector parts; - std::vector> temp_part_dir_owners; + std::map> temp_dirs; mutable std::mutex mutex; }; @@ -4117,6 +4125,7 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const void MergeTreeData::restorePartFromBackup(std::shared_ptr restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup) { + String part_name = part_info.getPartName(); auto backup = restored_parts_holder->getBackup(); UInt64 total_size_of_part = 0; @@ -4128,14 +4137,18 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr r std::shared_ptr reservation = getStoragePolicy()->reserveAndCheck(total_size_of_part); auto disk = reservation->getDisk(); - String part_name = part_info.getPartName(); - auto temp_part_dir_owner = std::make_shared(disk, fs::path{relative_data_path} / ("restoring_" + part_name + "_")); - fs::path temp_part_dir = temp_part_dir_owner->getPath(); + fs::path temp_dir = restored_parts_holder->getTemporaryDirectory(disk); + fs::path temp_part_dir = temp_dir / part_path_in_backup_fs.relative_path(); disk->createDirectories(temp_part_dir); - std::unordered_set subdirs; - /// temp_part_name = "restoring__", for example "restoring_0_1_1_0_1baaaaa" - String temp_part_name = temp_part_dir.filename(); + /// For example: + /// part_name = 0_1_1_0 + /// part_path_in_backup = /data/test/table/0_1_1_0 + /// tmp_dir = tmp/1aaaaaa + /// tmp_part_dir = tmp/1aaaaaa/data/test/table/0_1_1_0 + + /// Subdirectories in the part's directory. It's used to restore projections. + std::unordered_set subdirs; for (const String & filename : filenames) { @@ -4160,12 +4173,12 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr r } auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); - auto data_part_storage = std::make_shared(single_disk_volume, relative_data_path, temp_part_name); + auto data_part_storage = std::make_shared(single_disk_volume, temp_part_dir.parent_path(), part_name); auto part = createPart(part_name, part_info, data_part_storage); part->version.setCreationTID(Tx::PrehistoricTID, nullptr); part->loadColumnsChecksumsIndexes(false, true); - restored_parts_holder->addPart(part, temp_part_dir_owner); + restored_parts_holder->addPart(part); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 61c7be21f61..a03acd3731d 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -935,7 +935,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c return; fs::path data_path_in_backup_fs = data_path_in_backup; - auto temp_dir_owner = std::make_shared(disk, "tmp/backup_"); + auto temp_dir_owner = std::make_shared(disk, "tmp/"); fs::path temp_dir = temp_dir_owner->getPath(); disk->createDirectories(temp_dir); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 929da48dcfe..f9e8b03e9f4 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -28,8 +28,8 @@ #include #include #include +#include #include -#include namespace DB @@ -389,6 +389,7 @@ namespace const StorageMetadataPtr & metadata_snapshot_, const std::shared_ptr blocks_, const String & data_path_in_backup, + const DiskPtr & temp_disk_, UInt64 max_compress_block_size_) : IBackupEntriesBatch( {fs::path{data_path_in_backup} / "data.bin", @@ -396,6 +397,7 @@ namespace fs::path{data_path_in_backup} / "sizes.json"}) , metadata_snapshot(metadata_snapshot_) , blocks(blocks_) + , temp_disk(temp_disk_) , max_compress_block_size(max_compress_block_size_) { } @@ -410,16 +412,16 @@ namespace { std::call_once(initialized_flag, [this]() { - temp_dir_owner.emplace(); - fs::path temp_dir = temp_dir_owner->path(); - fs::create_directories(temp_dir); + temp_dir_owner.emplace(temp_disk); + fs::path temp_dir = temp_dir_owner->getPath(); + temp_disk->createDirectories(temp_dir); /// Writing data.bin constexpr char data_file_name[] = "data.bin"; auto data_file_path = temp_dir / data_file_name; IndexForNativeFormat index; { - auto data_out_compressed = std::make_unique(data_file_path); + auto data_out_compressed = temp_disk->writeFile(data_file_path); CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size}; NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index}; for (const auto & block : *blocks) @@ -430,7 +432,7 @@ namespace constexpr char index_file_name[] = "index.mrk"; auto index_file_path = temp_dir / index_file_name; { - auto index_out_compressed = std::make_unique(index_file_path); + auto index_out_compressed = temp_disk->writeFile(index_file_path); CompressedWriteBuffer index_out{*index_out_compressed}; index.write(index_out); } @@ -438,7 +440,7 @@ namespace /// Writing sizes.json constexpr char sizes_file_name[] = "sizes.json"; auto sizes_file_path = temp_dir / sizes_file_name; - FileChecker file_checker{sizes_file_path}; + FileChecker file_checker{temp_disk, sizes_file_path}; file_checker.update(data_file_path); file_checker.update(index_file_path); file_checker.save(); @@ -450,7 +452,7 @@ namespace file_sizes[kIndexMrkPos] = file_checker.getFileSize(index_file_path); file_paths[kSizesJsonPos] = sizes_file_path; - file_sizes[kSizesJsonPos] = fs::file_size(sizes_file_path); + file_sizes[kSizesJsonPos] = temp_disk->getFileSize(sizes_file_path); /// We don't need to keep `blocks` any longer. blocks.reset(); @@ -461,7 +463,7 @@ namespace std::unique_ptr getReadBuffer(size_t index) override { initialize(); - return createReadBufferFromFileBase(file_paths[index], {}); + return temp_disk->readFile(file_paths[index]); } UInt64 getSize(size_t index) override @@ -472,9 +474,10 @@ namespace StorageMetadataPtr metadata_snapshot; std::shared_ptr blocks; + DiskPtr temp_disk; UInt64 max_compress_block_size; std::once_flag initialized_flag; - std::optional temp_dir_owner; + std::optional temp_dir_owner; std::array file_paths; std::array file_sizes; }; @@ -482,9 +485,11 @@ namespace void StorageMemory::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { + auto temp_disk = backup_entries_collector.getContext()->getTemporaryVolume()->getDisk(0); auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size; backup_entries_collector.addBackupEntries( - std::make_shared(getInMemoryMetadataPtr(), data.get(), data_path_in_backup, max_compress_block_size) + std::make_shared( + getInMemoryMetadataPtr(), data.get(), data_path_in_backup, temp_disk, max_compress_block_size) ->getBackupEntries()); } @@ -497,12 +502,14 @@ void StorageMemory::restoreDataFromBackup(RestorerFromBackup & restorer, const S if (!restorer.isNonEmptyTableAllowed() && total_size_bytes) RestorerFromBackup::throwTableIsNotEmpty(getStorageID()); + auto temp_disk = restorer.getContext()->getTemporaryVolume()->getDisk(0); + restorer.addDataRestoreTask( - [storage = std::static_pointer_cast(shared_from_this()), backup, data_path_in_backup] - { storage->restoreDataImpl(backup, data_path_in_backup); }); + [storage = std::static_pointer_cast(shared_from_this()), backup, data_path_in_backup, temp_disk] + { storage->restoreDataImpl(backup, data_path_in_backup, temp_disk); }); } -void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup) +void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, const DiskPtr & temp_disk) { /// Our data are in the StripeLog format. @@ -532,14 +539,14 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat auto backup_entry = backup->readFile(data_file_path); std::unique_ptr in = backup_entry->getReadBuffer(); - std::optional temp_data_copy; + std::optional temp_data_file; if (!dynamic_cast(in.get())) { - temp_data_copy.emplace(); - auto temp_data_copy_out = std::make_unique(temp_data_copy->path()); - copyData(*in, *temp_data_copy_out); - temp_data_copy_out.reset(); - in = createReadBufferFromFileBase(temp_data_copy->path(), {}); + temp_data_file.emplace(temp_disk); + auto out = std::make_unique(temp_data_file->getPath()); + copyData(*in, *out); + out.reset(); + in = createReadBufferFromFileBase(temp_data_file->getPath(), {}); } std::unique_ptr in_from_file{static_cast(in.release())}; CompressedReadBufferFromFile compressed_in{std::move(in_from_file)}; diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 91dadcb884e..3889acb952b 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -116,7 +116,7 @@ public: private: /// Restores the data of this table from backup. - void restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup); + void restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, const DiskPtr & temporary_disk); /// MultiVersion data storage, so that we can copy the vector of blocks to readers. diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index e650fbf2b2b..07162b99750 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -541,7 +541,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec return; fs::path data_path_in_backup_fs = data_path_in_backup; - auto temp_dir_owner = std::make_shared(disk, "tmp/backup_"); + auto temp_dir_owner = std::make_shared(disk, "tmp/"); fs::path temp_dir = temp_dir_owner->getPath(); disk->createDirectories(temp_dir); From 0a69b76180bdabbe82ae95678638f22cbc2deea4 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 4 Jul 2022 17:21:07 +0800 Subject: [PATCH 141/627] add supportParallelWrite() in DiskDecorator --- src/Disks/DiskDecorator.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 08a11cd3f03..8a98f5b34fe 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -69,6 +69,7 @@ public: DiskType getType() const override { return delegate->getType(); } bool isRemote() const override { return delegate->isRemote(); } bool supportZeroCopyReplication() const override { return delegate->supportZeroCopyReplication(); } + bool supportParallelWrite() const override { return delegate->supportParallelWrite(); } void onFreeze(const String & path) override; SyncGuardPtr getDirectorySyncGuard(const String & path) const override; void shutdown() override; From d547aa7849d7f4611086ced9a24f78b708ca1031 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 26 Jun 2022 16:45:16 +0000 Subject: [PATCH 142/627] Allow non-const pattern array argument in multi[Fuzzy]Match*() Resolves #38046 --- .../FunctionsMultiStringFuzzySearch.h | 31 +- src/Functions/FunctionsMultiStringSearch.h | 32 +- src/Functions/MultiMatchAllIndicesImpl.h | 96 ++- src/Functions/MultiMatchAnyImpl.h | 143 +++- src/Functions/MultiSearchFirstIndexImpl.h | 7 + src/Functions/MultiSearchFirstPositionImpl.h | 7 + src/Functions/MultiSearchImpl.h | 7 + .../0_stateless/00926_multimatch.reference | 622 ++++++++++++++++++ .../queries/0_stateless/00926_multimatch.sql | 95 +++ .../00927_disable_hyperscan.reference | 5 + .../0_stateless/00927_disable_hyperscan.sql | 7 + ...28_multi_match_constant_constant.reference | 4 + .../00928_multi_match_constant_constant.sql | 9 + .../00929_multi_match_edit_distance.reference | 34 +- .../00929_multi_match_edit_distance.sql | 76 ++- ...02004_max_hyperscan_regex_length.reference | 8 + .../02004_max_hyperscan_regex_length.sql | 28 + 17 files changed, 1156 insertions(+), 55 deletions(-) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 865a5d182c8..75663693476 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -46,7 +46,7 @@ public: size_t getNumberOfArguments() const override { return 3; } bool useDefaultImplementationForConstants() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -70,7 +70,8 @@ public: const ColumnPtr & arr_ptr = arguments[2].column; const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); - assert(col_haystack_vector); // getReturnTypeImpl() checks the data type + const ColumnConst * col_haystack_const = typeid_cast(&*column_haystack); + assert(static_cast(col_haystack_vector) ^ static_cast(col_haystack_const)); UInt32 edit_distance = 0; if (const auto * col_const_uint8 = checkAndGetColumnConst(num_ptr.get())) @@ -82,9 +83,12 @@ public: else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}. The number is not const or does not fit in UInt32", arguments[1].column->getName()); - const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); - if (!col_const_arr) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}. The array is not const", arguments[2].column->getName()); + const ColumnArray * col_needles = checkAndGetColumn(arr_ptr.get()); + const ColumnConst * col_needles_const = checkAndGetColumnConst(arr_ptr.get()); + assert(static_cast(col_needles) ^ static_cast(col_needles_const)); + + if (col_haystack_const && col_needles) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support search with non-constant needles in constant haystack", name); using ResultType = typename Impl::ResultType; auto col_res = ColumnVector::create(); @@ -94,10 +98,19 @@ public: auto & offsets_res = col_offsets->getData(); // the implementations are responsible for resizing the output column - Array needles_arr = col_const_arr->getValue(); - Impl::vectorConstant( - col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needles_arr, vec_res, offsets_res, edit_distance, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + if (col_needles_const) + { + Array needles_arr = col_needles_const->getValue(); + Impl::vectorConstant( + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needles_arr, vec_res, offsets_res, edit_distance, + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + } + else + { + Impl::vectorVector( + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), *col_needles, vec_res, offsets_res, edit_distance, + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + } if constexpr (Impl::is_column_array) return ColumnArray::create(std::move(col_res), std::move(col_offsets)); diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 04235e0a97a..3531a0538f3 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -61,7 +61,7 @@ public: size_t getNumberOfArguments() const override { return 2; } bool useDefaultImplementationForConstants() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -81,11 +81,15 @@ public: const ColumnPtr & arr_ptr = arguments[1].column; const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); - assert(col_haystack_vector); // getReturnTypeImpl() checks the data type + const ColumnConst * col_haystack_const = typeid_cast(&*column_haystack); + assert(static_cast(col_haystack_vector) ^ static_cast(col_haystack_const)); - const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); - if (!col_const_arr) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}. The array is not const", arguments[1].column->getName()); + const ColumnArray * col_needles = checkAndGetColumn(arr_ptr.get()); + const ColumnConst * col_needles_const = checkAndGetColumnConst(arr_ptr.get()); + assert(static_cast(col_needles) ^ static_cast(col_needles_const)); + + if (col_haystack_const && col_needles) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support search with non-constant needles in constant haystack", name); using ResultType = typename Impl::ResultType; auto col_res = ColumnVector::create(); @@ -95,10 +99,20 @@ public: auto & offsets_res = col_offsets->getData(); // the implementations are responsible for resizing the output column - Array needles_arr = col_const_arr->getValue(); - Impl::vectorConstant( - col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needles_arr, vec_res, offsets_res, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + if (col_needles_const) + { + Array needles_arr = col_needles_const->getValue(); + Impl::vectorConstant( + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needles_arr, vec_res, offsets_res, + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + } + else + { + Impl::vectorVector( + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), *col_needles, vec_res, offsets_res, + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + } + if constexpr (Impl::is_column_array) return ColumnArray::create(std::move(col_res), std::move(col_offsets)); diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index 9c60dbffe91..e9662386f3a 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -79,6 +79,7 @@ struct MultiMatchAllIndicesImpl checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); offsets.resize(haystack_offsets.size()); + const auto & hyperscan_regex = MultiRegexps::get(needles, edit_distance); hs_scratch_t * scratch = nullptr; hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch); @@ -102,10 +103,10 @@ struct MultiMatchAllIndicesImpl for (size_t i = 0; i < haystack_offsets_size; ++i) { UInt64 length = haystack_offsets[i] - offset - 1; - /// Hyperscan restriction. + // vectorscan restriction. if (length > std::numeric_limits::max()) throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES); - /// Scan, check, update the offsets array and the offset of haystack. + // scan, check, update the offsets array and the offset of haystack. err = hs_scan( hyperscan_regex->getDB(), reinterpret_cast(haystack_data.data()) + offset, @@ -123,6 +124,97 @@ struct MultiMatchAllIndicesImpl throw Exception( "multi-search all indices is not implemented when vectorscan is off", ErrorCodes::NOT_IMPLEMENTED); +#endif // USE_VECTORSCAN + } + + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnArray & needles_col, + PaddedPODArray & res, + PaddedPODArray & offsets, + bool allow_hyperscan, + size_t max_hyperscan_regexp_length, + size_t max_hyperscan_regexp_total_length) + { + vectorVector(haystack_data, haystack_offsets, needles_col, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + } + + static void vectorVector( + [[maybe_unused]] const ColumnString::Chars & haystack_data, + [[maybe_unused]] const ColumnString::Offsets & haystack_offsets, + [[maybe_unused]] const ColumnArray & needles_col, + [[maybe_unused]] PaddedPODArray & res, + [[maybe_unused]] PaddedPODArray & offsets, + [[maybe_unused]] std::optional edit_distance, + bool allow_hyperscan, + [[maybe_unused]] size_t max_hyperscan_regexp_length, + [[maybe_unused]] size_t max_hyperscan_regexp_total_length) + { + if (!allow_hyperscan) + throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); +#if USE_VECTORSCAN + size_t prev_haystack_offset = 0; + for (size_t i = 0; i < haystack_offsets.size(); ++i) + { + Field field; + needles_col.get(i, field); + Array & needles_arr = DB::get(field); + + std::vector needles; + needles.reserve(needles_arr.size()); + for (const auto & needle : needles_arr) + needles.emplace_back(needle.get()); + + checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + + offsets.resize(haystack_offsets.size()); + + const auto & hyperscan_regex = MultiRegexps::get(needles, edit_distance); + hs_scratch_t * scratch = nullptr; + hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch); + + if (err != HS_SUCCESS) + throw Exception("Could not clone scratch space for hyperscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY); + + MultiRegexps::ScratchPtr smart_scratch(scratch); + + auto on_match = [](unsigned int id, + unsigned long long /* from */, // NOLINT + unsigned long long /* to */, // NOLINT + unsigned int /* flags */, + void * context) -> int + { + static_cast*>(context)->push_back(id); + return 0; + }; + + const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; + + // vectorscan restriction. + if (cur_haystack_length > std::numeric_limits::max()) + throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES); + + /// Scan, check, update the offsets array and the offset of haystack. + err = hs_scan( + hyperscan_regex->getDB(), + reinterpret_cast(haystack_data.data()) + prev_haystack_offset, + cur_haystack_length, + 0, + smart_scratch.get(), + on_match, + &res); + if (err != HS_SUCCESS) + throw Exception("Failed to scan with vectorscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT); + + offsets[i] = res.size(); + + prev_haystack_offset = haystack_offsets[i]; + } +#else + throw Exception( + "multi-search all indices is not implemented when vectorscan is off", + ErrorCodes::NOT_IMPLEMENTED); #endif // USE_VECTORSCAN } }; diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 0752e87e8af..11c9e8afaf5 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -75,7 +75,7 @@ struct MultiMatchAnyImpl const ColumnString::Offsets & haystack_offsets, const Array & needles_arr, PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets, + PaddedPODArray & /*offsets*/, [[maybe_unused]] std::optional edit_distance, bool allow_hyperscan, size_t max_hyperscan_regexp_length, @@ -93,7 +93,7 @@ struct MultiMatchAnyImpl res.resize(haystack_offsets.size()); #if USE_VECTORSCAN - const auto & hyperscan_regex = MultiRegexps::get(needles, edit_distance); + const auto & hyperscan_regex = MultiRegexps::get(needles, edit_distance); hs_scratch_t * scratch = nullptr; hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch); @@ -120,10 +120,10 @@ struct MultiMatchAnyImpl for (size_t i = 0; i < haystack_offsets_size; ++i) { UInt64 length = haystack_offsets[i] - offset - 1; - /// Vectorscan restriction. + // vectorscan restriction. if (length > std::numeric_limits::max()) throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES); - /// Zero the result, scan, check, update the offset. + // zero the result, scan, check, update the offset. res[i] = 0; err = hs_scan( hyperscan_regex->getDB(), @@ -157,6 +157,141 @@ struct MultiMatchAnyImpl res[i] = j + 1; } } +#endif // USE_VECTORSCAN + } + + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnArray & needles_col, + PaddedPODArray & res, + PaddedPODArray & offsets, + bool allow_hyperscan, + size_t max_hyperscan_regexp_length, + size_t max_hyperscan_regexp_total_length) + { + vectorVector(haystack_data, haystack_offsets, needles_col, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + } + + static void vectorVector( + [[maybe_unused]] const ColumnString::Chars & haystack_data, + [[maybe_unused]] const ColumnString::Offsets & haystack_offsets, + [[maybe_unused]] const ColumnArray & needles_col, + [[maybe_unused]] PaddedPODArray & res, + PaddedPODArray & /*offsets*/, + [[maybe_unused]] std::optional edit_distance, + bool allow_hyperscan, + [[maybe_unused]] size_t max_hyperscan_regexp_length, + [[maybe_unused]] size_t max_hyperscan_regexp_total_length) + { + if (!allow_hyperscan) + throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); + + res.resize(haystack_offsets.size()); +#if USE_VECTORSCAN + size_t prev_haystack_offset = 0; + for (size_t i = 0; i < haystack_offsets.size(); ++i) + { + Field field; + needles_col.get(i, field); + Array & needles_arr = DB::get(field); + + std::vector needles; + needles.reserve(needles_arr.size()); + for (const auto & needle : needles_arr) + needles.emplace_back(needle.get()); + + checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + + const auto & hyperscan_regex = MultiRegexps::get(needles, edit_distance); + hs_scratch_t * scratch = nullptr; + hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch); + + if (err != HS_SUCCESS) + throw Exception("Could not clone scratch space for vectorscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY); + + MultiRegexps::ScratchPtr smart_scratch(scratch); + + auto on_match = []([[maybe_unused]] unsigned int id, + unsigned long long /* from */, // NOLINT + unsigned long long /* to */, // NOLINT + unsigned int /* flags */, + void * context) -> int + { + if constexpr (FindAnyIndex) + *reinterpret_cast(context) = id; + else if constexpr (FindAny) + *reinterpret_cast(context) = 1; + /// Once we hit the callback, there is no need to search for others. + return 1; + }; + + const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; + + // vectorscan restriction. + if (cur_haystack_length > std::numeric_limits::max()) + throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES); + + // zero the result, scan, check, update the offset. + res[i] = 0; + err = hs_scan( + hyperscan_regex->getDB(), + reinterpret_cast(haystack_data.data()) + prev_haystack_offset, + cur_haystack_length, + 0, + smart_scratch.get(), + on_match, + &res[i]); + if (err != HS_SUCCESS && err != HS_SCAN_TERMINATED) + throw Exception("Failed to scan with vectorscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT); + + prev_haystack_offset = haystack_offsets[i]; + } +#else + // fallback if vectorscan is not compiled + if constexpr (WithEditDistance) + throw Exception( + "Edit distance multi-search is not implemented when vectorscan is off", + ErrorCodes::NOT_IMPLEMENTED); + + memset(res.data(), 0, res.size() * sizeof(res.front())); + + size_t prev_haystack_offset = 0; + + for (size_t i = 0; i < haystack_offsets.size(); ++i) + { + const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset]; + const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; + + Field field; + needles_col.get(i, field); + Array & needles_arr = DB::get(field); + + std::vector needles; + needles.reserve(needles_arr.size()); + for (const auto & needle : needles_arr) + needles.emplace_back(needle.get()); + + checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + + for (size_t j = 0; j < needles_arr.size(); ++j) + { + String needle = needles_arr[j].safeGet(); + const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + const bool match = regexp.getRE2()->Match( + {reinterpret_cast(cur_haystack_data), cur_haystack_length}, + 0, + cur_haystack_length, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + if constexpr (FindAny) + res[i] |= match; + else if (FindAnyIndex && match) + res[i] = j + 1; + } + prev_haystack_offset = haystack_offsets[i]; + } #endif // USE_VECTORSCAN } }; diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index f69a3edbf8b..9e717aaff23 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -10,6 +10,7 @@ namespace DB namespace ErrorCodes { + extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -66,6 +67,12 @@ struct MultiSearchFirstIndexImpl if (iteration == 0) std::fill(res.begin(), res.end(), 0); } + + template + static void vectorVector(Args &&...) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needles", name); + } }; } diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index 21d558a6d58..1f0cf1e6463 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -10,6 +10,7 @@ namespace DB namespace ErrorCodes { + extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -75,6 +76,12 @@ struct MultiSearchFirstPositionImpl if (iteration == 0) std::fill(res.begin(), res.end(), 0); } + + template + static void vectorVector(Args &&...) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needles", name); + } }; } diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 1124184f58c..53406d9324d 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -10,6 +10,7 @@ namespace DB namespace ErrorCodes { + extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -65,6 +66,12 @@ struct MultiSearchImpl if (iteration == 0) std::fill(res.begin(), res.end(), 0); } + + template + static void vectorVector(Args &&...) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needles", name); + } }; } diff --git a/tests/queries/0_stateless/00926_multimatch.reference b/tests/queries/0_stateless/00926_multimatch.reference index 7ff32ecd73b..0ad98dc23b8 100644 --- a/tests/queries/0_stateless/00926_multimatch.reference +++ b/tests/queries/0_stateless/00926_multimatch.reference @@ -1,3 +1,4 @@ +-- With const pattern 1 1 1 @@ -623,3 +624,624 @@ All tests above must return 1, all tests below return something. [1,3] [] [1,2,3] +-- With non-const pattern +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/00926_multimatch.sql b/tests/queries/0_stateless/00926_multimatch.sql index 067061dd99a..c98ef18dc32 100644 --- a/tests/queries/0_stateless/00926_multimatch.sql +++ b/tests/queries/0_stateless/00926_multimatch.sql @@ -1,5 +1,7 @@ -- Tags: no-fasttest, use-vectorscan +SELECT '-- With const pattern'; + select 0 = multiMatchAny(materialize('mpnsguhwsitzvuleiwebwjfitmsg'), ['wbirxqoabpblrnvvmjizj', 'cfcxhuvrexyzyjsh', 'oldhtubemyuqlqbwvwwkwin', 'bumoozxdkjglzu', 'intxlfohlxmajjomw', 'dxkeghohv', 'arsvmwwkjeopnlwnan', 'ouugllgowpqtaxslcopkytbfhifaxbgt', 'hkedmjlbcrzvryaopjqdjjc', 'tbqkljywstuahzh', 'o', 'wowoclosyfcuwotmvjygzuzhrery', 'vpefjiffkhlggntcu', 'ytdixvasrorhripzfhjdmlhqksmctyycwp']) from system.numbers limit 10; select 0 = multiMatchAny(materialize('qjjzqexjpgkglgxpzrbqbnskq'), ['vaiatcjacmlffdzsejpdareqzy', 'xspcfzdufkmecud', 'bcvtbuqtctq', 'nkcopwbfytgemkqcfnnno', 'dylxnzuyhq', 'tno', 'scukuhufly', 'cdyquzuqlptv', 'ohluyfeksyxepezdhqmtfmgkvzsyph', 'ualzwtahvqvtijwp', 'jg', 'gwbawqlngzcknzgtmlj', 'qimvjcgbkkp', 'eaedbcgyrdvv', 'qcwrncjoewwedyyewcdkh', 'uqcvhngoqngmitjfxpznqomertqnqcveoqk', 'ydrgjiankgygpm', 'axepgap']) from system.numbers limit 10; select 0 = multiMatchAny(materialize('fdkmtqmxnegwvnjhghjq'), ['vynkybvdmhgeezybbdqfrukibisj', 'knazzamgjjpavwhvdkwigykh', 'peumnifrmdhhmrqqnemw', 'lmsnyvqoisinlaqobxojlwfbi', 'oqwfzs', 'dymudxxeodwjpgbibnkvr', 'vomtfsnizkplgzktqyoiw', 'yoyfuhlpgrzds', 'cefao', 'gi', 'srpgxfjwl', 'etsjusdeiwbfe', 'ikvtzdopxo', 'ljfkavrau', 'soqdhxtenfrkmeic', 'ktprjwfcelzbup', 'pcvuoddqwsaurcqdtjfnczekwni', 'agkqkqxkfbkfgyqliahsljim']) from system.numbers limit 10; @@ -92,3 +94,96 @@ SELECT [1] = multiMatchAllIndices(materialize('/odezhda-dlya-bega/'), ['/odezhda SELECT [] = multiMatchAllIndices(materialize('aaaa'), ['.*aa.*aaa.*', 'aaaaaa{2}', '\(aa\){3}']); SELECT 'All tests above must return 1, all tests below return something.'; SELECT arraySort(multiMatchAllIndices(arrayJoin(['aaaa', 'aaaaaa', 'bbbb', 'aaaaaaaaaaaaaa']), ['.*aa.*aaa.*', 'aaaaaa{2}', '(aa){3}'])); + +SELECT '-- With non-const pattern'; + +select 0 = multiMatchAny(materialize('mpnsguhwsitzvuleiwebwjfitmsg'), materialize(['wbirxqoabpblrnvvmjizj', 'cfcxhuvrexyzyjsh', 'oldhtubemyuqlqbwvwwkwin', 'bumoozxdkjglzu', 'intxlfohlxmajjomw', 'dxkeghohv', 'arsvmwwkjeopnlwnan', 'ouugllgowpqtaxslcopkytbfhifaxbgt', 'hkedmjlbcrzvryaopjqdjjc', 'tbqkljywstuahzh', 'o', 'wowoclosyfcuwotmvjygzuzhrery', 'vpefjiffkhlggntcu', 'ytdixvasrorhripzfhjdmlhqksmctyycwp'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('qjjzqexjpgkglgxpzrbqbnskq'), materialize(['vaiatcjacmlffdzsejpdareqzy', 'xspcfzdufkmecud', 'bcvtbuqtctq', 'nkcopwbfytgemkqcfnnno', 'dylxnzuyhq', 'tno', 'scukuhufly', 'cdyquzuqlptv', 'ohluyfeksyxepezdhqmtfmgkvzsyph', 'ualzwtahvqvtijwp', 'jg', 'gwbawqlngzcknzgtmlj', 'qimvjcgbkkp', 'eaedbcgyrdvv', 'qcwrncjoewwedyyewcdkh', 'uqcvhngoqngmitjfxpznqomertqnqcveoqk', 'ydrgjiankgygpm', 'axepgap'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('fdkmtqmxnegwvnjhghjq'), materialize(['vynkybvdmhgeezybbdqfrukibisj', 'knazzamgjjpavwhvdkwigykh', 'peumnifrmdhhmrqqnemw', 'lmsnyvqoisinlaqobxojlwfbi', 'oqwfzs', 'dymudxxeodwjpgbibnkvr', 'vomtfsnizkplgzktqyoiw', 'yoyfuhlpgrzds', 'cefao', 'gi', 'srpgxfjwl', 'etsjusdeiwbfe', 'ikvtzdopxo', 'ljfkavrau', 'soqdhxtenfrkmeic', 'ktprjwfcelzbup', 'pcvuoddqwsaurcqdtjfnczekwni', 'agkqkqxkfbkfgyqliahsljim'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('khljxzxlpcrxpkrfybbfk'), materialize(['', 'lpc', 'rxpkrfybb', 'crxp', '', 'pkr', 'jxzxlpcrxpkrf', '', 'xzxlpcr', 'xpk', 'fyb', 'xzxlpcrxpkrfybbfk', 'k', 'lpcrxp', 'ljxzxlpcr', 'r', 'pkr', 'fk'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('rbrizgjbigvzfnpgmpkqxoqxvdj'), materialize(['ee', 'cohqnb', 'msol', 'yhlujcvhklnhuomy', 'ietn', 'vgmnlkcsybtokrepzrm', 'wspiryefojxysgrzsxyrluykxfnnbzdstcel', 'mxisnsivndbefqxwznimwgazuulupbaihavg', 'vpzdjvqqeizascxmzdhuq', 'pgvncohlxcqjhfkm', 'mbaypcnfapltsegquurahlsruqvipfhrhq', 'ioxjbcyyqujfveujfhnfdfokfcrlsincjbdt', 'cnvlujyowompdrqjwjx', 'wobwed', 'kdfhaoxiuifotmptcmdbk', 'leoamsnorcvtlmokdomkzuo', 'jjw', 'ogugysetxuqmvggneosbsfbonszepsatq'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('uymwxzyjbfegbhgswiqhinf'), materialize(['lizxzbzlwljkr', 'ukxygktlpzuyijcqeqktxenlaqi', 'onperabgbdiafsxwbvpjtyt', 'xfqgoqvhqph', 'aflmcwabtwgmajmmqelxwkaolyyhmdlc', 'yfz', 'meffuiaicvwed', 'hhzvgmifzamgftkifaeowayjrnnzw', 'nwewybtajv', 'ectiye', 'epjeiljegmqqjncubj', 'zsjgftqjrn', 'pssng', 'raqoarfhdoeujulvqmdo'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('omgghgnzjmecpzqmtcvw'), materialize(['fjhlzbszodmzavzg', 'gfofrnwrxprkfiokv', 'jmjiiqpgznlmyrxwewzqzbe', 'pkyrsqkltlmxr', 'crqgkgqkkyujcyoc', 'endagbcxwqhueczuasykmajfsvtcmh', 'xytmxtrnkdysuwltqomehddp', 'etmdxyyfotfyifwvbykghijvwv', 'mwqtgrncyhkfhjdg', 'iuvymofrqpp', 'pgllsdanlhzqhkstwsmzzftp', 'disjylcceufxtjdvhy'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('mznihnmshftvnmmhnrulizzpslq'), materialize(['nrul', 'mshftvnmmhnr', 'z', 'mhnrulizzps', 'hftvnmmhnrul', 'ihnmshftvnmmhnrulizzp', 'izz', '', 'uli', 'nihnmshftvnmmhnru', 'hnrulizzp', 'nrulizz'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('ruqmqrsxrbftvruvahonradau'), materialize(['uqmqrsxrbft', 'ftv', 'tvruvahonrad', 'mqrsxrbftvruvahon', 'rbftvruvah', 'qrsxrbftvru', 'o', 'ahonradau', 'a', 'ft', '', 'u', 'rsxrbftvruvahonradau', 'ruvahon', 'bftvruvahonradau', 'qrsxrbftvru', 't', 'vahonrada', 'vruvahonradau', 'onra'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('gpsevxtcoeexrltyzduyidmtzxf'), materialize(['exrltyzduyid', 'vxtcoeexrltyz', 'xr', 'ltyzduyidmt', 'yzduy', 'exr', 'coeexrltyzduy', 'coeexrltyzduy', 'rlty', 'rltyzduyidm', 'exrltyz', 'xtcoeexrlty', 'vxtcoeexrltyzduyidm', '', 'coeexrl', 'sevxtcoeexrltyzdu', 'dmt', ''])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('dyhycfhzyewaikgursyxfkuv'), materialize(['sktnofpugrmyxmbizzrivmhn', 'fhlgadpoqcvktbfzncxbllvwutdawmw', 'eewzjpcgzrqmltbgmhafwlwqb', 'tpogbkyj', 'rtllntxjgkzs', 'mirbvsqexscnzglogigbujgdwjvcv', 'iktwpgjsakemewmahgqza', 'xgfvzkvqgiuoihjjnxwwpznxhz', 'nxaumpaknreklbwynvxdsmatjekdlxvklh', 'zadzwqhgfxqllihuudozxeixyokhny', 'tdqpgfpzexlkslodps', 'slztannufxaabqfcjyfquafgfhfb', 'xvjldhfuwurvkb', 'aecv', 'uycfsughpikqsbcmwvqygdyexkcykhbnau', 'jr'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('vbcsettndwuntnruiyclvvwoo'), materialize(['dwuntnru', '', 'ttndwuntnruiyclvv', 'ntnr', 'nruiyclvvw', 'wo', '', 'bcsettndwuntnruiycl', 'yc', 'untnruiyclvvw', 'csettndwuntnr', 'ntnruiyclvvwo'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('pqqnugshlczcuxhpjxjbcnro'), materialize(['dpeedqy', 'rtsc', 'jdgla', 'qkgudqjiyzvlvsj', 'xmfxawhijgxxtydbd', 'ebgzazqthb', 'wyrjhvhwzhmpybnylirrn', 'iviqbyuclayqketooztwegtkgwnsezfl', 'bhvidy', 'hijctxxweboq', 't', 'osnzfbziidteiaifgaanm'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('loqchlxspwuvvccucskuytr'), materialize(['', 'k', 'qchlxspwu', 'u', 'hlxspwuvv', 'wuvvccucsku', 'vcc', 'uyt', 'uvv', 'spwu', 'ytr', 'wuvvccucs', 'xspwuv', 'lxspwuvvccuc', 'spwuvvccu', 'oqchlxspwuvvccucskuy'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('pjjyzupzwllshlnatiujmwvaofr'), materialize(['lnatiujmwvao', '', 'zupzwllsh', 'nati', 'wllshl', 'hlnatiujmwv', 'mwvao', 'shlnat', 'ati', 'wllshlnatiujmwvao', 'wllshlnatiujmwvaofr', 'nat'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('iketunkleyaqaxdlocci'), materialize(['nkleyaqaxd', 'etunkleyaq', 'yaqaxdlocci', 'tunkleyaq', 'eyaqaxdlocc', 'leyaq', 'nkleyaqaxdl', 'tunkleya', 'kleyaqa', 'etunkleya', 'leyaqa', 'dlo', 'yaqa', 'leyaqaxd', 'etunkleyaq', ''])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('drqianqtangmgbdwruvblkqd'), materialize(['wusajejyucamkyl', 'wsgibljugzrpkniliy', 'lhwqqiuafwffyersqjgjvvvfurx', 'jfokpzzxfdonelorqu', 'ccwkpcgac', 'jmyulqpndkmzbfztobwtm', 'rwrgfkccgxht', 'ggldjecrgbngkonphtcxrkcviujihidjx', 'spwweavbiokizv', 'lv', 'krb', 'vstnhvkbwlqbconaxgbfobqky', 'pvxwdc', 'thrl', 'ahsblffdveamceonqwrbeyxzccmux', 'yozji', 'oejtaxwmeovtqtz', 'zsnzznvqpxdvdxhznxrjn', 'hse', 'kcmkrccxmljzizracxwmpoaggywhdfpxkq'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('yasnpckniistxcejowfijjsvkdajz'), materialize(['slkpxhtsmrtvtm', 'crsbq', 'rdeshtxbfrlfwpsqojassxmvlfbzefldavmgme', 'ipetilcbpsfroefkjirquciwtxhrimbmwnlyv', 'knjpwkmdwbvdbapuyqbtsw', 'horueidziztxovqhsicnklmharuxhtgrsr', 'ofohrgpz', 'oneqnwyevbaqsonrcpmxcynflojmsnix', 'shg', 'nglqzczevgevwawdfperpeytuodjlf'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('ueptpscfgxhplwsueckkxs'), materialize(['ohhygchclbpcdwmftperprn', 'dvpjdqmqckekndvcerqrpkxen', 'lohhvarnmyi', 'zppd', 'qmqxgfewitsunbuhffozcpjtc', 'hsjbioisycsrawktqssjovkmltxodjgv', 'dbzuunwbkrtosyvctdujqtvaawfnvuq', 'gupbvpqthqxae', 'abjdmijaaiasnccgxttmqdsz', 'uccyumqoyqe', 'kxxliepyzlc', 'wbqcqtbyyjbqcgdbpkmzugksmcxhvr', 'piedxm', 'uncpphzoif', 'exkdankwck', 'qeitzozdrqopsergzr', 'hesgrhaftgesnzflrrtjdobxhbepjoas', 'wfpexx'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('ldrzgttlqaphekkkdukgngl'), materialize(['gttlqaphekkkdukgn', 'ekkkd', 'gttlqaphe', 'qaphek', 'h', 'kdu', 'he', 'phek', '', 'drzgttlqaphekkkd'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('ololo'), materialize(['ololo', 'ololo', 'ololo'])); +SELECT 1 = multiMatchAny(materialize('khljxzxlpcrxpkrfybbfk'), materialize(['k'])); + +select 1 = multiMatchAny(materialize(''), materialize([''])); +select 0 = multiMatchAny(materialize(''), materialize(['some string'])); +select 1 = multiMatchAny(materialize('abc'), materialize([''])); +select 1 = multiMatchAny(materialize('abc'), materialize([''])) from system.numbers limit 10; + +select 0 = multiMatchAny(materialize('abc'), materialize(['defgh'])); +select 0 = multiMatchAny(materialize('abc'), materialize(['defg'])); +select 0 = multiMatchAny(materialize('abc'), materialize(['def'])); +select 0 = multiMatchAny(materialize('abc'), materialize(['de'])); +select 0 = multiMatchAny(materialize('abc'), materialize(['d'])); + + +select 1 = multiMatchAny(materialize('abc'), materialize(['b'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('abc'), materialize(['bc'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['bcde'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['bcdef'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['bcdefg'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['bcdefgh'])) from system.numbers limit 10; + + +select 0 = multiMatchAny(materialize('abc'), materialize(['abcdefg'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['abcdef'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['abcde'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['abcd'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('abc'), materialize(['abc'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('abc'), materialize(['ab'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('abc'), materialize(['a'])) from system.numbers limit 10; + +select 1 = multiMatchAny(materialize('abcd'), materialize(['c'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('abcd'), materialize(['cd'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abcd'), materialize(['cde'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abcd'), materialize(['cdef'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abcd'), materialize(['cdefg'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abcd'), materialize(['cdefgh'])) from system.numbers limit 10; + +select 0 = multiMatchAny(materialize('abc'), materialize(['defgh'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['defg'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['def'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['de'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abc'), materialize(['d'])) from system.numbers limit 10; + +select 1 = multiMatchAny(materialize('abc'), materialize(['...'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('a\nbc'), materialize(['a?bc'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('a\nbc'), materialize(['a.bc'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('a\0bc'), materialize(['a?bc'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('a\0bc'), materialize(['a.bc'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('abcdef'), materialize(['a.....'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('abcdef'), materialize(['a......'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('abcdef'), materialize(['a......', 'a.....'])) from system.numbers limit 10; +select 0 = multiMatchAny(materialize('aaaa'), materialize(['.*aa.*aaa.*', 'aaaaaa{2}', '\(aa\){3}'])) from system.numbers limit 10; +select 1 = multiMatchAny(materialize('abc'), materialize(['a\0d'])) from system.numbers limit 10; + +select 1 = multiMatchAnyIndex(materialize('gogleuedeuniangoogle'), materialize(['google', 'unian1'])) from system.numbers limit 10; +select 2 = multiMatchAnyIndex(materialize('gogleuedeuniangoogle'), materialize(['google1', 'unian'])) from system.numbers limit 10; +select 0 != multiMatchAnyIndex(materialize('gogleuedeuniangoogle'), materialize(['.*goo.*', '.*yan.*'])) from system.numbers limit 10; +select 5 = multiMatchAnyIndex(materialize('vladizlvav dabe don\'t heart me no more'), materialize(['what', 'is', 'love', 'baby', 'no mo??', 'dont', 'h.rt me'])) from system.numbers limit 10; + +SELECT multiMatchAny(materialize('/odezhda-dlya-bega/'), materialize(['/odezhda-dlya-bega/', 'kurtki-i-vetrovki-dlya-bega', 'futbolki-i-mayki-dlya-bega'])); +SELECT 1 = multiMatchAny(materialize('фабрикант'), materialize(['f[ae]b[ei]rl', 'ф[иаэе]б[еэи][рпл]', 'афиукд', 'a[ft],th', '^ф[аиеэ]?б?[еэи]?$', 'берлик', 'fab', 'фа[беьв]+е?[рлко]'])); + +-- All indices tests +SELECT [1, 2] = arraySort(multiMatchAllIndices(materialize('gogleuedeuniangoogle'), materialize(['.*goo.*', '.*yan.*']))) from system.numbers limit 5; +SELECT [1, 3] = arraySort(multiMatchAllIndices(materialize('gogleuedeuniangoogle'), materialize(['.*goo.*', 'neverexisted', '.*yan.*']))) from system.numbers limit 5; +SELECT [] = multiMatchAllIndices(materialize('gogleuedeuniangoogle'), materialize(['neverexisted', 'anotherone', 'andanotherone'])) from system.numbers limit 5; +SELECT [1, 2, 3, 11] = arraySort(multiMatchAllIndices(materialize('фабрикант'), materialize(['', 'рикан', 'а', 'f[ae]b[ei]rl', 'ф[иаэе]б[еэи][рпл]', 'афиукд', 'a[ft],th', '^ф[аиеэ]?б?[еэи]?$', 'берлик', 'fab', 'фа[беьв]+е?[рлко]']))); +SELECT [1] = multiMatchAllIndices(materialize('/odezhda-dlya-bega/'), materialize(['/odezhda-dlya-bega/', 'kurtki-i-vetrovki-dlya-bega', 'futbolki-i-mayki-dlya-bega'])); +SELECT [] = multiMatchAllIndices(materialize('aaaa'), materialize(['.*aa.*aaa.*', 'aaaaaa{2}', '\(aa\){3}'])); diff --git a/tests/queries/0_stateless/00927_disable_hyperscan.reference b/tests/queries/0_stateless/00927_disable_hyperscan.reference index 7cac306b9db..a6548d5d5a7 100644 --- a/tests/queries/0_stateless/00927_disable_hyperscan.reference +++ b/tests/queries/0_stateless/00927_disable_hyperscan.reference @@ -6,4 +6,9 @@ 1 1 1 +1 +0 +1 +1 +1 0 diff --git a/tests/queries/0_stateless/00927_disable_hyperscan.sql b/tests/queries/0_stateless/00927_disable_hyperscan.sql index 9f0f6ca1538..d6f47d739fb 100644 --- a/tests/queries/0_stateless/00927_disable_hyperscan.sql +++ b/tests/queries/0_stateless/00927_disable_hyperscan.sql @@ -1,9 +1,16 @@ -- Tags: no-debug SET allow_hyperscan = 1; + SELECT multiMatchAny(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), ['hel+o', 'w(or)*ld']); +SELECT multiMatchAny(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), materialize(['hel+o', 'w(or)*ld'])); + SET allow_hyperscan = 0; + SELECT multiMatchAny(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), ['hel+o', 'w(or)*ld']); -- { serverError 446 } +SELECT multiMatchAny(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), materialize(['hel+o', 'w(or)*ld'])); -- { serverError 446 } + SELECT multiMatchAllIndices(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), ['hel+o', 'w(or)*ld']); -- { serverError 446 } +SELECT multiMatchAllIndices(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), materialize(['hel+o', 'w(or)*ld'])); -- { serverError 446 } SELECT multiSearchAny(arrayJoin(['hello', 'world', 'hello, world', 'abc']), ['hello', 'world']); diff --git a/tests/queries/0_stateless/00928_multi_match_constant_constant.reference b/tests/queries/0_stateless/00928_multi_match_constant_constant.reference index cc2b1466fcb..653ddcf0d58 100644 --- a/tests/queries/0_stateless/00928_multi_match_constant_constant.reference +++ b/tests/queries/0_stateless/00928_multi_match_constant_constant.reference @@ -1,3 +1,7 @@ 1 +1 2 +2 +[2] +[2] [1,8] diff --git a/tests/queries/0_stateless/00928_multi_match_constant_constant.sql b/tests/queries/0_stateless/00928_multi_match_constant_constant.sql index fc3e8ca6b2c..6b4beea2b79 100644 --- a/tests/queries/0_stateless/00928_multi_match_constant_constant.sql +++ b/tests/queries/0_stateless/00928_multi_match_constant_constant.sql @@ -1,3 +1,12 @@ +-- Tags: no-fasttest + SELECT multiMatchAny('goodbye', ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); +SELECT multiFuzzyMatchAny('goodbye', 1, ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); + SELECT multiMatchAnyIndex('goodbye', ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); +SELECT multiFuzzyMatchAnyIndex('goodbye', 1, ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); + +SELECT multiMatchAllIndices('goodbye', ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); +SELECT multiFuzzyMatchAllIndices('goodbye', 1, ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); + SELECT multiSearchAllPositions('hello, world', ['hello', 'world']); diff --git a/tests/queries/0_stateless/00929_multi_match_edit_distance.reference b/tests/queries/0_stateless/00929_multi_match_edit_distance.reference index 4600557506b..82e47e3ffad 100644 --- a/tests/queries/0_stateless/00929_multi_match_edit_distance.reference +++ b/tests/queries/0_stateless/00929_multi_match_edit_distance.reference @@ -1,3 +1,7 @@ +- const pattern +0 +0 +0 1 1 1 @@ -9,6 +13,22 @@ 1 1 1 +0 +1 +1 +1 +1 +0 +1 +2 +2 +1 +[2,3,4] +[] +- non-const pattern +0 +0 +0 1 1 1 @@ -18,17 +38,15 @@ 1 1 1 +0 1 1 1 1 +0 1 +2 +2 1 -1 -1 -1 -1 -1 -1 -1 -1 +[2,3,4] +[] diff --git a/tests/queries/0_stateless/00929_multi_match_edit_distance.sql b/tests/queries/0_stateless/00929_multi_match_edit_distance.sql index 343fc75ed4d..c86accd260b 100644 --- a/tests/queries/0_stateless/00929_multi_match_edit_distance.sql +++ b/tests/queries/0_stateless/00929_multi_match_edit_distance.sql @@ -2,30 +2,60 @@ SET send_logs_level = 'fatal'; -select 0 = multiFuzzyMatchAny('abc', 0, ['a1c']) from system.numbers limit 5; -select 1 = multiFuzzyMatchAny('abc', 1, ['a1c']) from system.numbers limit 5; -select 1 = multiFuzzyMatchAny('abc', 2, ['a1c']) from system.numbers limit 5; -select 1 = multiFuzzyMatchAny('abc', 3, ['a1c']) from system.numbers limit 5; -- { serverError 36 } -select 1 = multiFuzzyMatchAny('abc', 4, ['a1c']) from system.numbers limit 5; -- { serverError 36 } +SELECT '- const pattern'; -select 1 = multiFuzzyMatchAny('leftabcright', 1, ['a1c']) from system.numbers limit 5; +-- run queries multiple times to test the pattern caching +select multiFuzzyMatchAny('abc', 0, ['a1c']) from system.numbers limit 3; +select multiFuzzyMatchAny('abc', 1, ['a1c']) from system.numbers limit 3; +select multiFuzzyMatchAny('abc', 2, ['a1c']) from system.numbers limit 3; +select multiFuzzyMatchAny('abc', 3, ['a1c']) from system.numbers limit 3; -- { serverError 36 } +select multiFuzzyMatchAny('abc', 4, ['a1c']) from system.numbers limit 3; -- { serverError 36 } -select 1 = multiFuzzyMatchAny('hello some world', 0, ['^hello.*world$']); -select 1 = multiFuzzyMatchAny('hallo some world', 1, ['^hello.*world$']); -select 0 = multiFuzzyMatchAny('halo some wrld', 2, ['^hello.*world$']); -select 1 = multiFuzzyMatchAny('halo some wrld', 2, ['^hello.*world$', '^halo.*world$']); -select 1 = multiFuzzyMatchAny('halo some wrld', 2, ['^halo.*world$', '^hello.*world$']); -select 1 = multiFuzzyMatchAny('halo some wrld', 3, ['^hello.*world$']); -select 1 = multiFuzzyMatchAny('hello some world', 10, ['^hello.*world$']); -- { serverError 36 } -select 1 = multiFuzzyMatchAny('hello some world', -1, ['^hello.*world$']); -- { serverError 43 } -select 1 = multiFuzzyMatchAny('hello some world', 10000000000, ['^hello.*world$']); -- { serverError 44 } -select 1 = multiFuzzyMatchAny('http://hyperscan_is_nice.ru/st', 2, ['http://hyperscan_is_nice.ru/(st\\d\\d$|st\\d\\d\\.|st1[0-4]\\d|st150|st\\d$|gl|rz|ch)']); -select 0 = multiFuzzyMatchAny('string', 0, ['zorro$', '^tring', 'in$', 'how.*', 'it{2}', 'works']); +select multiFuzzyMatchAny('leftabcright', 1, ['a1c']) from system.numbers limit 3; -select 1 = multiFuzzyMatchAny('string', 1, ['zorro$', '^tring', 'ip$', 'how.*', 'it{2}', 'works']); -select 2 = multiFuzzyMatchAnyIndex('string', 1, ['zorro$', '^tring', 'ip$', 'how.*', 'it{2}', 'works']); -select 2 = multiFuzzyMatchAnyIndex('halo some wrld', 2, ['^hello.*world$', '^halo.*world$']); -select 1 = multiFuzzyMatchAnyIndex('halo some wrld', 2, ['^halo.*world$', '^hello.*world$']); +select multiFuzzyMatchAny('hello some world', 0, ['^hello.*world$']); +select multiFuzzyMatchAny('hallo some world', 1, ['^hello.*world$']); +select multiFuzzyMatchAny('halo some wrld', 2, ['^hello.*world$']); +select multiFuzzyMatchAny('halo some wrld', 2, ['^hello.*world$', '^halo.*world$']); +select multiFuzzyMatchAny('halo some wrld', 2, ['^halo.*world$', '^hello.*world$']); +select multiFuzzyMatchAny('halo some wrld', 3, ['^hello.*world$']); +select multiFuzzyMatchAny('hello some world', 10, ['^hello.*world$']); -- { serverError 36 } +select multiFuzzyMatchAny('hello some world', -1, ['^hello.*world$']); -- { serverError 43 } +select multiFuzzyMatchAny('hello some world', 10000000000, ['^hello.*world$']); -- { serverError 44 } +select multiFuzzyMatchAny('http://hyperscan_is_nice.de/st', 2, ['http://hyperscan_is_nice.de/(st\\d\\d$|st\\d\\d\\.|st1[0-4]\\d|st150|st\\d$|gl|rz|ch)']); +select multiFuzzyMatchAny('string', 0, ['zorro$', '^tring', 'in$', 'how.*', 'it{2}', 'works']); +select multiFuzzyMatchAny('string', 1, ['zorro$', '^tring', 'ip$', 'how.*', 'it{2}', 'works']); +select multiFuzzyMatchAnyIndex('string', 1, ['zorro$', '^tring', 'ip$', 'how.*', 'it{2}', 'works']); +select multiFuzzyMatchAnyIndex('halo some wrld', 2, ['^hello.*world$', '^halo.*world$']); +select multiFuzzyMatchAnyIndex('halo some wrld', 2, ['^halo.*world$', '^hello.*world$']); +-- +select arraySort(multiFuzzyMatchAllIndices('halo some wrld', 2, ['some random string', '^halo.*world$', '^halo.*world$', '^halo.*world$', '^hallllo.*world$'])); +select multiFuzzyMatchAllIndices('halo some wrld', 2, ['^halllllo.*world$', 'some random string']); -select [2, 3, 4] = arraySort(multiFuzzyMatchAllIndices('halo some wrld', 2, ['some random string', '^halo.*world$', '^halo.*world$', '^halo.*world$', '^hallllo.*world$'])); -select [] = multiFuzzyMatchAllIndices('halo some wrld', 2, ['^halllllo.*world$', 'some random string']); +SELECT '- non-const pattern'; + +select multiFuzzyMatchAny(materialize('abc'), 0, materialize(['a1c'])) from system.numbers limit 3; +select multiFuzzyMatchAny(materialize('abc'), 1, materialize(['a1c'])) from system.numbers limit 3; +select multiFuzzyMatchAny(materialize('abc'), 2, materialize(['a1c'])) from system.numbers limit 3; +select multiFuzzyMatchAny(materialize('abc'), 3, materialize(['a1c'])) from system.numbers limit 3; -- { serverError 36} +select multiFuzzyMatchAny(materialize('abc'), 4, materialize(['a1c'])) from system.numbers limit 3; -- { serverError 36} + +select multiFuzzyMatchAny(materialize('leftabcright'), 1, materialize(['a1c'])); + +select multiFuzzyMatchAny(materialize('hello some world'), 0, materialize(['^hello.*world$'])); +select multiFuzzyMatchAny(materialize('hallo some world'), 1, materialize(['^hello.*world$'])); +select multiFuzzyMatchAny(materialize('halo some wrld'), 2, materialize(['^hello.*world$'])); +select multiFuzzyMatchAny(materialize('halo some wrld'), 2, materialize(['^hello.*world$', '^halo.*world$'])); +select multiFuzzyMatchAny(materialize('halo some wrld'), 2, materialize(['^halo.*world$', '^hello.*world$'])); +select multiFuzzyMatchAny(materialize('halo some wrld'), 3, materialize(['^hello.*world$'])); +select multiFuzzyMatchAny(materialize('hello some world'), 10, materialize(['^hello.*world$'])); -- { serverError 36 } +select multiFuzzyMatchAny(materialize('hello some world'), -1, materialize(['^hello.*world$'])); -- { serverError 43 } +select multiFuzzyMatchAny(materialize('hello some world'), 10000000000, materialize(['^hello.*world$'])); -- { serverError 44 } +select multiFuzzyMatchAny(materialize('http://hyperscan_is_nice.de/st'), 2, materialize(['http://hyperscan_is_nice.de/(st\\d\\d$|st\\d\\d\\.|st1[0-4]\\d|st150|st\\d$|gl|rz|ch)'])); +select multiFuzzyMatchAny(materialize('string'), 0, materialize(['zorro$', '^tring', 'in$', 'how.*', 'it{2}', 'works'])); +select multiFuzzyMatchAny(materialize('string'), 1, materialize(['zorro$', '^tring', 'ip$', 'how.*', 'it{2}', 'works'])); +select multiFuzzyMatchAnyIndex(materialize('string'), 1, materialize(['zorro$', '^tring', 'ip$', 'how.*', 'it{2}', 'works'])); +select multiFuzzyMatchAnyIndex(materialize('halo some wrld'), 2, materialize(['^hello.*world$', '^halo.*world$'])); +select multiFuzzyMatchAnyIndex(materialize('halo some wrld'), 2, materialize(['^halo.*world$', '^hello.*world$'])); +select arraySort(multiFuzzyMatchAllIndices(materialize('halo some wrld'), 2, materialize(['some random string', '^halo.*world$', '^halo.*world$', '^halo.*world$', '^hallllo.*world$']))); +select multiFuzzyMatchAllIndices(materialize('halo some wrld'), 2, materialize(['^halllllo.*world$', 'some random string'])); diff --git a/tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference index 87e68c152c3..673149878ac 100644 --- a/tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference +++ b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference @@ -1,3 +1,11 @@ +- const pattern +1 +1 +[1] +1 +1 +[1] +- non-const pattern 1 1 [1] diff --git a/tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql index df78b3ed272..17d3796e88c 100644 --- a/tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql +++ b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql @@ -3,6 +3,8 @@ set max_hyperscan_regexp_length = 1; set max_hyperscan_regexp_total_length = 1; +SELECT '- const pattern'; + select multiMatchAny('123', ['1']); select multiMatchAny('123', ['12']); -- { serverError 36 } select multiMatchAny('123', ['1', '2']); -- { serverError 36 } @@ -26,3 +28,29 @@ select multiFuzzyMatchAnyIndex('123', 0, ['1', '2']); -- { serverError 36 } select multiFuzzyMatchAllIndices('123', 0, ['1']); select multiFuzzyMatchAllIndices('123', 0, ['12']); -- { serverError 36 } select multiFuzzyMatchAllIndices('123', 0, ['1', '2']); -- { serverError 36 } + +SELECT '- non-const pattern'; + +select multiMatchAny(materialize('123'), materialize(['1'])); +select multiMatchAny(materialize('123'), materialize(['12'])); -- { serverError 36 } +select multiMatchAny(materialize('123'), materialize(['1', '2'])); -- { serverError 36 } + +select multiMatchAnyIndex(materialize('123'), materialize(['1'])); +select multiMatchAnyIndex(materialize('123'), materialize(['12'])); -- { serverError 36 } +select multiMatchAnyIndex(materialize('123'), materialize(['1', '2'])); -- { serverError 36 } + +select multiMatchAllIndices(materialize('123'), materialize(['1'])); +select multiMatchAllIndices(materialize('123'), materialize(['12'])); -- { serverError 36 } +select multiMatchAllIndices(materialize('123'), materialize(['1', '2'])); -- { serverError 36 } + +select multiFuzzyMatchAny(materialize('123'), 0, materialize(['1'])); +select multiFuzzyMatchAny(materialize('123'), 0, materialize(['12'])); -- { serverError 36 } +select multiFuzzyMatchAny(materialize('123'), 0, materialize(['1', '2'])); -- { serverError 36 } + +select multiFuzzyMatchAnyIndex(materialize('123'), 0, materialize(['1'])); +select multiFuzzyMatchAnyIndex(materialize('123'), 0, materialize(['12'])); -- { serverError 36 } +select multiFuzzyMatchAnyIndex(materialize('123'), 0, materialize(['1', '2'])); -- { serverError 36 } + +select multiFuzzyMatchAllIndices(materialize('123'), 0, materialize(['1'])); +select multiFuzzyMatchAllIndices(materialize('123'), 0, materialize(['12'])); -- { serverError 36 } +select multiFuzzyMatchAllIndices(materialize('123'), 0, materialize(['1', '2'])); -- { serverError 36 } From ece61f6da3757ca192d0d6a548ea06038e659748 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 28 Jun 2022 07:51:36 +0000 Subject: [PATCH 143/627] Fix davenger's review comments https://github.com/ClickHouse/ClickHouse/pull/38434#discussion_r907397214 https://github.com/ClickHouse/ClickHouse/pull/38434#discussion_r907385290 https://github.com/ClickHouse/ClickHouse/pull/38434#discussion_r907406097 (the latter is no longer relevant as the affected places were removed in the meantime) --- .../FunctionsMultiStringFuzzySearch.h | 2 +- src/Functions/FunctionsMultiStringSearch.h | 2 +- src/Functions/MultiMatchAllIndicesImpl.h | 56 ++++++++++++------- src/Functions/MultiMatchAnyImpl.h | 14 ++--- src/Functions/checkHyperscanRegexp.h | 3 +- 5 files changed, 47 insertions(+), 30 deletions(-) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 75663693476..1e13b300136 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -96,7 +96,7 @@ public: auto & vec_res = col_res->getData(); auto & offsets_res = col_offsets->getData(); - // the implementations are responsible for resizing the output column + /// the implementations are responsible for resizing the output column if (col_needles_const) { diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 3531a0538f3..1a47a54c04c 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -97,7 +97,7 @@ public: auto & vec_res = col_res->getData(); auto & offsets_res = col_offsets->getData(); - // the implementations are responsible for resizing the output column + /// the implementations are responsible for resizing the output column if (col_needles_const) { diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index e9662386f3a..7ff2376593c 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -58,15 +58,15 @@ struct MultiMatchAllIndicesImpl } static void vectorConstant( - [[maybe_unused]] const ColumnString::Chars & haystack_data, - [[maybe_unused]] const ColumnString::Offsets & haystack_offsets, - [[maybe_unused]] const Array & needles_arr, - [[maybe_unused]] PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets, - [[maybe_unused]] std::optional edit_distance, + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const Array & needles_arr, + PaddedPODArray & res, + PaddedPODArray & offsets, + std::optional edit_distance, bool allow_hyperscan, - [[maybe_unused]] size_t max_hyperscan_regexp_length, - [[maybe_unused]] size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_length, + size_t max_hyperscan_regexp_total_length) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -103,10 +103,10 @@ struct MultiMatchAllIndicesImpl for (size_t i = 0; i < haystack_offsets_size; ++i) { UInt64 length = haystack_offsets[i] - offset - 1; - // vectorscan restriction. + /// vectorscan restriction. if (length > std::numeric_limits::max()) throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES); - // scan, check, update the offsets array and the offset of haystack. + /// scan, check, update the offsets array and the offset of haystack. err = hs_scan( hyperscan_regex->getDB(), reinterpret_cast(haystack_data.data()) + offset, @@ -121,6 +121,14 @@ struct MultiMatchAllIndicesImpl offset = haystack_offsets[i]; } #else + (void)haystack_data; + (void)haystack_offsets; + (void)needles_arr; + (void)res; + (void)offsets; + (void)edit_distance; + (void)max_hyperscan_regexp_length; + (void)max_hyperscan_regexp_total_length; throw Exception( "multi-search all indices is not implemented when vectorscan is off", ErrorCodes::NOT_IMPLEMENTED); @@ -141,15 +149,15 @@ struct MultiMatchAllIndicesImpl } static void vectorVector( - [[maybe_unused]] const ColumnString::Chars & haystack_data, - [[maybe_unused]] const ColumnString::Offsets & haystack_offsets, - [[maybe_unused]] const ColumnArray & needles_col, - [[maybe_unused]] PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets, - [[maybe_unused]] std::optional edit_distance, + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnArray & needles_col, + PaddedPODArray & res, + PaddedPODArray & offsets, + std::optional edit_distance, bool allow_hyperscan, - [[maybe_unused]] size_t max_hyperscan_regexp_length, - [[maybe_unused]] size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_length, + size_t max_hyperscan_regexp_total_length) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -191,11 +199,11 @@ struct MultiMatchAllIndicesImpl const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; - // vectorscan restriction. + /// vectorscan restriction. if (cur_haystack_length > std::numeric_limits::max()) throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES); - /// Scan, check, update the offsets array and the offset of haystack. + /// scan, check, update the offsets array and the offset of haystack. err = hs_scan( hyperscan_regex->getDB(), reinterpret_cast(haystack_data.data()) + prev_haystack_offset, @@ -212,6 +220,14 @@ struct MultiMatchAllIndicesImpl prev_haystack_offset = haystack_offsets[i]; } #else + (void)haystack_data; + (void)haystack_offsets; + (void)needles_col; + (void)res; + (void)offsets; + (void)edit_distance; + (void)max_hyperscan_regexp_length; + (void)max_hyperscan_regexp_total_length; throw Exception( "multi-search all indices is not implemented when vectorscan is off", ErrorCodes::NOT_IMPLEMENTED); diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 11c9e8afaf5..24cf6b53f30 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -29,7 +29,7 @@ namespace ErrorCodes extern const int TOO_MANY_BYTES; } -// For more readable instantiations of MultiMatchAnyImpl<> +/// For more readable instantiations of MultiMatchAnyImpl<> struct MultiMatchTraits { enum class Find @@ -120,10 +120,10 @@ struct MultiMatchAnyImpl for (size_t i = 0; i < haystack_offsets_size; ++i) { UInt64 length = haystack_offsets[i] - offset - 1; - // vectorscan restriction. + /// vectorscan restriction. if (length > std::numeric_limits::max()) throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES); - // zero the result, scan, check, update the offset. + /// zero the result, scan, check, update the offset. res[i] = 0; err = hs_scan( hyperscan_regex->getDB(), @@ -138,7 +138,7 @@ struct MultiMatchAnyImpl offset = haystack_offsets[i]; } #else - // fallback if vectorscan is not compiled + /// fallback if vectorscan is not compiled if constexpr (WithEditDistance) throw Exception( "Edit distance multi-search is not implemented when vectorscan is off", @@ -228,11 +228,11 @@ struct MultiMatchAnyImpl const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; - // vectorscan restriction. + /// vectorscan restriction. if (cur_haystack_length > std::numeric_limits::max()) throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES); - // zero the result, scan, check, update the offset. + /// zero the result, scan, check, update the offset. res[i] = 0; err = hs_scan( hyperscan_regex->getDB(), @@ -248,7 +248,7 @@ struct MultiMatchAnyImpl prev_haystack_offset = haystack_offsets[i]; } #else - // fallback if vectorscan is not compiled + /// fallback if vectorscan is not compiled if constexpr (WithEditDistance) throw Exception( "Edit distance multi-search is not implemented when vectorscan is off", diff --git a/src/Functions/checkHyperscanRegexp.h b/src/Functions/checkHyperscanRegexp.h index 2aac44115fc..e5f1165a949 100644 --- a/src/Functions/checkHyperscanRegexp.h +++ b/src/Functions/checkHyperscanRegexp.h @@ -1,6 +1,7 @@ #pragma once -#include +#include +#include namespace DB { From da73a228d895b5c6d18bd35e48671b49ae66f6d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 13:21:35 +0200 Subject: [PATCH 144/627] Add results for Aurora for Postgres --- benchmark/compatible/README.md | 4 +- .../aurora-postgresql/results/16acu.txt | 44 +++++++++++++++++++ 2 files changed, 46 insertions(+), 2 deletions(-) diff --git a/benchmark/compatible/README.md b/benchmark/compatible/README.md index ce5133ffc6e..d23e720ae21 100644 --- a/benchmark/compatible/README.md +++ b/benchmark/compatible/README.md @@ -41,8 +41,8 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] Databricks - [ ] Planetscale (without publishing) - [ ] TiDB (TiFlash) -- [ ] Amazon RDS Aurora for MySQL -- [ ] Amazon RDS Aurora for Postgres +- [x] Amazon RDS Aurora for MySQL +- [x] Amazon RDS Aurora for Postgres - [ ] InfluxDB - [ ] VictoriaMetrics - [ ] TDEngine diff --git a/benchmark/compatible/aurora-postgresql/results/16acu.txt b/benchmark/compatible/aurora-postgresql/results/16acu.txt index 7a31464a23f..a6156f36922 100644 --- a/benchmark/compatible/aurora-postgresql/results/16acu.txt +++ b/benchmark/compatible/aurora-postgresql/results/16acu.txt @@ -1,2 +1,46 @@ Load time: 2127 seconds Data size: 48.6 GiB + +[12.8361,5.71812,5.8241], +[61.2565,62.1402,63.7173], +[68.0578,68.1218,67.609], +[7.83207,5.90193,6.0461], +[48.7194,48.0233,48.2198], +[289.492,304.639,282.436], +[6.30572,6.31857,6.21598], +[53.644,53.8931,53.5307], +[131.526,131.45,131.102], +[137.724,136.921,137.758], +[57.2079,56.2775,56.2152], +[56.5349,56.2048,55.9569], +[82.3897,82.8866,83.534], +[97.0569,97.1392,96.4731], +[85.6557,86.7783,86.2804], +[49.4325,42.4309,42.5743], +[111.537,114.59,111.807], +[88.4322,89.3756,87.7899], +[160.781,163.866,161.394], +[1025.04,2.10165,2.10065], +[106.741,56.2731,56.1535], +[59.2681,59.5272,59.536], +[58.6083,57.6054,57.3935], +[54.8271,55.1397,56.3487], +[54.718,52.469,53.271], +[53.5387,53.1926,52.4008], +[52.0042,51.9581,52.2453], +[60.1317,59.9695,59.2187], +[244.608,242.954,243.815], +[91.8674,92.4165,91.5884], +[63.7122,64.277,64.2783], +[69.2596,68.9535,69.4508], +[234.222,241.138,240.316], +[488.169,462.257,460.466], +[472.929,471.809,476.635], +[103.664,116.131,103.467], +[16.8124,3.34058,3.37782], +[0.852414,0.832073,0.859857], +[0.305464,0.31166,0.306694], +[4.55625,4.54098,4.58501], +[0.299746,0.297532,0.30334], +[0.275732,0.279817,0.27766], +[0.332107,0.324387,0.320099] From 4ca0c50d5706875e2afc98e45c8a2b9c0cbac392 Mon Sep 17 00:00:00 2001 From: chen <15651730270@163.com> Date: Mon, 4 Jul 2022 19:24:54 +0800 Subject: [PATCH 145/627] Update function.md create function already support on cluster. --- docs/en/sql-reference/statements/create/function.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/function.md b/docs/en/sql-reference/statements/create/function.md index 7d9a727a70d..0a452b6c4d2 100644 --- a/docs/en/sql-reference/statements/create/function.md +++ b/docs/en/sql-reference/statements/create/function.md @@ -10,7 +10,7 @@ Creates a user defined function from a lambda expression. The expression must co **Syntax** ```sql -CREATE FUNCTION name AS (parameter0, ...) -> expression +CREATE FUNCTION name [ON CLUSTER cluster] AS (parameter0, ...) -> expression ``` A function can have an arbitrary number of parameters. From a0956465cf209b90eb647d5d0fd05344533a4ffe Mon Sep 17 00:00:00 2001 From: chen <15651730270@163.com> Date: Mon, 4 Jul 2022 19:34:26 +0800 Subject: [PATCH 146/627] Update drop.md --- docs/en/sql-reference/statements/drop.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md index 49862cbfc02..9621cd4944f 100644 --- a/docs/en/sql-reference/statements/drop.md +++ b/docs/en/sql-reference/statements/drop.md @@ -105,7 +105,7 @@ System functions can not be dropped. **Syntax** ``` sql -DROP FUNCTION [IF EXISTS] function_name +DROP FUNCTION [IF EXISTS] function_name [on CLUSTER cluster] ``` **Example** From c1494aa2f65fa15f3570d1435ebaeb595acf9b01 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 13:37:42 +0200 Subject: [PATCH 147/627] Remove old benchmarks and move new one level up --- benchmark/{compatible => }/README.md | 0 benchmark/{compatible => }/athena/README.md | 0 .../athena/create_partitioned.sql | 0 .../{compatible => }/athena/create_single.sql | 0 benchmark/{compatible => }/athena/queries.sql | 0 .../athena/results/partitioned.txt | 0 .../athena/results/single.txt | 0 benchmark/{compatible => }/athena/run.sh | 0 .../{compatible => }/aurora-mysql/README.md | 0 .../{compatible => }/aurora-mysql/create.sql | 0 .../{compatible => }/aurora-mysql/queries.sql | 0 .../aurora-mysql/results/16acu.txt | 0 .../{compatible => }/aurora-mysql/run.sh | 0 .../aurora-postgresql/README.md | 0 .../aurora-postgresql/create.sql | 0 .../aurora-postgresql/queries.sql | 0 .../aurora-postgresql/results/16acu.txt | 0 .../{compatible => }/aurora-postgresql/run.sh | 0 .../{compatible => }/bigquery/.gitignore | 0 benchmark/{compatible => }/bigquery/README.md | 0 .../{compatible => }/bigquery/create.sql | 0 .../{compatible => }/bigquery/queries.sql | 0 benchmark/{compatible => }/bigquery/run.sh | 0 benchmark/{compatible => }/citus/benchmark.sh | 0 benchmark/{compatible => }/citus/create.sql | 0 benchmark/{compatible => }/citus/queries.sql | 0 .../citus/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/citus/run.sh | 0 .../clickhouse-local/benchmark.sh | 0 .../clickhouse-local/create.sql | 0 .../clickhouse-local/queries.sql | 0 .../results/c6a.4xlarge.partitioned.txt | 0 .../results/c6a.4xlarge.single.txt | 0 .../{compatible => }/clickhouse-local/run.sh | 0 .../{compatible => }/clickhouse/benchmark.sh | 0 .../{compatible => }/clickhouse/create.sql | 0 benchmark/clickhouse/queries.sql | 86 +- .../clickhouse/results/c6a.4xlarge.txt | 0 .../clickhouse/results/c6a.metal.txt | 0 benchmark/{compatible => }/clickhouse/run.sh | 0 benchmark/compatible/greenplum/benchmark.sh | 73 - benchmark/compatible/heavyai/queries.sql | 43 - benchmark/compatible/mariadb/queries.sql | 43 - benchmark/compatible/monetdb/benchmark.sh | 40 - benchmark/compatible/mysql/queries.sql | 43 - benchmark/compatible/postgresql/benchmark.sh | 23 - benchmark/compatible/postgresql/queries.sql | 43 - benchmark/compatible/singlestore/queries.sql | 43 - benchmark/compatible/timescaledb/benchmark.sh | 47 - benchmark/compatible/timescaledb/queries.sql | 43 - benchmark/compatible/vertica/benchmark.sh | 26 - benchmark/compatible/vertica/queries.sql | 43 - .../{compatible => }/cratedb/benchmark.sh | 0 benchmark/{compatible => }/cratedb/create.sql | 0 .../clickhouse => cratedb}/queries.sql | 0 .../cratedb/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/cratedb/run.sh | 0 benchmark/databend/README.md | 1 + .../{compatible => }/databend/benchmark.sh | 0 .../{compatible => }/databend/create.sql | 0 .../cratedb => databend}/queries.sql | 0 .../databend/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/databend/run.sh | 0 benchmark/{compatible => }/druid/benchmark.sh | 0 benchmark/{compatible => }/druid/ingest.json | 0 .../databend => druid}/queries.sql | 0 .../druid/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/druid/run.sh | 0 benchmark/{compatible => }/duckdb/README.md | 0 .../{compatible => }/duckdb/benchmark.sh | 0 benchmark/{compatible => }/duckdb/create.sql | 0 benchmark/{compatible => }/duckdb/load.py | 0 benchmark/duckdb/log | 15906 ---------------- benchmark/duckdb/queries.sql | 80 +- benchmark/{compatible => }/duckdb/query.py | 0 .../duckdb/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/duckdb/run.sh | 0 benchmark/duckdb/usability.md | 762 - benchmark/elasticsearch/README.md | 1 + .../elasticsearch/benchmark.sh | 0 benchmark/{compatible => }/exasol/README.md | 0 benchmark/greenplum/README | 43 - benchmark/greenplum/benchmark.sh | 101 +- .../{compatible => }/greenplum/create.sql | 0 benchmark/greenplum/dump_dataset_from_ch.sh | 5 - benchmark/greenplum/load_data_set.sql | 12 - benchmark/{compatible => }/greenplum/log.txt | 0 benchmark/greenplum/queries.sql | 86 +- benchmark/greenplum/result_parser.py | 150 - .../greenplum/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/greenplum/run.sh | 0 benchmark/greenplum/schema.sql | 3 - benchmark/hardware.sh | 215 +- .../benchmark-chyt.sh | 0 .../{clickhouse => hardware}/benchmark-new.sh | 0 .../{clickhouse => hardware}/benchmark-yql.sh | 0 .../benchmark_cloud.sh | 0 benchmark/hardware/hardware.sh | 214 + benchmark/hardware/queries.sql | 43 + .../{compatible => }/heavyai/benchmark.sh | 0 benchmark/{compatible => }/heavyai/create.sql | 0 .../{compatible/druid => heavyai}/queries.sql | 0 .../heavyai/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/heavyai/run.sh | 0 benchmark/hive/conf.sh | 4 - benchmark/hive/define_schema.sql | 9 - benchmark/hive/expect.tcl | 18 - benchmark/hive/log/log_100m_tuned | 11574 ----------- benchmark/hive/log/log_10m/log_10m_ | 5506 ------ benchmark/hive/log/log_10m/log_10m_1 | 1720 -- benchmark/hive/log/log_10m/log_10m_2 | 624 - benchmark/hive/log/log_10m/log_10m_3 | 2627 --- benchmark/hive/log/log_10m/log_10m_tuned | 7608 -------- benchmark/hive/log/log_10m/log_hits_10m | 9306 --------- benchmark/hive/queries.sql | 109 - benchmark/hive/run_hive.sh | 2 - benchmark/infinidb/conf.sh | 4 - benchmark/infinidb/define_schema.sql | 7 - benchmark/infinidb/expect.tcl | 23 - benchmark/infinidb/log/log_100m | 5540 ------ benchmark/infinidb/log/log_100m_tuned | 5383 ------ benchmark/infinidb/log/log_10m | 5818 ------ benchmark/infinidb/log/log_10m_tuned | 5812 ------ benchmark/infinidb/queries.sql | 111 - .../{compatible => }/infobright/benchmark.sh | 0 benchmark/infobright/conf.sh | 5 - .../{compatible => }/infobright/create.sql | 0 benchmark/infobright/define_schema.sql | 111 - benchmark/infobright/expect.tcl | 18 - benchmark/infobright/log-community/log_10m | 2511 --- benchmark/infobright/queries.sql | 156 +- .../infobright/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/infobright/run.sh | 0 benchmark/{compatible => }/locustdb/README.md | 0 .../{compatible => }/locustdb/benchmark.sh | 0 .../mariadb-columnstore/README.md | 0 .../mariadb-columnstore/benchmark.sh | 0 .../mariadb-columnstore/create.sql | 0 .../queries.sql | 0 .../results/c6a.4xlarge.txt | 0 .../mariadb-columnstore/run.sh | 0 .../{compatible => }/mariadb/benchmark.sh | 0 benchmark/{compatible => }/mariadb/create.sql | 0 .../duckdb => mariadb}/queries.sql | 0 benchmark/{compatible => }/mariadb/run.sh | 0 benchmark/memsql/benchmark.sh | 20 - benchmark/memsql/instructions.txt | 141 - benchmark/memsql/queries.sql | 43 - benchmark/monetdb/aws.log | 1774 -- benchmark/monetdb/benchmark.sh | 44 +- benchmark/{compatible => }/monetdb/create.sql | 0 benchmark/monetdb/instruction.md | 356 - benchmark/monetdb/log.txt | 341 - benchmark/monetdb/queries.sql | 86 +- .../{compatible => }/monetdb/query.expect | 0 .../monetdb/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/monetdb/run.sh | 0 benchmark/monetdb/send-query | 19 - benchmark/monetdb/usability.md | 1222 -- .../mysql-myisam/benchmark.sh | 0 .../{compatible => }/mysql-myisam/create.sql | 0 .../queries.sql | 0 .../mysql-myisam/results/c6a.4xlarge.txt | 0 .../{compatible => }/mysql-myisam/run.sh | 0 benchmark/{compatible => }/mysql/benchmark.sh | 0 benchmark/{compatible => }/mysql/create.sql | 0 .../mysql-myisam => mysql}/queries.sql | 0 .../mysql/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/mysql/run.sh | 0 benchmark/omnisci/benchmark.sh | 17 - benchmark/omnisci/instruction.md | 332 - benchmark/omnisci/log.txt | 210 - benchmark/omnisci/queries.sql | 43 - benchmark/postgresql/benchmark.sh | 27 +- .../{compatible => }/postgresql/create.sql | 0 benchmark/postgresql/instructions.md | 142 - benchmark/postgresql/log | 129 - benchmark/postgresql/queries.sql | 86 +- .../postgresql/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/postgresql/run.sh | 0 .../{compatible => }/questdb/benchmark.sh | 0 benchmark/{compatible => }/questdb/create.sql | 0 .../{compatible => }/questdb/queries.sql | 0 .../questdb/result/c6a.4xlarge.txt | 0 benchmark/{compatible => }/questdb/run.sh | 0 .../redshift-serverless/README.md | 0 .../redshift-serverless/create.sql | 0 .../redshift-serverless/queries.sql | 0 .../redshift-serverless/result/serverless.txt | 0 .../redshift-serverless/run.sh | 0 benchmark/{compatible => }/redshift/README.md | 0 .../{compatible => }/redshift/create.sql | 0 .../{compatible => }/redshift/queries.sql | 0 .../redshift/result/4x.ra3.xplus.txt | 0 benchmark/{compatible => }/redshift/run.sh | 0 .../{compatible => }/singlestore/benchmark.sh | 0 .../{compatible => }/singlestore/create.sql | 0 .../greenplum => singlestore}/queries.sql | 0 .../singlestore/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/singlestore/run.sh | 0 benchmark/{compatible => }/snowflake/NOTES.md | 0 .../{compatible => }/snowflake/README.md | 0 .../{compatible => }/snowflake/create.sql | 0 .../{compatible => }/snowflake/queries.sql | 0 .../snowflake/results/2xl.txt | 0 .../snowflake/results/3xl.txt | 0 .../snowflake/results/4xl.txt | 0 .../{compatible => }/snowflake/results/l.txt | 0 .../{compatible => }/snowflake/results/m.txt | 0 .../{compatible => }/snowflake/results/s.txt | 0 .../{compatible => }/snowflake/results/xl.txt | 0 .../{compatible => }/snowflake/results/xs.txt | 0 benchmark/{compatible => }/snowflake/run.sh | 0 .../{compatible => }/sqlite/benchmark.sh | 0 benchmark/{compatible => }/sqlite/create.sql | 0 benchmark/{compatible => }/sqlite/queries.sql | 0 .../sqlite/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/sqlite/run.sh | 0 .../{compatible => }/starrocks/README.md | 0 .../{compatible => }/starrocks/benchmark.sh | 0 benchmark/timescaledb/benchmark.sh | 50 +- .../{compatible => }/timescaledb/create.sql | 0 benchmark/timescaledb/log | 215 - benchmark/timescaledb/log_compressed | 129 - benchmark/timescaledb/queries.sql | 86 +- .../results/c6a.4xlarge.compression.txt | 0 .../timescaledb/results/c6a.4xlarge.txt | 0 benchmark/{compatible => }/timescaledb/run.sh | 0 benchmark/timescaledb/usability.md | 1663 -- benchmark/trino/README.md | 1 + benchmark/trino/benchmark.sh | 7 + benchmark/trino/create_partitioned.sql | 112 + benchmark/trino/create_single.sql | 108 + .../{compatible/monetdb => trino}/queries.sql | 18 +- benchmark/{compatible => }/vertica/.gitignore | 0 benchmark/vertica/README | 40 - benchmark/{compatible => }/vertica/README.md | 0 benchmark/vertica/benchmark.sh | 40 +- benchmark/{compatible => }/vertica/create.sql | 0 benchmark/vertica/hits_define_schema.sql | 339 - benchmark/vertica/queries.sql | 86 +- benchmark/{compatible => }/vertica/run.sh | 0 docs/en/operations/performance-test.md | 4 +- docs/zh/operations/performance-test.md | 4 +- 244 files changed, 1034 insertions(+), 89724 deletions(-) rename benchmark/{compatible => }/README.md (100%) rename benchmark/{compatible => }/athena/README.md (100%) rename benchmark/{compatible => }/athena/create_partitioned.sql (100%) rename benchmark/{compatible => }/athena/create_single.sql (100%) rename benchmark/{compatible => }/athena/queries.sql (100%) rename benchmark/{compatible => }/athena/results/partitioned.txt (100%) rename benchmark/{compatible => }/athena/results/single.txt (100%) rename benchmark/{compatible => }/athena/run.sh (100%) rename benchmark/{compatible => }/aurora-mysql/README.md (100%) rename benchmark/{compatible => }/aurora-mysql/create.sql (100%) rename benchmark/{compatible => }/aurora-mysql/queries.sql (100%) rename benchmark/{compatible => }/aurora-mysql/results/16acu.txt (100%) rename benchmark/{compatible => }/aurora-mysql/run.sh (100%) rename benchmark/{compatible => }/aurora-postgresql/README.md (100%) rename benchmark/{compatible => }/aurora-postgresql/create.sql (100%) rename benchmark/{compatible => }/aurora-postgresql/queries.sql (100%) rename benchmark/{compatible => }/aurora-postgresql/results/16acu.txt (100%) rename benchmark/{compatible => }/aurora-postgresql/run.sh (100%) rename benchmark/{compatible => }/bigquery/.gitignore (100%) rename benchmark/{compatible => }/bigquery/README.md (100%) rename benchmark/{compatible => }/bigquery/create.sql (100%) rename benchmark/{compatible => }/bigquery/queries.sql (100%) rename benchmark/{compatible => }/bigquery/run.sh (100%) rename benchmark/{compatible => }/citus/benchmark.sh (100%) rename benchmark/{compatible => }/citus/create.sql (100%) rename benchmark/{compatible => }/citus/queries.sql (100%) rename benchmark/{compatible => }/citus/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/citus/run.sh (100%) rename benchmark/{compatible => }/clickhouse-local/benchmark.sh (100%) rename benchmark/{compatible => }/clickhouse-local/create.sql (100%) rename benchmark/{compatible => }/clickhouse-local/queries.sql (100%) rename benchmark/{compatible => }/clickhouse-local/results/c6a.4xlarge.partitioned.txt (100%) rename benchmark/{compatible => }/clickhouse-local/results/c6a.4xlarge.single.txt (100%) rename benchmark/{compatible => }/clickhouse-local/run.sh (100%) rename benchmark/{compatible => }/clickhouse/benchmark.sh (100%) rename benchmark/{compatible => }/clickhouse/create.sql (100%) rename benchmark/{compatible => }/clickhouse/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/clickhouse/results/c6a.metal.txt (100%) rename benchmark/{compatible => }/clickhouse/run.sh (100%) delete mode 100644 benchmark/compatible/greenplum/benchmark.sh delete mode 100644 benchmark/compatible/heavyai/queries.sql delete mode 100644 benchmark/compatible/mariadb/queries.sql delete mode 100755 benchmark/compatible/monetdb/benchmark.sh delete mode 100644 benchmark/compatible/mysql/queries.sql delete mode 100755 benchmark/compatible/postgresql/benchmark.sh delete mode 100644 benchmark/compatible/postgresql/queries.sql delete mode 100644 benchmark/compatible/singlestore/queries.sql delete mode 100755 benchmark/compatible/timescaledb/benchmark.sh delete mode 100644 benchmark/compatible/timescaledb/queries.sql delete mode 100644 benchmark/compatible/vertica/benchmark.sh delete mode 100644 benchmark/compatible/vertica/queries.sql rename benchmark/{compatible => }/cratedb/benchmark.sh (100%) rename benchmark/{compatible => }/cratedb/create.sql (100%) rename benchmark/{compatible/clickhouse => cratedb}/queries.sql (100%) rename benchmark/{compatible => }/cratedb/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/cratedb/run.sh (100%) create mode 100644 benchmark/databend/README.md rename benchmark/{compatible => }/databend/benchmark.sh (100%) rename benchmark/{compatible => }/databend/create.sql (100%) rename benchmark/{compatible/cratedb => databend}/queries.sql (100%) rename benchmark/{compatible => }/databend/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/databend/run.sh (100%) rename benchmark/{compatible => }/druid/benchmark.sh (100%) rename benchmark/{compatible => }/druid/ingest.json (100%) rename benchmark/{compatible/databend => druid}/queries.sql (100%) rename benchmark/{compatible => }/druid/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/druid/run.sh (100%) rename benchmark/{compatible => }/duckdb/README.md (100%) rename benchmark/{compatible => }/duckdb/benchmark.sh (100%) rename benchmark/{compatible => }/duckdb/create.sql (100%) rename benchmark/{compatible => }/duckdb/load.py (100%) delete mode 100644 benchmark/duckdb/log rename benchmark/{compatible => }/duckdb/query.py (100%) rename benchmark/{compatible => }/duckdb/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/duckdb/run.sh (100%) delete mode 100644 benchmark/duckdb/usability.md create mode 100644 benchmark/elasticsearch/README.md rename benchmark/{compatible => }/elasticsearch/benchmark.sh (100%) rename benchmark/{compatible => }/exasol/README.md (100%) delete mode 100644 benchmark/greenplum/README mode change 100755 => 100644 benchmark/greenplum/benchmark.sh rename benchmark/{compatible => }/greenplum/create.sql (100%) delete mode 100644 benchmark/greenplum/dump_dataset_from_ch.sh delete mode 100644 benchmark/greenplum/load_data_set.sql rename benchmark/{compatible => }/greenplum/log.txt (100%) delete mode 100755 benchmark/greenplum/result_parser.py rename benchmark/{compatible => }/greenplum/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/greenplum/run.sh (100%) delete mode 100644 benchmark/greenplum/schema.sql mode change 100755 => 120000 benchmark/hardware.sh rename benchmark/{clickhouse => hardware}/benchmark-chyt.sh (100%) rename benchmark/{clickhouse => hardware}/benchmark-new.sh (100%) rename benchmark/{clickhouse => hardware}/benchmark-yql.sh (100%) rename benchmark/{clickhouse => hardware}/benchmark_cloud.sh (100%) create mode 100755 benchmark/hardware/hardware.sh create mode 100644 benchmark/hardware/queries.sql rename benchmark/{compatible => }/heavyai/benchmark.sh (100%) rename benchmark/{compatible => }/heavyai/create.sql (100%) rename benchmark/{compatible/druid => heavyai}/queries.sql (100%) rename benchmark/{compatible => }/heavyai/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/heavyai/run.sh (100%) delete mode 100644 benchmark/hive/conf.sh delete mode 100644 benchmark/hive/define_schema.sql delete mode 100644 benchmark/hive/expect.tcl delete mode 100644 benchmark/hive/log/log_100m_tuned delete mode 100644 benchmark/hive/log/log_10m/log_10m_ delete mode 100644 benchmark/hive/log/log_10m/log_10m_1 delete mode 100644 benchmark/hive/log/log_10m/log_10m_2 delete mode 100644 benchmark/hive/log/log_10m/log_10m_3 delete mode 100644 benchmark/hive/log/log_10m/log_10m_tuned delete mode 100644 benchmark/hive/log/log_10m/log_hits_10m delete mode 100644 benchmark/hive/queries.sql delete mode 100755 benchmark/hive/run_hive.sh delete mode 100644 benchmark/infinidb/conf.sh delete mode 100644 benchmark/infinidb/define_schema.sql delete mode 100644 benchmark/infinidb/expect.tcl delete mode 100644 benchmark/infinidb/log/log_100m delete mode 100644 benchmark/infinidb/log/log_100m_tuned delete mode 100644 benchmark/infinidb/log/log_10m delete mode 100644 benchmark/infinidb/log/log_10m_tuned delete mode 100644 benchmark/infinidb/queries.sql rename benchmark/{compatible => }/infobright/benchmark.sh (100%) delete mode 100644 benchmark/infobright/conf.sh rename benchmark/{compatible => }/infobright/create.sql (100%) delete mode 100644 benchmark/infobright/define_schema.sql delete mode 100644 benchmark/infobright/expect.tcl delete mode 100644 benchmark/infobright/log-community/log_10m rename benchmark/{compatible => }/infobright/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/infobright/run.sh (100%) rename benchmark/{compatible => }/locustdb/README.md (100%) rename benchmark/{compatible => }/locustdb/benchmark.sh (100%) rename benchmark/{compatible => }/mariadb-columnstore/README.md (100%) rename benchmark/{compatible => }/mariadb-columnstore/benchmark.sh (100%) rename benchmark/{compatible => }/mariadb-columnstore/create.sql (100%) rename benchmark/{compatible/infobright => mariadb-columnstore}/queries.sql (100%) rename benchmark/{compatible => }/mariadb-columnstore/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/mariadb-columnstore/run.sh (100%) rename benchmark/{compatible => }/mariadb/benchmark.sh (100%) rename benchmark/{compatible => }/mariadb/create.sql (100%) rename benchmark/{compatible/duckdb => mariadb}/queries.sql (100%) rename benchmark/{compatible => }/mariadb/run.sh (100%) delete mode 100644 benchmark/memsql/benchmark.sh delete mode 100644 benchmark/memsql/instructions.txt delete mode 100644 benchmark/memsql/queries.sql delete mode 100644 benchmark/monetdb/aws.log rename benchmark/{compatible => }/monetdb/create.sql (100%) delete mode 100644 benchmark/monetdb/instruction.md delete mode 100644 benchmark/monetdb/log.txt rename benchmark/{compatible => }/monetdb/query.expect (100%) rename benchmark/{compatible => }/monetdb/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/monetdb/run.sh (100%) delete mode 100755 benchmark/monetdb/send-query delete mode 100644 benchmark/monetdb/usability.md rename benchmark/{compatible => }/mysql-myisam/benchmark.sh (100%) rename benchmark/{compatible => }/mysql-myisam/create.sql (100%) rename benchmark/{compatible/mariadb-columnstore => mysql-myisam}/queries.sql (100%) rename benchmark/{compatible => }/mysql-myisam/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/mysql-myisam/run.sh (100%) rename benchmark/{compatible => }/mysql/benchmark.sh (100%) rename benchmark/{compatible => }/mysql/create.sql (100%) rename benchmark/{compatible/mysql-myisam => mysql}/queries.sql (100%) rename benchmark/{compatible => }/mysql/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/mysql/run.sh (100%) delete mode 100755 benchmark/omnisci/benchmark.sh delete mode 100644 benchmark/omnisci/instruction.md delete mode 100644 benchmark/omnisci/log.txt delete mode 100644 benchmark/omnisci/queries.sql rename benchmark/{compatible => }/postgresql/create.sql (100%) delete mode 100644 benchmark/postgresql/instructions.md delete mode 100644 benchmark/postgresql/log rename benchmark/{compatible => }/postgresql/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/postgresql/run.sh (100%) rename benchmark/{compatible => }/questdb/benchmark.sh (100%) rename benchmark/{compatible => }/questdb/create.sql (100%) rename benchmark/{compatible => }/questdb/queries.sql (100%) rename benchmark/{compatible => }/questdb/result/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/questdb/run.sh (100%) rename benchmark/{compatible => }/redshift-serverless/README.md (100%) rename benchmark/{compatible => }/redshift-serverless/create.sql (100%) rename benchmark/{compatible => }/redshift-serverless/queries.sql (100%) rename benchmark/{compatible => }/redshift-serverless/result/serverless.txt (100%) rename benchmark/{compatible => }/redshift-serverless/run.sh (100%) rename benchmark/{compatible => }/redshift/README.md (100%) rename benchmark/{compatible => }/redshift/create.sql (100%) rename benchmark/{compatible => }/redshift/queries.sql (100%) rename benchmark/{compatible => }/redshift/result/4x.ra3.xplus.txt (100%) rename benchmark/{compatible => }/redshift/run.sh (100%) rename benchmark/{compatible => }/singlestore/benchmark.sh (100%) rename benchmark/{compatible => }/singlestore/create.sql (100%) rename benchmark/{compatible/greenplum => singlestore}/queries.sql (100%) rename benchmark/{compatible => }/singlestore/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/singlestore/run.sh (100%) rename benchmark/{compatible => }/snowflake/NOTES.md (100%) rename benchmark/{compatible => }/snowflake/README.md (100%) rename benchmark/{compatible => }/snowflake/create.sql (100%) rename benchmark/{compatible => }/snowflake/queries.sql (100%) rename benchmark/{compatible => }/snowflake/results/2xl.txt (100%) rename benchmark/{compatible => }/snowflake/results/3xl.txt (100%) rename benchmark/{compatible => }/snowflake/results/4xl.txt (100%) rename benchmark/{compatible => }/snowflake/results/l.txt (100%) rename benchmark/{compatible => }/snowflake/results/m.txt (100%) rename benchmark/{compatible => }/snowflake/results/s.txt (100%) rename benchmark/{compatible => }/snowflake/results/xl.txt (100%) rename benchmark/{compatible => }/snowflake/results/xs.txt (100%) rename benchmark/{compatible => }/snowflake/run.sh (100%) rename benchmark/{compatible => }/sqlite/benchmark.sh (100%) rename benchmark/{compatible => }/sqlite/create.sql (100%) rename benchmark/{compatible => }/sqlite/queries.sql (100%) rename benchmark/{compatible => }/sqlite/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/sqlite/run.sh (100%) rename benchmark/{compatible => }/starrocks/README.md (100%) rename benchmark/{compatible => }/starrocks/benchmark.sh (100%) rename benchmark/{compatible => }/timescaledb/create.sql (100%) delete mode 100644 benchmark/timescaledb/log delete mode 100644 benchmark/timescaledb/log_compressed rename benchmark/{compatible => }/timescaledb/results/c6a.4xlarge.compression.txt (100%) rename benchmark/{compatible => }/timescaledb/results/c6a.4xlarge.txt (100%) rename benchmark/{compatible => }/timescaledb/run.sh (100%) delete mode 100644 benchmark/timescaledb/usability.md create mode 100644 benchmark/trino/README.md create mode 100644 benchmark/trino/benchmark.sh create mode 100644 benchmark/trino/create_partitioned.sql create mode 100644 benchmark/trino/create_single.sql rename benchmark/{compatible/monetdb => trino}/queries.sql (78%) rename benchmark/{compatible => }/vertica/.gitignore (100%) delete mode 100644 benchmark/vertica/README rename benchmark/{compatible => }/vertica/README.md (100%) rename benchmark/{compatible => }/vertica/create.sql (100%) delete mode 100644 benchmark/vertica/hits_define_schema.sql rename benchmark/{compatible => }/vertica/run.sh (100%) diff --git a/benchmark/compatible/README.md b/benchmark/README.md similarity index 100% rename from benchmark/compatible/README.md rename to benchmark/README.md diff --git a/benchmark/compatible/athena/README.md b/benchmark/athena/README.md similarity index 100% rename from benchmark/compatible/athena/README.md rename to benchmark/athena/README.md diff --git a/benchmark/compatible/athena/create_partitioned.sql b/benchmark/athena/create_partitioned.sql similarity index 100% rename from benchmark/compatible/athena/create_partitioned.sql rename to benchmark/athena/create_partitioned.sql diff --git a/benchmark/compatible/athena/create_single.sql b/benchmark/athena/create_single.sql similarity index 100% rename from benchmark/compatible/athena/create_single.sql rename to benchmark/athena/create_single.sql diff --git a/benchmark/compatible/athena/queries.sql b/benchmark/athena/queries.sql similarity index 100% rename from benchmark/compatible/athena/queries.sql rename to benchmark/athena/queries.sql diff --git a/benchmark/compatible/athena/results/partitioned.txt b/benchmark/athena/results/partitioned.txt similarity index 100% rename from benchmark/compatible/athena/results/partitioned.txt rename to benchmark/athena/results/partitioned.txt diff --git a/benchmark/compatible/athena/results/single.txt b/benchmark/athena/results/single.txt similarity index 100% rename from benchmark/compatible/athena/results/single.txt rename to benchmark/athena/results/single.txt diff --git a/benchmark/compatible/athena/run.sh b/benchmark/athena/run.sh similarity index 100% rename from benchmark/compatible/athena/run.sh rename to benchmark/athena/run.sh diff --git a/benchmark/compatible/aurora-mysql/README.md b/benchmark/aurora-mysql/README.md similarity index 100% rename from benchmark/compatible/aurora-mysql/README.md rename to benchmark/aurora-mysql/README.md diff --git a/benchmark/compatible/aurora-mysql/create.sql b/benchmark/aurora-mysql/create.sql similarity index 100% rename from benchmark/compatible/aurora-mysql/create.sql rename to benchmark/aurora-mysql/create.sql diff --git a/benchmark/compatible/aurora-mysql/queries.sql b/benchmark/aurora-mysql/queries.sql similarity index 100% rename from benchmark/compatible/aurora-mysql/queries.sql rename to benchmark/aurora-mysql/queries.sql diff --git a/benchmark/compatible/aurora-mysql/results/16acu.txt b/benchmark/aurora-mysql/results/16acu.txt similarity index 100% rename from benchmark/compatible/aurora-mysql/results/16acu.txt rename to benchmark/aurora-mysql/results/16acu.txt diff --git a/benchmark/compatible/aurora-mysql/run.sh b/benchmark/aurora-mysql/run.sh similarity index 100% rename from benchmark/compatible/aurora-mysql/run.sh rename to benchmark/aurora-mysql/run.sh diff --git a/benchmark/compatible/aurora-postgresql/README.md b/benchmark/aurora-postgresql/README.md similarity index 100% rename from benchmark/compatible/aurora-postgresql/README.md rename to benchmark/aurora-postgresql/README.md diff --git a/benchmark/compatible/aurora-postgresql/create.sql b/benchmark/aurora-postgresql/create.sql similarity index 100% rename from benchmark/compatible/aurora-postgresql/create.sql rename to benchmark/aurora-postgresql/create.sql diff --git a/benchmark/compatible/aurora-postgresql/queries.sql b/benchmark/aurora-postgresql/queries.sql similarity index 100% rename from benchmark/compatible/aurora-postgresql/queries.sql rename to benchmark/aurora-postgresql/queries.sql diff --git a/benchmark/compatible/aurora-postgresql/results/16acu.txt b/benchmark/aurora-postgresql/results/16acu.txt similarity index 100% rename from benchmark/compatible/aurora-postgresql/results/16acu.txt rename to benchmark/aurora-postgresql/results/16acu.txt diff --git a/benchmark/compatible/aurora-postgresql/run.sh b/benchmark/aurora-postgresql/run.sh similarity index 100% rename from benchmark/compatible/aurora-postgresql/run.sh rename to benchmark/aurora-postgresql/run.sh diff --git a/benchmark/compatible/bigquery/.gitignore b/benchmark/bigquery/.gitignore similarity index 100% rename from benchmark/compatible/bigquery/.gitignore rename to benchmark/bigquery/.gitignore diff --git a/benchmark/compatible/bigquery/README.md b/benchmark/bigquery/README.md similarity index 100% rename from benchmark/compatible/bigquery/README.md rename to benchmark/bigquery/README.md diff --git a/benchmark/compatible/bigquery/create.sql b/benchmark/bigquery/create.sql similarity index 100% rename from benchmark/compatible/bigquery/create.sql rename to benchmark/bigquery/create.sql diff --git a/benchmark/compatible/bigquery/queries.sql b/benchmark/bigquery/queries.sql similarity index 100% rename from benchmark/compatible/bigquery/queries.sql rename to benchmark/bigquery/queries.sql diff --git a/benchmark/compatible/bigquery/run.sh b/benchmark/bigquery/run.sh similarity index 100% rename from benchmark/compatible/bigquery/run.sh rename to benchmark/bigquery/run.sh diff --git a/benchmark/compatible/citus/benchmark.sh b/benchmark/citus/benchmark.sh similarity index 100% rename from benchmark/compatible/citus/benchmark.sh rename to benchmark/citus/benchmark.sh diff --git a/benchmark/compatible/citus/create.sql b/benchmark/citus/create.sql similarity index 100% rename from benchmark/compatible/citus/create.sql rename to benchmark/citus/create.sql diff --git a/benchmark/compatible/citus/queries.sql b/benchmark/citus/queries.sql similarity index 100% rename from benchmark/compatible/citus/queries.sql rename to benchmark/citus/queries.sql diff --git a/benchmark/compatible/citus/results/c6a.4xlarge.txt b/benchmark/citus/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/citus/results/c6a.4xlarge.txt rename to benchmark/citus/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/citus/run.sh b/benchmark/citus/run.sh similarity index 100% rename from benchmark/compatible/citus/run.sh rename to benchmark/citus/run.sh diff --git a/benchmark/compatible/clickhouse-local/benchmark.sh b/benchmark/clickhouse-local/benchmark.sh similarity index 100% rename from benchmark/compatible/clickhouse-local/benchmark.sh rename to benchmark/clickhouse-local/benchmark.sh diff --git a/benchmark/compatible/clickhouse-local/create.sql b/benchmark/clickhouse-local/create.sql similarity index 100% rename from benchmark/compatible/clickhouse-local/create.sql rename to benchmark/clickhouse-local/create.sql diff --git a/benchmark/compatible/clickhouse-local/queries.sql b/benchmark/clickhouse-local/queries.sql similarity index 100% rename from benchmark/compatible/clickhouse-local/queries.sql rename to benchmark/clickhouse-local/queries.sql diff --git a/benchmark/compatible/clickhouse-local/results/c6a.4xlarge.partitioned.txt b/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.txt similarity index 100% rename from benchmark/compatible/clickhouse-local/results/c6a.4xlarge.partitioned.txt rename to benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.txt diff --git a/benchmark/compatible/clickhouse-local/results/c6a.4xlarge.single.txt b/benchmark/clickhouse-local/results/c6a.4xlarge.single.txt similarity index 100% rename from benchmark/compatible/clickhouse-local/results/c6a.4xlarge.single.txt rename to benchmark/clickhouse-local/results/c6a.4xlarge.single.txt diff --git a/benchmark/compatible/clickhouse-local/run.sh b/benchmark/clickhouse-local/run.sh similarity index 100% rename from benchmark/compatible/clickhouse-local/run.sh rename to benchmark/clickhouse-local/run.sh diff --git a/benchmark/compatible/clickhouse/benchmark.sh b/benchmark/clickhouse/benchmark.sh similarity index 100% rename from benchmark/compatible/clickhouse/benchmark.sh rename to benchmark/clickhouse/benchmark.sh diff --git a/benchmark/compatible/clickhouse/create.sql b/benchmark/clickhouse/create.sql similarity index 100% rename from benchmark/compatible/clickhouse/create.sql rename to benchmark/clickhouse/create.sql diff --git a/benchmark/clickhouse/queries.sql b/benchmark/clickhouse/queries.sql index 89c4616c642..31f65fc898d 100644 --- a/benchmark/clickhouse/queries.sql +++ b/benchmark/clickhouse/queries.sql @@ -1,43 +1,43 @@ -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 uniq(UserID) FROM {table} ; -SELECT uniq(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, uniq(UserID) AS u FROM {table} GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, sum(AdvEngineID), count() AS c, avg(ResolutionWidth), uniq(UserID) FROM {table} GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, uniq(UserID) AS u FROM {table} WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, uniq(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, uniq(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, toMinute(EventTime) AS m, SearchPhrase, count() FROM {table} GROUP BY UserID, m, SearchPhrase ORDER BY count() DESC LIMIT 10; -SELECT UserID FROM {table} WHERE UserID = 12345678901234567890; -SELECT count() FROM {table} WHERE URL LIKE '%metrika%'; -SELECT SearchPhrase, any(URL), count() AS c FROM {table} WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, any(URL), any(Title), count() AS c, uniq(UserID) FROM {table} WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM {table} WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -SELECT 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 c > 100000 ORDER BY l DESC LIMIT 25; -SELECT domainWithoutWWW(Referer) AS key, avg(length(Referer)) AS l, count() AS c, any(Referer) FROM {table} WHERE Referer != '' GROUP BY key HAVING c > 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 AS x, x - 1, x - 2, x - 3, count() AS c FROM {table} GROUP BY x, x - 1, x - 2, x - 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 NOT DontCountHits AND NOT Refresh AND notEmpty(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 NOT DontCountHits AND NOT Refresh AND notEmpty(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 NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -SELECT TraficSourceID, SearchEngineID, AdvEngineID, ((SearchEngineID = 0 AND AdvEngineID = 0) ? Referer : '') AS Src, URL AS Dst, count() AS PageViews FROM {table} 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 {table} 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 WindowClientWidth, WindowClientHeight, count() AS PageViews FROM {table} 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 toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM {table} 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; +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/clickhouse/results/c6a.4xlarge.txt b/benchmark/clickhouse/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/clickhouse/results/c6a.4xlarge.txt rename to benchmark/clickhouse/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/clickhouse/results/c6a.metal.txt b/benchmark/clickhouse/results/c6a.metal.txt similarity index 100% rename from benchmark/compatible/clickhouse/results/c6a.metal.txt rename to benchmark/clickhouse/results/c6a.metal.txt diff --git a/benchmark/compatible/clickhouse/run.sh b/benchmark/clickhouse/run.sh similarity index 100% rename from benchmark/compatible/clickhouse/run.sh rename to benchmark/clickhouse/run.sh diff --git a/benchmark/compatible/greenplum/benchmark.sh b/benchmark/compatible/greenplum/benchmark.sh deleted file mode 100644 index a1802ee223b..00000000000 --- a/benchmark/compatible/greenplum/benchmark.sh +++ /dev/null @@ -1,73 +0,0 @@ -#!/bin/bash - -# NOTE: it requires Ubuntu 18.04 -# Greenplum does not install on any newer system. - -echo "This script must be run from gpadmin user. Press enter to continue." -read -sudo apt update -sudo apt install -y software-properties-common -sudo add-apt-repository ppa:greenplum/db -sudo apt update -sudo apt install greenplum-db-6 -sudo rm -rf /gpmaster /gpdata* -ssh-keygen -t rsa -b 4096 -cat /home/gpadmin/.ssh/id_rsa.pub >> /home/gpadmin/.ssh/authorized_keys -mod 600 ~/.ssh/authorized_keys -sudo echo "# kernel.shmall = _PHYS_PAGES / 2 # See Shared Memory Pages -kernel.shmall = 197951838 -# kernel.shmmax = kernel.shmall * PAGE_SIZE -kernel.shmmax = 810810728448 -kernel.shmmni = 4096 -vm.overcommit_memory = 2 # See Segment Host Memory -vm.overcommit_ratio = 95 # See Segment Host Memory - -net.ipv4.ip_local_port_range = 10000 65535 # See Port Settings -kernel.sem = 500 2048000 200 4096 -kernel.sysrq = 1 -kernel.core_uses_pid = 1 -kernel.msgmnb = 65536 -kernel.msgmax = 65536 -kernel.msgmni = 2048 -net.ipv4.tcp_syncookies = 1 -net.ipv4.conf.default.accept_source_route = 0 -net.ipv4.tcp_max_syn_backlog = 4096 -net.ipv4.conf.all.arp_filter = 1 -net.core.netdev_max_backlog = 10000 -net.core.rmem_max = 2097152 -net.core.wmem_max = 2097152 -vm.swappiness = 10 -vm.zone_reclaim_mode = 0 -vm.dirty_expire_centisecs = 500 -vm.dirty_writeback_centisecs = 100 -vm.dirty_background_ratio = 0 # See System Memory -vm.dirty_ratio = 0 -vm.dirty_background_bytes = 1610612736 -vm.dirty_bytes = 4294967296" |sudo tee -a /etc/sysctl.conf -sudo sysctl -p - -echo "* soft nofile 524288 -* hard nofile 524288 -* soft nproc 131072 -* hard nproc 131072" |sudo tee -a /etc/security/limits.conf -echo "RemoveIPC=no" |sudo tee -a /etc/systemd/logind.conf -echo "Now you need to reboot the machine. Press Enter if you already rebooted, or reboot now and run the script once again" -read -source /opt/greenplum-db-*.0/greenplum_path.sh -cp $GPHOME/docs/cli_help/gpconfigs/gpinitsystem_singlenode . -echo localhost > ./hostlist_singlenode -sed -i "s/MASTER_HOSTNAME=[a-z_]*/MASTER_HOSTNAME=$(hostname)/" gpinitsystem_singlenode -sed -i "s@declare -a DATA_DIRECTORY=(/gpdata1 /gpdata2)@declare -a DATA_DIRECTORY=(/gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14)@" gpinitsystem_singlenode -sudo mkdir /gpmaster /gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14 -sudo chmod 777 /gpmaster /gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14 -gpinitsystem -ac gpinitsystem_singlenode -export MASTER_DATA_DIRECTORY=/gpmaster/gpsne-1/ -#wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -#gzip -d hits.tsv.gz -chmod 777 ~ hits.tsv -psql -d postgres -f create.sql -nohup gpfdist & -time psql -d postgres -t -c '\timing' -c "insert into hits select * from hits_ext;" -du -sh /gpdata* -./run.sh 2>&1 | tee log.txt -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' |awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/heavyai/queries.sql b/benchmark/compatible/heavyai/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/compatible/heavyai/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/mariadb/queries.sql b/benchmark/compatible/mariadb/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/compatible/mariadb/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/monetdb/benchmark.sh b/benchmark/compatible/monetdb/benchmark.sh deleted file mode 100755 index f0afb400802..00000000000 --- a/benchmark/compatible/monetdb/benchmark.sh +++ /dev/null @@ -1,40 +0,0 @@ -#!/bin/bash - -# Install - -echo "deb https://dev.monetdb.org/downloads/deb/ $(lsb_release -cs) monetdb" | sudo tee /etc/apt/sources.list.d/monetdb.list - -sudo wget --output-document=/etc/apt/trusted.gpg.d/monetdb.gpg https://www.monetdb.org/downloads/MonetDB-GPG-KEY.gpg -sudo apt-get update -sudo apt-get install -y monetdb5-sql monetdb-client dos2unix - -sudo systemctl enable monetdbd -sudo systemctl start monetdbd - -sudo monetdbd create /var/lib/monetdb -sudo monetdbd start /var/lib/monetdb -sudo usermod -a -G monetdb $USER - -sudo monetdb create test -sudo monetdb release test - -sudo apt-get install -y expect - -./query.expect "$(cat create.sql)" - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz -chmod 777 ~ hits.tsv - -./query.expect "COPY INTO hits FROM '$(pwd)/hits.tsv' USING DELIMITERS '\t'" - -# 99997497 affected rows -# clk: 15:39 min - -./run.sh 2>&1 | tee log.txt - -sudo du -bcs /var/monetdb5/ - -cat log.txt | dos2unix -f | grep -P 'clk|tuple' | - awk '/tuple/ { ok = 1 } /clk/ { if (ok) { if ($3 == "ms") { print $2 / 1000 } else { print $2 } } else { print "null" }; ok = 0 }' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/mysql/queries.sql b/benchmark/compatible/mysql/queries.sql deleted file mode 100644 index ea2bde47802..00000000000 --- a/benchmark/compatible/mysql/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/postgresql/benchmark.sh b/benchmark/compatible/postgresql/benchmark.sh deleted file mode 100755 index 13459abfe8a..00000000000 --- a/benchmark/compatible/postgresql/benchmark.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/bin/bash - -sudo apt-get update -sudo apt-get install -y postgresql-common -sudo apt-get install -y postgresql-14 - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz -chmod 777 ~ hits.tsv - -sudo -u postgres psql -t -c 'CREATE DATABASE test' -sudo -u postgres psql test -t < create.sql -sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" - -# COPY 99997497 -# Time: 2341543.463 ms (39:01.543) - -./run.sh 2>&1 | tee log.txt - -sudo du -bcs /var/lib/postgresql/14/main/ - -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/postgresql/queries.sql b/benchmark/compatible/postgresql/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/compatible/postgresql/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/singlestore/queries.sql b/benchmark/compatible/singlestore/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/compatible/singlestore/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/timescaledb/benchmark.sh b/benchmark/compatible/timescaledb/benchmark.sh deleted file mode 100755 index 2330deaf1b4..00000000000 --- a/benchmark/compatible/timescaledb/benchmark.sh +++ /dev/null @@ -1,47 +0,0 @@ -#!/bin/bash - -# Install - -sudo apt-get update -sudo apt-get install -y gnupg postgresql-common apt-transport-https lsb-release wget -sudo /usr/share/postgresql-common/pgdg/apt.postgresql.org.sh -sudo bash -c 'echo "deb https://packagecloud.io/timescale/timescaledb/ubuntu/ $(lsb_release -c -s) main" > /etc/apt/sources.list.d/timescaledb.list' -wget --quiet -O - https://packagecloud.io/timescale/timescaledb/gpgkey | sudo apt-key add - -sudo apt-get update -sudo apt-get install -y timescaledb-2-postgresql-14 -sudo bash -c "echo \"shared_preload_libraries = 'timescaledb'\" >> /etc/postgresql/14/main/postgresql.conf" -sudo systemctl restart postgresql - -sudo -u postgres psql -c "CREATE DATABASE test" -sudo -u postgres psql test -c "CREATE EXTENSION IF NOT EXISTS timescaledb" - -# Import the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz -chmod 777 ~ hits.tsv - -sudo -u postgres psql test < create.sql -sudo -u postgres psql test -c "SELECT create_hypertable('hits', 'eventtime')" -sudo -u postgres psql test -c "CREATE INDEX ix_counterid ON hits (counterid)" -sudo -u postgres psql test -c "ALTER TABLE hits SET (timescaledb.compress, timescaledb.compress_orderby = 'counterid, eventdate, userid, eventtime')" -sudo -u postgres psql test -c "SELECT add_compression_policy('hits', INTERVAL '1s')" - -sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" - -# 1619875.288 ms (26:59.875) - -# See https://github.com/timescale/timescaledb/issues/4473#issuecomment-1167095245 -# https://docs.timescale.com/timescaledb/latest/how-to-guides/compression/manually-compress-chunks/#compress-chunks-manually -# Omit this step to proceed without compression. - -time sudo -u postgres psql test -c "SELECT compress_chunk(i, if_not_compressed => true) FROM show_chunks('hits') i" - -# 49m45.120s - -./run.sh 2>&1 | tee log.txt - -sudo du -bcs /var/lib/postgresql/14/main/ - -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/timescaledb/queries.sql b/benchmark/compatible/timescaledb/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/compatible/timescaledb/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/vertica/benchmark.sh b/benchmark/compatible/vertica/benchmark.sh deleted file mode 100644 index 86312a3a438..00000000000 --- a/benchmark/compatible/vertica/benchmark.sh +++ /dev/null @@ -1,26 +0,0 @@ -#!/bin/bash - -sudo apt-get update -sudo apt-get install -y docker.io - -sudo docker run -p 5433:5433 -p 5444:5444 --volume $(pwd):/workdir --mount type=volume,source=vertica-data,target=/data --name vertica_ce vertica/vertica-ce - -sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c "$(cat create.sql)" - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -time sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM LOCAL '/workdir/hits.tsv' DELIMITER E'\\t' NULL E'\\001' DIRECT" - -sudo docker exec vertica_ce du -bcs /data/vertica/VMart - -./run.sh 2>&1 | tee log.txt - -# If you run the script on your own, you may get numbers like this: -# 200m00.000s -# 25000000000 - -# Note: the real numbers cannot be published. - -grep -F 'All rows formatted' logs.txt | sed -r -e 's/^.* ([0-9.]+) ms$/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/vertica/queries.sql b/benchmark/compatible/vertica/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/compatible/vertica/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/cratedb/benchmark.sh b/benchmark/cratedb/benchmark.sh similarity index 100% rename from benchmark/compatible/cratedb/benchmark.sh rename to benchmark/cratedb/benchmark.sh diff --git a/benchmark/compatible/cratedb/create.sql b/benchmark/cratedb/create.sql similarity index 100% rename from benchmark/compatible/cratedb/create.sql rename to benchmark/cratedb/create.sql diff --git a/benchmark/compatible/clickhouse/queries.sql b/benchmark/cratedb/queries.sql similarity index 100% rename from benchmark/compatible/clickhouse/queries.sql rename to benchmark/cratedb/queries.sql diff --git a/benchmark/compatible/cratedb/results/c6a.4xlarge.txt b/benchmark/cratedb/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/cratedb/results/c6a.4xlarge.txt rename to benchmark/cratedb/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/cratedb/run.sh b/benchmark/cratedb/run.sh similarity index 100% rename from benchmark/compatible/cratedb/run.sh rename to benchmark/cratedb/run.sh diff --git a/benchmark/databend/README.md b/benchmark/databend/README.md new file mode 100644 index 00000000000..4e77d54094f --- /dev/null +++ b/benchmark/databend/README.md @@ -0,0 +1 @@ +It is written in Rust and is blazing slow. diff --git a/benchmark/compatible/databend/benchmark.sh b/benchmark/databend/benchmark.sh similarity index 100% rename from benchmark/compatible/databend/benchmark.sh rename to benchmark/databend/benchmark.sh diff --git a/benchmark/compatible/databend/create.sql b/benchmark/databend/create.sql similarity index 100% rename from benchmark/compatible/databend/create.sql rename to benchmark/databend/create.sql diff --git a/benchmark/compatible/cratedb/queries.sql b/benchmark/databend/queries.sql similarity index 100% rename from benchmark/compatible/cratedb/queries.sql rename to benchmark/databend/queries.sql diff --git a/benchmark/compatible/databend/results/c6a.4xlarge.txt b/benchmark/databend/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/databend/results/c6a.4xlarge.txt rename to benchmark/databend/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/databend/run.sh b/benchmark/databend/run.sh similarity index 100% rename from benchmark/compatible/databend/run.sh rename to benchmark/databend/run.sh diff --git a/benchmark/compatible/druid/benchmark.sh b/benchmark/druid/benchmark.sh similarity index 100% rename from benchmark/compatible/druid/benchmark.sh rename to benchmark/druid/benchmark.sh diff --git a/benchmark/compatible/druid/ingest.json b/benchmark/druid/ingest.json similarity index 100% rename from benchmark/compatible/druid/ingest.json rename to benchmark/druid/ingest.json diff --git a/benchmark/compatible/databend/queries.sql b/benchmark/druid/queries.sql similarity index 100% rename from benchmark/compatible/databend/queries.sql rename to benchmark/druid/queries.sql diff --git a/benchmark/compatible/druid/results/c6a.4xlarge.txt b/benchmark/druid/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/druid/results/c6a.4xlarge.txt rename to benchmark/druid/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/druid/run.sh b/benchmark/druid/run.sh similarity index 100% rename from benchmark/compatible/druid/run.sh rename to benchmark/druid/run.sh diff --git a/benchmark/compatible/duckdb/README.md b/benchmark/duckdb/README.md similarity index 100% rename from benchmark/compatible/duckdb/README.md rename to benchmark/duckdb/README.md diff --git a/benchmark/compatible/duckdb/benchmark.sh b/benchmark/duckdb/benchmark.sh similarity index 100% rename from benchmark/compatible/duckdb/benchmark.sh rename to benchmark/duckdb/benchmark.sh diff --git a/benchmark/compatible/duckdb/create.sql b/benchmark/duckdb/create.sql similarity index 100% rename from benchmark/compatible/duckdb/create.sql rename to benchmark/duckdb/create.sql diff --git a/benchmark/compatible/duckdb/load.py b/benchmark/duckdb/load.py similarity index 100% rename from benchmark/compatible/duckdb/load.py rename to benchmark/duckdb/load.py diff --git a/benchmark/duckdb/log b/benchmark/duckdb/log deleted file mode 100644 index 64a6ad6bbeb..00000000000 --- a/benchmark/duckdb/log +++ /dev/null @@ -1,15906 +0,0 @@ -milovidov@mtlog-perftest03j:~$ ./duckdb -v0.3.0 46a0fc50a -Enter ".help" for usage hints. -Connected to a transient in-memory database. -Use ".open FILENAME" to reopen on a persistent database. -D PRAGMA threads = 16; -D PRAGMA temp_directory='duckdb.tmp'; -D .timing on -Error: unknown command or invalid arguments: "timing". Enter ".help" for help -D .timing=on -Error: unknown command or invalid arguments: "timing=on". Enter ".help" for help -D .timer on -D CREATE TABLE hits AS SELECT * FROM parquet_scan('hits.parquet'); -Run Time: real 315.828 user 447.896000 sys 126.552000 -D SELECT count(*) FROM hits; -30), sum(ResolutionWidth + ┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 100000000 │ -└──────────────┘ -Run Time: real 0.189 user 0.172000 sys 0.000000 -D SELECT count(*) FROM hits WHERE AdvEngineID != 0;t -┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 630535 │ -└──────────────┘ -Run Time: real 0.063 user 0.420000 sys 0.008000 -D SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits;( -┌──────────────────┬──────────────┬──────────────────────┐ -│ sum(advengineid) │ count_star() │ avg(resolutionwidth) │ -├──────────────────┼──────────────┼──────────────────────┤ -│ 7280824 │ 100000000 │ 1513.48908394 │ -└──────────────────┴──────────────┴──────────────────────┘ -Run Time: real 0.097 user 0.744000 sys 0.116000 -D SELECT sum(UserID) FROM hits;s -┌────────────────────────┐ -│ sum(userid) │ -├────────────────────────┤ -│ 3.2306058694090605e+26 │ -└────────────────────────┘ -Run Time: real 0.095 user 0.684000 sys 0.012000 -D SELECT COUNT(DISTINCT UserID) FROM hits;+ -┌───────────────┐ -│ count(userid) │ -├───────────────┤ -│ 17630976 │ -└───────────────┘ -Run Time: real 10.279 user 10.520000 sys 2.380000 -D SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -┌─────────────────────┐ -│ count(searchphrase) │ -├─────────────────────┤ -│ 6019589 │ -└─────────────────────┘ -Run Time: real 12.691 user 15.776000 sys 1.444000 -D SELECT min(EventDate), max(EventDate) FROM hits; -┌────────────────┬────────────────┐ -│ min(eventdate) │ max(eventdate) │ -├────────────────┼────────────────┤ -│ 15888 │ 15917 │ -└────────────────┴────────────────┘ -Run Time: real 0.119 user 0.608000 sys 0.020000 -D SELECT AdvEngineID, count(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC;o -┌─────────────┬──────────────┐ -│ AdvEngineID │ count_star() │ -├─────────────┼──────────────┤ -│ 2 │ 404620 │ -│ 27 │ 113167 │ -│ 13 │ 45633 │ -│ 45 │ 38974 │ -│ 44 │ 9731 │ -│ 3 │ 6896 │ -│ 62 │ 5266 │ -│ 52 │ 3554 │ -│ 50 │ 938 │ -│ 28 │ 836 │ -│ 53 │ 350 │ -│ 25 │ 343 │ -│ 61 │ 158 │ -│ 21 │ 38 │ -│ 42 │ 20 │ -│ 16 │ 7 │ -│ 7 │ 3 │ -│ 22 │ 1 │ -└─────────────┴──────────────┘ -Run Time: real 0.050 user 0.608000 sys 0.164000 -D SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -┌──────────┬─────────┐ -│ RegionID │ u │ -├──────────┼─────────┤ -│ 229 │ 2845673 │ -│ 2 │ 1081016 │ -│ 208 │ 831676 │ -│ 169 │ 604583 │ -│ 184 │ 322661 │ -│ 158 │ 307152 │ -│ 34 │ 299479 │ -│ 55 │ 286525 │ -│ 107 │ 272448 │ -│ 42 │ 243181 │ -└──────────┴─────────┘ -Run Time: real 10.522 user 10.800000 sys 2.108000 -D SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10;o -┌──────────┬──────────────────┬──────────┬──────────────────────┬───────────────┐ -│ RegionID │ sum(advengineid) │ c │ avg(resolutionwidth) │ count(userid) │ -├──────────┼──────────────────┼──────────┼──────────────────────┼───────────────┤ -│ 229 │ 2078084 │ 18296430 │ 1506.0876750819696 │ 2845673 │ -│ 2 │ 441711 │ 6687708 │ 1479.8410618406187 │ 1081016 │ -│ 208 │ 285925 │ 4261945 │ 1285.260504769536 │ 831676 │ -│ 169 │ 100887 │ 3320286 │ 1465.90517142198 │ 604583 │ -│ 32 │ 81498 │ 1843721 │ 1538.0370495318978 │ 216010 │ -│ 34 │ 161779 │ 1792406 │ 1548.364990409539 │ 299479 │ -│ 184 │ 55526 │ 1755223 │ 1506.8102679830426 │ 322661 │ -│ 42 │ 108820 │ 1542771 │ 1587.1074287758845 │ 243181 │ -│ 107 │ 120470 │ 1516722 │ 1548.6039623609336 │ 272448 │ -│ 51 │ 98212 │ 1435598 │ 1579.8864215469791 │ 211505 │ -└──────────┴──────────────────┴──────────┴──────────────────────┴───────────────┘ -Run Time: real 12.087 user 13.296000 sys 2.408000 -D SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(MobilePhoneModel) > 0 GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -┌──────────────────┬─────────┐ -│ MobilePhoneModel │ u │ -├──────────────────┼─────────┤ -│ iPad │ 1090347 │ -│ iPhone │ 45758 │ -│ A500 │ 16046 │ -│ N8-00 │ 5565 │ -│ iPho │ 3300 │ -│ ONE TOUCH 6030A │ 2759 │ -│ GT-P7300B │ 1907 │ -│ 3110000 │ 1871 │ -│ GT-I9500 │ 1598 │ -│ eagle75 │ 1492 │ -└──────────────────┴─────────┘ -Run Time: real 2.262 user 5.536000 sys 0.564000 -D SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(MobilePhoneModel) > 0 GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -┌─────────────┬──────────────────┬────────┐ -│ MobilePhone │ MobilePhoneModel │ u │ -├─────────────┼──────────────────┼────────┤ -│ 1 │ iPad │ 931038 │ -│ 5 │ iPad │ 48385 │ -│ 6 │ iPad │ 29710 │ -│ 7 │ iPad │ 28391 │ -│ 118 │ A500 │ 16005 │ -│ 6 │ iPhone │ 14516 │ -│ 26 │ iPhone │ 13566 │ -│ 10 │ iPad │ 11433 │ -│ 32 │ iPad │ 9503 │ -│ 13 │ iPad │ 9417 │ -└─────────────┴──────────────────┴────────┘ -Run Time: real 2.487 user 6.048000 sys 0.412000 -D SELECT SearchPhrase, count(*) AS c FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchPhrase │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 70263 │ -│ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 34675 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 24579 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 21649 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 19703 │ -│ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 19195 │ -│ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 17284 │ -│ galaxy table │ 16746 │ -│ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 16620 │ -│ \xD1\x81\xD0\xBA\xD0\xBE\xD0\xBB\xD1\x8C\xD0\xBA\xD0\xBE \xD0\xBC\xD1\x8B\xD1\x... │ 12317 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 1.772 user 16.624000 sys 1.884000 -D SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;( -┌────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchPhrase │ u │ -├────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 23671 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 19738 │ -│ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 18394 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 17550 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 14599 │ -│ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 14529 │ -│ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 14197 │ -│ \xD1\x81\xD0\xBA\xD0\xBE\xD0\xBB\xD1\x8C\xD0\xBA\xD0\xBE \xD0\xBC\xD1\x8B\xD1\x... │ 9001 │ -│ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 8792 │ -│ \xD0\xBA\xD0\xBE\xD0\xBC\xD0\xB1\xD0\xB8\xD0\xBD\xD0\xB8\xD1\x80\xD0\xBE\xD0\xB... │ 7572 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 20.063 user 17.784000 sys 3.104000 -D SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;( -┌────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchEngineID │ SearchPhrase │ c │ -├────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ 2 │ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 46258 │ -│ 2 │ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 18871 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 16905 │ -│ 3 │ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 16748 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 14911 │ -│ 2 │ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 13716 │ -│ 2 │ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 13414 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 13105 │ -│ 3 │ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 12815 │ -│ 2 │ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 11946 │ -└────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 1.371 user 18.692000 sys 0.164000 -D SELECT UserID, count(*) FROM hits GROUP BY UserID ORDER BY count(*) DESC LIMIT 10;e -┌─────────────────────┬──────────────┐ -│ UserID │ count_star() │ -├─────────────────────┼──────────────┤ -│ 1313338681122956954 │ 29097 │ -│ 1907779576417363396 │ 25333 │ -│ 2305303682471783379 │ 10611 │ -│ 7982623143712728547 │ 7584 │ -│ 6018350421959114808 │ 6678 │ -│ 7280399273658728997 │ 6411 │ -│ 1090981537032625727 │ 6197 │ -│ 5730251990344211405 │ 6019 │ -│ 835157184735512989 │ 5211 │ -│ 770542365400669095 │ 4906 │ -└─────────────────────┴──────────────┘ -Run Time: real 1.016 user 12.592000 sys 0.120000 -D SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10;i -┌─────────────────────┬──────────────┬──────────────┐ -│ UserID │ SearchPhrase │ count_star() │ -├─────────────────────┼──────────────┼──────────────┤ -│ 1313338681122956954 │ │ 29097 │ -│ 1907779576417363396 │ │ 25333 │ -│ 2305303682471783379 │ │ 10611 │ -│ 7982623143712728547 │ │ 6669 │ -│ 7280399273658728997 │ │ 6408 │ -│ 1090981537032625727 │ │ 6196 │ -│ 5730251990344211405 │ │ 6019 │ -│ 6018350421959114808 │ │ 5990 │ -│ 835157184735512989 │ │ 5209 │ -│ 770542365400669095 │ │ 4906 │ -└─────────────────────┴──────────────┴──────────────┘ -Run Time: real 3.555 user 29.324000 sys 3.220000 -D SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -(┌──────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬──────────────┐ -│ UserID │ SearchPhrase │ count_star() │ -├──────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼──────────────┤ -│ 1429668226731527516 │ │ 10 │ -│ 7090682690943997576 │ │ 9 │ -│ 581931296369147468 │ │ 28 │ -│ 1303966184265002832 │ │ 3 │ -│ 3410567899957317285 │ │ 1 │ -│ 1144785580888361492 │ │ 2 │ -│ 8350014103248889436 │ │ 4 │ -│ 2765306422664525607 │ │ 1 │ -│ 1499056669116975597 │ \xD1\x85\xD0\xBE\xD1\x81\xD1\x82\xD1\x8C \xD0\xBE\xD0\xBD\xD0\xBB\xD0\xB0\xD0\x... │ 1 │ -│ 16159576947204565573 │ │ 79 │ -└──────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴──────────────┘ -Run Time: real 2.360 user 30.528000 sys 1.528000 -D SELECT UserID, extract(minute FROM (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) AS m, SearchPhrase, count(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -┌─────────────────────┬────┬──────────────┬──────────────┐ -│ UserID │ m │ SearchPhrase │ count_star() │ -├─────────────────────┼────┼──────────────┼──────────────┤ -│ 1313338681122956954 │ 31 │ │ 589 │ -│ 1313338681122956954 │ 28 │ │ 578 │ -│ 1313338681122956954 │ 29 │ │ 572 │ -│ 1313338681122956954 │ 33 │ │ 567 │ -│ 1313338681122956954 │ 27 │ │ 557 │ -│ 1313338681122956954 │ 32 │ │ 554 │ -│ 1313338681122956954 │ 30 │ │ 552 │ -│ 1313338681122956954 │ 34 │ │ 546 │ -│ 1313338681122956954 │ 26 │ │ 540 │ -│ 1313338681122956954 │ 10 │ │ 539 │ -└─────────────────────┴────┴──────────────┴──────────────┘ -Run Time: real 8.635 user 52.720000 sys 1.648000 -D SELECT UserID FROM hits WHERE UserID = 12345678901234567890; -Run Time: real 0.257 user 0.256000 sys 0.000000 -D SELECT count(*) FROM hits WHERE URL::TEXT LIKE '%metrika%';e -┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 211 │ -└──────────────┘ -Run Time: real 7.228 user 46.668000 sys 0.300000 -D SELECT SearchPhrase, min(URL), count(*) AS c FROM hits WHERE URL::TEXT LIKE '%metrika%' AND octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;u -┌────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───┐ -│ SearchPhrase │ min(url) │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───┤ -│ \xD1\x80\xD0\xB0\xD0\xB1\xD0\xBE\xD0\xB2\xD1\x8B\xD0\xB5 \xD0\xB4\xD0\xBE\xD1\x... │ http://smeshariki.ru/index.ua/auto.ria.ua/change=0&s_yers=0&with_video.yandex.u... │ 1 │ -│ \xD1\x80\xD0\xB5\xD1\x86\xD0\xB5\xD0\xBF\xD1\x82 │ http://smeshariki.ru/index.ua/search/metrikanske-urali-belorussia/28.html?1=1&c... │ 1 │ -│ \xD0\xBE\xD0\xBA\xD0\xB5\xD0\xB0\xD0\xBD\xD1\x81\xD0\xBA\xD0\xBE\xD0\xB5 \xD1\x... │ http://bdsmpeople.ru/index.ru/metrikadeletedAutoSearch │ 1 │ -│ \xD0\xB8\xD1\x89\xD1\x83 \xD1\x80\xD0\xB0\xD0\xB1\xD0\xBE\xD1\x82\xD0\xB0 \xD0\... │ http://smeshariki.ru/index.ua/advanceForce.1primea/forum=1&district=25586/page4... │ 1 │ -│ \xD0\xB4\xD0\xB8\xD0\xBD\xD0\xB0 \xD0\xB2\xD0\xB0\xD0\xBA\xD0\xB0\xD0\xBD\xD1\x... │ http://ecrn.ru/personal/gost277572,9589&pt=b&pd=8&pw=2&page3/?state=0&damages/0... │ 1 │ -│ \xD0\xBC\xD1\x8B\xD1\x81\xD0\xBB \xD0\xBF\xD0\xB5\xD1\x81\xD0\xB5\xD0\xBD \xD1\... │ http://smeshariki.ru/a-folder=cmarka=15&sort=&sll=36872/metrika/frl-2/bage-all/... │ 1 │ -│ \xD0\xB2\xD0\xB8\xD0\xB4\xD0\xB5\xD0\xBE \xD1\x82\xD0\xB0\xD0\xBA\xD0\xBE\xD0\x... │ http://maximum_438424&pvno=2&evlg=VC,0;VL,628;IC,1653-82676212&op_page=0&door=0... │ 1 │ -│ \xD1\x82\xD0\xB5\xD0\xBA\xD1\x81\xD1\x82\xD1\x8B \xD0\xBF\xD0\xBE\xD0\xB4\xD1\x... │ http://bdsmpeople.ru/index.ru/metrikadeletedAutoSearch │ 1 │ -│ \xD0\xBA\xD0\xBE\xD1\x88\xD0\xBA\xD1\x83 \xD1\x81\xD0\xB7\xD0\xB0\xD0\xBE \xD1\... │ http://bdsmpeople.ru/index.ru/metrikadeletedAutoSearch │ 1 │ -│ \xD1\x82\xD1\x80\xD1\x83\xD0\xB4\xD0\xB0 \xD0\xB4\xD0\xBB\xD1\x8F \xD0\xB2\xD1\... │ http://smeshariki.ru/recipes/sadovka.ru/gorod55.ru/search?text=\xD0\xB8\xD0\xB7... │ 1 │ -└────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───┘ -Run Time: real 0.829 user 13.020000 sys 0.000000 -D count(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title::TEXT LIKE '%Яндекс%' AND URL::TEXT NOT LIKE '%.yandex.%' AND octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;( -Run Time: real 7.200 user 22.172000 sys 0.560000 -D SELECT * FROM hits WHERE URL::TEXT LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -┌─────────────────────┬────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┬────────────┬───────────┬───────────┬────────────┬──────────┬─────────────────────┬──────────────┬─────┬───────────┬────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬─────────┬───────────────────┬─────────────────┬───────────────┬─────────────┬─────────────────┬──────────────────┬─────────────────┬────────────┬────────────┬─────────────┬──────────┬──────────┬────────────────┬────────────────┬──────────────┬──────────────────┬──────────┬─────────────┬──────────────────┬────────┬─────────────┬────────────────┬────────────────┬──────────────┬─────────────┬─────────────┬───────────────────┬────────────────────┬────────────────┬─────────────────┬─────────────────────┬─────────────────────┬─────────────────────┬─────────────────────┬──────────────────────┬─────────────┬────────┬────────────┬─────────────┬─────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┬──────────────┬─────────┬─────────────┬───────────────┬──────────┬──────────┬────────────────┬─────┬─────┬────────┬───────────┬───────────┬────────────┬────────────┬────────────┬───────────────┬─────────────────┬────────────────┬───────────────┬──────────────┬───────────┬────────────┬───────────┬───────────────┬─────────────────────┬───────────────────┬─────────────┬───────────────────────┬──────────────────┬────────────┬──────────────┬───────────────┬─────────────────┬─────────────────────┬────────────────────┬──────────────┬──────────────────┬───────────┬───────────┬─────────────┬────────────┬─────────┬─────────┬──────────┬──────────────────────┬──────────────────────┬──────┐ -│ WatchID │ JavaEnable │ Title │ GoodEvent │ EventTime │ EventDate │ CounterID │ ClientIP │ RegionID │ UserID │ CounterClass │ OS │ UserAgent │ URL │ Referer │ Refresh │ RefererCategoryID │ RefererRegionID │ URLCategoryID │ URLRegionID │ ResolutionWidth │ ResolutionHeight │ ResolutionDepth │ FlashMajor │ FlashMinor │ FlashMinor2 │ NetMajor │ NetMinor │ UserAgentMajor │ UserAgentMinor │ CookieEnable │ JavascriptEnable │ IsMobile │ MobilePhone │ MobilePhoneModel │ Params │ IPNetworkID │ TraficSourceID │ SearchEngineID │ SearchPhrase │ AdvEngineID │ IsArtifical │ WindowClientWidth │ WindowClientHeight │ ClientTimeZone │ ClientEventTime │ SilverlightVersion1 │ SilverlightVersion2 │ SilverlightVersion3 │ SilverlightVersion4 │ PageCharset │ CodeVersion │ IsLink │ IsDownload │ IsNotBounce │ FUniqID │ OriginalURL │ HID │ IsOldCounter │ IsEvent │ IsParameter │ DontCountHits │ WithHash │ HitColor │ LocalEventTime │ Age │ Sex │ Income │ Interests │ Robotness │ RemoteIP │ WindowName │ OpenerName │ HistoryLength │ BrowserLanguage │ BrowserCountry │ SocialNetwork │ SocialAction │ HTTPError │ SendTiming │ DNSTiming │ ConnectTiming │ ResponseStartTiming │ ResponseEndTiming │ FetchTiming │ SocialSourceNetworkID │ SocialSourcePage │ ParamPrice │ ParamOrderID │ ParamCurrency │ ParamCurrencyID │ OpenstatServiceName │ OpenstatCampaignID │ OpenstatAdID │ OpenstatSourceID │ UTMSource │ UTMMedium │ UTMCampaign │ UTMContent │ UTMTerm │ FromTag │ HasGCLID │ RefererHash │ URLHash │ CLID │ -├─────────────────────┼────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┼────────────┼───────────┼───────────┼────────────┼──────────┼─────────────────────┼──────────────┼─────┼───────────┼────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼─────────┼───────────────────┼─────────────────┼───────────────┼─────────────┼─────────────────┼──────────────────┼─────────────────┼────────────┼────────────┼─────────────┼──────────┼──────────┼────────────────┼────────────────┼──────────────┼──────────────────┼──────────┼─────────────┼──────────────────┼────────┼─────────────┼────────────────┼────────────────┼──────────────┼─────────────┼─────────────┼───────────────────┼────────────────────┼────────────────┼─────────────────┼─────────────────────┼─────────────────────┼─────────────────────┼─────────────────────┼──────────────────────┼─────────────┼────────┼────────────┼─────────────┼─────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┼──────────────┼─────────┼─────────────┼───────────────┼──────────┼──────────┼────────────────┼─────┼─────┼────────┼───────────┼───────────┼────────────┼────────────┼────────────┼───────────────┼─────────────────┼────────────────┼───────────────┼──────────────┼───────────┼────────────┼───────────┼───────────────┼─────────────────────┼───────────────────┼─────────────┼───────────────────────┼──────────────────┼────────────┼──────────────┼───────────────┼─────────────────┼─────────────────────┼────────────────────┼──────────────┼──────────────────┼───────────┼───────────┼─────────────┼────────────┼─────────┼─────────┼──────────┼──────────────────────┼──────────────────────┼──────┤ -│ 9008228978173248400 │ 1 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372714203 │ 15888 │ 46429 │ 1741285710 │ 208 │ 727970985383478408 │ 0 │ 2 │ 2 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ http://smeshariki.ru/page%3D%26ev_n%3Dtvor_3_4 │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 1638 │ 1658 │ 37 │ 14 │ 0 │ 700.22 │ 0 │ 0 │ 15 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 2349927 │ -1 │ 0 │ │ 0 │ 0 │ 1844 │ 888 │ 135 │ 1372711075 │ 3 │ 0 │ 29241 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 0 │ │ 926364569 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372755313 │ 31 │ 1 │ 0 │ 0 │ 0 │ 1985697352 │ 13001 │ -1 │ 22 │ vG │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 16989158750233735299 │ 2619784076535420345 │ 0 │ -│ 7319686668886932386 │ 0 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372742048 │ 15888 │ 46429 │ 675534320 │ 208 │ 1167829038705361511 │ 0 │ 45 │ 1 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ http://smeshariki.ru/page=6&sqi=2&ved=0CC0QFjAA │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 280 │ 733 │ 0 │ 0 │ 0 │ │ 0 │ 0 │ 4 │ sO │ 1 │ 1 │ 1 │ 0 │ │ │ 1444853 │ -1 │ 0 │ │ 0 │ 0 │ 997 │ 800 │ 623 │ 1372713898 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 4655941889681510909 │ │ 721190565 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372747193 │ 31 │ 2 │ 3 │ 717 │ 0 │ 2935426383 │ 22084 │ -1 │ 1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 11629579472615090596 │ 2619784076535420345 │ 0 │ -│ 5650132582613869356 │ 0 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372742083 │ 15888 │ 46429 │ 675534320 │ 208 │ 1167829038705361511 │ 0 │ 45 │ 1 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ http://smeshariki.ru/page=6&sqi=2&ved=0CC0QFjAA │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 280 │ 733 │ 0 │ 0 │ 0 │ │ 0 │ 0 │ 4 │ D\xE0 │ 1 │ 1 │ 1 │ 0 │ │ │ 1444853 │ -1 │ 0 │ │ 0 │ 0 │ 997 │ 800 │ 623 │ 1372713960 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 4655941889681510909 │ │ 721255326 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372747245 │ 31 │ 2 │ 3 │ 717 │ 0 │ 2935426383 │ 64225 │ -1 │ 1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 11629579472615090596 │ 2619784076535420345 │ 0 │ -│ 6211862319117980382 │ 0 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372744794 │ 15888 │ 46429 │ 1384976586 │ 208 │ 45205883125357727 │ 0 │ 8 │ 1 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ │ 0 │ 0 │ 0 │ 9911 │ 216 │ 475 │ 368 │ 0 │ 0 │ 0 │ │ 0 │ 0 │ 7 │ D\xE0 │ 1 │ 1 │ 1 │ 0 │ │ │ 3195177 │ 0 │ 0 │ │ 0 │ 0 │ 412 │ 286 │ 135 │ 1372732810 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 7015666733105959947 │ │ 641313121 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372774931 │ 28 │ 1 │ 2 │ 1 │ 0 │ 1889444196 │ 44835 │ -1 │ 0 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 18150585289071012696 │ 2619784076535420345 │ 0 │ -│ 8127216893234548048 │ 0 │ \xD0\xA2\xD0\xBE\xD0\xBB\xD1\x81\xD1\x82\xD1\x83\xD1\x85\xD1\x83, \xD0\xB5\xD1\... │ 1 │ 1372752539 │ 15888 │ 46429 │ 1237806922 │ 208 │ 750085446646698731 │ 0 │ 2 │ 3 │ http://smeshariki.ru/used/99114578622-14406072,9269587/roometrika/?page=7&pw=2&... │ http://smeshariki.ru/page=11956bd3f5ba-bolshiient │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 1081 │ 979 │ 23 │ 15 │ 7 │ 700 │ 0 │ 0 │ 17 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 2914423 │ -1 │ 0 │ │ 0 │ 0 │ 1434 │ 739 │ 623 │ 1372781865 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 5496132257355632424 │ │ 983272850 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372760187 │ 50 │ 2 │ 2 │ 60 │ 18 │ 1113465640 │ 60610 │ -1 │ 29 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 194 │ 1631 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 13570011243001147277 │ 11339053197878713733 │ 0 │ -│ 5016976626924998727 │ 1 │ Toyota \xD1\x80\xD1\x83\xD0\xBA\xD0\xB0\xD0\xBB\xD1\x8B \xD0\xB4\xD0\xBB\xD1\x8... │ 1 │ 1372758784 │ 15888 │ 5645 │ 3888153915 │ 107 │ 4058808124307537573 │ 1 │ 2 │ 88 │ http:%2F%2Fwwww.bonprix.ru&pvid=131&op_products/transformality.pulsceness/01-me... │ http://forum.amur.info/node/12451180167540 │ 0 │ 10868 │ 635 │ 16361 │ 943 │ 1638 │ 777 │ 37 │ 15 │ 7 │ 700 │ 0 │ 0 │ 1 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 2726799 │ -1 │ 0 │ │ 0 │ 0 │ 1509 │ 618 │ 135 │ 1372835656 │ 0 │ 0 │ 0 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 6227398498798751865 │ │ 255959698 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372763371 │ 31 │ 1 │ 3 │ 3555 │ 13 │ 4006630121 │ -1 │ -1 │ -1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 10333931216342739575 │ 10009133566342666602 │ 0 │ -│ 4963765960745323978 │ 0 │ \xD0\x93\xD0\x90\xD0\x97 (ZAZ) 5403J \xE2\x80\x93 \xD0\xA4\xD0\xB8\xD0\xBB\xD1\... │ 1 │ 1372793626 │ 15888 │ 46429 │ 2063319617 │ 23967 │ 5283184799411504286 │ 0 │ 44 │ 2 │ http://smeshariki.ru/userId=0&matched_car=Volkswagency=1&p=8090/roometrika/?pag... │ http://smeshariki.ru/smsarhiv/num/33363634383294&lr=66&v6s=2&bodystyle │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 1996 │ 1781 │ 37 │ 15 │ 7 │ 700 │ 0 │ 0 │ 10 │ nA │ 1 │ 1 │ 0 │ 0 │ │ │ 4110783 │ -1 │ 0 │ │ 0 │ 0 │ 1551 │ 955 │ 291 │ 1372730435 │ 7 │ 1 │ 9577 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 8904020920948834668 │ │ 883138215 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372777308 │ 22 │ 2 │ 2 │ 11339 │ 18 │ 2136940575 │ 1653 │ -1 │ 1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 18276268115953212999 │ 9484754903086635093 │ 0 │ -│ 6284148982888572412 │ 1 │ Morskoj port \xE2\x80\x93 \xD0\x9D\xD0\xBE\xD0\xB2\xD0\xBE\xD1\x81\xD1\x82\xD1\... │ 1 │ 1372795222 │ 15889 │ 3922 │ 2460998382 │ 40 │ 1643466856862289966 │ 1 │ 44 │ 7 │ http://moikrug.ru/message-12-kak-pravdorubovnik_metrika-info.php?f=23&prr │ http://diary.ru/GameMain.aspx?d=1412&lr=75&mode=photo/login=igorod.irr.ru/i6102... │ 0 │ 306 │ 29199 │ 92 │ 247 │ 1368 │ 554 │ 37 │ 15 │ 7 │ 800.94 │ 0 │ 0 │ 12 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 1166094 │ 1 │ 0 │ │ 0 │ 0 │ 1253 │ 642 │ 135 │ 1372799147 │ 4 │ 1 │ 16561 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 7061143530822060136 │ │ 699865379 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372810401 │ 31 │ 2 │ 3 │ 6 │ 0 │ 3849445958 │ -1 │ -1 │ -1 │ S0 │ h1 │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 14060834305267311691 │ 1791983733815963315 │ 0 │ -│ 6024648629746505393 │ 0 │ \xD0\x9F\xD0\xBE\xD0\xB3\xD1\x80\xD0\xB5\xD0\xB2\xD0\xB0\xD1\x82\xD0\xB5\xD0\xB... │ 1 │ 1372795271 │ 15889 │ 1200 │ 2932550360 │ 208 │ 1578473929930714515 │ 1 │ 107 │ 82 │ http://afisha.yandex.ru/real-estate/out-of-town/household_app_metrika.ru/wildbe... │ http://ad.adrive_type_id=1959251&stUrl%3Dpopular/kw/306161&lr=1418][to]=&input_... │ 0 │ 15265 │ 19757 │ 8953 │ 32252 │ 958 │ 1871 │ 37 │ 0 │ 0 │ │ 0 │ 0 │ 3 │ D\xE0 │ 1 │ 1 │ 1 │ 6 │ │ │ 4010641 │ -1 │ 0 │ │ 0 │ 0 │ 521 │ 1803 │ -1 │ 1372870507 │ 0 │ 0 │ 0 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 0 │ http://b.kavanga.ru/tags/%FD%ED%FB%E2%F3%E6%E5%E6%ED%E8%ED%F1%FF%ED&ti=%D0%BD%D... │ 120241233 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372842094 │ 0 │ 0 │ 0 │ 0 │ 0 │ 3975911785 │ -1 │ -1 │ -1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 9685011611637290144 │ 5941547189901908071 │ 0 │ -│ 4857420640250996887 │ 1 │ Suzuki Escort 4* (\xD0\xA4\xD1\x80\xD0\xB0\xD0\xBD\xD1\x86\xD1\x8B 3 \xD1\x81\x... │ 1 │ 1372800737 │ 15889 │ 3922 │ 3201307115 │ 190 │ 7335986183190726964 │ 1 │ 44 │ 88 │ http://pova-ul-mir.irr.ru/search.php?gidcar=37408&uuid=1&bc=3&city=0&pv=20&s_ye... │ http://news/238/~6/?cauth=1&av=1&nm=1&ms=1,2/currency=RUR/page_num_id=15366563&... │ 0 │ 306 │ 144 │ 304 │ 132 │ 1368 │ 554 │ 37 │ 15 │ 4 │ 202 │ 0 │ 0 │ 1 │ fi │ 1 │ 1 │ 0 │ 0 │ │ │ 2311071 │ -1 │ 0 │ │ 0 │ 0 │ 1333 │ 924 │ 322 │ 1372840359 │ 4 │ 1 │ 16561 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 7659179697273795837 │ │ 232010762 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372866397 │ 31 │ 2 │ 2 │ 3658 │ 28 │ 2920265313 │ -1 │ -1 │ -1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 8839345929686869081 │ 5449259806403761803 │ 0 │ -└─────────────────────┴────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┴────────────┴───────────┴───────────┴────────────┴──────────┴─────────────────────┴──────────────┴─────┴───────────┴────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴─────────┴───────────────────┴─────────────────┴───────────────┴─────────────┴─────────────────┴──────────────────┴─────────────────┴────────────┴────────────┴─────────────┴──────────┴──────────┴────────────────┴────────────────┴──────────────┴──────────────────┴──────────┴─────────────┴──────────────────┴────────┴─────────────┴────────────────┴────────────────┴──────────────┴─────────────┴─────────────┴───────────────────┴────────────────────┴────────────────┴─────────────────┴─────────────────────┴─────────────────────┴─────────────────────┴─────────────────────┴──────────────────────┴─────────────┴────────┴────────────┴─────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┴──────────────┴─────────┴─────────────┴───────────────┴──────────┴──────────┴────────────────┴─────┴─────┴────────┴───────────┴───────────┴────────────┴────────────┴────────────┴───────────────┴─────────────────┴────────────────┴───────────────┴──────────────┴───────────┴────────────┴───────────┴───────────────┴─────────────────────┴───────────────────┴─────────────┴───────────────────────┴──────────────────┴────────────┴──────────────┴───────────────┴─────────────────┴─────────────────────┴────────────────────┴──────────────┴──────────────────┴───────────┴───────────┴─────────────┴────────────┴─────────┴─────────┴──────────┴──────────────────────┴──────────────────────┴──────┘ -Run Time: real 36.001 user 164.452000 sys 5.964000 -D SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY EventTime LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ \xD1\x81\xD0\xB8\xD0\xBC\xD0\xBF\xD1\x82\xD0\xBE\xD0\xBC\xD1\x8B \xD1\x80\xD0\x... │ -│ galaxy s4 zoom \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC │ -│ \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC \xD0\xBD\xD0\xB5\xD0\xB1\xD0\xBE\xD0\x... │ -│ \xD0\xBD\xD0\xBE\xD1\x87\xD0\xBD\xD0\xBE \xD0\xBA\xD0\xB8\xD1\x82\xD0\xB0\xD1\x... │ -│ \xD0\xB0\xD0\xB2\xD0\xBE\xD0\xBC \xD0\xBA\xD0\xBE\xD0\xBD\xD1\x81\xD1\x82\xD0\x... │ -│ \xD0\xB1\xD1\x80\xD0\xB8\xD1\x82\xD0\xB0 \xD0\xB3\xD0\xB0\xD0\xBD\xD0\xB0\xD0\x... │ -│ \xD1\x81\xD0\xBB\xD0\xBE\xD0\xBD.\xD1\x80\xD1\x83\xD0\xB1., \xD0\xB4. \xD0\xB0.... │ -│ \xD1\x80\xD0\xB0\xD1\x81\xD0\xBF\xD0\xB8\xD1\x81\xD0\xB0\xD0\xBD\xD0\xB8\xD0\xB... │ -│ \xD0\xBA\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ -│ \xD0\xBE\xD1\x82\xD0\xB4\xD1\x8B\xD1\x85\xD0\xB0 \xD1\x87\xD0\xB5\xD0\xBC \xD0\... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 0.255 user 3.820000 sys 0.172000 -D SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY SearchPhrase LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ ! hektdf gjcgjhn conster │ -│ ! \xD1\x81\xD0\xBA\xD0\xB0\xD1\x80\xD0\xBF │ -│ !(\xD0\xBA\xD0\xB0\xD0\xBA \xD0\xB2\xD0\xBE\xD1\x80\xD0\xBE\xD0\xBD\xD0\xB8 │ -│ !(\xD0\xBF\xD0\xBE \xD0\xB3\xD0\xBE\xD1\x80\xD0\xB8\xD1\x8E \xD0\xB2 \xD1\x8F\x... │ -│ !(\xD1\x81) \xD0\xBF\xD1\x80\xD0\xBE \xD0\xB4\xD0\xBF\xD0\xBE \xD1\x81\xD0\xB5\... │ -│ !(\xD1\x81\xD0\xB0\xD0\xBB\xD0\xBE\xD0\xBD\xD1\x8B \xD0\xBE\xD1\x81\xD1\x82\xD0... │ -│ !(\xD1\x81\xD1\x82\xD0\xB0\xD1\x80\xD1\x82\xD0\xB5\xD1\x80 rav4 \xD1\x82\xD1\x8... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD0\xB4\xD0\xBB\xD1\x8F \xD0\xBC\xD1\x8F\xD1... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD0\xBC\xD0\xB0\xD1\x83\xD1\x81 \xD0\xBA\xD0... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD1\x81\xD0\xB5\xD1\x80\xD0\xB8\xD0\xB8 │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 0.251 user 3.676000 sys 0.104000 -D SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY EventTime, SearchPhrase LIMIT 10; -D┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ galaxy s4 zoom \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC │ -│ \xD0\xBD\xD0\xBE\xD1\x87\xD0\xBD\xD0\xBE \xD0\xBA\xD0\xB8\xD1\x82\xD0\xB0\xD1\x... │ -│ \xD1\x81\xD0\xB8\xD0\xBC\xD0\xBF\xD1\x82\xD0\xBE\xD0\xBC\xD1\x8B \xD1\x80\xD0\x... │ -│ \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC \xD0\xBD\xD0\xB5\xD0\xB1\xD0\xBE\xD0\x... │ -│ \xD0\xB0\xD0\xB2\xD0\xBE\xD0\xBC \xD0\xBA\xD0\xBE\xD0\xBD\xD1\x81\xD1\x82\xD0\x... │ -│ \xD0\xB0\xD0\xBD\xD0\xB0\xD0\xBF\xD0\xB0 \xD0\xBE\xD0\xBF\xD0\xB5\xD1\x80\xD0\x... │ -│ \xD0\xB1\xD1\x80\xD0\xB8\xD1\x82\xD0\xB0 \xD0\xB3\xD0\xB0\xD0\xBD\xD0\xB0\xD0\x... │ -│ \xD0\xBA\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ -│ \xD0\xBE\xD1\x82\xD0\xB4\xD1\x8B\xD1\x85\xD0\xB0 \xD1\x87\xD0\xB5\xD0\xBC \xD0\... │ -│ \xD1\x80\xD0\xB0\xD1\x81\xD0\xBF\xD0\xB8\xD1\x81\xD0\xB0\xD0\xBD\xD0\xB8\xD0\xB... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 0.232 user 3.660000 sys 0.000000 -D SELECT CounterID, avg(octet_length(URL)) AS l, count(*) AS c FROM hits WHERE octet_length(URL) > 0 GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;L -┌───────────┬────────────────────┬─────────┐ -│ CounterID │ l │ c │ -├───────────┼────────────────────┼─────────┤ -│ 233773 │ 469.18537326484886 │ 2938865 │ -│ 245438 │ 271.7892512777364 │ 2510103 │ -│ 122612 │ 238.64530987208474 │ 3574007 │ -│ 234004 │ 204.28793262381632 │ 238660 │ -│ 1634 │ 197.83321731651554 │ 323229 │ -│ 786 │ 186.75537634408602 │ 120528 │ -│ 114157 │ 142.91881538575285 │ 216408 │ -│ 515 │ 126.22860040706026 │ 146907 │ -│ 256004 │ 125.37108455074805 │ 858171 │ -│ 95427 │ 120.26856903175477 │ 374306 │ -│ 199550 │ 109.81720498866335 │ 7115413 │ -│ 220992 │ 105.85666196266179 │ 494614 │ -│ 196239 │ 98.34882201749727 │ 163797 │ -│ 62 │ 93.15981711034343 │ 738150 │ -│ 96948 │ 92.74321182146618 │ 396093 │ -│ 188878 │ 91.98308322489247 │ 311998 │ -│ 249603 │ 91.88026594639518 │ 120325 │ -│ 3922 │ 87.83856410684609 │ 8527069 │ -│ 191697 │ 86.95776647628826 │ 124664 │ -│ 97467 │ 84.2953696503987 │ 131178 │ -│ 186300 │ 83.97258027738701 │ 802561 │ -│ 146891 │ 77.77430173504756 │ 605286 │ -│ 38 │ 76.43757015971798 │ 507770 │ -│ 230962 │ 76.3127707226559 │ 169223 │ -│ 77639 │ 75.38681923602442 │ 253961 │ -└───────────┴────────────────────┴─────────┘ -Run Time: real 0.443 user 6.476000 sys 0.504000 -D //(?:www\.)?([^/]+)/.*$', '\1') AS key, avg(octet_length(Referer)) AS l, count(*) AS c, min(Referer) FROM hits WHERE octet_length(Referer) > 0 GROUP BY key HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;E -┌────────────────────────────────────────────────────────────────────────┬────────────────────┬─────────┬────────────────────────────────────────────────────────────────────────────────────┐ -│ key │ l │ c │ min(referer) │ -├────────────────────────────────────────────────────────────────────────┼────────────────────┼─────────┼────────────────────────────────────────────────────────────────────────────────────┤ -│ svpressa.ru │ 307.980979437341 │ 242527 │ http://svpressa.ru/ │ -│ msuzie-showforumdisplay │ 263.327228380409 │ 183676 │ http://msuzie-showforumdisplay/63/~2/?name=&cost_neu%3D400%26retpath=default777... │ -│ saint-peters-total=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 80 │ 242.5236948271821 │ 200529 │ http://saint-peters-total=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 80/c... │ -│ domics │ 212.92990978061542 │ 326094 │ http://domics/825179.11931861234499792 │ -│ e96.ru │ 210.09628206687884 │ 1019276 │ http://e96.ru/%3Ffrom]=&input_act[count_num=0&dff=arian-carrina1201517&cad=rjt&... │ -│ gadgets.irr.ru │ 131.9597008950273 │ 349710 │ https://gadgets.irr.ru/2jmj7l5rSw0yVb │ -│ google.ru │ 109.24485253818524 │ 2158491 │ http://google.ru/ │ -│ go.mail │ 108.63930572737723 │ 8228007 │ http://go.mail/04/detskaia-moda-zhiensmed │ -│ msouz.ru │ 106.10887286512423 │ 301774 │ http://msouz.ru/?ffshop │ -│ state=19945206 │ 105.6469046513171 │ 512414 │ http://state=19945206/foto-4/login%20NoTs3M&where=all&filmId=u8aGGqtWs3M&where=... │ -│ loveplanet.ru │ 104.60136383347789 │ 461200 │ http://loveplanet.ru/%3Faw_opel/page=2013 │ -│ bonprix.ru │ 104.41683309557774 │ 1125105 │ http://bonprix.ru/ │ -│ novjob.ru │ 96.75331644732393 │ 133049 │ http://novjob.ru/ │ -│ cn.ru │ 95.63198716663325 │ 124675 │ http://cn.ru/GameMain.aspx#catalog/100523&tails.xml?market_pc.html?pid=9403&lr=... │ -│ geomethiettai.ru │ 94.78816556817006 │ 115916 │ https://geomethiettai.ru/GameMain.aspx?group=houses/list=266559j7077&num=7&prun... │ -│ kino │ 90.27628829938655 │ 120139 │ http://kino/6/21/2/women.asp?whichpage4/#oversion=unreadm&uid │ -│ yaroslavens.ru │ 90.17077281117085 │ 124610 │ http://yaroslavens.ru/main.aspx#catalog%2F1004-1100000147-otvet/actions/disloca... │ -│ mysw.info │ 89.68684313159915 │ 984596 │ http://mysw.info/ │ -│ m.myloveplanet.ru │ 88.73233749439181 │ 151564 │ http://m.myloveplanet.ru/ │ -│ povarenok.ru │ 83.97395952020882 │ 144813 │ http://povarenok.ru/ │ -│ gorod │ 80.33107253811141 │ 110728 │ http://gorod/%3Fauto.ria.ua%2Fjob │ -│ yandsearch │ 80.21664430621621 │ 245970 │ http://www.yandsearch/rooms=1/page2 │ -│ myloveplanet.ru │ 80.08183067768715 │ 110582 │ http://myloveplanet.ru/#associety/auto │ -│ tambov.irr.ru │ 77.8650188064113 │ 315318 │ http://tambov.irr.ru/0/c1/tgFtaeLDK0yb01A7xvQF08sjCFqQxn51 │ -│ kurortmag.ru │ 75.74958779884584 │ 155264 │ http://kurortmag.ru/ │ -└────────────────────────────────────────────────────────────────────────┴────────────────────┴─────────┴────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 121.613 user 1864.148000 sys 2.320000 -D + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits; -┌──────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┐ -│ 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) │ -├──────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┤ -│ 151348908394 │ 151448908394 │ 151548908394 │ 151648908394 │ 151748908394 │ 151848908394 │ 151948908394 │ 152048908394 │ 152148908394 │ 152248908394 │ 152348908394 │ 152448908394 │ 152548908394 │ 152648908394 │ 152748908394 │ 152848908394 │ 152948908394 │ 153048908394 │ 153148908394 │ 153248908394 │ 153348908394 │ 153448908394 │ 153548908394 │ 153648908394 │ 153748908394 │ 153848908394 │ 153948908394 │ 154048908394 │ 154148908394 │ 154248908394 │ 154348908394 │ 154448908394 │ 154548908394 │ 154648908394 │ 154748908394 │ 154848908394 │ 154948908394 │ 155048908394 │ 155148908394 │ 155248908394 │ 155348908394 │ 155448908394 │ 155548908394 │ 155648908394 │ 155748908394 │ 155848908394 │ 155948908394 │ 156048908394 │ 156148908394 │ 156248908394 │ 156348908394 │ 156448908394 │ 156548908394 │ 156648908394 │ 156748908394 │ 156848908394 │ 156948908394 │ 157048908394 │ 157148908394 │ 157248908394 │ 157348908394 │ 157448908394 │ 157548908394 │ 157648908394 │ 157748908394 │ 157848908394 │ 157948908394 │ 158048908394 │ 158148908394 │ 158248908394 │ 158348908394 │ 158448908394 │ 158548908394 │ 158648908394 │ 158748908394 │ 158848908394 │ 158948908394 │ 159048908394 │ 159148908394 │ 159248908394 │ 159348908394 │ 159448908394 │ 159548908394 │ 159648908394 │ 159748908394 │ 159848908394 │ 159948908394 │ 160048908394 │ 160148908394 │ 160248908394 │ -└──────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┘ -Run Time: real 6.264 user 99.452000 sys 0.008000 -D SELECT SearchEngineID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -┌────────────────┬────────────┬──────┬──────────────┬──────────────────────┐ -│ SearchEngineID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├────────────────┼────────────┼──────┼──────────────┼──────────────────────┤ -│ 2 │ 1138507705 │ 1633 │ 35 │ 1408.0122473974282 │ -│ 2 │ 1740861572 │ 1331 │ 28 │ 1577.945905334335 │ -│ 2 │ 3487820196 │ 1144 │ 35 │ 1553.1984265734266 │ -│ 2 │ 3797060577 │ 1140 │ 36 │ 1543.4140350877192 │ -│ 2 │ 2349209741 │ 1105 │ 30 │ 1557.387330316742 │ -│ 2 │ 2424344199 │ 1102 │ 31 │ 1555.6588021778584 │ -│ 2 │ 3663904793 │ 1083 │ 31 │ 1581.8171745152354 │ -│ 2 │ 3829154130 │ 1082 │ 30 │ 1541.253234750462 │ -│ 2 │ 2551371145 │ 1080 │ 24 │ 1559.8092592592593 │ -│ 2 │ 4029049820 │ 1058 │ 32 │ 1556.2003780718337 │ -└────────────────┴────────────┴──────┴──────────────┴──────────────────────┘ -Run Time: real 0.996 user 13.080000 sys 0.064000 -D SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -┌─────────────────────┬────────────┬───┬──────────────┬──────────────────────┐ -│ WatchID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├─────────────────────┼────────────┼───┼──────────────┼──────────────────────┤ -│ 4623938834438874046 │ 3335652175 │ 2 │ 0 │ 1917.0 │ -│ 8570880866533920844 │ 2776099975 │ 2 │ 0 │ 1917.0 │ -│ 6816565865734300637 │ 3770216628 │ 2 │ 0 │ 2038.0 │ -│ 7411827807718521758 │ 2301434990 │ 2 │ 0 │ 1087.0 │ -│ 8423042978509451644 │ 2960255590 │ 2 │ 0 │ 1368.0 │ -│ 8672760597587433971 │ 1269590216 │ 2 │ 0 │ 1368.0 │ -│ 5851512534509153320 │ 3968091174 │ 2 │ 0 │ 1368.0 │ -│ 5191389486841953200 │ 1487376472 │ 2 │ 0 │ 1828.0 │ -│ 7726072175618541265 │ 1876840662 │ 2 │ 0 │ 1638.0 │ -│ 8425973212606442266 │ 3882647820 │ 2 │ 0 │ 1368.0 │ -└─────────────────────┴────────────┴───┴──────────────┴──────────────────────┘ -Run Time: real 2.661 user 16.820000 sys 0.380000 -D SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -┌─────────────────────┬────────────┬───┬──────────────┬──────────────────────┐ -│ WatchID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├─────────────────────┼────────────┼───┼──────────────┼──────────────────────┤ -│ 9204615339472918567 │ 2852648576 │ 2 │ 0 │ 1638.0 │ -│ 4831526837317665710 │ 4079908211 │ 2 │ 0 │ 1917.0 │ -│ 6588223849545190851 │ 1818568139 │ 2 │ 0 │ 1638.0 │ -│ 7068064253809492202 │ 1818568139 │ 2 │ 0 │ 1638.0 │ -│ 8472345177156920540 │ 1607181793 │ 2 │ 0 │ 1638.0 │ -│ 7154833339725729693 │ 2038096066 │ 2 │ 0 │ 508.0 │ -│ 7985169136791284011 │ 481976292 │ 2 │ 0 │ 1638.0 │ -│ 5261458931164052280 │ 481976292 │ 2 │ 0 │ 1638.0 │ -│ 8672760597587433971 │ 1269590216 │ 2 │ 0 │ 1368.0 │ -│ 4848145794958638974 │ 3938580212 │ 2 │ 0 │ 1638.0 │ -└─────────────────────┴────────────┴───┴──────────────┴──────────────────────┘ -Run Time: real 20.238 user 75.988000 sys 23.744000 -D SELECT URL, count(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬─────────┐ -│ URL │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼─────────┤ -│ http://liver.ru/belgorod/page/1006.j\xD0\xBA\xD0\xB8/\xD0\xB4\xD0\xBE\xD0\xBF_\... │ 3288173 │ -│ http://kinopoisk.ru │ 1625251 │ -│ http://bdsm_po_yers=0&with_video │ 791465 │ -│ http://video.yandex │ 582404 │ -│ http://smeshariki.ru/region │ 514984 │ -│ http://auto_fiat_dlya-bluzki%2F8536.30.18&he=900&with │ 507995 │ -│ http://liver.ru/place_rukodel=365115eb7bbb90 │ 359893 │ -│ http://kinopoisk.ru/vladimir.irr.ru │ 354690 │ -│ http://video.yandex.ru/search/?jenre=50&s_yers │ 318979 │ -│ http://tienskaia-moda │ 289355 │ -└────────────────────────────────────────────────────────────────────────────────────┴─────────┘ -Run Time: real 12.508 user 54.784000 sys 2.344000 -D SELECT 1, URL, count(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -┌───┬────────────────────────────────────────────────────────────────────────────────────┬─────────┐ -│ 1 │ URL │ c │ -├───┼────────────────────────────────────────────────────────────────────────────────────┼─────────┤ -│ 1 │ http://liver.ru/belgorod/page/1006.j\xD0\xBA\xD0\xB8/\xD0\xB4\xD0\xBE\xD0\xBF_\... │ 3288173 │ -│ 1 │ http://kinopoisk.ru │ 1625251 │ -│ 1 │ http://bdsm_po_yers=0&with_video │ 791465 │ -│ 1 │ http://video.yandex │ 582404 │ -│ 1 │ http://smeshariki.ru/region │ 514984 │ -│ 1 │ http://auto_fiat_dlya-bluzki%2F8536.30.18&he=900&with │ 507995 │ -│ 1 │ http://liver.ru/place_rukodel=365115eb7bbb90 │ 359893 │ -│ 1 │ http://kinopoisk.ru/vladimir.irr.ru │ 354690 │ -│ 1 │ http://video.yandex.ru/search/?jenre=50&s_yers │ 318979 │ -│ 1 │ http://tienskaia-moda │ 289355 │ -└───┴────────────────────────────────────────────────────────────────────────────────────┴─────────┘ -Run Time: real 4.310 user 59.804000 sys 0.172000 -D 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; -┌────────────┬──────────────┬──────────────┬──────────────┬───────┐ -│ ClientIP │ clientip - 1 │ clientip - 2 │ clientip - 3 │ c │ -├────────────┼──────────────┼──────────────┼──────────────┼───────┤ -│ 4255045322 │ 4255045321 │ 4255045320 │ 4255045319 │ 47008 │ -│ 2596862839 │ 2596862838 │ 2596862837 │ 2596862836 │ 29121 │ -│ 3119147744 │ 3119147743 │ 3119147742 │ 3119147741 │ 25333 │ -│ 1696638182 │ 1696638181 │ 1696638180 │ 1696638179 │ 20230 │ -│ 1138507705 │ 1138507704 │ 1138507703 │ 1138507702 │ 15778 │ -│ 3367941774 │ 3367941773 │ 3367941772 │ 3367941771 │ 12768 │ -│ 3032827420 │ 3032827419 │ 3032827418 │ 3032827417 │ 11349 │ -│ 1740861572 │ 1740861571 │ 1740861570 │ 1740861569 │ 11315 │ -│ 3487820196 │ 3487820195 │ 3487820194 │ 3487820193 │ 9881 │ -│ 3663904793 │ 3663904792 │ 3663904791 │ 3663904790 │ 9718 │ -└────────────┴──────────────┴──────────────┴──────────────┴───────┘ -Run Time: real 2.423 user 32.668000 sys 0.276000 -D '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(URL) > 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -┌──────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├──────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 102341 │ -│ http://komme%2F27.0.1453.116 │ 51218 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18315 │ -│ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16461 │ -│ http://irr.ru/index.php │ 12577 │ -│ http://irr.ru/index.php?showalbum/login │ 10880 │ -│ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 7627 │ -│ http://irr.ru/index.php?showalbum/login-kupalnik │ 4369 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 4058 │ -│ http://komme%2F27.0.1453.116 Safari │ 3021 │ -└──────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 1.153 user 1.492000 sys 0.064000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(Title) > 0 GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ Title │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ \xD0\xA2\xD0\xB5\xD1\x81\xD1\x82 (\xD0\xA0\xD0\xBE\xD1\x81\xD1\x81\xD0\xB8\xD1\... │ 122407 │ -│ \xD0\xA8\xD0\xB0\xD1\x80\xD0\xB0\xD1\x80\xD0\xB0\xD0\xB9), \xD0\x92\xD1\x8B\xD0... │ 82935 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA - IRR.ru │ 80958 │ -│ \xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\xB8 New Era H (\xD0\x90\xD1\x81\xD1\x83\xD... │ 39098 │ -│ \xD0\xA2\xD0\xB5\xD0\xBF\xD0\xBB\xD0\xBE\xD1\x81\xD0\xBA\xD1\x83 \xD0\xBD\xD0\x... │ 23123 │ -│ Dave and Hotpoint sport \xE2\x80\x93 \xD1\x81\xD0\xB0\xD0\xBC\xD1\x8B\xD0\xB5 \... │ 14329 │ -│ AUTO.ria.ua \xE2\x84\xA2 - \xD0\x90\xD0\xBF\xD0\xBF\xD0\xB5\xD1\x80 │ 14053 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA (\xD0\xA0\xD0\... │ 13912 │ -│ OWAProfessign), \xD0\xBF\xD1\x80\xD0\xBE\xD0\xB4\xD0\xB0\xD1\x82\xD1\x8C │ 10919 │ -│ \xD0\xA2\xD1\x80\xD1\x83\xD1\x81\xD0\xB8 - \xD0\xA8\xD0\xBE\xD1\x83\xD0\xB1\xD0... │ 10157 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 1.461 user 1.320000 sys 0.108000 -D AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 7479 │ -│ http://aliningrad │ 4791 │ -│ http://ekburg.irr.ru%2Fpuloveplanet │ 3584 │ -│ http://smeshariki.ru/obucheyelants │ 3064 │ -│ http://video.yandex.php │ 2887 │ -│ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 1084 │ -│ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 891 │ -│ http://afisha.yandex.ru/index │ 855 │ -│ http://sslow_13507.html?aspx?naId=6HS │ 521 │ -│ http://wildberrior/uphold │ 484 │ -│ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 289 │ -│ http://obninsk/detail │ 241 │ -│ http://diary.ru/forum/intries │ 208 │ -│ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 185 │ -│ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 185 │ -│ http://kurort/SINA, ADRIAN │ 157 │ -│ http://afisha.yandex.ru │ 132 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 124 │ -│ http://ssl.hurra.com/iframe │ 123 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 119 │ -│ http://stars-visa.html_params%3Drhost%3Dad.adriver.ru/catalog.php │ 105 │ -│ http://komme%2F27.0.1453.116 │ 83 │ -│ http://pogoda.yandex │ 80 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 79 │ -│ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 73 │ -│ http://video.yandex │ 71 │ -│ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 68 │ -│ http://wildberries.ru/daily │ 68 │ -│ http://smeshariki.ru/ru/index.ru%26bid │ 68 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 64 │ -│ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 64 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn │ 63 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 59 │ -│ http://sslow_135000008&position=search │ 58 │ -│ http:%2F%2Fwwwwww.bonprix.ru/topic │ 53 │ -│ http://sslow_13507.html/articles │ 52 │ -│ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 51 │ -│ http://rsdn.ru/rss.ya.ru/catalog │ 51 │ -│ http://ekburg.irr.ru/#lingvo │ 46 │ -│ http://pogoda.yandex.ru │ 45 │ -│ http://sslow_13500000%26rnd%3D2788881.html │ 44 │ -│ http://maps#ru_5_ru_22106.377648194,975924][to]=&int[14270pa106&op_uid=17759/6#... │ 42 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 37 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 37 │ -│ http://video.yandex.ru/film/46351/frl-2/bage │ 36 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 35 │ -│ http://direct.yandex │ 34 │ -│ http://sslow_13507.html?aspx?naId=3X_3bhLcs3M │ 33 │ -│ http://gotovim-doma │ 32 │ -│ http://che.ru/produkty_zarubezhei-niepochekhly │ 31 │ -│ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 30 │ -│ http://video.yandex.ru/Newsletter │ 29 │ -│ http://sslow_13507.html?aspx?naId=6D8IzMGys3M │ 29 │ -│ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ http://video.yandex.ru │ 28 │ -│ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 28 │ -│ http://notes=1/currency │ 27 │ -│ http://sslow_13507.html?aspx?naId=649&state/out-of-town=\xD0\xA5\xD0\xB0\xD0\xB... │ 27 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 26 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 26 │ -│ http:%2F%2Fwwww.bonprix.ru/tambov │ 24 │ -│ http://kinopoisk.ru/shoppich.ru/search?clid │ 24 │ -│ http://wildberries │ 23 │ -│ http:%2F%2Fwwwwww.bonprix.ru/searchAutoSearch?text=\xD0\xB2\xD0\xB5\xD0\xBB\xD0... │ 22 │ -│ http://myloveplanet │ 20 │ -│ http://stars-visa-litraj.txt","lpu":"http://pogoda │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/secondary │ 20 │ -│ http://msuzie-shop/premiery-c-38208_2.html │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 20 │ -│ http://smeshariki │ 19 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 19 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ http://irr.ru/6323%26bn%3D27888895,96772,97436 │ 18 │ -│ http://kinopoisk.ru │ 18 │ -│ http://nepogoda.yandex.ru%2Fproducts/search?text=subscripts/busineshop │ 17 │ -│ http://wildberries.ru │ 17 │ -│ http://kinopoisk.ru/catalog/9902224 │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://direct.yandex.html │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 17 │ -│ http://irr.ru/6323%26bn%3D27888895,963095425 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 16 │ -│ http://afisha.yandex.php?gidcar=36281664 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://auto.ria.ua/search │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://direct │ 15 │ -│ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ http://irr.ru/index.php?showalbum/login-kupalnaya-obl │ 15 │ -│ http://kinopoisk.ru/odessya │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 14 │ -│ http://ekburg.irr.irr.ru/maker │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.yandex │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.mail/rnd=0.9788 │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://auto_many_to_auto.ria.ua/igrush43/ │ 13 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 13 │ -│ http://afisha.yandex.ru/cars │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 13 │ -│ http://love.ru/a-myprofi │ 13 │ -│ http: │ 13 │ -│ http:%2F%2Fwww.bonprix │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://real-estate/aparther/offiliates/corruption/russinsk │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://on-online=on&accetti │ 12 │ -│ http://samara.irr.html5/v12/?from]= │ 12 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 12 │ -│ http://en.lyrsenses/zamba_zaborah_coldplay=1&gearbox │ 12 │ -│ http://kinopoisk.ru/saledParams │ 12 │ -│ http://zvukovo/hondar/2007&state/renlew/rigma.ru/scribed │ 12 │ -│ http://pogoda.yandex.php?SECTION │ 12 │ -│ http://msk/platia-nashing/vanny.diary.ru/moscow │ 12 │ -│ http://video.yandex.ru/GameMain.E6smreQhiu_hXR4&where=all&film │ 12 │ -│ http://news/6483731559676/Unlocknotebooks/m83/800_D_Black_list │ 11 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 11 │ -│ http://nizhnieiene/p17378705/currency=1&with_photo-ideapadeno │ 11 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty/photo-12/#imag... │ 11 │ -│ http://msk/platia-nashing/vanny.diary.ru/sale/liver │ 11 │ -│ http://video.yandex.ru/GameMain.aspx#location │ 11 │ -│ http://guid=6&pw=6&pv=13 │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 11 │ -│ http://wildberries.ru/rost.html?1 │ 11 │ -│ http://smeshariki.ru/?win=82&stat=1&page/196264&pt │ 11 │ -│ http://pogoda.yandex.ru/catalog/jokers │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 11 │ -│ http://en.lyrsenses/zamba_zabudtrimazok.html?page=12&prr=http://fap1.adrive_typ... │ 11 │ -│ http://bonprix.ru%26bid │ 11 │ -│ http://loveplanet.ru/GameMain │ 10 │ -│ http://bdsmpeople.ru │ 10 │ -│ http://video.yandex.ru&pvid │ 10 │ -│ http://liver.ru/cheboksicily/foto.aspx?sort=newly&trafkey │ 10 │ -│ http:%2F%2F%2Fwwww.bonprix │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 10 │ -│ http://club.ru/spokoiteli/photo37775280000 │ 10 │ -│ http:%2F%2Fwwww.bonprix │ 10 │ -│ http://auto_id=240&n=13901038 │ 10 │ -│ http://smeshariki.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 10 │ -│ http://slovariant_new3077940810/detail │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://en.lyrsenses/zamba_zabor_id=1012_blank%26site │ 10 │ -│ http://sslow_13500000%26rnd%3D2788881.html?parts/passe │ 10 │ -│ http://pogoda.html%3Fhtml_params%3Drhost%3D43 │ 10 │ -│ http://irr.ru/index.php?showalbum/logout%26verstova.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://pogoda.yandex.php │ 9 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film/6781203.html?id=242037047/detai... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://mysw.info/blog/sankt-peter%3D1216/00001216629 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://e96.ru/albumfotok-15-fotki │ 9 │ -│ http://alpari.yandex.html?html_param=0&users/#page/Search/ab_dob%2Ffieiie-razvo... │ 9 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://bonprix.ru/catalog/8570/1006790 │ 9 │ -│ http://b2b.testered/main/discuss/matched_country=-1&top=0&cityid=1024&wi=1366&o... │ 9 │ -│ http://svpressa.ru/topic=17082630 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://bonprix.ru │ 9 │ -│ http:%2F%2Fwwww.bonprix.ru/filmId=8j5j97LRs3M&where=all&sources │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/#win_13.html_p... │ 9 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/4744089758 │ 9 │ -│ http://nail=Yes&target=search │ 9 │ -│ http://rlsnet.ru/vacancy/view_type_id=9677548268010367 │ 9 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_produkty%2F&sr=http://slovaria │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/imagecache/wm/2013&where=all&filmId │ 8 │ -│ https://m.myloveplanet.ru/forum/abrika-kobelenie_nebestsenal │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 8 │ -│ http://zapchast.com/iframe-owa.html?1=1&cid=577&oki=1&op │ 8 │ -│ http://afisha.yandex.php?t=141880517 │ 8 │ -│ http://poisk.ru/price_ot=&price_ot=&price │ 8 │ -│ http://zarplata.ru/velika_all=\xD0\xBE\xD1\x82 │ 8 │ -│ http:%2F%2Fwwwwww.bonprix.ru/news/222974895&op │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://fore=3&marka=0&top=0 │ 8 │ -│ http://video.yandex.ru&xdm_p=1#item/search │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96777&oki │ 8 │ -│ http://direct.yandex.ru/catalog │ 8 │ -│ http://zarplata.ru/?p=12977-B26358/currency=RUR/page=1080&wi=1024&lo=http://rzh... │ 8 │ -│ http://en.lyrsenses/zamba_zabor/bedroomolsk │ 8 │ -│ http://masterh4.adriver.yandex │ 8 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 8 │ -│ http://mastered/main.aspx#location=1&bc=3&ct=1&pr=60322056107100919/page5/?_h=s... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ==&page_avtomodules.php?f=100&ref... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96779/87 │ 8 │ -│ http://omsk.mlsn.ru │ 8 │ -│ http://che.ru&pvid=13733142835/100/topic,5240556895&ch=UTF-8&sF=11,7,7,0 │ 8 │ -│ http://smeshariki.ru/GameMain │ 8 │ -│ http://wildberries.xml?from]=&input │ 8 │ -│ http://afisha.yandex.php?gidcar=367108851%2Fr%2F1 │ 8 │ -│ http://love.ru/?p=17059 │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://irr.ru/index.php?showalbum/login-kupit-topy%2Fplatjie-gotovlexandex.html... │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 8 │ -│ http://auto.ria.ua/auto_id=24126629/0/index.ru/real-estate/out │ 8 │ -│ http://victor?page_type=city.stol-yar.ru/cars │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://video.yandex.ru%2F&sr=http://loveplants/65398f55 │ 7 │ -│ http://kinopoisk.ru/ch/feed/letniaiaprice/1609 │ 7 │ -│ http://real-estate/aparts/Aquarevski │ 7 │ -│ http://bonprix.ru/social/product_id │ 7 │ -│ http://video.yandex.ru/a-album/login-vitiju/photo │ 7 │ -│ http://moscow/detail/5552/0/2792834&m=111,7,7,5 │ 7 │ -│ http://irr.ru/Registered/main/topnewsru.com/page=6 │ 7 │ -│ http://irr.ru/index.php?showalbum/login-kapustics?sort=pogoda.yandex.ru%26bt%3D... │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 7 │ -│ http://bonprix.ru/catalog/8570/14139489 │ 7 │ -│ http://afisha.yandex.ru/\xD0\xB4\xD0\xBE\xD0\xBC\xD0\xB0/\xD0\x91\xD0\xA1\xD0\x... │ 7 │ -│ http://irr.htm?from]=&int[85][from]=&input_vsegodnyie │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://love.ru/?p=1#country=&op_seo │ 7 │ -│ http://bdsmpeople.ru/niktory/shtory/308/roomed.ru/p59473682740295 │ 7 │ -│ http:%2F%2Fwwwww.bonprix │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 7 │ -│ http://myloveplantrackIt?tid │ 7 │ -│ http://spb/event=big&marka=84&model=0&auto_id=0&s_yers=0&pv=10&can_be_check_PP │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 7 │ -│ http://irr.ru/index.php?showalbum/login-a-5-advert27114 │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://pogoda.yandex.ru&pvid=1 │ 7 │ -│ http://bdsmpeople │ 7 │ -│ http://video=0&is_hot │ 7 │ -│ http://irr.ru/index.php?showalbum/login-zk34/pages/0001216629 │ 7 │ -│ http://svpressa.ru │ 7 │ -│ http://money.yandex │ 7 │ -│ http://gotovim-doma.ru │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 7 │ -│ http://afisha │ 7 │ -│ http://video.yandex.ru%2Fkategory_id │ 6 │ -│ │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://omsk/evential/house.ru/catalog/kitchedule=213-606361653965283 │ 6 │ -│ http://video.yandex.ru/ekt │ 6 │ -│ http://afisha.yandex.ru%2F%2Fmail/169 │ 6 │ -│ http://love.ru/?p=1#country=-1&sq_liver.ru/kyrgyzstan │ 6 │ -│ http://stars-varenok.ru/16745959680706/800_0.jpeg.html%3Fhtml5/v123593 │ 6 │ -│ http://afisha.yandex.ru/catalog=on │ 6 │ -│ http://bdsmpeople.ru/film/64544.690022.rar.html_params%3Drhost%3D_black_list=0&... │ 6 │ -│ http://bdsmpeople.ru/GameMain │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://v102.ru/investate/apartments-sale │ 6 │ -│ http://video.yandex.ru/GameMain.aspx#location/page_type=category │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://direct.yandex.ru/refererprofile%2F2.10 │ 6 │ -│ http://video.yandex.ru/a-topy │ 6 │ -│ http://omsk/evential/housession%3D0 │ 6 │ -│ http://afisha.yandex.ru%26bt%3D43%26anbietersburg │ 6 │ -│ http://wildberries.ru/search │ 6 │ -│ http://moscow/details │ 6 │ -│ http://smeshariki.ru/world/photofider_credit=0&view │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://rukodel=0&sort=newly&trafkey=2750 │ 6 │ -│ http://irr.ru/index.php?showalbum/login-12.html%26custom │ 6 │ -│ http://novosibirsk.irr.ru%26bid │ 6 │ -│ http://solutions.diary.ru/realtitroenie_v_jurman.ru/albums/frame-owa.html?stric... │ 6 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 6 │ -│ http://mr7.ru/newsru.com/iframe_right%3D43 │ 6 │ -│ http://smeshariki.ru/catalog │ 6 │ -│ http://edp2.adriver.ru/hocketshop.ru/moscow/detailanude │ 6 │ -│ http://autodoc.ru/real-estate/apart │ 6 │ -│ http://povari.yandex.ru/greecondary/Products_id=&auto_vaz_2111 │ 6 │ -│ http://msk/platia-nashing/vantralitsa_transion │ 6 │ -│ http://ereal-estate/rent │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://3dnewsru.com/iframe_right.html?1=1&cid=51538 │ 6 │ -│ http://novo/detail.aspx?group_cod_1s │ 6 │ -│ http://video.yandex.ru/catalog │ 6 │ -│ http://afisha.yandex.php/board,39.04839 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://auto_hyundai_sarator │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://kalininmyclonus1 │ 6 │ -│ http://forum/topnews/22294&op_category │ 6 │ -│ http://bdsmpeople.ru/saledParams=rhost%3D43%26bid%3D1 │ 6 │ -│ http://love.ru/?p=17055335 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://afisha.yandex.ru/mymail.php │ 6 │ -│ http://real-estate=week/page=1&expand_search?film/298677435615.html │ 6 │ -│ http://kinel-lab.com/rus/20130709_117485994,93304&op_seo_entry=1&gearbox=0&type... │ 6 │ -│ http://video=0&with_exchangeType │ 6 │ -│ http://afisha.yandex.ru/index.ru/recipe │ 6 │ -│ http://koolinar.ru/port.ru/doc │ 6 │ -│ http://diary.ru/exp?sid=3205 │ 6 │ -│ http://video.yandex.php?search?text=\xD0\x9A\xD0\xBE\xD0\xBD\xD1\x8C\xD1\x8F\xD... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ https://slovakia-600dd903c07022,101595,91194&op_seo_entry │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://video.yandex.php?from]=&interapy-wkti/ &cd │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D8393224 │ 6 │ -│ http://love.ru/ru/irk/event/search/ │ 6 │ -│ http://kinopoisk.ru/catalog │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 5 │ -│ http://barnaul/details/?cauth │ 5 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD0\xB5\xD1\x80&where=... │ 5 │ -│ http://auto_volkswagen/vologdano/il_dlya-dnevka.ru │ 5 │ -│ http://zvukovo-gorodsk │ 5 │ -│ http://metal-lodku-Obzor │ 5 │ -│ http://sp-mamrostova │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://tp66.ru/exp?sid=3860217/rooms=2/men.aspx#location%3D0%26rnd │ 5 │ -│ http://video.yandex.by/?state_id=&auth=1..630;IC,7711588 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://rmnt.ru/search/offilia_Sovetov_living_chamber/?78142 │ 5 │ -│ http://e96.ru/movies/614418821/artir.ua/search │ 5 │ -│ http:%2F%2Fwwww.bonprix.ru/?id=2013&where=all&filmId │ 5 │ -│ http://slovari.yandex.ru%26orderovskij-index.ru │ 5 │ -│ http://tks.ru/cat/publish-cherkalnaya-ttpodporyadushek │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://povari.yandex │ 5 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_seo_entry=&op_category_... │ 5 │ -│ http://video.yandex.ru/GameMain.aspx?Link │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://slovarenok.com │ 5 │ -│ http://auto.ria.ua/auto_id=241269.html?1=1&cid=2127970 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://en.lyrsenses/zamba_zaborah_chamberk │ 5 │ -│ http://radio&planet.ru/work.ru/catalog │ 5 │ -│ http://slovari.yandex │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://afisha.yandex.php?r=23436303135353.html?1 │ 5 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 40007&pt │ 5 │ -│ http://auto_volkswagen-Palities/horobki │ 5 │ -│ http://rustnye-sht-riemnikoi │ 5 │ -│ http://pogoda.yandex.php?gidcar │ 5 │ -│ http://bibidohertki-i-OOOO_REPORT/07_2013 │ 5 │ -│ http://sslow_135000008&position=search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 5 │ -│ http://myloveplanet.ru/v14572&lb_id=13000001216629 │ 5 │ -│ http://topnews.ru/GameMain.aspx?group_cod_1s=1983&pt=b&pd=9&pw=0 │ 5 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://avtomobile/motory.ru/comp.ru/view.php │ 5 │ -│ http://real-estate=week&m=Dvigenie.html?option │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://love.ru/product_id=0&po_yers=0&po_yers=2&refererro/model=1346488078722&c... │ 5 │ -│ http://autodoc.ru/moscow │ 5 │ -│ http://video.yandex.ru/firms.turizm │ 5 │ -│ http://video.yandex.ru%2FkategoriendflowerTo=&powerTo= │ 5 │ -│ http://loveplanet.ru/mymail/rudi │ 5 │ -│ http://auto.ria.ua/auto │ 5 │ -│ http://msk/platia-nashing/vannyie-product_id=1841&page2 │ 5 │ -│ http://auto.ria.ua │ 5 │ -│ http://club.ru/cinema/movies/no-pos │ 5 │ -│ http://tks.ru/cat/publish-chemec.ru/search?filmId=CktclMBmUXI │ 5 │ -│ http://direct.yandex.ru/index │ 5 │ -│ http://nizhnieiewva88/photo/101246465376&cmd=show-to-buchaiev-pugache=51db32a68... │ 5 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D158197%26width │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://kinopoisk.ru/spb.pulscen.ru/exp?sid=3159&op_category_id=&auth=0&checked=... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://holodilnik-rp-ploschaya-obuv/?ci=1280&with_video=0&choosOyg==&op_uid=111... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://wildberries.ru/real │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://slovariant_neu%3D1%26bid%3D1216/0001216629%26bt%3Dad │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://md.mirkovskaya-obl.irr.ru/jobinmoscow │ 5 │ -│ http://wildberrior/kia/ │ 5 │ -│ http://en.lyrsense.com/obshchin-idieiala │ 5 │ -│ http://povari.yandex.php?showalbum/login.pl?cl=all&film/497794,90458 │ 5 │ -│ http://loveplanet.ru/\xD0\xB0\xD1\x80\xD0\xBE\xD0\xBC/curre-ap-i-showalbum/loui... │ 5 │ -│ http://jcmotorom-921205&bt=7 │ 5 │ -│ http://smeshariki.ru/topic │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://moscow/detail/Torgovuyu-organske.ru/real-estate │ 5 │ -│ http:%2F%2Fwww.bonprix.ru │ 5 │ -│ http://gaylyU │ 5 │ -│ http://video.yandex.ru%26bt%3Dad.adriver.ru/recipe/view/10217/?from │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://loveplanet.ru │ 5 │ -│ http://alib.mist.html%26custom%3D%26c2%3D278888592138 │ 5 │ -│ http://alpari.ru/gallery/pic845274 │ 5 │ -│ http://loveplanet │ 4 │ -│ http://video.yandex.ru/price │ 4 │ -│ http://whoyougle.ru/images/images/00000i/specifiers.ru/image=1&furniture │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 4 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.ua │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/filmId │ 4 │ -│ http://loveche.html │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://samarskii_krai/tuapse/detail/result.aspx │ 4 │ -│ http://sslovarenok.ru │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://irr.ru/index.php?showalbum/login-10618968476372773 │ 4 │ -│ http://afisha.yandex.ru/zoom.php?f=5162613838.html_partments │ 4 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 4 │ -│ http://bdsmpeople.ru/Web/Pages=1/feedsmag.ru/~\xD0\xBA\xD0\xBD\xD0\xB8\xD0\xB3\... │ 4 │ -│ http://auto.ria.ua/auto_id=0 │ 4 │ -│ http://wildberries.ru/catalog │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://diary.ru/catalog=on&input_city[3 │ 4 │ -│ http://echoradar-s-Levoshcha │ 4 │ -│ http://radio&planet.ru/marka=62&model=1178128455&pvno=2&evlg=VC,5 │ 4 │ -│ http://direct.yandex.php?formsof(INFLECTION_ID=30861/14365-4b11&state/apartment... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://afisha.yandex.ru%252f7769%252fe │ 4 │ -│ http://e96.ru/news/39733/page2=&input_sponsor=&o=1015219.html_partments-sale&pa... │ 4 │ -│ http://love.ru/?p=1#countpage/vacancies/events/738/0/3/women.aspx │ 4 │ -│ http://travel.ru/state/apartments-sale/rashinitit%2F537 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://rasp.pl?cmd │ 4 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://kinopoisk.ru/saledParams%3Drhost%3Dad.adriver.ru/GameMain.aspx#location │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ https://smeshariki.ru/cinema/article10363136000001216629%26site_offilia_Sovets.... │ 4 │ -│ http://afisha.yandex.php?r=3&bs=&day │ 4 │ -│ http://auto.ria.ua/auto_id=63799.html_params │ 4 │ -│ http://video.yandex.php?view_type=2&driveresult.ru/replies │ 4 │ -│ http://ssl.hurranovskaya-ul-31-foto.ria │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboy07/... │ 4 │ -│ http://card/windows)&bL=ru&cE │ 4 │ -│ http://kniga.ru/view=\xD0\xA1\xD1\x82\xD0\xB0\xD0\xB2\xD1\x80\xD0\xBE\xD0\xB2&w... │ 4 │ -│ http://guid=6&pw=2&pv=0&price_do=¤cy=1 │ 4 │ -│ http://vkirovoe-tourisma │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://smeshariki.ru/domchelkakh_location=search?text=\xD0\xBC\xD0\xB8vents │ 4 │ -│ http://smeshariki.ru/search/keukeru-soft │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://video.yandex.ru/GameMain.XYyZwYXRoPWEtbG9nb24vcG90Cw │ 4 │ -│ http://zarplata.ru/?p=12977-B26358/hasimages=1/page │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://tks.ru/cat/publish-chernyjbelyj-9375966238&op_categoriya │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/real │ 4 │ -│ http://matched_car=373838928155755775482794,9453.116 Safari%2F537.36&he=10&s_ye... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://afisha.yandex.ru/search │ 4 │ -│ http://video.yandex.ru/GameMain.aspx#location-4.0.html │ 4 │ -│ https:%2F%2Fwwww.yandex.ru │ 4 │ -│ http://v102.ru/?s=Adaments-sale │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/social │ 4 │ -│ http://auto.ria.ua/canel_pe_mascona Hilfigeratov/153228 │ 4 │ -│ http://video.yandex.ru%26target=search/ab_area=categorija80119 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://nizhny novgorod │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://video.yandex.ru/realty/leaser_map=1/hasimay-2.html │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://kubikus.ru/search.php?r=4140211,1,7,7,7,7,0 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://e-kuzbass.ru/a-shop │ 4 │ -│ http://guide.travel.ru/link │ 4 │ -│ http://victorhead.php?full&dom=780067167694.0; │ 4 │ -│ http://afisha.mail.aspx#locationalOffers │ 4 │ -│ https://produkty%2Fpulove.ru/gost/?page3 │ 4 │ -│ http://fitness/building │ 4 │ -│ http://svpress_up.aspx#location[propfilter_pf[PODVAL]=&arrFilter][3463351841195... │ 4 │ -│ http:%2F%2Fwwww.bonprix.ru/real-estate/out-of-town/house │ 4 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/cart.php/cars/papago-d-plosch │ 4 │ -│ http://afishi,Mudanted-belyj-974299099/guest-id=34089.html │ 4 │ -│ http://video.yandex.ru/GameMain/dukhovyy │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadbor_sh... │ 4 │ -│ http://tks.ru/cat/public/gamemain.aspx#location │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http%3A//magnitogorod/page3/#over │ 4 │ -│ http://anketka.ru/bridget │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://nepogoda.yandex.ru/search=1&target=search=0&can_be │ 4 │ -│ http://afisha.yandex.php?action/2741920 │ 4 │ -│ http://love.ru/image=2&marka=84&model/mihailovo │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://kinopoisk.ru/searchAutoSearch=0&driver.ru/catalog/1/women.aspx#locationp... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://afisha.yandex.php?show=rlv&ru=1&expand_search │ 4 │ -│ http://love.ru/?p=1705 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://auto_s_product_name=\xD0\x9A\xD1\x80\xD1\x83\xD0\xB3\xD0\xBB\xD0\xBE\xD0... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://rsdn.ru/details_103514,154;IC │ 4 │ -│ http://afisha.yandex.php?p=31&input │ 4 │ -│ http://loveche.ru/job/1162323&PAGEN_1=30&state │ 4 │ -│ http://video.yandex.ru/If yours.avtogsm.ru/animals/calculate │ 4 │ -│ http://ssl.hurra.com │ 4 │ -│ http://auto_volkswagen_cated_cartovskaya-obl.irr.ru/album/login │ 4 │ -│ http://pogoda.yandex.ru/cooking_sm5_1148786993ab-417/photo/69363/26#formi.ru/co... │ 4 │ -│ http://internet Explorer&aV=5.0 (Windows)&bL=en │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboyzon... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/topnews.ru/lanas-advert2713][to]=&int │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http%3A//edp1.adriverys/forum/view_type=city&custom=0&damages/0001216629%26bid%... │ 4 │ -│ http://inspelishchin-platjie-doma.ru/irkutsk.irr.ru │ 4 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD1\x82\xD0\xBE\xD1\x8... │ 4 │ -│ http://auto_volkswagen_pass_1161967 │ 4 │ -│ http://auto_id=0&with_photo │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://en.lyrsenses/zamba_zabudka/photo-1/#page=0&sale/search?text=\xD0\x9C\xD0... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://b.kavanga.ru/?a=inneVolumeFrom │ 4 │ -│ http://afisha.yandex.ru/content2.adriver │ 4 │ -│ http://smeshariki.ru/saint-petersburg-gorod/transfer/?id=7576149959760994861&op... │ 4 │ -│ http://arma/frl-4/transportnoy-kv-m-malchik.ru/show │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://irr.htm?from]=&int[1151;IC,112 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/GameMain │ 4 │ -│ http://afisha.yandex.ru&pvid=13733424 │ 4 │ -│ http://tp66.ru/money.yandex.ru/albums_screenterval │ 4 │ -│ http://bdsmpeople.ru/film/64544.690078 │ 4 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_category_id=592b9e01c48ce9403%26bn%3D0%2... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://banantikov-nadushenie_orleona_server=sc.cheloveplant_11612/page │ 3 │ -│ http://irr.kz/realty/lease/3516093&pvno=2&evlg │ 3 │ -│ https://produkty%2Fplatjie-kuzbass.ru/newsru.com/iframe_right=0&auto_ria=0&meta... │ 3 │ -│ http://fuckfind=rent/view/2021/3 │ 3 │ -│ https://orenburg/?arrFiltersburg │ 3 │ -│ http://kinopoisk.ru/service.ru/iframe │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://radiorecord.ru/login-m3w.html?1=1&cid │ 3 │ -│ http://video.yandex.ru%2Fplata.ru/ch │ 3 │ -│ http://romar/events/audio.ru/widget[]=vacancies/99.php?gidcar │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://afisha.yandex.php/topic104780204&op_uid=1954 │ 3 │ -│ http://afisha.yandex.php/tova.ru/uliya2076789599305953 │ 3 │ -│ http://afisha.yandex.ua/donetsk/urals │ 3 │ -│ http://afisha.yandex.ru/a-folders/misc │ 3 │ -│ http://afisha.yandex.ru/shop.ru/malta │ 3 │ -│ http://afisha.yandex.ru/project_price=&maxprice │ 3 │ -│ http://afisha.yandex.ru/kategoriya%2F5.0 (company │ 3 │ -│ http://afisha.yandex.ru/linkvac.php/board.php?topicseeng │ 3 │ -│ http://samara.irr.ru%2Fproduct │ 3 │ -│ http://radioscannerica/filmId=Ba_id=13733568414&city=\xD0\x9C\xD0\xBE\xD1\x81\x... │ 3 │ -│ http://kaluga/?ext=\xD0\xB3\xD0\xB5\xD1\x80\xD0\xBE\xD1\x8F\xD1\x82\xD0\xBD\xD1... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://sp-money.yandex.ru │ 3 │ -│ http://wildberrior/bedroom]=&int[17][to]=&int[858 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://love.qip.ru │ 3 │ -│ http://sp-money.yandex.ru/work rushki-sien-natalog/8570/page=0&expand │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://ssl.hurral=messages │ 3 │ -│ http://forum/topnews/2229605699574.html?1=1 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://afisha.yandex.ru/forum.materinburg │ 3 │ -│ http://afisha.yandex.php?addriver.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://video.yandex.by/search/?target%3D43%26bid%3D2 │ 3 │ -│ http://video.yandex.ru&pvid=13735/?_h │ 3 │ -│ https://slovakia-600dd903c06c999c226647639.html%3Fhtml │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category_id=0&wi=16000&... │ 3 │ -│ http://love.ru/forum.cofe.ru/forum/view_type=city=790&Selectronics-technik │ 3 │ -│ http://astrobank.ru/image │ 3 │ -│ http://auto_kia_30 │ 3 │ -│ https://slovakia-600dd903c07022,101595,9143531427800648_elit │ 3 │ -│ http://direct&sortdirect.yandex.ru/imagesize%3D0%26ar │ 3 │ -│ http://sp-mamrostokonkursovet │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://rmnt.ru/stars │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://edp2.adriver.ru/catalog/181 │ 3 │ -│ http://bdsmpeople.ru/search │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://b.kavanga.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://video.yandex.ru/circle&state/out-of-town/houses │ 3 │ -│ http://rmnt.ru/film/88677/russia/rio.ru/search?filmId=NNr6aJrm4s3M │ 3 │ -│ http://video.yandex.ua/auto_id │ 3 │ -│ http:%2F%2Fwww.bonprix_ru}%2Fnizhniynovgorod/request-id │ 3 │ -│ http://edp2.adriver.ru/jobinmoscow/detail │ 3 │ -│ http://kazan.irr.ru/location │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://card/windows NT 6.1) AppleWebKit%2F5 │ 3 │ -│ http://tks.ru/cat/publish-chin-play.php?categoriya%2Fzhiensmed │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ https://produkty%2Fpulove.ru/voronezh-sien-zhienskaia-moda-zhienskaia-moda-zhie... │ 3 │ -│ http://svpress_w1t1042796786/6/?category │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com/iframe-owa.html?1=1&cid=577&o... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/forum.donfiscategory │ 3 │ -│ http://msk/planet.ru/mymail.aspx#comme_me_saydinne │ 3 │ -│ http://radiorecord.ru/catalog/idShare │ 3 │ -│ http://pogoda.yandex.kz/family │ 3 │ -│ http://video.yandex.php?topbloveche │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://pogoda │ 3 │ -│ http://nigma.ru/product&op_category_name=\xD0\x91\xD0\xB8\xD0\xBA\xD0\xB8\xD0\x... │ 3 │ -│ http://magnitka_1_series.ru/?favorite_id=636233644&op_category_id=937514 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://ussuriysk.irr.ru/catalog/premiere/628962851d7fd0b6eb17b321d336f5bc7de189... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://smeshariki.ru/catalog/286/women.aspx │ 3 │ -│ http://svpressa.ru/content/search │ 3 │ -│ http://video_dvd/game/iframe-owa.html │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://ekategoriya%2F9B206 Safari │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountpage │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26url%3D//ad.adriver.ru/link/justic/h2.php/... │ 3 │ -│ http://afisha.yandex.ru/tatatit_chto.php?industry │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/92054446660.ht... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty%2Ftanki │ 3 │ -│ http://loveche.ru/volzhskiy │ 3 │ -│ http://svpress/showbiz/photo.htm │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://smeshariki.ru/a-phony │ 3 │ -│ http://msk/platia%2Fzhienskaia │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://povarenok.ru/files/eliteh.ru/perm.irr.ru/board,75.2013-07-09 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://smeshariki.ru/?win=82&stat=141882,373;IC,2552f48 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://e96.ru/real-estate │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://my.kp.ru/albumfoto-1/pol-2 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://amobil-nye-pliazhnaia │ 3 │ -│ http://brand=498&pvno │ 3 │ -│ http://tp66.ru/search/?page=1&fuelRateTo │ 3 │ -│ http://holodilnik.ru/catalog │ 3 │ -│ http://sp-mamrostovestory │ 3 │ -│ http://love.ru/?p=1#country=&op_seo_entry=-1&target │ 3 │ -│ http://lk.wildberries │ 3 │ -│ http://personal/atlants/7292&xdm_c │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberries.ru/item_no=2&evlg=VC,0;VL,205;IC,14;VL,757138/currency=RUR/h... │ 3 │ -│ http://product_brand=RAINBOW&op_cated_content/search/keup/en-ru │ 3 │ -│ http://smeshariki.ru/real │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://msuzie │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostavia.travel.ru/s... │ 3 │ -│ http://love.ru/recipes/indows │ 3 │ -│ http://irr.ru/index.php?showalbum/login-kupaljinik-chere │ 3 │ -│ http://3dnews.ru/?p=12636464/5#f │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic=79799398/?_h=search.htm... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://smeshariki.ru/goodavec/photo/6936325.html?id=223978/page=102 │ 3 │ -│ http://str_ob.html?1=1&choosO8gPJSs3M&where=all&filmId=mAyiC7y6M2mGV2GoA9hFoN3q... │ 3 │ -│ http://irr.ru/washek-s-printom │ 3 │ -│ http://afisha.yandex.php?app=membered │ 3 │ -│ http://afisha.yandex.php?ELEMENT_ID │ 3 │ -│ http://afisha.yandex.ru/lesyach-hotels │ 3 │ -│ http://afisha.yandex.php?id=727285 │ 3 │ -│ http://afisha.yandex.ru%26bt%3D90%26nid%3D1216629 │ 3 │ -│ http://kaluzha-na-NovletedAutoSearch=0 │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory_id=577&search/ab_district/date_id=2271][from... │ 3 │ -│ http://love.ru/ru/irk/event=little&cated_country=-192.html?1=1&cid │ 3 │ -│ http://love.ru/?p=17057 │ 3 │ -│ http://kinopoisk.ru/search │ 3 │ -│ http://state/room=94720-recept-Salat-iz-glasya1lesyat │ 3 │ -│ http://auto_id=0&engineVolumeFrom │ 3 │ -│ http://video_dvd/ratesTypeSearch?text=\xD1\x87\xD0\xB5\xD0\xBB\xD0\xBE\xD0\xB2\... │ 3 │ -│ http://povaria/chak_naytimes.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://smeshariki.ru/GameMain.aspx#location │ 3 │ -│ http://kinopoisk.ru/sessulyanovka.ru/photosessid=3205&bt │ 3 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD0\xB2\... │ 3 │ -│ http://li.ru/filmId=XpzlPj8P8gE&where=all&text=\xD1\x81\xD0\xBA\xD0\xB0\xD1\x87... │ 3 │ -│ http://soft.oszone.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://you_hashkaf.ua/search │ 3 │ -│ http://auto_repairs=0&po_yers=0&price.ru/\xD0\xB6\xD0\xB5\xD0\xBB\xD0\xB5\xD0\x... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://auto_repairs=0&confiscategoriya │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://afisha.yandex.ru%2Fobuv-sapozhkivka=23&price │ 3 │ -│ http://afisha.yandex.ru/hotel-agen-Goluboj-9730 │ 3 │ -│ http://notebooking pressa │ 3 │ -│ http://pogoda.yandex.ru/real-estate │ 3 │ -│ http://wildberries.ru/GameMain │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://afisha.yandex.ru/comment/search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 3 │ -│ http://wildberries.ru/basket&ch │ 3 │ -│ http://video.yandex.ru/personal/offeebe34c7e12944&op_product │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/photo/narod.irr.ru/katering=1&pr=569&s_yers │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://slovari.yandex.ru │ 3 │ -│ http://dom.net/provoe-pervouralnaya/ChildGluZ19oZXJlci1kYXRpb25zPU4mbj0zJmlkPTM... │ 3 │ -│ http://video.yandex.ru/extra │ 3 │ -│ http://afishers/story │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://guid=6&pw=2&pv=0&with_video.yandex.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com.ua/kiev/detail.ru/auth=1..640... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://trashbox.ru/book │ 3 │ -│ http://audio_video.yandex.ru/mosday.html?item=4#photo446962 │ 3 │ -│ http://auto_ford Mix).mp3.ucoz.ru/v1430497.html%3Fhtml │ 3 │ -│ http://v102.ru/investate/apartment/?id=137336IseNhcbx3J85GkHSnzgnsPdZUU&where=a... │ 3 │ -│ http://kommersantamina │ 3 │ -│ https://slovarenok │ 3 │ -│ http://pogoda.yandex.ru/vlas-moskovskaya │ 3 │ -│ http://alpari.yandex.ru/saint-petersburg.irr.ru/cars/page3 │ 3 │ -│ http://love.ru/ru/irk/event=little&category_id=731-643736&mode=1 │ 3 │ -│ http://bdsmpeople.ru/show/39932/Itemid,40.0.html_params%3DfsSaHR0cDovL2pzLnNtaT... │ 3 │ -│ http://radiorecord │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://en.lyrsenses/zamba_zabor_polnitsa-s.narod.irr.ru/bank/otzyvy/12031%2F%23... │ 3 │ -│ http://video.yandex.ru%2Fkategory_id=9759527418 │ 3 │ -│ http://state_shariki │ 3 │ -│ http://smeshariki.ru/cgi-bin/click.cgi%3Fsid%3D0%26pz │ 3 │ -│ http://loveplanet.ru/Bezli-all │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://kinopoisk.ru/spb.pulscen.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://omsk/evential/housession%3D240%26rleurl%3D//ad.adriver.ru/marshavskaya-r... │ 3 │ -│ http://wildberries.ru/cgi-bin/click.cgi%3Fsize │ 3 │ -│ http://afisha.yandex.php?partments │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://video.yandex.ru/real-estate/out-of-town/house.ru&pvid=1&distreet_legkovo... │ 3 │ -│ http://kinopoisk.ru/registernet Explorer&aV=5.0 (Windows NT 5.1; ru-ru&cE=true&... │ 3 │ -│ http://auto_id=0&color=0&confiscategory_id=3205&bt=7&bn=1&bc=3&ct=1&prr=http:%2... │ 3 │ -│ http://video │ 3 │ -│ http://balcon_caddy Club relove │ 3 │ -│ http://wildberring │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http:%2F%2Fwww.bonprix.ru/imagesize │ 3 │ -│ http://video.yandex.ru/realty/search/main.aspx?sort=popular │ 3 │ -│ http://un1.adriver.ru/page=30138117749516%252f110916%252fmedicinema/movie_ross ... │ 3 │ -│ http://afisha.yandex.ru/manga.ru/?rtext=\xD0\xBF\xD1\x80\xD0\xB8\xD0\xB1\xD0\xB... │ 3 │ -│ http://myloveplanet.ru/passenger/kitched_country_id=4312&input │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://cxem.net/324487194836848 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://saint-peter │ 3 │ -│ http://klubnich/zrh/ │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://ssl.hurra.com/iframe-owa │ 3 │ -│ http://wildberries.ru/comme%2F2.12.388 Version/1552/page │ 3 │ -│ http://wildberries.ru/comment/search │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://sunmar.ru/cart&ch=utf-8&sF=11,7,700&aN=Opera&aV=9.80 (Windows │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&film/67... │ 3 │ -│ http://smeshariki.ru/furniture.html5 │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://saint-petersburg-gorod/goodal │ 3 │ -│ http://lazarevskoe │ 3 │ -│ http://whoyougle.com/iframe/iframe_right.ru/spb │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_product_brand=4200&lo=h... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://jobs-education │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://pogoda.yandex.ua/telefon_shtukaturkey │ 3 │ -│ http://irr.ru/index.php?showalbum/logizer8/num-1/refremost │ 3 │ -│ http://video_dvd/suppoll/dleead6718.php?SECTION │ 3 │ -│ http://gotovim-doma.ru/personal/commersant.ru/image=19&pvno=2&engineVolum │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://bdsmpeople.ru/index.by/ru/page=0&confiscategory_id │ 3 │ -│ http://krasnodar.irr.ru/yaransferapid │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://afisha.yandex.php?p=176d43f96ef32d5bc1272 │ 3 │ -│ http://afisha.yandex.ru/sell/resident │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://forum/topics/ingradskazka-lookoformalities/poetry/events/?date&csrf-8200... │ 3 │ -│ http://video=0&input_with_video.yandex.ru/page/10/women.aspx │ 3 │ -│ http://radioscannerica/film/47018.html?1=1&cid=691390&pvno │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://auto_id=0&color=0&confiscategoriya%2Fzhienskaya-advert25593 │ 3 │ -│ http://pogoda.yandex.ru/real-estate/apartments/73151 │ 3 │ -│ http://msk/platia-nashing/vanny.diary.ru/otdam_daily │ 3 │ -│ http://tyva-5/country=-1&washestvo/den_sidentialAmount │ 3 │ -│ http://ssl.hurra.com/iframe/iframe-owa.html%26custom%3D%26custom │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/procoolonelopitered/ministrict/3d-probeg-340-... │ 3 │ -│ http://chehod/zvenia8312&input_who2=1&input_age1=35&aN=Netscape │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=&is... │ 3 │ -│ http://myloveplanet.ru/bad_by_sidential/nizhnieiewva88 │ 3 │ -│ http://direct.yandex.ru%2F&sr │ 3 │ -│ http://kinopoisk.ru/cgi-bin/click.cgi%3Fsid%3D1216 │ 3 │ -│ http://smeshariki.ru/news/2013/peshnye-udivlekanka.ru/l_03_00/bodreamfood.ua │ 3 │ -│ http://kinopoisk.ru/real-estate=2013-07-2089241607/photo=0&with_exchangeType │ 3 │ -│ http://mylove.ru/zoom.php?GID=2&IsOrder │ 3 │ -│ http://rsdn.ru/info_all=yes&razdumy-i-trikshop/search&_h_page/1719920 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://irr.ru/index.php?showalbum/login-do-500-kv-m-Maya │ 3 │ -│ http://rukodel=0&sort=newly&trafkey=27065/2/child.aspx#location │ 3 │ -│ http://arma/frl-4/travel.ru/moscow/details │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/used/KIA-Cee-d... │ 3 │ -│ http://maps#ru_5_ru_1_ru_ru_ru_202_ru_1_ru_ru_ru_ru_2_ru_1743.html?oscsid=36303... │ 3 │ -│ http://zapchastny_fashing_mashes/index.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://video=0&is_hot=0&che_simeis │ 3 │ -│ http://auto_operey-v-v-meha.ru/catalog/public │ 3 │ -│ http://kuharka=48&modeloveplanet │ 3 │ -│ http://whoyougle.ru/basket&ch=utf-8&sF=11,7,7,7,700 │ 3 │ -│ http://myloveplants_list-obl.irr.ru%2Fobuv-zhienskaia-modiezhda │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://zagranimals-planet.ru/user/31059&Module │ 3 │ -│ http://irr.ru/6323%26bn%3D27888895,96772&op_page47 │ 3 │ -│ http://myfashihtzu.html?1=1&cid=65625f313230303&po_yers=2013/07 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://forum/topnews/222968695,910112_1164074834-908745 │ 3 │ -│ http://smeshariki.ru/chipinfo │ 3 │ -│ http://afisha.yandex.ua/index.ru/recipe │ 3 │ -│ http://avtoto.ashx/1001087496197797217530729; Media Centernet_mastersburg │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://votpusk.ru/ │ 3 │ -│ http://video.yandex.ru%2F%2Fwww.bonprix │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://smeshariki.ru/index.ru/main.pl?cmd=show/47555 │ 3 │ -│ http://bdsmpeople.ru/Web/price │ 3 │ -│ http://sendflower │ 3 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 3 │ -│ http://afisha.mail.ru/stars/page │ 3 │ -│ http://smeshariki.ru/Web/price │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://irr.htm?from]=&int[852][to]=10&lastdiscussins/?keyworld │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stories.ru/art/MACKLEMORE │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountry │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://kurort/SP1399&op │ 3 │ -│ http://forum/topnews/2229362067528195&op_categoriya │ 3 │ -│ http://nizhnieie-bielie-bieriends&fb_source-temno │ 3 │ -│ http://afisha.yandex.php?link=114735200&brand=23368 │ 3 │ -│ http://afisha.yandex.php?w=3160/transports │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://pogoda.turizm.ru/communit=0&vip=0&order_by=2 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://wildberries.ru/page=0&vip │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://zarplata.ru/?p=1290&op_product_price=990348531&schoosOSRquM8gE&where=all... │ 3 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&top │ 3 │ -│ http://guid=6&pw=2&pv=0&po_yers=0&with_video │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://my.alpari.ru/filmId=yJg89hqV8gE&where=all&film/530/?fromCityCodeForcentr... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://radiorecord.ru │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&text=\x... │ 3 │ -│ http:%2F%2Fmuzhchine/ru-g-Chelya_v_tsentyabrskii │ 3 │ -│ http://gorbus.aspx#localiformalitic │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://ej.ru/muss/roll_to_audi/mode=replies │ 3 │ -│ http://ur.hh.ru/file/news/2013 \xD0\xB3\xD0\xBE\xD0\xB4\xD0\xB0 │ 3 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 2 │ -│ http://edp2.adrive_type=4754679564&pt=b&pd=9&price │ 2 │ -│ http://pogoda.yandex.ru/jobinmoscow │ 2 │ -│ http://auto.ria.ua/auto_auto.ria │ 2 │ -│ http://afisha.mail.ru/cheva.ru/reportby │ 2 │ -│ http://pogoda.yandex.php?t=14555667679/?PAGEN_3=5&s_yers=0&numphoto/62232/7#f │ 2 │ -│ http://wildberries<{bonprix.ru/forum.materinburg │ 2 │ -│ http://smeshariki.ru/kitched_country=-1&target= │ 2 │ -│ http://balance/#21792/guest=\xD1\x89\xD0\xB5\xD0\xBD\xD0\xBA\xD0\xBE \xD0\xB8 \... │ 2 │ -│ http://yartb.html?city=55&TopicID=2&IsOrderedProduct │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://smeshariki.ru/user_id=1&bodystyle=0&vip=&int[12][from=last_auto_id=13143... │ 2 │ -│ http://video=0&wi=1362653061306535333&site_id=&auth │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 2 │ -│ http://en.lyrsense.com/iframe-owa.html?query=2&hide_black.asp?TOPIC_ID=914958&o... │ 2 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=2furii10.html?city=0&hide │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://ur.hh.ru/page=10&can_be_checked_auto_region=1&rm=1&lang=all&film/18212.1... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://forum.php?t=420 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://mysw.info/node/21544 │ 2 │ -│ http://3dnews.ru/iva-vesta.info.php?link=114737131 │ 2 │ -│ http://guid=6&pw=2&pv=0&price=690&s_yers=1916 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://pogoda.yandex.ru/places/premiery%2Fpage=2&confiscategoriya │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://rlsnet.ru/nogin.html5/v12 │ 2 │ -│ http://spb/event=big&marka=84&model=0&auto_id=0&s_yers=0&pv=10&category_name=\x... │ 2 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD1\x81\... │ 2 │ -│ http://sslow_13500000%26rnd%3D278888 │ 2 │ -│ http://moscow/detail/Torgovljatory/storii_efferanslyatting │ 2 │ -│ http://wildberries.ru/editem_no=100¤cy=1#country=&op_proizvodskaya-obuv-z... │ 2 │ -│ http://wildberries.ru/filmId=4920/room=1&lang=all&filmId=DnGbyVQVUXI&wheretomug... │ 2 │ -│ http://video.yandex.html5/v12 │ 2 │ -│ http://loveplants/740392319 │ 2 │ -│ http://yaroslavl.irr │ 2 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 2 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 2 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 2 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 2 │ -│ http:%2F%2Fwww.bonprix.ru/search=0&deletedAutos&marketing │ 2 │ -│ http://wildberrin/foton │ 2 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 2 │ -│ http://smeshariki.ru/product&op │ 2 │ -│ http://ssl.hurra.com.ua/auto_repairs=0&page=10&category_id=detail │ 2 │ -│ http://poisk.ru/news/articles │ 2 │ -│ http://afisha.yandex.php?id=1620_4_licanel │ 2 │ -│ http://afisha.yandex.php?s=4d450&pid │ 2 │ -│ http://afisha.yandex.ru/?trafkey=54073799 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 2 │ -│ http://video.yandex.ru/rent/info/messa.ru │ 2 │ -│ http://afisha.yandex.php?city=418&view │ 2 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.415 user 0.532000 sys 0.028000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -┌────────────────┬────────────────┬─────────────┬────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ TraficSourceID │ SearchEngineID │ AdvEngineID │ src │ dst │ pageviews │ -├────────────────┼────────────────┼─────────────┼────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33069 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 24703 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 15817 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 14233 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 │ 6549 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 │ 5257 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 5257 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 3547 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login │ 3538 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik │ 3371 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php │ 3333 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 3309 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 2923 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 2616 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 2441 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php │ 2418 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 2390 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari │ 2367 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 2349 │ -│ 1 │ 0 │ 0 │ https://google.com.ua/url?sa=t&rct │ http://komme%2F27.0.1453.116 │ 2228 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 1920 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://komme%2F27.0.1453.116 │ 1679 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ekburg.irr.ru%2Fpuloveplanet │ 1340 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://aliningrad │ 1270 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1254 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1213 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1193 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1186 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1142 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F27.0.1453.116 │ 1007 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 954 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://aliningrad │ 859 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login │ 821 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 791 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 777 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 757 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 722 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://video.yandex.php │ 702 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 691 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/obucheyelants │ 684 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 632 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 593 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 577 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://smeshariki.ru/obucheyelants │ 566 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.php │ 452 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 445 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBA\xD0\xBE\xD0\xBD\xD1\x82\xD1\x80\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 422 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 421 │ -│ 0 │ 0 │ 0 │ │ http://obninsk/detail │ 407 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 386 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug.ru │ http://irr.ru/index.php │ 374 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru/index │ 360 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 348 │ -│ 2 │ 0 │ 13 │ │ http://komme%2F27.0.1453.116 │ 347 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-2011/43597 │ 293 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 293 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6HS │ 282 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 268 │ -│ 3 │ 72 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 267 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 259 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 257 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari │ 256 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 │ 241 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login │ 237 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 │ 236 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/28435&lr=157 │ http://komme%2F27.0.1453.116 │ 233 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/ru/photo/6936313555&text=\xD1\x85\xD0\xBA \xD0\xBB\xD0\xBE\xD... │ http://komme%2F27.0.1453.116 │ 224 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php │ 223 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cat/dushkirillovyj │ http://komme%2F27.0.1453.116 │ 222 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=359&op_page2/... │ 218 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kriminally-bezhevsk │ 217 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0 │ 212 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 207 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 202 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 199 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://ekburg.irr.ru%2Fpuloveplanet │ 199 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 198 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 197 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 195 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/414526863.xlsx │ 191 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CDMQFjAJ │ http://irr.ru/index.php?showalbum/login-kiriskaya-obl.irr.ru/index.ru/GameMain.... │ 190 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=0&with_photo.ashx/101/4/?cat=6257271 │ 188 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=partments-sale/search?text=\xD1\x81\xD0... │ 187 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://afisha.yandex.ru/index │ 183 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://ekburg.irr.ru%2Fpuloveplanet │ 171 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 168 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 164 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 163 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://ekburg.irr.ru%2Fpuloveplanet │ 160 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 160 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/life.ru/cars/misc/travel.ru/?trafkey=058143&p... │ 155 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 │ 154 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=577&oki=1&oby=&op_s... │ 152 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://wildberrior/uphold │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6HS │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 149 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 146 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustic/meterburg │ 144 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 144 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 144 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E │ 141 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 140 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 136 │ -│ 1 │ 0 │ 0 │ http://kombardighantnie │ http://komme%2F27.0.1453.116 │ 136 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login │ 135 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kysjacevtika │ 132 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://komme%2F27.0.1453.116 Safari │ 124 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 121 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category/stroy/dachines... │ 114 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/?state │ http://komme%2F27.0.1453.116 │ 113 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/GameMain │ http://komme%2F27.0.1453.116 │ 113 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 111 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login │ 110 │ -│ 3 │ 85 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 110 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kardigan │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 107 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosoftwarenok.ru/projects/zhbi.po... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 105 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268143.html?1=1&cid=577&o... │ 104 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 104 │ -│ 0 │ 0 │ 0 │ │ http://tvidi.ru/photo=0&confiscategory_id=0&engineVolumeFrom=&fuelRateFrom=type... │ 103 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 102 │ -│ 3 │ 14 │ 0 │ │ http://komme%2F27.0.1453.116 │ 102 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 100 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 98 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://video.yandex.php │ 97 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 96 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.21150895 │ 96 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?sort=price │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 95 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login │ 95 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.12.388 │ 93 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 93 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 92 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberrior/uphold │ 92 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 91 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-tanks/search=1&ady=62&modeloveplanet.ru │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 90 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.145332.15&he=10&category_id=2740387993 │ 89 │ -│ 1 │ 0 │ 0 │ http://in-the-weightEnd=2351&numphoto=&isExclusiver.ru/alertljus │ http://komme%2F27.0.1453.116 │ 88 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fprodazha_Italja_unit=1&av=1&nm=1&lang=ru │ http://irr.ru/index.ru/show/414526863.xlsx │ 87 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 86 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb1.html?item_no=386703/?bundle=7172&msid=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 86 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 │ 85 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 84 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733582852/ │ 84 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupit/action │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 82 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 82 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?bodystyle │ http://komme%2F27.0.1453.116 │ 81 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F22.14&he=768486 │ 81 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusk.ru/ru/lookaginitial/Prodayu-Dach... │ 80 │ -│ 3 │ 4 │ 0 │ │ http://komme%2F27.0.1453.116 │ 80 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 79 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://aliningrad │ 78 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 78 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login │ 76 │ -│ 1 │ 0 │ 0 │ https://go.1ps.ru/show&showforum │ http://komme%2F27.0.1453.116 │ 76 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandex.ru │ http://komme%2F27.0.1453.116 │ 76 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26221/detail.ru/v1496366&... │ 75 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php │ 75 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 74 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%2F&sr=http://bonprix.ru... │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len80/page/product │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filtr/all/perm.pulscen... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 73 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 73 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F1.7.1364.172 │ 72 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://diary.ru/forum/intries │ 72 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/show/414526863_112 │ 72 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2618561&pp=1059&op_produc... │ 72 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://video.yandex.php │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 71 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 71 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari │ 71 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://aliningrad │ 71 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=product_id=42&Selection.chelov.ru/searc... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/yandsearch?te... │ http://irr.ru/index.php │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/server=sc.chel.ru/main... │ http://irr.ru/index.php │ 70 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification/?year=\xD0\xB1\... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2791954~43.87725656132&op... │ 69 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://komme%2F27.0.1453.116 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/produkty/bleacs/udilis... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 69 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&wi=1366&br... │ 68 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 67 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731&CgID=124jc&where=all... │ 66 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/Yozh/Goodda │ http://komme%2F27.0.1453.116 │ 66 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2725215195&ti=\xD0\x9F\xD... │ 66 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://ekburg.irr.ru%2Fpuloveplanet │ 65 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru/index │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 64 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://afisha.yandex.ru/index │ 64 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 63 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nmg.com%2F4.0 Safari%26clients-sale/search&e... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 63 │ -│ 2 │ 13 │ 13 │ │ http://komme%2F27.0.1453.116 │ 62 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 62 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login │ 62 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://ekburg.irr.ru%2Fpuloveplanet │ 61 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 61 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 61 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26092.html?s_text=\x5C\x5... │ 59 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/thenon-houses/public/g... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27755f32316.30; .NET CLR ... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742/details/?cauth=0&dam... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fautoad/kniga.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26004/?_h=galle/\xD0\xBF\... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert27930555&sob=1&p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2692.html%3Fhtml?period=3... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/world/photo31469:Album... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=leting "HalUXI&where=\xD0\x9F\xD0\xB5\xD1\x8... │ http://irr.ru/index.php?showalbum/lofiver.ru/articles/86121%26uid%3D139750%26ad... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27000,224648804-recept-so... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/koshka.com/ig/iframe-o... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/realty/suntime-5/extre... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory=cinema.perm.p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742304][from]=&int[14670... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742357/detskii_gosts.xml... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694336/photo/suzannason/... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Forum25/top... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 57 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupit/action │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268320995,968650f45491882... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1375605&ga... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://video.yandex.php │ 56 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga/choice/... │ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_product_... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?showtopic,5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26017/quarius_Moscow/cavi... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2726125413975d77cf&search... │ 55 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 54 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266848223/review_type=pro... │ 53 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 53 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26989%26bt%3Dad.adriver.r... │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/album/login-1800002&pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 52 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0 │ 51 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2698172,93932353064614618... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605085268a9c4d61-9862.49... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19&numphoto/login=A-CL-MS-36575c72937][to]=&int[12822304 │ http://irr.ru/index.ru/widgetchrome%2F&ti=no&dom_v_bordovye-printime.ru │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/investate/comple/make/?page5/&docid=jlMNIrXw... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 50 │ -│ 0 │ 0 │ 0 │ │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 50 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F11.7.1364 │ 50 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 50 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26051.htBeg=6&NightRegist... │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xBE\xD1\x81\xD0\xB8\xD0\xB1... │ 49 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 49 │ -│ 5 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 49 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?group=days=3&text=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27410/photo-2.xhtml&serve... │ 48 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandsearch │ http://komme%2F27.0.1453.116 │ 48 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 48 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusty-i-vkont.at.ua/search=0&userId=0... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266128182&op_seo_entry/de... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?sort=&br... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/intrumen │ 46 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://afisha.yandex.ru/index │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumanegenre=33155?analog/kitchen_mini... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik │ 46 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login%2Flick.g.doubleclick.diary.ru/car/kw/3061/us... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://afisha.yandex.ru/index │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734.html_params%3Dfh_loc... │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/GameMain.p1Yo4A │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ua/searchads/jo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605906/frl-2/sportal.ru/... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fshow/lpp/cre.ru/pers... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.asp?search&ev... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logize.ru/msk/events_liver.ru/russert-plies.r... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26099486633033&countpage=... │ 44 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 43 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://sslow_135000008&position=search │ 43 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischet-solik/odezhda-plos... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert277553/38021/66936575776/... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271050&with_photo.kurortm... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 43 │ -│ 1 │ 0 │ 0 │ http://forums/liii-kuler S22oHgBJTngegotavgorod55.ru/filtr[2]=42 │ http://komme%2F27.0.1453.116 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-folders/?end=9a08488... │ http://irr.ru/index.php?showalbum/litamak.irr.ru/catalog/8570/travel.ru/cgi-bin... │ 41 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 41 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 41 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/0001216629/#top_by │ 41 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26040931&s_yers=0&with_vi... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://ekburg.irr.ru%2Fpuloveplanet │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.137508&s_yers=200... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://aliningrad │ 40 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logito8ergo_6470/page_type_id=20872/0/001:12:... │ 40 │ -│ 0 │ 0 │ 0 │ │ http://afisha.mail.ru/dmitrij │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/readar-nashi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27924563724&key=46960/med... │ 40 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://lib.ru/exp?sid=3205&bt=7&bn │ 40 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 39 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credit │ http://komme%2F27.0.1453.116 │ 38 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=8570/page=5&s_yers=2006 │ 38 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2679/detail/5482,935033/2... │ 38 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 38 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login │ 37 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dnevnoy-kv-Samara.irr.ru/searchAutos&marka=4... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 37 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://komme%2F27.0.1453.116 │ 37 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=theating&page=... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?group │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2784389/room=39695,966681... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/forum/topicID=269&stat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/votkim-rukav-i-kova-ul-adve... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wolf-club478561/OPEL | \xD0\xB0\xD0\xB2\xD1\... │ http://irr.ru/index.php?showalbum/login-marka=13&city&custom=0&damages/0001216 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filmId=xFX1UbTNJjxe4yF... │ http://irr.ru/index.php?showalbum/login.asp?razdel7/test/matched=115909d9_dsc07... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27553&s_yers=0&po_yers/46... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert275226607660?design=6efxq... │ 36 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=show=&fgroup-tab-mara.irinables/yearT... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278171&size=52094&op_seo_... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27330528/photo/6223799%26... │ 36 │ -│ 5 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://smeshariki.ru/obucheyelants │ 36 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36&he │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=post_neu%3D22%26pz%3D0%26rleurl%3D%26xp... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 35 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/8569 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/index.ru/katego... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vip.ru/book.com/photo/7086/page=0&state=mont... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-cukickdrops_bak_yant4tlGtZgy3BQ&bvm=bv.49478... │ http://irr.ru/index.php?showalbum/login-501-98552&op_uid=1060948/6#f │ 35 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (Windows │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19&m_staraya-obl │ http://irr.ru/index.ru/show/414526863_112 │ 34 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.12.388 │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/promo=C-Tease/rent/dat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 34 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_uid=1... │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/list │ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 34 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/directadvert/kavanga_728x90.html │ http://komme%2F27.0.1453.116 │ 34 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshops.xml?typ │ http://komme%2F27.0.1453.116 │ 33 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?cPath=default.ru/yandex.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33 │ -│ 1 │ 0 │ 0 │ http://auto.ria.ua/\xD0\x9A\xD0\xBB\xD0\xB8\xD0\xBC\xD0\xB0\xD1\x82\xD1\x8C\xD1... │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 33 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://video.yandex.php │ 33 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 33 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser │ 33 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265748ba3f755671_1162684&... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertif?sle=85000 \xD0\xB4\xD0... │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364 │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/pub │ 32 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zxcvbnm1990&search/jobinmoscow.ru/yandex.by/... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements-obschaya │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fproduct/foto-4/login... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&op_category_id=1174]... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniturers_id=575386110 │ http://irr.ru/index.php?showalbum/login.exit/rem/php/board/10_kruizer_8267049&o... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiepochto_id=93231238][to... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://afisha.yandex.ru/index │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ssl.hurra.com/iframe │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischuk_Kerasivye-klavik/?... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/jobs-education/m_acces... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26791e3a7c543c9bf9ef5c0-9... │ 31 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt │ http://komme%2F27.0.1453.116 │ 31 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertificia/?letteo_985428161/... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_api.php?cid=41389/rooms/perm.pulscen.ru/sear... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 30 │ -│ 1 │ 0 │ 0 │ http://domchelov.html&lr=55&text=\xD0\xB4\xD1\x80\xD1\x83\xD0\xB3\xD0\xB8\xD0\x... │ http://komme%2F27.0.1453.116 │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoryID=12452929587/... │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 30 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 30 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268630533&site_ot=&price_... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-k608i.trashbox.ru/cars/page5/&docid=577&lr=2... │ http://irr.ru/index.php?showalbum/logii_58247.php?forum.borovichy77/page5 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2716/~3/150160947694,9745... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2696374/#page/tab=com_avt... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/products/?category_id=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 30 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.14&he=768&wi=1920 │ 30 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/topic │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605446518%2F&sr=http://e... │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26979553039353936694&ch=u... │ 30 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://smeshariki.ru/obucheyelants │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-carava/fotok-8-steklove.ru │ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26928142&op_products/view... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kupibatikov/?recrete/personal/klimat.ru/work... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_countpage/1 │ 29 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x9A\xD0\xBE\xD1\x80\xD0\xBE\xD1\... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertifications[0][150]=60&inp... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/wedditeli_foto-2/__48/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274229696.0 (Windows)&bL=... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/click.net/paged=696&lo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-14132&with_photo=1&srtime-zone.net/catalog/t... │ http://irr.ru/index.php?showalbum/login-sierra%2Fdlia-zhienskaia-moda-zhiensmed │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 29 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704.html%3Fhtml5/v12/?fr... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2664952&xdm_p=1#country=-... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734/fotographic_2017666&... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategoriya/zhienskaya-... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9649.html%3... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/small_mercial-shoppicc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27350350394&ch=UTF-8&sF=1... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_seo_entr... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-baza/gadge... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27425.3120%26ntype_id=2&r... │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 28 │ -│ 1 │ 0 │ 0 │ https://directions[0][1541769377921968 │ http://komme%2F1.7.1364.172 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert272291684/hasimage=0&view... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13733097&op_categ... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/mann japanel/?text=\xD1\x81\xD0\xBC\xD0... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269500469412216388/detail... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-my-sovies/bebybum.homestion bonprix.ru/... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikarbox=2&tz=9b81998&po_y... │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?lang=all&fi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-176-kakaya-obl.irr.ru/search?lr=191&ei=N6rrg... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\xA4\... │ 28 │ -│ 3 │ 95 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-eve-nn.ru/cars/passenger/\xD0\x92\xD0\x90\xD... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/anime-2/#album/l... │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274236&pv=15&prr=8&sid=35... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/41452626u1zIq0SGLXCI │ 28 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic/8939858-59929477/detailshop │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://ekburg.irr.ru/#lingvo │ 28 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F22.14&he=768486 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678/page=2&marka=84&mode... │ 27 │ -│ 3 │ 2 │ 0 │ │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/search?cl4url=http://yandex.ru/online... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27013.ya.ru/sportalznakom... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert24061.jpg.html%3Fhtml?1=1... │ 27 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031343-sh-898675/sort=pri... │ http://irr.ru/index.php?showalbum/login-5158&jenre/slyudya_metroman.ru/GameMain... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19195/offset=101&district=&city&with_photo=¤cy │ http://irr.ru/img/catalog/53485785/topic/8939850.php?page=3&marka=84&mm │ 27 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 27 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 27 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fhit_list.html?s... │ http://irr.ru/index.php?showalbum/login-irake-myservierk-supierts%2F&ti=\xD0\x9... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-oleginnesujeli-i-centum=etarget=search?clid=... │ http://irr.ru/index.php?showalbum/login-kupe.html_params%3Drhost%3Dad.adriver.r... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27928092_price=\xD0\xBC\x... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/mazda-dievochnye_avari... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-19.xhtml?city_to_one=All&film]/on/orders/810... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/1.19.mobile_photo=6001... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27630/?item_id=0&bodystyl... │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ 1 │ 0 │ 0 │ http://bonprix_ru_11559&lr=12&usln │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lizova_ii_1112_1150&option[price_do=300... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E&wher... │ 26 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://obninsk/detail │ 26 │ -│ 1 │ 0 │ 0 │ http://yandex.ua/?target=search/tab=user │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detall/seconomicsmovinki │ http://komme%2F27.0.1453.116 │ 26 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login-jjjekrju-s-pring=1&price[price=\xD0\xBC... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/loginsk.irr.ru%2Fkategory_id=9204471-650/?_h=... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lejandsearch?text=\xD1\x82\xD0\xB8\xD1\x85\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert21124631/?Search.php │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification.ru/messa.ru/rea... │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 26 │ -│ -1 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_category... │ 26 │ -│ 3 │ 197 │ 0 │ │ http://komme%2F27.0.1453.116 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?messa.ru/worlds/page2/7277932582&text=\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 26 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-kupe-20010120652838799.html?n=7148.html... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemey-volvom80Cw&where=a... │ 26 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=607&state=4... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.22 (KHTML, like Gecko) │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 25 │ -│ 1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 25 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/people.ru/cars/passets... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731/?city=0&pvno=2&evlg=... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vivarki/redki_sdelaet-law.ru/produkty/kartgo... │ http://irr.ru/index.php?showalbum/login=driff/en-ru/?p=2#messages/00001216629 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 25 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert264314953); ru) AppleWebK... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.html&ei=GcXrUYe8F2DAU-RbuRlLMczoJ--5uDCx8 │ http://irr.ru/index.php?showalbum/login-132/#images/0000.h95/\xD1\x82\xD1\x8E\x... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search?text=myaccountry │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login │ 25 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://obninsk/detail │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguard-payments-sale/se... │ http://irr.ru/index.php?showalbum/login-88i8LaDyEkCVv6-DhRfEDcw==&action │ 25 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/yandex.ru/?f... │ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.ru/film/46351/frl-2/bage │ 25 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605838.html5/v12/?from=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-link=1147373-1&p=1&p=2&text=\xD0\xBC\xD0\xBE... │ http://irr.ru/index.php?showalbum/logisterlingvo/#! │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741559&t=5204/make=Chrys... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694589/detail/555-javata... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=shops.html?id=1&body_type=\xD0\xBF\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781472265&op_seo_entry=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626rleurl%3D//ad.adriver... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685/price_ot=&price=\xD0... │ 24 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://diary.ru/forum/intries │ 24 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-117031&op_category&op_seo_entry=&op_uid │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26685857947301_Zoryatiya/... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671291&fridgets/2012&pri... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27399992.php/board,13.0) ... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguar.ru/page4/?emain.a... │ http://irr.ru/index.php?showalbum/login=ogabass.ru/filmId=9WOqzzitive&view │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-annative-expressageId=0oVXx21hUXI&where=all&... │ http://irr.ru/index.php?showalbum/login-rybnovlevojj_mamountry=-1&type%3D0 │ 24 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://video.yandex.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo/7... │ http://irr.ru/index.php?showalbum/login-kupit-plitka-s-korsiety/art/151/n4p/160... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/intrumen │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kupalnaya_elit=&id=3&clid=9403&lr=1913849 │ http://irr.ru/index.php?showalbum/login-gallebia_zhurnalCall D.R.E.mp3ex.net/ph... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lolacake/tatyaka.html?1=1&cid=577&oki=1 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinglish.rus-l7-p70505-1... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/yandsearch?lr=1... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-yvaken/topic=7702.jpg.html?1=1&cid=577&... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=1&bc=3&ct=1&pr │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-547552/1/?sma=RAINBOW&op_uid=fdd1b9d2721728&... │ http://irr.ru/index.php?showalbum/login-kapusta-advertika/search/room=1&damage=... │ 23 │ -│ 1 │ 0 │ 0 │ http://auto_volkswagency=1&text │ http://komme%2F27.0.1453.116 │ 23 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgarials/commend/searchvage-2.html&dt=13734... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-tuniki_510... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263090024189-528619c2077/... │ 23 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/ru/index.ru%26bid │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333572620201709/page/10... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26939.html?page/3/#count=... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news/radio&clid=198555... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/kategory_id=5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2740974%26pz%3D0%26ar_sli... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 23 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/resultatya9176da22f1a521a5853.html&... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26600721376439322%26pz%3D... │ 23 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga_728x90.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://afisha.yandex.ru │ 22 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemennoke0070553995f27d6... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2697574697.html_param=0&u... │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-105vr/ │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=search.tutung/s... │ 22 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/wheel.ceratornaya_solnechka │ http://komme%2F27.0.1453.116 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-10760b31b65633eZateq eb806e887d9f15ccf593280... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2091&local=1&TypeSearch?mail.ru/yandsearch&c... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26058&nnum=s4746835895&op... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-the-ili-tempera-minsk.irr.ru/produkty/bluzki... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodny │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2649.html?pg=menu_29.jpg&... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739693071/pomer... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/v1465][from]=&int[157281 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-business/sale/ru/search?mail.ru/film%2Fbarna... │ http://irr.ru/index.php?showalbum/login-zakon_type=1&fuelRatesTypeSearch │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267830840994,95688781470%... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 22 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27200,55.7654.html_parat-... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26601629][from=&fuelRateT... │ 22 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&l... │ http://irr.ru/introlux_page5/2/page/Renaul.irr.ru/start=1500-rublic/gamemain/Tu... │ 22 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://ekburg.irr.ru%2Fpuloveplanet │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liliate_id=24145602&with_phoney.yandex.ru;ya... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-natashkarola.irr.ru%2Fviewforum/MsgList.html... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26900/technics-technics-t... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&city[7]=23239330794,25826044_Black_lis... │ 21 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?price_till beli.php?cx=015216684_36 │ http://komme%2F27.0.1453.116 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691_ru_17832523.html_par... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694,978825315f373400/det... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1) AppleWebKit%2F537 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/world/photo/41/62b1dfa450/3/women-clother%26tms%3D%26ev_ltx%3D... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/basketshop.ru/yandsear... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/forum%2Fukhov/l... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27025&Select[4][]=32598.h... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265228.html?1=1&cid=577&o... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo44953.116.php?t=4... │ http://irr.ru/index.php?showalbum/login=ko&page=0&view.aspx?group_cod_1s=85&key... │ 21 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://wildberrior/uphold │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/?from │ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/houses/passenge... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-dance/index.ru/?favorite_off=1&encoding&hl=\... │ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logino-s-grigerator/page1=&input_age1 │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cars/commeddesk.ru/google_s&12.5.746.59954.30 (KHTML, like/\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login.html?1=1&input_who1=2&input_who2=1&inpu... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/logii-bin/click.cgi%3Fsid%3D158195,97987231-9... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ramblery/pic/893985650697... │ http://smeshariki.ru/obucheyelants │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26737865/?page/12186/#/ec... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dietsik.ru/razdel_id=111,7,700&w=728x90.html... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929&from%3D%26custom=0&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/club.ru/anime-zone.ru/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-brie... │ 21 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739353/details.... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2186/detail.aspx#location... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kupalnik │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification%2F4.0; XBLWP7; ... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/9-0--navam-zhien... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741465][from]=&input_wit... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_uid=1147... │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login=uzpp2XnEw&bvm=bv.49784469][from │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 20 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-Larki-i-korobeg-1124-95367/guestblowinp... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26219&wi=1280&lo=http://w... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estatic.diary.ru/... │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=58&Selectronics-technics/k... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/yandex.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b706234651&op_uid=1&b... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?er=2&text=\xD0\xBA\xD1\x83\xD0\xBF\xD0\xB8\... │ http://irr.ru/index.php?showalbum/login=razer2.moikrug.ru/recipes/shop.spb │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://video.yandex.php │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=NeIdeaCenternet Explorer&aV=5.0 │ 20 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&price │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertificirovka.ru/noteating-i... │ 20 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271161076&st=261&t=170977... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate=0&ReturnUr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313163307/?from]=&int[15... │ http://che.ru/produkty_zarubezhei-niepochekhly │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_api.php?id=7262882,9454472&op_product_n... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604370757034dea482207549... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 20 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/cars │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-karbox=0&top=0&category/1961/#images/00... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26637&model=1705.html?1=1... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumilora481 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt=13733129/room │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-iulyanovskaya-obl/lyubvi?page=166733&eid=485... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27502/detail/?folders/#pa... │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://video.yandex.php │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626u1zIq0SGLXCI&where=al... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=240&u_cd=242035165&input_who2=1... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news_id_72387877055/nu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikulina.ru/real-estate/ou... │ 19 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?sort=popup/casualg... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBB\xD0\xB8\xD0\xBA\xD0\xBE\xD0\xBB\xD0\... │ http://irr.ru/index.php │ 19 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/filmId=eRaB4pYAXCI │ http://komme%2F27.0.1453.116 │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.ru/daily │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260540718534/?dateOn=1&bo... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login=perfect.yandex.ru%2Fkategoriya%2F537 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26672655462.html?city.ru/... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261921&pr=2441185112.html... │ 19 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostok.ru/cars │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671&Destige-all.html_par... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27404895,95923.html%26cus... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/check=1&redir=1&lang=a... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313437&pvno=2&evlg=VC,2;... │ http://irr.ru/index.php?showalbum/login-rizova.ru/look/timashirtsevm.cofe │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 19 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2638/?from=&raceTo=&power... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6HS │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166333/price[price_till]=&pri... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F_liveresum... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27355f3132366336601&uuid=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.php?f=11340.1j1j2.12...5998-103358071/8/page... │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/00007/10/1320006&p... │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto_repai... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://smeshariki.ru/obucheyelants │ 19 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.15751 Safari%2F5.0 (Windows │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2675406828ac956028692581/... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://obninsk/detail │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26968794618/irr.ru/real-e... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2702483,631;IC,444041%2F&... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2629&lr=65&rstr=-213&text=back=False&withmos... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=http://video.yandex.ru/real-estate │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26669E116-1-1550046560013... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?search?text=\xD1\x80\xD1\x83\xD1\x81\xD0\xB... │ http://irr.ru/index.php?showalbum/login-inadlinny-sched_car=0&city=0&price=18&p... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274026181891&rn=52&brand=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/dlia-zhiensk... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&checked=0&state/room=10002-g-v-program... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-165&jenre]/49047&lr=163&text=samsung ga-moda... │ http://irr.ru/index.php?showalbum/login-2-komn-v-na-92f5-6ccf-fef3-013f9f926a21... │ 18 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ru/index │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5 │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?elmt=popular&where=all&film/723/num │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/restate/out-of-town-resh=1&cnt=1373... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2012-recept-Ragu-iz-2-x │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupaljinik-slidi/places/?PAGEN_1=2 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 18 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=deltec&type=citymo=&version%2F4.0 (Linu... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27534.30 (KHTML, like Gec... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kochtinsk.pulscendinskiy-krasnodar.irr.ru │ http://irr.ru/index.php?showalbum/login-kuplyuskij/zapchastorii_state/out │ 18 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login.moikrug.ru/catalog/otchenko-ul-advert27... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zzzi.ru/a-shop and blademas/?id=1&marka=0&ma... │ http://irr.ru/index.php?showalbum/login-2003prev=/search/index.ru/meteor.html%3... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\... │ 18 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27132069304;IC,1511e8d158... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278453&with_exchange=0&ex... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=367132&pv=2... │ 18 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27071038_114736895,923401... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x9A\... │ 18 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/8018&l=malkogo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27420780790931.html5/v12/... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kanza_tab_2499472&lr=1074&sort=popular │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-natura.ru/jobs-education=post_137336ID0... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/?target=search?... │ http://irr.ru/index.php?showalbum/login-bigutensiysk.irr.ru/regular&bid=3205&bt... │ 18 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-21102&text=\xD0\xB1\xD0\xB0\xD0\xB1\xD0\xBA\... │ http://irr.ru/index.php?showalbum/login.mospsy.ru/super-na-kostronics/tv-audio.... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/obuv%2F │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685&numphoto=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626p/ │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertimeshariki.ru/cgi-bin/cli... │ 18 │ -│ 1 │ 0 │ 0 │ https://diary.ru/magnitogorsk │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sien-zhienskaya-i-kvartir.html?id=1955451-01... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260908861%26point-peter=&... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeSearch?text=\xD0\xB2\xD0\xBA\xD1\x83\xD1... │ 18 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1453.116 Safari%2F8536 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=yators/?jumping.ru/series.ru/priceup&page=3&... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinger/\xD0\x93\xD0\x90\... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=36427929&stat... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929750%26ar_sliceid%3D2... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga1788778.137329TVmbFNmbVNaR0YwW$UyNDg4fxS... │ http://irr.ru/index.php?showalbum/login-sienskaia-moda-zhienskikh_rabotjaga.ru/... │ 18 │ -│ 4 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691c352005&pvno=2&evlg=V... │ 17 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27422544/3/womens.ru/ufa.... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru%2Ffilm/4858?pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/cars/communalux.ru/dos... │ http://irr.ru/index.php?showalbum/loshaya-obl.irr.ru/film/35562C?analytics/kto-... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271527091/frl-2/bage-2.ht... │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#7878... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.103402.html%3Fhtml │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=artira-Lyubyat │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.aspx#location │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=yes&y1=2013-07-07.ya.ru/5_search?text=\... │ 17 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://aliningrad │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.15.html?1=1 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266149_51928/detail.ru/ka... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%26rleurl%3D%26CompPath%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2670308&po_yers=0&price_o... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2613168312.71 Safari%2F__... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_Bolers-device.aspx#commersanta_premiery... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=8&ch=UTF-8&ifr=1&av=1&nm=1&ved=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 17 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6HS │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-dlia-zhiensmed.ru/el... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginyazevskaya%2F5.0 (Windows NT 5.1) Prestorygoo... │ http://irr.ru/index.php?showalbum/login-kupilomatic.aspx#location=rr&days&city&... │ 17 │ -│ 1 │ 0 │ 0 │ http://bdsmpeople.ru/register2123 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logistereda.ru/photo/70861/6#f │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263161366&bid=2823&numpho... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowthreadreplies=24&ids=74&cu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313&ussp=mstp&usln=1&inp... │ http://irr.ru/index.php?showalbum/login-3767436/roomootofile/obzor.com/iframe │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26041.htm%3Fsid%3D0%26pz%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27595,9305f3931339322%26n... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273363331251798796&op_seo... │ 17 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/stan/index.kz/yandex │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertistia.html?partment/searc... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-search%3Futm_source=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031323532363&win=70&mode=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 3 │ 22 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://diary.ru/forum/intries │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 17 │ -│ 1 │ 0 │ 0 │ http://bibieters_sig=5ea8QizwGSPy0yGcm14ATH74D4Bw&usg=AFQjCNHxMZlonova.ru/p1847... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_style... │ 17 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/messar.ru/professig │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-futbolki-k... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lenfiki-zhienskaia-moda-muzhchin-sitafa ishq... │ http://irr.ru/index.php?showalbum/logical.ru/link=11485-9065383235348%2F&sr=htt... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261636/detail/odnoklassim... │ 17 │ -│ 1 │ 0 │ 0 │ http://acase.php?input_who1=2&input_who2=1 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.10 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694&option%2F12.14&he=76... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733145625/details... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268814453957595,94406/det... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26618531&t=111246.html?p=... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=Like Gecko) Chrome%2Fpotnik=1&price_do=&curr... │ http://irr.ru/index.php?showalbum/login-the-poxudet_sponsor=&o=6000708498/?caut... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-irina19910735%2F27.0.1453.116 Safari │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=bestwo.diary/details&id=9160/0/posti.ri... │ 17 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-53584715,1327288/belgium/arch=1&themec.... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b76faact/less=1&ru=1&... │ 17 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 16 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/category=cinem rasmusic │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27034195,9520160173/5/wom... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgas.rQRAX2Ua3IF1iHJcaz4ATv34DABw │ http://irr.ru/index.php?showalbum/login/?page=1024&wi=144438-88-3906740/currenc... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267044005320AA76)&vendor_... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098707209463401090/deta... │ 16 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search?clid=19554786.0.8.0.2.2003558_3497926... │ http://irr.ru/index.php?showalbum/logie-niz-doktops/electronics/4185570c72 │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-herokee_tatus=1&rm=18606bAxFEcQT6smBB4W... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search/cuZXdzJTJGJmxyPTI4fQ,... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26396284046.xhtml%3Fhtml%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-plan... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&pt=b&pd=7&pw=2&page5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260612d7cf.4082813,74,756... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate/out-of-tow... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/page=91552&q=\... │ http://irr.ru/index.php │ 16 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/movinki... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?&clients-sale&siteurl%3D//ads/search?text... │ http://irr.ru/index.php?showalbum/login-kapusta-advert273786/foto-5/#photo=2817... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login.aspx?sle=13&s_yers=0&page_type=0&door=0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnaja-prost-petersburg/detail/?fold... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-krasok.html_params%3Drhost%3Dad.adriver │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661232&streamfood.com/if... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiensmed.ru/dl/\xD0\xB7\x... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti/redmonds %2F offset=403134333.2201560&c... │ http://irr.ru/index.php?showalbum/login-v40-velopiter.feriod=0&scroll_to_auto.r... │ 16 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/70948/35.htm?id=... │ http://irr.ru/index.php?showalbum/logiy-luchaiev%2F&sr=http:%2F%2Fwww.bonprix c... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166303133334%2F&ei=horia+iudi... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26765.html_params%3Drhost... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-57548811426&text=\xD0\xBB\xD1\x83\xD1\x87\xD... │ http://irr.ru/index.php?showalbum/login-coolonellana-Molodilnik.ru/index.php?vi... │ 16 │ -│ 1 │ 0 │ 0 │ http://histore/profile;u= │ http://irr.ru/index.php?showalbum/login-kupe-20013&uuid=1373356164564_1sort/sta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/showtopic,803813.html&... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_catalog/1250/?itemsg/cd... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2755597.html%3Fhtml?1=1&c... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=findjob.ru/planet.ru/index.ru/carinov.h... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert260477/cities/visas/exp?s... │ 16 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://video.yandex.php │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27429&input_bdsm_position... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2728024/detail/508/?instv... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26887301/Zona/simferopol.... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-153-4b9c4.4331626.htm?size%3D0%26rleurl │ 16 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect │ http://komme%2F27.0.1453.116 │ 16 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://msuzie-shop/premiery-c-38208_2.html │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-marka=23&model&desting?field_town/search/pri... │ http://irr.ru/index.php?showalbum/login-492ea9&show/417582,9626512874":10157628... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/posts&q=\xD0\xB2\xD0\xB5\xD0\xB4\xD1\x83\xD1... │ http://irr.ru/index.php?showalbum/login-mistore/#CATALOG_LIST-s-ovosibirsk/deta... │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/sch/price][min]=41&lr=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27364229586121%26url%3D%2... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ 1 │ 0 │ 0 │ http://volgografiyah_27_iyunya_50_let%2FgetId │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333%26bid%3D1%26rleurl%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-maritkiN │ http://irr.ru/index.php?showalbum/login-ladimir.irr.ru/remiery-telefon.ru │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new=127201148][fr... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661251-6efa-d61f-fef3-01... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/tv-audi/a7-spec=9894797179698712/p1/8... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti%2F%3Ff%3D100%&http://afisha.yandsearch&... │ http://irr.ru/index.php?showalbum/login-6030d.html_params%3Drhost%3D90%26height │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/7427510/detail/5... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269899458/price/3815&pvno... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=view/5/item3963... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logiste's a di galaxy-s4-story1.aspx#location... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5.0 (Win... │ 16 │ -│ 1 │ 0 │ 0 │ http://yandsearch?lr=2&color │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-gorod/search?p=7&oprnd=9902.jpg&img_url=http... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27222.vk.me/u3166 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2753][from]=&pricedownloa... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new10152954vac │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2735030373434-152495.php?... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makeevka.ru/perm.irr.ru/slingvo/#1\xD0\... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691.html?1=1&input_age2/... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26708662307][]=&selection... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313033393633310,932803][... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2010000087378687/pic/8939375f303839303935373... │ http://irr.ru/index.php?showalbum/login-souse=2/path=39_1.html?1=1&cid=577&oki=... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-irinakai/page_num_read2306e4574&ei=Ot_rUfjPG... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/artira │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/?strict=5710... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertists/736850/?item_no=737&... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2695/19545602cf94d77/repl... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumy.ua/search?text=\xD1\x81\xD0\xB5\... │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikal.ru/search/room=1&dam... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login/?do=ready-mansion%3D0%26url%3D%26bn%3D0... │ 15 │ -│ 1 │ 0 │ 0 │ http://forums/liiiervierk-suppohudeemvmesting/294465&op_categoriya │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb=1&prepairs=0&city[1 │ http://irr.ru/index.php?showalbum/lounona-muzhchin-kupaljinik-chernyj-odnoe/san... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&model=1173... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=new&cad=rjt&fu=0&input_country_id=228&lr=961... │ http://irr.ru/index.php?showalbum/login-cam.shtml#objdesc=true&uA=Mozilla │ 15 │ -│ 1 │ 0 │ 0 │ http://launcher-searchads/search │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-4433140796851/detail/536180&all=False&i... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26981583/page=1089592445/... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-album/login.2/second... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781308;IC,238208836746/0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginsk.eu/ru/Pagesize=13&m1=07 │ http://irr.ru/index.php?showalbum/login-3muda/truction/vacancies/750207190765.h... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_produkty/kiev.org/forum... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273631253711/foto-6320166... │ 15 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F1.7.1364.172 │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945HPS?analog/r10418/sort=price_do=200&site_off=1 │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 15 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshop/id_art_type=7&s_yers │ http://irr.ru/index.php?showalbum/login │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2724/?_h=search?text=\xD0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273389; U; Android 4.0.14... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-goluboj-podserial&dfs=13 │ http://irr.ru/index.php?showalbum/login-haus.html#news/228670,257&pci=3012/frl │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#67 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266356163f39ea6f8/7a8745&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/washes/built-in_two_ch... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26605566254/room=1.6&wher... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list.html?typ=SMA&anbieter=aleks_evilkos.com │ 15 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/state=1909644 │ http://afisha.yandex.ua/auto_id=0&with_photo.kurortmag.ru/razdnitsya-vishnury │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://irr.ru/index.ru/show/414526863_112 │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-land_search?text=\xD0\xB0\xD0\xBC\xD0\xB1\xD... │ http://irr.ru/6323%26bn%3D27888895,963095425 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?page37/?sta... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?Topic │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/index.ru/ange=0&s_yers=(3000&static.diary.ru... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/produkty%2Fplatj... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginskikh-viana.irr.ru/GameMain.aspx?sle=1/house.... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?transk.i... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678&op_page=60000/curren... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?Id=9582 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liloveplanet.ru/futurer533/women.aspx?group-... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2631920&lo=http://person_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.1Uv1UfqCA8SonYC4BQ&usg=AFQjCNGB3pBUuKY1jJPP3... │ http://irr.ru/index.php?showalbum/login-tank-sale/search%3Fmode=&page_type │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-tumbov.irr.ru/page1=&input_who1=2&id=29... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=alini... │ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2718599/photo=0&is_hot=0&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27515/418695&st=327-B110Q │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.php?city[1]=700003 │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2699364224073532663835386... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26761,55.654289600/detail... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login=vladimir/page_type=0&expand_search?text... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?dok=001&rightbox/week&From=0&user │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/frl-4/trailer/view/3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-lential/secondary/details9.html_params%... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27265484158197%26width%3D... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page9/#14... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=25127691%2F... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/ch/floore troubleclick... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-676216b8af/4fd00fa61b3185631821/page_ty... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/gamemain.asp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27877437494,943082&stat_t... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2730675595,9292fa-d61f-fe... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1955451&lr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienshchin-planet.ru/myma... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_appliazh-pliance/mista-bez-uchaiev/000202&cl... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginy_name/3196906481/currency=6465373200&price=\... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2776/?date=0&doorov.irr.r... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-102-821922237&arrFilter2013/08-iH4AT5zIGQDA │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_type=&freetao.diary.ru/sea... │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 15 │ -│ 3 │ 123 │ 0 │ │ http://komme%2F27.0.1453.116 │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Ford-antent... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 15 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nashinger/users/search?textilead&353%26ev_pl... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert276278-ploschaya_checked_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9584%26pz%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert262997385f32313335781094&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2693_763613.html_params%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/877/ru/buildin... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274298799461981/?date=129... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://ekburg.irr.ru%2Fpuloveplanet │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.10 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=on&input_onliner... │ http://irr.ru/index.php?showalbum/login.html%26custom%3D%26CompPath.2; WOW64; r... │ 15 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credir=1 │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/gosthelp.ru/auto.ria.... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=on&in... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/litlip/saint-peter557932E-8C62-4917%26ad%3D64... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19&numphoto/login=A-CL-MS-smi.org%2F40 (KHTML, like Gecko) Version... │ http://irr.ru/index.ru/widgetchrome%2F&ti=no&dom_sell/pansiyskaya-obl.irr.html%... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search?text=\xD1\x84\x... │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert269257798044.html_partmen... │ 15 │ -└────────────────┴────────────────┴─────────────┴────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.683 user 1.444000 sys 0.132000 -D 07-01' AND (DATE '1970-01-01' + 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; -Run Time: real 1.480 user 0.196000 sys 0.048000 -D D (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND DontCountHits = 0 AND URLHash = 686716256552154761 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Run Time: real 1.744 user 0.252000 sys 0.172000 -D " = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) ORDER BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))); -Run Time: real 0.508 user 0.712000 sys 0.024000 -D -D -D -D -D -D -D SELECT count(*) FROM hits;+ -┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 100000000 │ -└──────────────┘ -Run Time: real 0.010 user 0.080000 sys 0.000000 -D SELECT count(*) FROM hits WHERE AdvEngineID != 0;d -┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 630535 │ -└──────────────┘ -Run Time: real 0.639 user 0.520000 sys 0.060000 -D SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits;m -┌──────────────────┬──────────────┬──────────────────────┐ -│ sum(advengineid) │ count_star() │ avg(resolutionwidth) │ -├──────────────────┼──────────────┼──────────────────────┤ -│ 7280824 │ 100000000 │ 1513.48908394 │ -└──────────────────┴──────────────┴──────────────────────┘ -Run Time: real 0.053 user 0.780000 sys 0.044000 -D SELECT sum(UserID) FROM hits; -┌───────────────────────┐ -│ sum(userid) │ -├───────────────────────┤ -│ 3.230605869408804e+26 │ -└───────────────────────┘ -Run Time: real 0.039 user 0.604000 sys 0.004000 -D SELECT COUNT(DISTINCT UserID) FROM hits; -┌───────────────┐ -│ count(userid) │ -├───────────────┤ -│ 17630976 │ -└───────────────┘ -Run Time: real 8.105 user 9.812000 sys 0.996000 -D SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -o┌─────────────────────┐ -│ count(searchphrase) │ -├─────────────────────┤ -│ 6019589 │ -└─────────────────────┘ -Run Time: real 12.975 user 17.084000 sys 1.176000 -D SELECT min(EventDate), max(EventDate) FROM hits; -o┌────────────────┬────────────────┐ -│ min(eventdate) │ max(eventdate) │ -├────────────────┼────────────────┤ -│ 15888 │ 15917 │ -└────────────────┴────────────────┘ -Run Time: real 0.338 user 0.560000 sys 0.040000 -D SELECT AdvEngineID, count(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC;s -┌─────────────┬──────────────┐ -│ AdvEngineID │ count_star() │ -├─────────────┼──────────────┤ -│ 2 │ 404620 │ -│ 27 │ 113167 │ -│ 13 │ 45633 │ -│ 45 │ 38974 │ -│ 44 │ 9731 │ -│ 3 │ 6896 │ -│ 62 │ 5266 │ -│ 52 │ 3554 │ -│ 50 │ 938 │ -│ 28 │ 836 │ -│ 53 │ 350 │ -│ 25 │ 343 │ -│ 61 │ 158 │ -│ 21 │ 38 │ -│ 42 │ 20 │ -│ 16 │ 7 │ -│ 7 │ 3 │ -│ 22 │ 1 │ -└─────────────┴──────────────┘ -Run Time: real 0.030 user 0.456000 sys 0.000000 -D SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -┌──────────┬─────────┐ -│ RegionID │ u │ -├──────────┼─────────┤ -│ 229 │ 2845673 │ -│ 2 │ 1081016 │ -│ 208 │ 831676 │ -│ 169 │ 604583 │ -│ 184 │ 322661 │ -│ 158 │ 307152 │ -│ 34 │ 299479 │ -│ 55 │ 286525 │ -│ 107 │ 272448 │ -│ 42 │ 243181 │ -└──────────┴─────────┘ -Run Time: real 9.257 user 11.444000 sys 1.204000 -D SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10;i -┌──────────┬──────────────────┬──────────┬──────────────────────┬───────────────┐ -│ RegionID │ sum(advengineid) │ c │ avg(resolutionwidth) │ count(userid) │ -├──────────┼──────────────────┼──────────┼──────────────────────┼───────────────┤ -│ 229 │ 2078084 │ 18296430 │ 1506.0876750819696 │ 2845673 │ -│ 2 │ 441711 │ 6687708 │ 1479.8410618406187 │ 1081016 │ -│ 208 │ 285925 │ 4261945 │ 1285.260504769536 │ 831676 │ -│ 169 │ 100887 │ 3320286 │ 1465.90517142198 │ 604583 │ -│ 32 │ 81498 │ 1843721 │ 1538.0370495318978 │ 216010 │ -│ 34 │ 161779 │ 1792406 │ 1548.364990409539 │ 299479 │ -│ 184 │ 55526 │ 1755223 │ 1506.8102679830426 │ 322661 │ -│ 42 │ 108820 │ 1542771 │ 1587.1074287758845 │ 243181 │ -│ 107 │ 120470 │ 1516722 │ 1548.6039623609336 │ 272448 │ -│ 51 │ 98212 │ 1435598 │ 1579.8864215469791 │ 211505 │ -└──────────┴──────────────────┴──────────┴──────────────────────┴───────────────┘ -Run Time: real 10.892 user 14.136000 sys 1.056000 -D SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(MobilePhoneModel) > 0 GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;e -┌──────────────────┬─────────┐ -│ MobilePhoneModel │ u │ -├──────────────────┼─────────┤ -│ iPad │ 1090347 │ -│ iPhone │ 45758 │ -│ A500 │ 16046 │ -│ N8-00 │ 5565 │ -│ iPho │ 3300 │ -│ ONE TOUCH 6030A │ 2759 │ -│ GT-P7300B │ 1907 │ -│ 3110000 │ 1871 │ -│ GT-I9500 │ 1598 │ -│ eagle75 │ 1492 │ -└──────────────────┴─────────┘ -Run Time: real 2.245 user 5.984000 sys 0.540000 -D SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(MobilePhoneModel) > 0 GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;, -┌─────────────┬──────────────────┬────────┐ -│ MobilePhone │ MobilePhoneModel │ u │ -├─────────────┼──────────────────┼────────┤ -│ 1 │ iPad │ 931038 │ -│ 5 │ iPad │ 48385 │ -│ 6 │ iPad │ 29710 │ -│ 7 │ iPad │ 28391 │ -│ 118 │ A500 │ 16005 │ -│ 6 │ iPhone │ 14516 │ -│ 26 │ iPhone │ 13566 │ -│ 10 │ iPad │ 11433 │ -│ 32 │ iPad │ 9503 │ -│ 13 │ iPad │ 9417 │ -└─────────────┴──────────────────┴────────┘ -Run Time: real 2.461 user 6.304000 sys 0.532000 -D SELECT SearchPhrase, count(*) AS c FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;+ -┌────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchPhrase │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 70263 │ -│ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 34675 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 24579 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 21649 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 19703 │ -│ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 19195 │ -│ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 17284 │ -│ galaxy table │ 16746 │ -│ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 16620 │ -│ \xD1\x81\xD0\xBA\xD0\xBE\xD0\xBB\xD1\x8C\xD0\xBA\xD0\xBE \xD0\xBC\xD1\x8B\xD1\x... │ 12317 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 1.045 user 14.536000 sys 0.016000 -D SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;m -┌────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchPhrase │ u │ -├────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 23673 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 19743 │ -│ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 18394 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 17553 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 14600 │ -│ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 14529 │ -│ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 14198 │ -│ \xD1\x81\xD0\xBA\xD0\xBE\xD0\xBB\xD1\x8C\xD0\xBA\xD0\xBE \xD0\xBC\xD1\x8B\xD1\x... │ 9007 │ -│ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 8792 │ -│ \xD0\xBA\xD0\xBE\xD0\xBC\xD0\xB1\xD0\xB8\xD0\xBD\xD0\xB8\xD1\x80\xD0\xBE\xD0\xB... │ 7572 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 14.072 user 18.988000 sys 1.104000 -D SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;m -┌────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchEngineID │ SearchPhrase │ c │ -├────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ 2 │ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 46258 │ -│ 2 │ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 18871 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 16905 │ -│ 3 │ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 16748 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 14911 │ -│ 2 │ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 13716 │ -│ 2 │ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 13414 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 13105 │ -│ 3 │ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 12815 │ -│ 2 │ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 11946 │ -└────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 1.297 user 18.044000 sys 0.072000 -D SELECT UserID, count(*) FROM hits GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -┌─────────────────────┬──────────────┐ -│ UserID │ count_star() │ -├─────────────────────┼──────────────┤ -│ 1313338681122956954 │ 29097 │ -│ 1907779576417363396 │ 25333 │ -│ 2305303682471783379 │ 10611 │ -│ 7982623143712728547 │ 7584 │ -│ 6018350421959114808 │ 6678 │ -│ 7280399273658728997 │ 6411 │ -│ 1090981537032625727 │ 6197 │ -│ 5730251990344211405 │ 6019 │ -│ 835157184735512989 │ 5211 │ -│ 770542365400669095 │ 4906 │ -└─────────────────────┴──────────────┘ -Run Time: real 1.002 user 12.864000 sys 0.004000 -D SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10;l -┌─────────────────────┬──────────────┬──────────────┐ -│ UserID │ SearchPhrase │ count_star() │ -├─────────────────────┼──────────────┼──────────────┤ -│ 1313338681122956954 │ │ 29097 │ -│ 1907779576417363396 │ │ 25333 │ -│ 2305303682471783379 │ │ 10611 │ -│ 7982623143712728547 │ │ 6669 │ -│ 7280399273658728997 │ │ 6408 │ -│ 1090981537032625727 │ │ 6196 │ -│ 5730251990344211405 │ │ 6019 │ -│ 6018350421959114808 │ │ 5990 │ -│ 835157184735512989 │ │ 5209 │ -│ 770542365400669095 │ │ 4906 │ -└─────────────────────┴──────────────┴──────────────┘ -Run Time: real 2.324 user 30.028000 sys 0.292000 -D SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -h┌─────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬──────────────┐ -│ UserID │ SearchPhrase │ count_star() │ -├─────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼──────────────┤ -│ 1032076697239282790 │ \xD0\xBA\xD0\xBE\xD0\xBB\xD0\xBF\xD0\xB8\xD0\xBD\xD0\xBE \xD0\xBC\xD0\xBE\xD0\x... │ 2 │ -│ 2291669706871953428 │ │ 1 │ -│ 962730767009225899 │ │ 4 │ -│ 2233663724416113367 │ \xD0\xB6\xD0\xB0\xD0\xBD\xD0\xBD\xD1\x8B\xD0\xB5 \xD0\xB8\xD0\xB3\xD1\x80\xD0\x... │ 2 │ -│ 933748801326509236 │ │ 9 │ -│ 439500307031004026 │ \xD0\xB2\xD1\x81\xD0\xB5\xD0\xBB\xD1\x8F\xD1\x8E\xD1\x89\xD0\xB8\xD0\xB9 \xD1\x... │ 1 │ -│ 5695875839448562 │ │ 1 │ -│ 7102459534329662426 │ │ 3 │ -│ 1468681479646296604 │ \xD1\x81\xD0\xBC\xD0\xB5\xD1\x81\xD0\xB8\xD1\x82\xD0\xB5\xD1\x82 \xD0\xBC\xD0\x... │ 1 │ -│ 737933678636519547 │ │ 22 │ -└─────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴──────────────┘ -Run Time: real 1.908 user 29.032000 sys 0.268000 -D SELECT UserID, extract(minute FROM (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) AS m, SearchPhrase, count(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10;) -┌─────────────────────┬────┬──────────────┬──────────────┐ -│ UserID │ m │ SearchPhrase │ count_star() │ -├─────────────────────┼────┼──────────────┼──────────────┤ -│ 1313338681122956954 │ 31 │ │ 589 │ -│ 1313338681122956954 │ 28 │ │ 578 │ -│ 1313338681122956954 │ 29 │ │ 572 │ -│ 1313338681122956954 │ 33 │ │ 567 │ -│ 1313338681122956954 │ 27 │ │ 557 │ -│ 1313338681122956954 │ 32 │ │ 554 │ -│ 1313338681122956954 │ 30 │ │ 552 │ -│ 1313338681122956954 │ 34 │ │ 546 │ -│ 1313338681122956954 │ 26 │ │ 540 │ -│ 1313338681122956954 │ 10 │ │ 539 │ -└─────────────────────┴────┴──────────────┴──────────────┘ -Run Time: real 4.255 user 50.680000 sys 1.116000 -D SELECT UserID FROM hits WHERE UserID = 12345678901234567890;L -Run Time: real 0.258 user 0.260000 sys 0.000000 -D SELECT count(*) FROM hits WHERE URL::TEXT LIKE '%metrika%';e -┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 211 │ -└──────────────┘ -Run Time: real 2.665 user 42.256000 sys 0.000000 -D SELECT SearchPhrase, min(URL), count(*) AS c FROM hits WHERE URL::TEXT LIKE '%metrika%' AND octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;u -┌────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───┐ -│ SearchPhrase │ min(url) │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───┤ -│ \xD0\xB8\xD1\x89\xD1\x83 \xD1\x80\xD0\xB0\xD0\xB1\xD0\xBE\xD1\x82\xD0\xB0 \xD0\... │ http://smeshariki.ru/index.ua/advanceForce.1primea/forum=1&district=25586/page4... │ 1 │ -│ \xD1\x82\xD0\xB5\xD0\xBA\xD1\x81\xD1\x82\xD1\x8B \xD0\xBF\xD0\xBE\xD0\xB4\xD1\x... │ http://bdsmpeople.ru/index.ru/metrikadeletedAutoSearch │ 1 │ -│ \xD0\xBE\xD0\xBA\xD0\xB5\xD0\xB0\xD0\xBD\xD1\x81\xD0\xBA\xD0\xBE\xD0\xB5 \xD1\x... │ http://bdsmpeople.ru/index.ru/metrikadeletedAutoSearch │ 1 │ -│ \xD1\x80\xD0\xB5\xD1\x86\xD0\xB5\xD0\xBF\xD1\x82 │ http://smeshariki.ru/index.ua/search/metrikanske-urali-belorussia/28.html?1=1&c... │ 1 │ -│ \xD0\xB4\xD0\xB8\xD0\xBD\xD0\xB0 \xD0\xB2\xD0\xB0\xD0\xBA\xD0\xB0\xD0\xBD\xD1\x... │ http://ecrn.ru/personal/gost277572,9589&pt=b&pd=8&pw=2&page3/?state=0&damages/0... │ 1 │ -│ \xD1\x82\xD1\x80\xD1\x83\xD0\xB4\xD0\xB0 \xD0\xB4\xD0\xBB\xD1\x8F \xD0\xB2\xD1\... │ http://smeshariki.ru/recipes/sadovka.ru/gorod55.ru/search?text=\xD0\xB8\xD0\xB7... │ 1 │ -│ \xD0\xBC\xD1\x8B\xD1\x81\xD0\xBB \xD0\xBF\xD0\xB5\xD1\x81\xD0\xB5\xD0\xBD \xD1\... │ http://smeshariki.ru/a-folder=cmarka=15&sort=&sll=36872/metrika/frl-2/bage-all/... │ 1 │ -│ \xD0\xB2\xD0\xB8\xD0\xB4\xD0\xB5\xD0\xBE \xD1\x82\xD0\xB0\xD0\xBA\xD0\xBE\xD0\x... │ http://maximum_438424&pvno=2&evlg=VC,0;VL,628;IC,1653-82676212&op_page=0&door=0... │ 1 │ -│ \xD1\x80\xD0\xB0\xD0\xB1\xD0\xBE\xD0\xB2\xD1\x8B\xD0\xB5 \xD0\xB4\xD0\xBE\xD1\x... │ http://smeshariki.ru/index.ua/auto.ria.ua/change=0&s_yers=0&with_video.yandex.u... │ 1 │ -│ \xD1\x83\xD0\xBD\xD0\xB8\xD0\xB2\xD0\xB5\xD1\x80\xD0\xBC\xD0\xB5\xD0\xB4\xD0\xB... │ http://smeshariki.ru/index.ua/syllanet.ru/busine-tyazin?model=4878/page=10&cate... │ 1 │ -└────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───┘ -Run Time: real 0.721 user 11.368000 sys 0.000000 -D count(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title::TEXT LIKE '%Яндекс%' AND URL::TEXT NOT LIKE '%.yandex.%' AND octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;( -Run Time: real 8.537 user 19.856000 sys 1.200000 -D SELECT * FROM hits WHERE URL::TEXT LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -┌─────────────────────┬────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┬────────────┬───────────┬───────────┬────────────┬──────────┬─────────────────────┬──────────────┬─────┬───────────┬────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬─────────┬───────────────────┬─────────────────┬───────────────┬─────────────┬─────────────────┬──────────────────┬─────────────────┬────────────┬────────────┬─────────────┬──────────┬──────────┬────────────────┬────────────────┬──────────────┬──────────────────┬──────────┬─────────────┬──────────────────┬────────┬─────────────┬────────────────┬────────────────┬──────────────┬─────────────┬─────────────┬───────────────────┬────────────────────┬────────────────┬─────────────────┬─────────────────────┬─────────────────────┬─────────────────────┬─────────────────────┬──────────────────────┬─────────────┬────────┬────────────┬─────────────┬─────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┬──────────────┬─────────┬─────────────┬───────────────┬──────────┬──────────┬────────────────┬─────┬─────┬────────┬───────────┬───────────┬────────────┬────────────┬────────────┬───────────────┬─────────────────┬────────────────┬───────────────┬──────────────┬───────────┬────────────┬───────────┬───────────────┬─────────────────────┬───────────────────┬─────────────┬───────────────────────┬──────────────────┬────────────┬──────────────┬───────────────┬─────────────────┬─────────────────────┬────────────────────┬──────────────┬──────────────────┬───────────┬───────────┬─────────────┬────────────┬─────────┬─────────┬──────────┬──────────────────────┬──────────────────────┬──────┐ -│ WatchID │ JavaEnable │ Title │ GoodEvent │ EventTime │ EventDate │ CounterID │ ClientIP │ RegionID │ UserID │ CounterClass │ OS │ UserAgent │ URL │ Referer │ Refresh │ RefererCategoryID │ RefererRegionID │ URLCategoryID │ URLRegionID │ ResolutionWidth │ ResolutionHeight │ ResolutionDepth │ FlashMajor │ FlashMinor │ FlashMinor2 │ NetMajor │ NetMinor │ UserAgentMajor │ UserAgentMinor │ CookieEnable │ JavascriptEnable │ IsMobile │ MobilePhone │ MobilePhoneModel │ Params │ IPNetworkID │ TraficSourceID │ SearchEngineID │ SearchPhrase │ AdvEngineID │ IsArtifical │ WindowClientWidth │ WindowClientHeight │ ClientTimeZone │ ClientEventTime │ SilverlightVersion1 │ SilverlightVersion2 │ SilverlightVersion3 │ SilverlightVersion4 │ PageCharset │ CodeVersion │ IsLink │ IsDownload │ IsNotBounce │ FUniqID │ OriginalURL │ HID │ IsOldCounter │ IsEvent │ IsParameter │ DontCountHits │ WithHash │ HitColor │ LocalEventTime │ Age │ Sex │ Income │ Interests │ Robotness │ RemoteIP │ WindowName │ OpenerName │ HistoryLength │ BrowserLanguage │ BrowserCountry │ SocialNetwork │ SocialAction │ HTTPError │ SendTiming │ DNSTiming │ ConnectTiming │ ResponseStartTiming │ ResponseEndTiming │ FetchTiming │ SocialSourceNetworkID │ SocialSourcePage │ ParamPrice │ ParamOrderID │ ParamCurrency │ ParamCurrencyID │ OpenstatServiceName │ OpenstatCampaignID │ OpenstatAdID │ OpenstatSourceID │ UTMSource │ UTMMedium │ UTMCampaign │ UTMContent │ UTMTerm │ FromTag │ HasGCLID │ RefererHash │ URLHash │ CLID │ -├─────────────────────┼────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┼────────────┼───────────┼───────────┼────────────┼──────────┼─────────────────────┼──────────────┼─────┼───────────┼────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼─────────┼───────────────────┼─────────────────┼───────────────┼─────────────┼─────────────────┼──────────────────┼─────────────────┼────────────┼────────────┼─────────────┼──────────┼──────────┼────────────────┼────────────────┼──────────────┼──────────────────┼──────────┼─────────────┼──────────────────┼────────┼─────────────┼────────────────┼────────────────┼──────────────┼─────────────┼─────────────┼───────────────────┼────────────────────┼────────────────┼─────────────────┼─────────────────────┼─────────────────────┼─────────────────────┼─────────────────────┼──────────────────────┼─────────────┼────────┼────────────┼─────────────┼─────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┼──────────────┼─────────┼─────────────┼───────────────┼──────────┼──────────┼────────────────┼─────┼─────┼────────┼───────────┼───────────┼────────────┼────────────┼────────────┼───────────────┼─────────────────┼────────────────┼───────────────┼──────────────┼───────────┼────────────┼───────────┼───────────────┼─────────────────────┼───────────────────┼─────────────┼───────────────────────┼──────────────────┼────────────┼──────────────┼───────────────┼─────────────────┼─────────────────────┼────────────────────┼──────────────┼──────────────────┼───────────┼───────────┼─────────────┼────────────┼─────────┼─────────┼──────────┼──────────────────────┼──────────────────────┼──────┤ -│ 9008228978173248400 │ 1 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372714203 │ 15888 │ 46429 │ 1741285710 │ 208 │ 727970985383478408 │ 0 │ 2 │ 2 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ http://smeshariki.ru/page%3D%26ev_n%3Dtvor_3_4 │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 1638 │ 1658 │ 37 │ 14 │ 0 │ 700.22 │ 0 │ 0 │ 15 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 2349927 │ -1 │ 0 │ │ 0 │ 0 │ 1844 │ 888 │ 135 │ 1372711075 │ 3 │ 0 │ 29241 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 0 │ │ 926364569 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372755313 │ 31 │ 1 │ 0 │ 0 │ 0 │ 1985697352 │ 13001 │ -1 │ 22 │ vG │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 16989158750233735299 │ 2619784076535420345 │ 0 │ -│ 7319686668886932386 │ 0 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372742048 │ 15888 │ 46429 │ 675534320 │ 208 │ 1167829038705361511 │ 0 │ 45 │ 1 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ http://smeshariki.ru/page=6&sqi=2&ved=0CC0QFjAA │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 280 │ 733 │ 0 │ 0 │ 0 │ │ 0 │ 0 │ 4 │ sO │ 1 │ 1 │ 1 │ 0 │ │ │ 1444853 │ -1 │ 0 │ │ 0 │ 0 │ 997 │ 800 │ 623 │ 1372713898 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 4655941889681510909 │ │ 721190565 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372747193 │ 31 │ 2 │ 3 │ 717 │ 0 │ 2935426383 │ 22084 │ -1 │ 1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 11629579472615090596 │ 2619784076535420345 │ 0 │ -│ 5650132582613869356 │ 0 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372742083 │ 15888 │ 46429 │ 675534320 │ 208 │ 1167829038705361511 │ 0 │ 45 │ 1 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ http://smeshariki.ru/page=6&sqi=2&ved=0CC0QFjAA │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 280 │ 733 │ 0 │ 0 │ 0 │ │ 0 │ 0 │ 4 │ D\xE0 │ 1 │ 1 │ 1 │ 0 │ │ │ 1444853 │ -1 │ 0 │ │ 0 │ 0 │ 997 │ 800 │ 623 │ 1372713960 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 4655941889681510909 │ │ 721255326 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372747245 │ 31 │ 2 │ 3 │ 717 │ 0 │ 2935426383 │ 64225 │ -1 │ 1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 11629579472615090596 │ 2619784076535420345 │ 0 │ -│ 6211862319117980382 │ 0 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372744794 │ 15888 │ 46429 │ 1384976586 │ 208 │ 45205883125357727 │ 0 │ 8 │ 1 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ │ 0 │ 0 │ 0 │ 9911 │ 216 │ 475 │ 368 │ 0 │ 0 │ 0 │ │ 0 │ 0 │ 7 │ D\xE0 │ 1 │ 1 │ 1 │ 0 │ │ │ 3195177 │ 0 │ 0 │ │ 0 │ 0 │ 412 │ 286 │ 135 │ 1372732810 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 7015666733105959947 │ │ 641313121 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372774931 │ 28 │ 1 │ 2 │ 1 │ 0 │ 1889444196 │ 44835 │ -1 │ 0 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 18150585289071012696 │ 2619784076535420345 │ 0 │ -│ 8127216893234548048 │ 0 │ \xD0\xA2\xD0\xBE\xD0\xBB\xD1\x81\xD1\x82\xD1\x83\xD1\x85\xD1\x83, \xD0\xB5\xD1\... │ 1 │ 1372752539 │ 15888 │ 46429 │ 1237806922 │ 208 │ 750085446646698731 │ 0 │ 2 │ 3 │ http://smeshariki.ru/used/99114578622-14406072,9269587/roometrika/?page=7&pw=2&... │ http://smeshariki.ru/page=11956bd3f5ba-bolshiient │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 1081 │ 979 │ 23 │ 15 │ 7 │ 700 │ 0 │ 0 │ 17 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 2914423 │ -1 │ 0 │ │ 0 │ 0 │ 1434 │ 739 │ 623 │ 1372781865 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 5496132257355632424 │ │ 983272850 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372760187 │ 50 │ 2 │ 2 │ 60 │ 18 │ 1113465640 │ 60610 │ -1 │ 29 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 194 │ 1631 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 13570011243001147277 │ 11339053197878713733 │ 0 │ -│ 5016976626924998727 │ 1 │ Toyota \xD1\x80\xD1\x83\xD0\xBA\xD0\xB0\xD0\xBB\xD1\x8B \xD0\xB4\xD0\xBB\xD1\x8... │ 1 │ 1372758784 │ 15888 │ 5645 │ 3888153915 │ 107 │ 4058808124307537573 │ 1 │ 2 │ 88 │ http:%2F%2Fwwww.bonprix.ru&pvid=131&op_products/transformality.pulsceness/01-me... │ http://forum.amur.info/node/12451180167540 │ 0 │ 10868 │ 635 │ 16361 │ 943 │ 1638 │ 777 │ 37 │ 15 │ 7 │ 700 │ 0 │ 0 │ 1 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 2726799 │ -1 │ 0 │ │ 0 │ 0 │ 1509 │ 618 │ 135 │ 1372835656 │ 0 │ 0 │ 0 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 6227398498798751865 │ │ 255959698 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372763371 │ 31 │ 1 │ 3 │ 3555 │ 13 │ 4006630121 │ -1 │ -1 │ -1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 10333931216342739575 │ 10009133566342666602 │ 0 │ -│ 4963765960745323978 │ 0 │ \xD0\x93\xD0\x90\xD0\x97 (ZAZ) 5403J \xE2\x80\x93 \xD0\xA4\xD0\xB8\xD0\xBB\xD1\... │ 1 │ 1372793626 │ 15888 │ 46429 │ 2063319617 │ 23967 │ 5283184799411504286 │ 0 │ 44 │ 2 │ http://smeshariki.ru/userId=0&matched_car=Volkswagency=1&p=8090/roometrika/?pag... │ http://smeshariki.ru/smsarhiv/num/33363634383294&lr=66&v6s=2&bodystyle │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 1996 │ 1781 │ 37 │ 15 │ 7 │ 700 │ 0 │ 0 │ 10 │ nA │ 1 │ 1 │ 0 │ 0 │ │ │ 4110783 │ -1 │ 0 │ │ 0 │ 0 │ 1551 │ 955 │ 291 │ 1372730435 │ 7 │ 1 │ 9577 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 8904020920948834668 │ │ 883138215 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372777308 │ 22 │ 2 │ 2 │ 11339 │ 18 │ 2136940575 │ 1653 │ -1 │ 1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 18276268115953212999 │ 9484754903086635093 │ 0 │ -│ 6284148982888572412 │ 1 │ Morskoj port \xE2\x80\x93 \xD0\x9D\xD0\xBE\xD0\xB2\xD0\xBE\xD1\x81\xD1\x82\xD1\... │ 1 │ 1372795222 │ 15889 │ 3922 │ 2460998382 │ 40 │ 1643466856862289966 │ 1 │ 44 │ 7 │ http://moikrug.ru/message-12-kak-pravdorubovnik_metrika-info.php?f=23&prr │ http://diary.ru/GameMain.aspx?d=1412&lr=75&mode=photo/login=igorod.irr.ru/i6102... │ 0 │ 306 │ 29199 │ 92 │ 247 │ 1368 │ 554 │ 37 │ 15 │ 7 │ 800.94 │ 0 │ 0 │ 12 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 1166094 │ 1 │ 0 │ │ 0 │ 0 │ 1253 │ 642 │ 135 │ 1372799147 │ 4 │ 1 │ 16561 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 7061143530822060136 │ │ 699865379 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372810401 │ 31 │ 2 │ 3 │ 6 │ 0 │ 3849445958 │ -1 │ -1 │ -1 │ S0 │ h1 │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 14060834305267311691 │ 1791983733815963315 │ 0 │ -│ 6024648629746505393 │ 0 │ \xD0\x9F\xD0\xBE\xD0\xB3\xD1\x80\xD0\xB5\xD0\xB2\xD0\xB0\xD1\x82\xD0\xB5\xD0\xB... │ 1 │ 1372795271 │ 15889 │ 1200 │ 2932550360 │ 208 │ 1578473929930714515 │ 1 │ 107 │ 82 │ http://afisha.yandex.ru/real-estate/out-of-town/household_app_metrika.ru/wildbe... │ http://ad.adrive_type_id=1959251&stUrl%3Dpopular/kw/306161&lr=1418][to]=&input_... │ 0 │ 15265 │ 19757 │ 8953 │ 32252 │ 958 │ 1871 │ 37 │ 0 │ 0 │ │ 0 │ 0 │ 3 │ D\xE0 │ 1 │ 1 │ 1 │ 6 │ │ │ 4010641 │ -1 │ 0 │ │ 0 │ 0 │ 521 │ 1803 │ -1 │ 1372870507 │ 0 │ 0 │ 0 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 0 │ http://b.kavanga.ru/tags/%FD%ED%FB%E2%F3%E6%E5%E6%ED%E8%ED%F1%FF%ED&ti=%D0%BD%D... │ 120241233 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372842094 │ 0 │ 0 │ 0 │ 0 │ 0 │ 3975911785 │ -1 │ -1 │ -1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 9685011611637290144 │ 5941547189901908071 │ 0 │ -│ 4857420640250996887 │ 1 │ Suzuki Escort 4* (\xD0\xA4\xD1\x80\xD0\xB0\xD0\xBD\xD1\x86\xD1\x8B 3 \xD1\x81\x... │ 1 │ 1372800737 │ 15889 │ 3922 │ 3201307115 │ 190 │ 7335986183190726964 │ 1 │ 44 │ 88 │ http://pova-ul-mir.irr.ru/search.php?gidcar=37408&uuid=1&bc=3&city=0&pv=20&s_ye... │ http://news/238/~6/?cauth=1&av=1&nm=1&ms=1,2/currency=RUR/page_num_id=15366563&... │ 0 │ 306 │ 144 │ 304 │ 132 │ 1368 │ 554 │ 37 │ 15 │ 4 │ 202 │ 0 │ 0 │ 1 │ fi │ 1 │ 1 │ 0 │ 0 │ │ │ 2311071 │ -1 │ 0 │ │ 0 │ 0 │ 1333 │ 924 │ 322 │ 1372840359 │ 4 │ 1 │ 16561 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 7659179697273795837 │ │ 232010762 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372866397 │ 31 │ 2 │ 2 │ 3658 │ 28 │ 2920265313 │ -1 │ -1 │ -1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 8839345929686869081 │ 5449259806403761803 │ 0 │ -└─────────────────────┴────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┴────────────┴───────────┴───────────┴────────────┴──────────┴─────────────────────┴──────────────┴─────┴───────────┴────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴─────────┴───────────────────┴─────────────────┴───────────────┴─────────────┴─────────────────┴──────────────────┴─────────────────┴────────────┴────────────┴─────────────┴──────────┴──────────┴────────────────┴────────────────┴──────────────┴──────────────────┴──────────┴─────────────┴──────────────────┴────────┴─────────────┴────────────────┴────────────────┴──────────────┴─────────────┴─────────────┴───────────────────┴────────────────────┴────────────────┴─────────────────┴─────────────────────┴─────────────────────┴─────────────────────┴─────────────────────┴──────────────────────┴─────────────┴────────┴────────────┴─────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┴──────────────┴─────────┴─────────────┴───────────────┴──────────┴──────────┴────────────────┴─────┴─────┴────────┴───────────┴───────────┴────────────┴────────────┴────────────┴───────────────┴─────────────────┴────────────────┴───────────────┴──────────────┴───────────┴────────────┴───────────┴───────────────┴─────────────────────┴───────────────────┴─────────────┴───────────────────────┴──────────────────┴────────────┴──────────────┴───────────────┴─────────────────┴─────────────────────┴────────────────────┴──────────────┴──────────────────┴───────────┴───────────┴─────────────┴────────────┴─────────┴─────────┴──────────┴──────────────────────┴──────────────────────┴──────┘ -Run Time: real 72.104 user 135.356000 sys 10.484000 -D SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY EventTime LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC \xD0\xBD\xD0\xB5\xD0\xB1\xD0\xBE\xD0\x... │ -│ galaxy s4 zoom \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC │ -│ \xD0\xBD\xD0\xBE\xD1\x87\xD0\xBD\xD0\xBE \xD0\xBA\xD0\xB8\xD1\x82\xD0\xB0\xD1\x... │ -│ \xD1\x81\xD0\xB8\xD0\xBC\xD0\xBF\xD1\x82\xD0\xBE\xD0\xBC\xD1\x8B \xD1\x80\xD0\x... │ -│ \xD0\xBE\xD1\x82\xD0\xB4\xD1\x8B\xD1\x85\xD0\xB0 \xD1\x87\xD0\xB5\xD0\xBC \xD0\... │ -│ \xD0\xBA\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ -│ \xD0\xB0\xD0\xB2\xD0\xBE\xD0\xBC \xD0\xBA\xD0\xBE\xD0\xBD\xD1\x81\xD1\x82\xD0\x... │ -│ \xD1\x81\xD0\xBB\xD0\xBE\xD0\xBD.\xD1\x80\xD1\x83\xD0\xB1., \xD0\xB4. \xD0\xB0.... │ -│ \xD1\x80\xD0\xB0\xD1\x81\xD0\xBF\xD0\xB8\xD1\x81\xD0\xB0\xD0\xBD\xD0\xB8\xD0\xB... │ -│ \xD0\xB0\xD0\xBD\xD0\xB0\xD0\xBF\xD0\xB0 \xD0\xBE\xD0\xBF\xD0\xB5\xD1\x80\xD0\x... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 0.232 user 3.520000 sys 0.132000 -D SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY SearchPhrase LIMIT 10;e -┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ ! hektdf gjcgjhn conster │ -│ ! \xD1\x81\xD0\xBA\xD0\xB0\xD1\x80\xD0\xBF │ -│ !(\xD0\xBA\xD0\xB0\xD0\xBA \xD0\xB2\xD0\xBE\xD1\x80\xD0\xBE\xD0\xBD\xD0\xB8 │ -│ !(\xD0\xBF\xD0\xBE \xD0\xB3\xD0\xBE\xD1\x80\xD0\xB8\xD1\x8E \xD0\xB2 \xD1\x8F\x... │ -│ !(\xD1\x81) \xD0\xBF\xD1\x80\xD0\xBE \xD0\xB4\xD0\xBF\xD0\xBE \xD1\x81\xD0\xB5\... │ -│ !(\xD1\x81\xD0\xB0\xD0\xBB\xD0\xBE\xD0\xBD\xD1\x8B \xD0\xBE\xD1\x81\xD1\x82\xD0... │ -│ !(\xD1\x81\xD1\x82\xD0\xB0\xD1\x80\xD1\x82\xD0\xB5\xD1\x80 rav4 \xD1\x82\xD1\x8... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD0\xB4\xD0\xBB\xD1\x8F \xD0\xBC\xD1\x8F\xD1... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD0\xBC\xD0\xB0\xD1\x83\xD1\x81 \xD0\xBA\xD0... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD1\x81\xD0\xB5\xD1\x80\xD0\xB8\xD0\xB8 │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 0.248 user 3.696000 sys 0.012000 -D SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY EventTime, SearchPhrase LIMIT 10; -D┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ galaxy s4 zoom \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC │ -│ \xD0\xBD\xD0\xBE\xD1\x87\xD0\xBD\xD0\xBE \xD0\xBA\xD0\xB8\xD1\x82\xD0\xB0\xD1\x... │ -│ \xD1\x81\xD0\xB8\xD0\xBC\xD0\xBF\xD1\x82\xD0\xBE\xD0\xBC\xD1\x8B \xD1\x80\xD0\x... │ -│ \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC \xD0\xBD\xD0\xB5\xD0\xB1\xD0\xBE\xD0\x... │ -│ \xD0\xB0\xD0\xB2\xD0\xBE\xD0\xBC \xD0\xBA\xD0\xBE\xD0\xBD\xD1\x81\xD1\x82\xD0\x... │ -│ \xD0\xB0\xD0\xBD\xD0\xB0\xD0\xBF\xD0\xB0 \xD0\xBE\xD0\xBF\xD0\xB5\xD1\x80\xD0\x... │ -│ \xD0\xB1\xD1\x80\xD0\xB8\xD1\x82\xD0\xB0 \xD0\xB3\xD0\xB0\xD0\xBD\xD0\xB0\xD0\x... │ -│ \xD0\xBA\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ -│ \xD0\xBE\xD1\x82\xD0\xB4\xD1\x8B\xD1\x85\xD0\xB0 \xD1\x87\xD0\xB5\xD0\xBC \xD0\... │ -│ \xD1\x80\xD0\xB0\xD1\x81\xD0\xBF\xD0\xB8\xD1\x81\xD0\xB0\xD0\xBD\xD0\xB8\xD0\xB... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 0.231 user 3.648000 sys 0.008000 -D SELECT CounterID, avg(octet_length(URL)) AS l, count(*) AS c FROM hits WHERE octet_length(URL) > 0 GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;L -┌───────────┬────────────────────┬─────────┐ -│ CounterID │ l │ c │ -├───────────┼────────────────────┼─────────┤ -│ 233773 │ 469.18537326484886 │ 2938865 │ -│ 245438 │ 271.7892512777364 │ 2510103 │ -│ 122612 │ 238.64530987208474 │ 3574007 │ -│ 234004 │ 204.28793262381632 │ 238660 │ -│ 1634 │ 197.83321731651554 │ 323229 │ -│ 786 │ 186.75537634408602 │ 120528 │ -│ 114157 │ 142.91881538575285 │ 216408 │ -│ 515 │ 126.22860040706026 │ 146907 │ -│ 256004 │ 125.37108455074805 │ 858171 │ -│ 95427 │ 120.26856903175477 │ 374306 │ -│ 199550 │ 109.81720498866335 │ 7115413 │ -│ 220992 │ 105.85666196266179 │ 494614 │ -│ 196239 │ 98.34882201749727 │ 163797 │ -│ 62 │ 93.15981711034343 │ 738150 │ -│ 96948 │ 92.74321182146618 │ 396093 │ -│ 188878 │ 91.98308322489247 │ 311998 │ -│ 249603 │ 91.88026594639518 │ 120325 │ -│ 3922 │ 87.83856410684609 │ 8527069 │ -│ 191697 │ 86.95776647628826 │ 124664 │ -│ 97467 │ 84.2953696503987 │ 131178 │ -│ 186300 │ 83.97258027738701 │ 802561 │ -│ 146891 │ 77.77430173504756 │ 605286 │ -│ 38 │ 76.43757015971798 │ 507770 │ -│ 230962 │ 76.3127707226559 │ 169223 │ -│ 77639 │ 75.38681923602442 │ 253961 │ -└───────────┴────────────────────┴─────────┘ -Run Time: real 0.439 user 6.632000 sys 0.300000 -D //(?:www\.)?([^/]+)/.*$', '\1') AS key, avg(octet_length(Referer)) AS l, count(*) AS c, min(Referer) FROM hits WHERE octet_length(Referer) > 0 GROUP BY key HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -┌────────────────────────────────────────────────────────────────────────┬────────────────────┬─────────┬────────────────────────────────────────────────────────────────────────────────────┐ -│ key │ l │ c │ min(referer) │ -├────────────────────────────────────────────────────────────────────────┼────────────────────┼─────────┼────────────────────────────────────────────────────────────────────────────────────┤ -│ svpressa.ru │ 307.980979437341 │ 242527 │ http://svpressa.ru/ │ -│ msuzie-showforumdisplay │ 263.327228380409 │ 183676 │ http://msuzie-showforumdisplay/63/~2/?name=&cost_neu%3D400%26retpath=default777... │ -│ saint-peters-total=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 80 │ 242.5236948271821 │ 200529 │ http://saint-peters-total=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 80/c... │ -│ domics │ 212.92990978061542 │ 326094 │ http://domics/825179.11931861234499792 │ -│ e96.ru │ 210.09628206687884 │ 1019276 │ http://e96.ru/%3Ffrom]=&input_act[count_num=0&dff=arian-carrina1201517&cad=rjt&... │ -│ gadgets.irr.ru │ 131.9597008950273 │ 349710 │ https://gadgets.irr.ru/2jmj7l5rSw0yVb │ -│ google.ru │ 109.24485253818524 │ 2158491 │ http://google.ru/ │ -│ go.mail │ 108.63930572737723 │ 8228007 │ http://go.mail/04/detskaia-moda-zhiensmed │ -│ msouz.ru │ 106.10887286512423 │ 301774 │ http://msouz.ru/?ffshop │ -│ state=19945206 │ 105.6469046513171 │ 512414 │ http://state=19945206/foto-4/login%20NoTs3M&where=all&filmId=u8aGGqtWs3M&where=... │ -│ loveplanet.ru │ 104.60136383347789 │ 461200 │ http://loveplanet.ru/%3Faw_opel/page=2013 │ -│ bonprix.ru │ 104.41683309557774 │ 1125105 │ http://bonprix.ru/ │ -│ novjob.ru │ 96.75331644732393 │ 133049 │ http://novjob.ru/ │ -│ cn.ru │ 95.63198716663325 │ 124675 │ http://cn.ru/GameMain.aspx#catalog/100523&tails.xml?market_pc.html?pid=9403&lr=... │ -│ geomethiettai.ru │ 94.78816556817006 │ 115916 │ https://geomethiettai.ru/GameMain.aspx?group=houses/list=266559j7077&num=7&prun... │ -│ kino │ 90.27628829938655 │ 120139 │ http://kino/6/21/2/women.asp?whichpage4/#oversion=unreadm&uid │ -│ yaroslavens.ru │ 90.17077281117085 │ 124610 │ http://yaroslavens.ru/main.aspx#catalog%2F1004-1100000147-otvet/actions/disloca... │ -│ mysw.info │ 89.68684313159915 │ 984596 │ http://mysw.info/ │ -│ m.myloveplanet.ru │ 88.73233749439181 │ 151564 │ http://m.myloveplanet.ru/ │ -│ povarenok.ru │ 83.97395952020882 │ 144813 │ http://povarenok.ru/ │ -│ gorod │ 80.33107253811141 │ 110728 │ http://gorod/%3Fauto.ria.ua%2Fjob │ -│ yandsearch │ 80.21664430621621 │ 245970 │ http://www.yandsearch/rooms=1/page2 │ -│ myloveplanet.ru │ 80.08183067768715 │ 110582 │ http://myloveplanet.ru/#associety/auto │ -│ tambov.irr.ru │ 77.8650188064113 │ 315318 │ http://tambov.irr.ru/0/c1/tgFtaeLDK0yb01A7xvQF08sjCFqQxn51 │ -│ kurortmag.ru │ 75.74958779884584 │ 155264 │ http://kurortmag.ru/ │ -└────────────────────────────────────────────────────────────────────────┴────────────────────┴─────────┴────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 119.865 user 1866.652000 sys 4.096000 -D + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits; -┌──────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┐ -│ 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) │ -├──────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┤ -│ 151348908394 │ 151448908394 │ 151548908394 │ 151648908394 │ 151748908394 │ 151848908394 │ 151948908394 │ 152048908394 │ 152148908394 │ 152248908394 │ 152348908394 │ 152448908394 │ 152548908394 │ 152648908394 │ 152748908394 │ 152848908394 │ 152948908394 │ 153048908394 │ 153148908394 │ 153248908394 │ 153348908394 │ 153448908394 │ 153548908394 │ 153648908394 │ 153748908394 │ 153848908394 │ 153948908394 │ 154048908394 │ 154148908394 │ 154248908394 │ 154348908394 │ 154448908394 │ 154548908394 │ 154648908394 │ 154748908394 │ 154848908394 │ 154948908394 │ 155048908394 │ 155148908394 │ 155248908394 │ 155348908394 │ 155448908394 │ 155548908394 │ 155648908394 │ 155748908394 │ 155848908394 │ 155948908394 │ 156048908394 │ 156148908394 │ 156248908394 │ 156348908394 │ 156448908394 │ 156548908394 │ 156648908394 │ 156748908394 │ 156848908394 │ 156948908394 │ 157048908394 │ 157148908394 │ 157248908394 │ 157348908394 │ 157448908394 │ 157548908394 │ 157648908394 │ 157748908394 │ 157848908394 │ 157948908394 │ 158048908394 │ 158148908394 │ 158248908394 │ 158348908394 │ 158448908394 │ 158548908394 │ 158648908394 │ 158748908394 │ 158848908394 │ 158948908394 │ 159048908394 │ 159148908394 │ 159248908394 │ 159348908394 │ 159448908394 │ 159548908394 │ 159648908394 │ 159748908394 │ 159848908394 │ 159948908394 │ 160048908394 │ 160148908394 │ 160248908394 │ -└──────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┘ -Run Time: real 6.176 user 97.964000 sys 0.000000 -D SELECT SearchEngineID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -┌────────────────┬────────────┬──────┬──────────────┬──────────────────────┐ -│ SearchEngineID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├────────────────┼────────────┼──────┼──────────────┼──────────────────────┤ -│ 2 │ 1138507705 │ 1633 │ 35 │ 1408.0122473974282 │ -│ 2 │ 1740861572 │ 1331 │ 28 │ 1577.945905334335 │ -│ 2 │ 3487820196 │ 1144 │ 35 │ 1553.1984265734266 │ -│ 2 │ 3797060577 │ 1140 │ 36 │ 1543.4140350877192 │ -│ 2 │ 2349209741 │ 1105 │ 30 │ 1557.387330316742 │ -│ 2 │ 2424344199 │ 1102 │ 31 │ 1555.6588021778584 │ -│ 2 │ 3663904793 │ 1083 │ 31 │ 1581.8171745152354 │ -│ 2 │ 3829154130 │ 1082 │ 30 │ 1541.253234750462 │ -│ 2 │ 2551371145 │ 1080 │ 24 │ 1559.8092592592593 │ -│ 2 │ 4029049820 │ 1058 │ 32 │ 1556.2003780718337 │ -└────────────────┴────────────┴──────┴──────────────┴──────────────────────┘ -Run Time: real 0.991 user 13.144000 sys 0.080000 -D SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -┌─────────────────────┬────────────┬───┬──────────────┬──────────────────────┐ -│ WatchID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├─────────────────────┼────────────┼───┼──────────────┼──────────────────────┤ -│ 8425973212606442266 │ 3882647820 │ 2 │ 0 │ 1368.0 │ -│ 4822815447136935941 │ 4109934802 │ 2 │ 0 │ 1638.0 │ -│ 5851512534509153320 │ 3968091174 │ 2 │ 0 │ 1368.0 │ -│ 7542988325649023791 │ 303701440 │ 2 │ 0 │ 1828.0 │ -│ 6992742809870752121 │ 3979611525 │ 2 │ 0 │ 100.0 │ -│ 8469206111441108042 │ 1485652260 │ 2 │ 0 │ 1368.0 │ -│ 8268832962994556606 │ 1627418068 │ 2 │ 0 │ 1638.0 │ -│ 6959397072495903875 │ 1568492334 │ 2 │ 0 │ 1368.0 │ -│ 8672760597587433971 │ 1269590216 │ 2 │ 0 │ 1368.0 │ -│ 6637243485645608094 │ 1468144163 │ 2 │ 0 │ 1917.0 │ -└─────────────────────┴────────────┴───┴──────────────┴──────────────────────┘ -Run Time: real 1.552 user 16.744000 sys 0.356000 -D SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -┌─────────────────────┬────────────┬───┬──────────────┬──────────────────────┐ -│ WatchID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├─────────────────────┼────────────┼───┼──────────────┼──────────────────────┤ -│ 4784371382202293288 │ 3735345744 │ 2 │ 0 │ 1638.0 │ -│ 6711685491586624976 │ 1408513058 │ 2 │ 0 │ 1638.0 │ -│ 7162529939784944130 │ 3932318335 │ 2 │ 0 │ 1996.0 │ -│ 8775946809982825654 │ 3518457715 │ 2 │ 0 │ 1368.0 │ -│ 8716405327873619806 │ 3518457715 │ 2 │ 0 │ 1368.0 │ -│ 8723645685735855902 │ 2003188719 │ 2 │ 0 │ 1594.0 │ -│ 9007142022850874662 │ 4006219740 │ 2 │ 0 │ 1638.0 │ -│ 7542988325649023791 │ 303701440 │ 2 │ 0 │ 1828.0 │ -│ 4632233196500506143 │ 93692316 │ 2 │ 0 │ 1750.0 │ -│ 7726072175618541265 │ 1876840662 │ 2 │ 0 │ 1638.0 │ -└─────────────────────┴────────────┴───┴──────────────┴──────────────────────┘ -Run Time: real 55.584 user 81.372000 sys 17.980000 -D SELECT URL, count(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬─────────┐ -│ URL │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼─────────┤ -│ http://liver.ru/belgorod/page/1006.j\xD0\xBA\xD0\xB8/\xD0\xB4\xD0\xBE\xD0\xBF_\... │ 3288173 │ -│ http://kinopoisk.ru │ 1625251 │ -│ http://bdsm_po_yers=0&with_video │ 791465 │ -│ http://video.yandex │ 582404 │ -│ http://smeshariki.ru/region │ 514984 │ -│ http://auto_fiat_dlya-bluzki%2F8536.30.18&he=900&with │ 507995 │ -│ http://liver.ru/place_rukodel=365115eb7bbb90 │ 359893 │ -│ http://kinopoisk.ru/vladimir.irr.ru │ 354690 │ -│ http://video.yandex.ru/search/?jenre=50&s_yers │ 318979 │ -│ http://tienskaia-moda │ 289355 │ -└────────────────────────────────────────────────────────────────────────────────────┴─────────┘ -Run Time: real 14.208 user 54.440000 sys 3.008000 -D SELECT 1, URL, count(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -┌───┬────────────────────────────────────────────────────────────────────────────────────┬─────────┐ -│ 1 │ URL │ c │ -├───┼────────────────────────────────────────────────────────────────────────────────────┼─────────┤ -│ 1 │ http://liver.ru/belgorod/page/1006.j\xD0\xBA\xD0\xB8/\xD0\xB4\xD0\xBE\xD0\xBF_\... │ 3288173 │ -│ 1 │ http://kinopoisk.ru │ 1625251 │ -│ 1 │ http://bdsm_po_yers=0&with_video │ 791465 │ -│ 1 │ http://video.yandex │ 582404 │ -│ 1 │ http://smeshariki.ru/region │ 514984 │ -│ 1 │ http://auto_fiat_dlya-bluzki%2F8536.30.18&he=900&with │ 507995 │ -│ 1 │ http://liver.ru/place_rukodel=365115eb7bbb90 │ 359893 │ -│ 1 │ http://kinopoisk.ru/vladimir.irr.ru │ 354690 │ -│ 1 │ http://video.yandex.ru/search/?jenre=50&s_yers │ 318979 │ -│ 1 │ http://tienskaia-moda │ 289355 │ -└───┴────────────────────────────────────────────────────────────────────────────────────┴─────────┘ -Run Time: real 4.586 user 65.340000 sys 0.188000 -D 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; -┌────────────┬──────────────┬──────────────┬──────────────┬───────┐ -│ ClientIP │ clientip - 1 │ clientip - 2 │ clientip - 3 │ c │ -├────────────┼──────────────┼──────────────┼──────────────┼───────┤ -│ 4255045322 │ 4255045321 │ 4255045320 │ 4255045319 │ 47008 │ -│ 2596862839 │ 2596862838 │ 2596862837 │ 2596862836 │ 29121 │ -│ 3119147744 │ 3119147743 │ 3119147742 │ 3119147741 │ 25333 │ -│ 1696638182 │ 1696638181 │ 1696638180 │ 1696638179 │ 20230 │ -│ 1138507705 │ 1138507704 │ 1138507703 │ 1138507702 │ 15778 │ -│ 3367941774 │ 3367941773 │ 3367941772 │ 3367941771 │ 12768 │ -│ 3032827420 │ 3032827419 │ 3032827418 │ 3032827417 │ 11349 │ -│ 1740861572 │ 1740861571 │ 1740861570 │ 1740861569 │ 11315 │ -│ 3487820196 │ 3487820195 │ 3487820194 │ 3487820193 │ 9881 │ -│ 3663904793 │ 3663904792 │ 3663904791 │ 3663904790 │ 9718 │ -└────────────┴──────────────┴──────────────┴──────────────┴───────┘ -Run Time: real 4.737 user 38.872000 sys 0.148000 -D '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(URL) > 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -┌──────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├──────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 102341 │ -│ http://komme%2F27.0.1453.116 │ 51218 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18315 │ -│ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16461 │ -│ http://irr.ru/index.php │ 12577 │ -│ http://irr.ru/index.php?showalbum/login │ 10880 │ -│ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 7627 │ -│ http://irr.ru/index.php?showalbum/login-kupalnik │ 4369 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 4058 │ -│ http://komme%2F27.0.1453.116 Safari │ 3021 │ -└──────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.819 user 1.016000 sys 0.152000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(Title) > 0 GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ Title │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ \xD0\xA2\xD0\xB5\xD1\x81\xD1\x82 (\xD0\xA0\xD0\xBE\xD1\x81\xD1\x81\xD0\xB8\xD1\... │ 122407 │ -│ \xD0\xA8\xD0\xB0\xD1\x80\xD0\xB0\xD1\x80\xD0\xB0\xD0\xB9), \xD0\x92\xD1\x8B\xD0... │ 82935 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA - IRR.ru │ 80958 │ -│ \xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\xB8 New Era H (\xD0\x90\xD1\x81\xD1\x83\xD... │ 39098 │ -│ \xD0\xA2\xD0\xB5\xD0\xBF\xD0\xBB\xD0\xBE\xD1\x81\xD0\xBA\xD1\x83 \xD0\xBD\xD0\x... │ 23123 │ -│ Dave and Hotpoint sport \xE2\x80\x93 \xD1\x81\xD0\xB0\xD0\xBC\xD1\x8B\xD0\xB5 \... │ 14329 │ -│ AUTO.ria.ua \xE2\x84\xA2 - \xD0\x90\xD0\xBF\xD0\xBF\xD0\xB5\xD1\x80 │ 14053 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA (\xD0\xA0\xD0\... │ 13912 │ -│ OWAProfessign), \xD0\xBF\xD1\x80\xD0\xBE\xD0\xB4\xD0\xB0\xD1\x82\xD1\x8C │ 10919 │ -│ \xD0\xA2\xD1\x80\xD1\x83\xD1\x81\xD0\xB8 - \xD0\xA8\xD0\xBE\xD1\x83\xD0\xB1\xD0... │ 10157 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.870 user 1.416000 sys 0.040000 -D AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 7479 │ -│ http://aliningrad │ 4791 │ -│ http://ekburg.irr.ru%2Fpuloveplanet │ 3584 │ -│ http://smeshariki.ru/obucheyelants │ 3064 │ -│ http://video.yandex.php │ 2887 │ -│ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 1084 │ -│ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 891 │ -│ http://afisha.yandex.ru/index │ 855 │ -│ http://sslow_13507.html?aspx?naId=6HS │ 521 │ -│ http://wildberrior/uphold │ 484 │ -│ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 289 │ -│ http://obninsk/detail │ 241 │ -│ http://diary.ru/forum/intries │ 208 │ -│ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 185 │ -│ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 185 │ -│ http://kurort/SINA, ADRIAN │ 157 │ -│ http://afisha.yandex.ru │ 132 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 124 │ -│ http://ssl.hurra.com/iframe │ 123 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 119 │ -│ http://stars-visa.html_params%3Drhost%3Dad.adriver.ru/catalog.php │ 105 │ -│ http://komme%2F27.0.1453.116 │ 83 │ -│ http://pogoda.yandex │ 80 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 79 │ -│ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 73 │ -│ http://video.yandex │ 71 │ -│ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 68 │ -│ http://wildberries.ru/daily │ 68 │ -│ http://smeshariki.ru/ru/index.ru%26bid │ 68 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 64 │ -│ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 64 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn │ 63 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 59 │ -│ http://sslow_135000008&position=search │ 58 │ -│ http:%2F%2Fwwwwww.bonprix.ru/topic │ 53 │ -│ http://sslow_13507.html/articles │ 52 │ -│ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 51 │ -│ http://rsdn.ru/rss.ya.ru/catalog │ 51 │ -│ http://ekburg.irr.ru/#lingvo │ 46 │ -│ http://pogoda.yandex.ru │ 45 │ -│ http://sslow_13500000%26rnd%3D2788881.html │ 44 │ -│ http://maps#ru_5_ru_22106.377648194,975924][to]=&int[14270pa106&op_uid=17759/6#... │ 42 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 37 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 37 │ -│ http://video.yandex.ru/film/46351/frl-2/bage │ 36 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 35 │ -│ http://direct.yandex │ 34 │ -│ http://sslow_13507.html?aspx?naId=3X_3bhLcs3M │ 33 │ -│ http://gotovim-doma │ 32 │ -│ http://che.ru/produkty_zarubezhei-niepochekhly │ 31 │ -│ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 30 │ -│ http://video.yandex.ru/Newsletter │ 29 │ -│ http://sslow_13507.html?aspx?naId=6D8IzMGys3M │ 29 │ -│ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ http://video.yandex.ru │ 28 │ -│ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 28 │ -│ http://notes=1/currency │ 27 │ -│ http://sslow_13507.html?aspx?naId=649&state/out-of-town=\xD0\xA5\xD0\xB0\xD0\xB... │ 27 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 26 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 26 │ -│ http:%2F%2Fwwww.bonprix.ru/tambov │ 24 │ -│ http://kinopoisk.ru/shoppich.ru/search?clid │ 24 │ -│ http://wildberries │ 23 │ -│ http:%2F%2Fwwwwww.bonprix.ru/searchAutoSearch?text=\xD0\xB2\xD0\xB5\xD0\xBB\xD0... │ 22 │ -│ http://myloveplanet │ 20 │ -│ http://stars-visa-litraj.txt","lpu":"http://pogoda │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/secondary │ 20 │ -│ http://msuzie-shop/premiery-c-38208_2.html │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 20 │ -│ http://smeshariki │ 19 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 19 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ http://irr.ru/6323%26bn%3D27888895,96772,97436 │ 18 │ -│ http://kinopoisk.ru │ 18 │ -│ http://nepogoda.yandex.ru%2Fproducts/search?text=subscripts/busineshop │ 17 │ -│ http://wildberries.ru │ 17 │ -│ http://kinopoisk.ru/catalog/9902224 │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://direct.yandex.html │ 17 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 17 │ -│ http://irr.ru/6323%26bn%3D27888895,963095425 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 16 │ -│ http://afisha.yandex.php?gidcar=36281664 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://auto.ria.ua/search │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://direct │ 15 │ -│ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ http://irr.ru/index.php?showalbum/login-kupalnaya-obl │ 15 │ -│ http://kinopoisk.ru/odessya │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 14 │ -│ http://ekburg.irr.irr.ru/maker │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.yandex │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.mail/rnd=0.9788 │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://auto_many_to_auto.ria.ua/igrush43/ │ 13 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 13 │ -│ http://afisha.yandex.ru/cars │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 13 │ -│ http://love.ru/a-myprofi │ 13 │ -│ http: │ 13 │ -│ http:%2F%2Fwww.bonprix │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://real-estate/aparther/offiliates/corruption/russinsk │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 12 │ -│ http://on-online=on&accetti │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 12 │ -│ http://en.lyrsenses/zamba_zaborah_coldplay=1&gearbox │ 12 │ -│ http://kinopoisk.ru/saledParams │ 12 │ -│ http://samara.irr.html5/v12/?from]= │ 12 │ -│ http://zvukovo/hondar/2007&state/renlew/rigma.ru/scribed │ 12 │ -│ http://pogoda.yandex.php?SECTION │ 12 │ -│ http://msk/platia-nashing/vanny.diary.ru/moscow │ 12 │ -│ http://video.yandex.ru/GameMain.E6smreQhiu_hXR4&where=all&film │ 12 │ -│ http://nizhnieiene/p17378705/currency=1&with_photo-ideapadeno │ 11 │ -│ http://news/6483731559676/Unlocknotebooks/m83/800_D_Black_list │ 11 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 11 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty/photo-12/#imag... │ 11 │ -│ http://msk/platia-nashing/vanny.diary.ru/sale/liver │ 11 │ -│ http://video.yandex.ru/GameMain.aspx#location │ 11 │ -│ http://guid=6&pw=6&pv=13 │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 11 │ -│ http://wildberries.ru/rost.html?1 │ 11 │ -│ http://smeshariki.ru/?win=82&stat=1&page/196264&pt │ 11 │ -│ http://pogoda.yandex.ru/catalog/jokers │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 11 │ -│ http://en.lyrsenses/zamba_zabudtrimazok.html?page=12&prr=http://fap1.adrive_typ... │ 11 │ -│ http://bonprix.ru%26bid │ 11 │ -│ http://loveplanet.ru/GameMain │ 10 │ -│ http://bdsmpeople.ru │ 10 │ -│ http://liver.ru/cheboksicily/foto.aspx?sort=newly&trafkey │ 10 │ -│ http://video.yandex.ru&pvid │ 10 │ -│ http:%2F%2F%2Fwwww.bonprix │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://club.ru/spokoiteli/photo37775280000 │ 10 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 10 │ -│ http:%2F%2Fwwww.bonprix │ 10 │ -│ http://auto_id=240&n=13901038 │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 10 │ -│ http://smeshariki.ru │ 10 │ -│ http://slovariant_new3077940810/detail │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://en.lyrsenses/zamba_zabor_id=1012_blank%26site │ 10 │ -│ http://sslow_13500000%26rnd%3D2788881.html?parts/passe │ 10 │ -│ http://pogoda.html%3Fhtml_params%3Drhost%3D43 │ 10 │ -│ http://irr.ru/index.php?showalbum/logout%26verstova.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://pogoda.yandex.php │ 9 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film/6781203.html?id=242037047/detai... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://mysw.info/blog/sankt-peter%3D1216/00001216629 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://e96.ru/albumfotok-15-fotki │ 9 │ -│ http://alpari.yandex.html?html_param=0&users/#page/Search/ab_dob%2Ffieiie-razvo... │ 9 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu │ 9 │ -│ http://bonprix.ru/catalog/8570/1006790 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://b2b.testered/main/discuss/matched_country=-1&top=0&cityid=1024&wi=1366&o... │ 9 │ -│ http://svpressa.ru/topic=17082630 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://bonprix.ru │ 9 │ -│ http:%2F%2Fwwww.bonprix.ru/filmId=8j5j97LRs3M&where=all&sources │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/#win_13.html_p... │ 9 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/4744089758 │ 9 │ -│ http://nail=Yes&target=search │ 9 │ -│ http://rlsnet.ru/vacancy/view_type_id=9677548268010367 │ 9 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_produkty%2F&sr=http://slovaria │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/imagecache/wm/2013&where=all&filmId │ 8 │ -│ https://m.myloveplanet.ru/forum/abrika-kobelenie_nebestsenal │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 8 │ -│ http://zapchast.com/iframe-owa.html?1=1&cid=577&oki=1&op │ 8 │ -│ http://afisha.yandex.php?t=141880517 │ 8 │ -│ http://poisk.ru/price_ot=&price_ot=&price │ 8 │ -│ http://zarplata.ru/velika_all=\xD0\xBE\xD1\x82 │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http:%2F%2Fwwwwww.bonprix.ru/news/222974895&op │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://fore=3&marka=0&top=0 │ 8 │ -│ http://direct.yandex.ru/catalog │ 8 │ -│ http://video.yandex.ru&xdm_p=1#item/search │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96777&oki │ 8 │ -│ http://zarplata.ru/?p=12977-B26358/currency=RUR/page=1080&wi=1024&lo=http://rzh... │ 8 │ -│ http://masterh4.adriver.yandex │ 8 │ -│ http://en.lyrsenses/zamba_zabor/bedroomolsk │ 8 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://mastered/main.aspx#location=1&bc=3&ct=1&pr=60322056107100919/page5/?_h=s... │ 8 │ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ==&page_avtomodules.php?f=100&ref... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://omsk.mlsn.ru │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96779/87 │ 8 │ -│ http://afisha.yandex.php?gidcar=367108851%2Fr%2F1 │ 8 │ -│ http://che.ru&pvid=13733142835/100/topic,5240556895&ch=UTF-8&sF=11,7,7,0 │ 8 │ -│ http://smeshariki.ru/GameMain │ 8 │ -│ http://love.ru/?p=17059 │ 8 │ -│ http://wildberries.xml?from]=&input │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://irr.ru/index.php?showalbum/login-kupit-topy%2Fplatjie-gotovlexandex.html... │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 8 │ -│ http://auto.ria.ua/auto_id=24126629/0/index.ru/real-estate/out │ 8 │ -│ http://victor?page_type=city.stol-yar.ru/cars │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://kinopoisk.ru/ch/feed/letniaiaprice/1609 │ 7 │ -│ http://video.yandex.ru%2F&sr=http://loveplants/65398f55 │ 7 │ -│ http://real-estate/aparts/Aquarevski │ 7 │ -│ http://bonprix.ru/social/product_id │ 7 │ -│ http://moscow/detail/5552/0/2792834&m=111,7,7,5 │ 7 │ -│ http://video.yandex.ru/a-album/login-vitiju/photo │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 7 │ -│ http://irr.ru/index.php?showalbum/login-kapustics?sort=pogoda.yandex.ru%26bt%3D... │ 7 │ -│ http://irr.ru/Registered/main/topnewsru.com/page=6 │ 7 │ -│ http://bonprix.ru/catalog/8570/14139489 │ 7 │ -│ http://afisha.yandex.ru/\xD0\xB4\xD0\xBE\xD0\xBC\xD0\xB0/\xD0\x91\xD0\xA1\xD0\x... │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://love.ru/?p=1#country=&op_seo │ 7 │ -│ http://irr.htm?from]=&int[85][from]=&input_vsegodnyie │ 7 │ -│ http://bdsmpeople.ru/niktory/shtory/308/roomed.ru/p59473682740295 │ 7 │ -│ http:%2F%2Fwwwww.bonprix │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 7 │ -│ http://myloveplantrackIt?tid │ 7 │ -│ http://spb/event=big&marka=84&model=0&auto_id=0&s_yers=0&pv=10&can_be_check_PP │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 7 │ -│ http://pogoda.yandex.ru&pvid=1 │ 7 │ -│ http://irr.ru/index.php?showalbum/login-a-5-advert27114 │ 7 │ -│ http://video=0&is_hot │ 7 │ -│ http://bdsmpeople │ 7 │ -│ http://irr.ru/index.php?showalbum/login-zk34/pages/0001216629 │ 7 │ -│ http://svpressa.ru │ 7 │ -│ http://money.yandex │ 7 │ -│ http://gotovim-doma.ru │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 7 │ -│ http://afisha │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://afisha.yandex.ru%2F%2Fmail/169 │ 6 │ -│ http://omsk/evential/house.ru/catalog/kitchedule=213-606361653965283 │ 6 │ -│ http://video.yandex.ru%2Fkategory_id │ 6 │ -│ │ 6 │ -│ http://love.ru/?p=1#country=-1&sq_liver.ru/kyrgyzstan │ 6 │ -│ http://video.yandex.ru/ekt │ 6 │ -│ http://stars-varenok.ru/16745959680706/800_0.jpeg.html%3Fhtml5/v123593 │ 6 │ -│ http://afisha.yandex.ru/catalog=on │ 6 │ -│ http://bdsmpeople.ru/film/64544.690022.rar.html_params%3Drhost%3D_black_list=0&... │ 6 │ -│ http://v102.ru/investate/apartments-sale │ 6 │ -│ http://bdsmpeople.ru/GameMain │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://video.yandex.ru/GameMain.aspx#location/page_type=category │ 6 │ -│ http://afisha.yandex.ru%26bt%3D43%26anbietersburg │ 6 │ -│ http://wildberries.ru/search │ 6 │ -│ http://omsk/evential/housession%3D0 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://video.yandex.ru/a-topy │ 6 │ -│ http://direct.yandex.ru/refererprofile%2F2.10 │ 6 │ -│ http://rukodel=0&sort=newly&trafkey=2750 │ 6 │ -│ http://moscow/details │ 6 │ -│ http://smeshariki.ru/world/photofider_credit=0&view │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://novosibirsk.irr.ru%26bid │ 6 │ -│ http://irr.ru/index.php?showalbum/login-12.html%26custom │ 6 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 6 │ -│ http://solutions.diary.ru/realtitroenie_v_jurman.ru/albums/frame-owa.html?stric... │ 6 │ -│ http://mr7.ru/newsru.com/iframe_right%3D43 │ 6 │ -│ http://smeshariki.ru/catalog │ 6 │ -│ http://autodoc.ru/real-estate/apart │ 6 │ -│ http://edp2.adriver.ru/hocketshop.ru/moscow/detailanude │ 6 │ -│ http://povari.yandex.ru/greecondary/Products_id=&auto_vaz_2111 │ 6 │ -│ http://ereal-estate/rent │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://afisha.yandex.php/board,39.04839 │ 6 │ -│ http://msk/platia-nashing/vantralitsa_transion │ 6 │ -│ http://novo/detail.aspx?group_cod_1s │ 6 │ -│ http://video.yandex.ru/catalog │ 6 │ -│ http://3dnewsru.com/iframe_right.html?1=1&cid=51538 │ 6 │ -│ http://forum/topnews/22294&op_category │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://auto_hyundai_sarator │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://kalininmyclonus1 │ 6 │ -│ http://bdsmpeople.ru/saledParams=rhost%3D43%26bid%3D1 │ 6 │ -│ http://love.ru/?p=17055335 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://afisha.yandex.ru/mymail.php │ 6 │ -│ http://real-estate=week/page=1&expand_search?film/298677435615.html │ 6 │ -│ http://koolinar.ru/port.ru/doc │ 6 │ -│ http://afisha.yandex.ru/index.ru/recipe │ 6 │ -│ http://kinel-lab.com/rus/20130709_117485994,93304&op_seo_entry=1&gearbox=0&type... │ 6 │ -│ http://video=0&with_exchangeType │ 6 │ -│ http://diary.ru/exp?sid=3205 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://video.yandex.php?search?text=\xD0\x9A\xD0\xBE\xD0\xBD\xD1\x8C\xD1\x8F\xD... │ 6 │ -│ https://slovakia-600dd903c07022,101595,91194&op_seo_entry │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://video.yandex.php?from]=&interapy-wkti/ &cd │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D8393224 │ 6 │ -│ http://love.ru/ru/irk/event/search/ │ 6 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 5 │ -│ http://kinopoisk.ru/catalog │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://auto_volkswagen/vologdano/il_dlya-dnevka.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://barnaul/details/?cauth │ 5 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD0\xB5\xD1\x80&where=... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://sp-mamrostova │ 5 │ -│ http://zvukovo-gorodsk │ 5 │ -│ http://metal-lodku-Obzor │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 5 │ -│ http://tp66.ru/exp?sid=3860217/rooms=2/men.aspx#location%3D0%26rnd │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://video.yandex.by/?state_id=&auth=1..630;IC,7711588 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 5 │ -│ http:%2F%2Fwwww.bonprix.ru/?id=2013&where=all&filmId │ 5 │ -│ http://rmnt.ru/search/offilia_Sovetov_living_chamber/?78142 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://e96.ru/movies/614418821/artir.ua/search │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://slovari.yandex.ru%26orderovskij-index.ru │ 5 │ -│ http://povari.yandex │ 5 │ -│ http://tks.ru/cat/publish-cherkalnaya-ttpodporyadushek │ 5 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_seo_entry=&op_category_... │ 5 │ -│ http://slovarenok.com │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://video.yandex.ru/GameMain.aspx?Link │ 5 │ -│ http://auto.ria.ua/auto_id=241269.html?1=1&cid=2127970 │ 5 │ -│ http://slovari.yandex │ 5 │ -│ http://rustnye-sht-riemnikoi │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://radio&planet.ru/work.ru/catalog │ 5 │ -│ http://auto_volkswagen-Palities/horobki │ 5 │ -│ http://afisha.yandex.php?r=23436303135353.html?1 │ 5 │ -│ http://en.lyrsenses/zamba_zaborah_chamberk │ 5 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 40007&pt │ 5 │ -│ http://love.ru/product_id=0&po_yers=0&po_yers=2&refererro/model=1346488078722&c... │ 5 │ -│ http://myloveplanet.ru/v14572&lb_id=13000001216629 │ 5 │ -│ http://pogoda.yandex.php?gidcar │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://sslow_135000008&position=search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 5 │ -│ http://bibidohertki-i-OOOO_REPORT/07_2013 │ 5 │ -│ http://autodoc.ru/moscow │ 5 │ -│ http://avtomobile/motory.ru/comp.ru/view.php │ 5 │ -│ http://topnews.ru/GameMain.aspx?group_cod_1s=1983&pt=b&pd=9&pw=0 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://real-estate=week&m=Dvigenie.html?option │ 5 │ -│ http://loveplanet.ru/mymail/rudi │ 5 │ -│ http://video.yandex.ru/firms.turizm │ 5 │ -│ http://video.yandex.ru%2FkategoriendflowerTo=&powerTo= │ 5 │ -│ http://auto.ria.ua/auto │ 5 │ -│ http://auto.ria.ua │ 5 │ -│ http://nizhnieiewva88/photo/101246465376&cmd=show-to-buchaiev-pugache=51db32a68... │ 5 │ -│ http://msk/platia-nashing/vannyie-product_id=1841&page2 │ 5 │ -│ http://tks.ru/cat/publish-chemec.ru/search?filmId=CktclMBmUXI │ 5 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D158197%26width │ 5 │ -│ http://direct.yandex.ru/index │ 5 │ -│ http://club.ru/cinema/movies/no-pos │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://kinopoisk.ru/spb.pulscen.ru/exp?sid=3159&op_category_id=&auth=0&checked=... │ 5 │ -│ http://holodilnik-rp-ploschaya-obuv/?ci=1280&with_video=0&choosOyg==&op_uid=111... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://slovariant_neu%3D1%26bid%3D1216/0001216629%26bt%3Dad │ 5 │ -│ http://wildberries.ru/real │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://md.mirkovskaya-obl.irr.ru/jobinmoscow │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://wildberrior/kia/ │ 5 │ -│ http://jcmotorom-921205&bt=7 │ 5 │ -│ http://loveplanet.ru/\xD0\xB0\xD1\x80\xD0\xBE\xD0\xBC/curre-ap-i-showalbum/loui... │ 5 │ -│ http://en.lyrsense.com/obshchin-idieiala │ 5 │ -│ http://povari.yandex.php?showalbum/login.pl?cl=all&film/497794,90458 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://video.yandex.ru%26bt%3Dad.adriver.ru/recipe/view/10217/?from │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://smeshariki.ru/topic │ 5 │ -│ http://moscow/detail/Torgovuyu-organske.ru/real-estate │ 5 │ -│ http:%2F%2Fwww.bonprix.ru │ 5 │ -│ http://gaylyU │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://loveplanet.ru │ 5 │ -│ http://alib.mist.html%26custom%3D%26c2%3D278888592138 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://alpari.ru/gallery/pic845274 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://whoyougle.ru/images/images/00000i/specifiers.ru/image=1&furniture │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 4 │ -│ http://loveplanet │ 4 │ -│ http://video.yandex.ru/price │ 4 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 4 │ -│ http://video.yandex.ua │ 4 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 4 │ -│ http://sslovarenok.ru │ 4 │ -│ http://auto.ria.ua/auto_id=0 │ 4 │ -│ http://afisha.yandex.ru/zoom.php?f=5162613838.html_partments │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://loveche.html │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/filmId │ 4 │ -│ http://samarskii_krai/tuapse/detail/result.aspx │ 4 │ -│ http://irr.ru/index.php?showalbum/login-10618968476372773 │ 4 │ -│ http://bdsmpeople.ru/Web/Pages=1/feedsmag.ru/~\xD0\xBA\xD0\xBD\xD0\xB8\xD0\xB3\... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://e96.ru/news/39733/page2=&input_sponsor=&o=1015219.html_partments-sale&pa... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://radio&planet.ru/marka=62&model=1178128455&pvno=2&evlg=VC,5 │ 4 │ -│ http://wildberries.ru/catalog │ 4 │ -│ http://afisha.yandex.ru%252f7769%252fe │ 4 │ -│ http://echoradar-s-Levoshcha │ 4 │ -│ http://love.ru/?p=1#countpage/vacancies/events/738/0/3/women.aspx │ 4 │ -│ http://direct.yandex.php?formsof(INFLECTION_ID=30861/14365-4b11&state/apartment... │ 4 │ -│ http://diary.ru/catalog=on&input_city[3 │ 4 │ -│ http://rasp.pl?cmd │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 4 │ -│ http://travel.ru/state/apartments-sale/rashinitit%2F537 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.php?view_type=2&driveresult.ru/replies │ 4 │ -│ https://smeshariki.ru/cinema/article10363136000001216629%26site_offilia_Sovets.... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://afisha.yandex.php?r=3&bs=&day │ 4 │ -│ http://ssl.hurranovskaya-ul-31-foto.ria │ 4 │ -│ http://kinopoisk.ru/saledParams%3Drhost%3Dad.adriver.ru/GameMain.aspx#location │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://auto.ria.ua/auto_id=63799.html_params │ 4 │ -│ http://vkirovoe-tourisma │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://smeshariki.ru/domchelkakh_location=search?text=\xD0\xBC\xD0\xB8vents │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboy07/... │ 4 │ -│ http://card/windows)&bL=ru&cE │ 4 │ -│ http://kniga.ru/view=\xD0\xA1\xD1\x82\xD0\xB0\xD0\xB2\xD1\x80\xD0\xBE\xD0\xB2&w... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://smeshariki.ru/search/keukeru-soft │ 4 │ -│ http://guid=6&pw=2&pv=0&price_do=¤cy=1 │ 4 │ -│ http://v102.ru/?s=Adaments-sale │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/real │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://afisha.yandex.ru/search │ 4 │ -│ http://zarplata.ru/?p=12977-B26358/hasimages=1/page │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://matched_car=373838928155755775482794,9453.116 Safari%2F537.36&he=10&s_ye... │ 4 │ -│ http://video.yandex.ru/GameMain.aspx#location-4.0.html │ 4 │ -│ http://video.yandex.ru/GameMain.XYyZwYXRoPWEtbG9nb24vcG90Cw │ 4 │ -│ http://tks.ru/cat/publish-chernyjbelyj-9375966238&op_categoriya │ 4 │ -│ https:%2F%2Fwwww.yandex.ru │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://nizhny novgorod │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/social │ 4 │ -│ http://video.yandex.ru/realty/leaser_map=1/hasimay-2.html │ 4 │ -│ http://auto.ria.ua/canel_pe_mascona Hilfigeratov/153228 │ 4 │ -│ http://video.yandex.ru%26target=search/ab_area=categorija80119 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://guide.travel.ru/link │ 4 │ -│ http://victorhead.php?full&dom=780067167694.0; │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://kubikus.ru/search.php?r=4140211,1,7,7,7,7,0 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://e-kuzbass.ru/a-shop │ 4 │ -│ http://svpress_up.aspx#location[propfilter_pf[PODVAL]=&arrFilter][3463351841195... │ 4 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/cart.php/cars/papago-d-plosch │ 4 │ -│ http://afisha.mail.aspx#locationalOffers │ 4 │ -│ http:%2F%2Fwwww.bonprix.ru/real-estate/out-of-town/house │ 4 │ -│ https://produkty%2Fpulove.ru/gost/?page3 │ 4 │ -│ http://fitness/building │ 4 │ -│ http://afishi,Mudanted-belyj-974299099/guest-id=34089.html │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://anketka.ru/bridget │ 4 │ -│ http%3A//magnitogorod/page3/#over │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://afisha.yandex.php?action/2741920 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadbor_sh... │ 4 │ -│ http://tks.ru/cat/public/gamemain.aspx#location │ 4 │ -│ http://video.yandex.ru/GameMain/dukhovyy │ 4 │ -│ http://nepogoda.yandex.ru/search=1&target=search=0&can_be │ 4 │ -│ http://love.ru/image=2&marka=84&model/mihailovo │ 4 │ -│ http://kinopoisk.ru/searchAutoSearch=0&driver.ru/catalog/1/women.aspx#locationp... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://afisha.yandex.php?show=rlv&ru=1&expand_search │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://love.ru/?p=1705 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://internet Explorer&aV=5.0 (Windows)&bL=en │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://pogoda.yandex.ru/cooking_sm5_1148786993ab-417/photo/69363/26#formi.ru/co... │ 4 │ -│ http://auto_volkswagen_cated_cartovskaya-obl.irr.ru/album/login │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://afisha.yandex.php?p=31&input │ 4 │ -│ http://loveche.ru/job/1162323&PAGEN_1=30&state │ 4 │ -│ http://video.yandex.ru/If yours.avtogsm.ru/animals/calculate │ 4 │ -│ http://ssl.hurra.com │ 4 │ -│ http://auto_s_product_name=\xD0\x9A\xD1\x80\xD1\x83\xD0\xB3\xD0\xBB\xD0\xBE\xD0... │ 4 │ -│ http://rsdn.ru/details_103514,154;IC │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://inspelishchin-platjie-doma.ru/irkutsk.irr.ru │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://auto_volkswagen_pass_1161967 │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD1\x82\xD0\xBE\xD1\x8... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboyzon... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.ru/topnews.ru/lanas-advert2713][to]=&int │ 4 │ -│ http%3A//edp1.adriverys/forum/view_type=city&custom=0&damages/0001216629%26bid%... │ 4 │ -│ http://auto_id=0&with_photo │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://afisha.yandex.ru/content2.adriver │ 4 │ -│ http://arma/frl-4/transportnoy-kv-m-malchik.ru/show │ 4 │ -│ http://en.lyrsenses/zamba_zabudka/photo-1/#page=0&sale/search?text=\xD0\x9C\xD0... │ 4 │ -│ http://b.kavanga.ru/?a=inneVolumeFrom │ 4 │ -│ http://smeshariki.ru/saint-petersburg-gorod/transfer/?id=7576149959760994861&op... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://afisha.yandex.ru&pvid=13733424 │ 4 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_category_id=592b9e01c48ce9403%26bn%3D0%2... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/GameMain │ 4 │ -│ http://tp66.ru/money.yandex.ru/albums_screenterval │ 4 │ -│ http://irr.htm?from]=&int[1151;IC,112 │ 4 │ -│ http://bdsmpeople.ru/film/64544.690078 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://romar/events/audio.ru/widget[]=vacancies/99.php?gidcar │ 3 │ -│ http://banantikov-nadushenie_orleona_server=sc.cheloveplant_11612/page │ 3 │ -│ http://afisha.yandex.php/topic104780204&op_uid=1954 │ 3 │ -│ http://afisha.yandex.php/tova.ru/uliya2076789599305953 │ 3 │ -│ http://afisha.yandex.ua/donetsk/urals │ 3 │ -│ http://afisha.yandex.ru/a-folders/misc │ 3 │ -│ http://afisha.yandex.ru/shop.ru/malta │ 3 │ -│ http://afisha.yandex.ru/project_price=&maxprice │ 3 │ -│ http://afisha.yandex.ru/kategoriya%2F5.0 (company │ 3 │ -│ http://afisha.yandex.ru/linkvac.php/board.php?topicseeng │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://kinopoisk.ru/service.ru/iframe │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://irr.kz/realty/lease/3516093&pvno=2&evlg │ 3 │ -│ https://produkty%2Fplatjie-kuzbass.ru/newsru.com/iframe_right=0&auto_ria=0&meta... │ 3 │ -│ http://fuckfind=rent/view/2021/3 │ 3 │ -│ https://orenburg/?arrFiltersburg │ 3 │ -│ http://radioscannerica/filmId=Ba_id=13733568414&city=\xD0\x9C\xD0\xBE\xD1\x81\x... │ 3 │ -│ http://radiorecord.ru/login-m3w.html?1=1&cid │ 3 │ -│ http://video.yandex.ru%2Fplata.ru/ch │ 3 │ -│ http://samara.irr.ru%2Fproduct │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://sp-money.yandex.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://astrobank.ru/image │ 3 │ -│ http://forum/topnews/2229605699574.html?1=1 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://love.qip.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://afisha.yandex.ru/forum.materinburg │ 3 │ -│ http://afisha.yandex.php?addriver.ru │ 3 │ -│ http://video.yandex.by/search/?target%3D43%26bid%3D2 │ 3 │ -│ http://love.ru/forum.cofe.ru/forum/view_type=city=790&Selectronics-technik │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ https://slovakia-600dd903c07022,101595,9143531427800648_elit │ 3 │ -│ http://wildberrior/bedroom]=&int[17][to]=&int[858 │ 3 │ -│ http://sp-money.yandex.ru/work rushki-sien-natalog/8570/page=0&expand │ 3 │ -│ https://slovakia-600dd903c06c999c226647639.html%3Fhtml │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category_id=0&wi=16000&... │ 3 │ -│ http://direct&sortdirect.yandex.ru/imagesize%3D0%26ar │ 3 │ -│ http://auto_kia_30 │ 3 │ -│ http://ssl.hurral=messages │ 3 │ -│ http://sp-mamrostokonkursovet │ 3 │ -│ http://kaluga/?ext=\xD0\xB3\xD0\xB5\xD1\x80\xD0\xBE\xD1\x8F\xD1\x82\xD0\xBD\xD1... │ 3 │ -│ http://video.yandex.ru&pvid=13735/?_h │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://rmnt.ru/stars │ 3 │ -│ http://edp2.adriver.ru/catalog/181 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://video.yandex.ru/circle&state/out-of-town/houses │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://bdsmpeople.ru/search │ 3 │ -│ http://b.kavanga.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http:%2F%2Fwww.bonprix_ru}%2Fnizhniynovgorod/request-id │ 3 │ -│ http://video.yandex.ua/auto_id │ 3 │ -│ http://rmnt.ru/film/88677/russia/rio.ru/search?filmId=NNr6aJrm4s3M │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://kazan.irr.ru/location │ 3 │ -│ http://pogoda │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://pogoda.yandex.kz/family │ 3 │ -│ http://edp2.adriver.ru/jobinmoscow/detail │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/forum.donfiscategory │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://video.yandex.php?topbloveche │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com/iframe-owa.html?1=1&cid=577&o... │ 3 │ -│ http://nigma.ru/product&op_category_name=\xD0\x91\xD0\xB8\xD0\xBA\xD0\xB8\xD0\x... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://card/windows NT 6.1) AppleWebKit%2F5 │ 3 │ -│ http://tks.ru/cat/publish-chin-play.php?categoriya%2Fzhiensmed │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ https://produkty%2Fpulove.ru/voronezh-sien-zhienskaia-moda-zhienskaia-moda-zhie... │ 3 │ -│ http://svpress_w1t1042796786/6/?category │ 3 │ -│ http://radiorecord.ru/catalog/idShare │ 3 │ -│ http://msk/planet.ru/mymail.aspx#comme_me_saydinne │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountpage │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://ussuriysk.irr.ru/catalog/premiere/628962851d7fd0b6eb17b321d336f5bc7de189... │ 3 │ -│ http://ekategoriya%2F9B206 Safari │ 3 │ -│ http://magnitka_1_series.ru/?favorite_id=636233644&op_category_id=937514 │ 3 │ -│ http://afisha.yandex.ru/tatatit_chto.php?industry │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://smeshariki.ru/a-phony │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26url%3D//ad.adriver.ru/link/justic/h2.php/... │ 3 │ -│ http://smeshariki.ru/catalog/286/women.aspx │ 3 │ -│ http://svpress/showbiz/photo.htm │ 3 │ -│ http://svpressa.ru/content/search │ 3 │ -│ http://video_dvd/game/iframe-owa.html │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/92054446660.ht... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty%2Ftanki │ 3 │ -│ http://loveche.ru/volzhskiy │ 3 │ -│ http://sp-mamrostovestory │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://e96.ru/real-estate │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://my.kp.ru/albumfoto-1/pol-2 │ 3 │ -│ http://holodilnik.ru/catalog │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://smeshariki.ru/?win=82&stat=141882,373;IC,2552f48 │ 3 │ -│ http://personal/atlants/7292&xdm_c │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://msk/platia%2Fzhienskaia │ 3 │ -│ http://tp66.ru/search/?page=1&fuelRateTo │ 3 │ -│ http://povarenok.ru/files/eliteh.ru/perm.irr.ru/board,75.2013-07-09 │ 3 │ -│ http://love.ru/?p=1#country=&op_seo_entry=-1&target │ 3 │ -│ http://amobil-nye-pliazhnaia │ 3 │ -│ http://lk.wildberries │ 3 │ -│ http://brand=498&pvno │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory_id=577&search/ab_district/date_id=2271][from... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostavia.travel.ru/s... │ 3 │ -│ http://irr.ru/washek-s-printom │ 3 │ -│ http://irr.ru/index.php?showalbum/login-kupaljinik-chere │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://product_brand=RAINBOW&op_cated_content/search/keup/en-ru │ 3 │ -│ http://wildberries.ru/item_no=2&evlg=VC,0;VL,205;IC,14;VL,757138/currency=RUR/h... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://afisha.yandex.php?app=membered │ 3 │ -│ http://afisha.yandex.php?ELEMENT_ID │ 3 │ -│ http://afisha.yandex.ru/lesyach-hotels │ 3 │ -│ http://msuzie │ 3 │ -│ http://afisha.yandex.ru%26bt%3D90%26nid%3D1216629 │ 3 │ -│ http://afisha.yandex.php?id=727285 │ 3 │ -│ http://smeshariki.ru/goodavec/photo/6936325.html?id=223978/page=102 │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://love.ru/recipes/indows │ 3 │ -│ http://str_ob.html?1=1&choosO8gPJSs3M&where=all&filmId=mAyiC7y6M2mGV2GoA9hFoN3q... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic=79799398/?_h=search.htm... │ 3 │ -│ http://3dnews.ru/?p=12636464/5#f │ 3 │ -│ http://love.ru/ru/irk/event=little&cated_country=-192.html?1=1&cid │ 3 │ -│ http://love.ru/?p=17057 │ 3 │ -│ http://smeshariki.ru/real │ 3 │ -│ http://kinopoisk.ru/search │ 3 │ -│ http://kaluzha-na-NovletedAutoSearch=0 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://pogoda.yandex.ru/real-estate │ 3 │ -│ http://soft.oszone.ru │ 3 │ -│ http://povaria/chak_naytimes.ru │ 3 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD0\xB2\... │ 3 │ -│ http://li.ru/filmId=XpzlPj8P8gE&where=all&text=\xD1\x81\xD0\xBA\xD0\xB0\xD1\x87... │ 3 │ -│ http://kinopoisk.ru/sessulyanovka.ru/photosessid=3205&bt │ 3 │ -│ http://wildberries.ru/basket&ch │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberries.ru/GameMain │ 3 │ -│ http://afisha.yandex.ru/comment/search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 3 │ -│ http://afisha.yandex.ru%2Fobuv-sapozhkivka=23&price │ 3 │ -│ http://afisha.yandex.ru/hotel-agen-Goluboj-9730 │ 3 │ -│ http://notebooking pressa │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://state/room=94720-recept-Salat-iz-glasya1lesyat │ 3 │ -│ http://you_hashkaf.ua/search │ 3 │ -│ http://auto_repairs=0&confiscategoriya │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://auto_id=0&engineVolumeFrom │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://video_dvd/ratesTypeSearch?text=\xD1\x87\xD0\xB5\xD0\xBB\xD0\xBE\xD0\xB2\... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://auto_repairs=0&po_yers=0&price.ru/\xD0\xB6\xD0\xB5\xD0\xBB\xD0\xB5\xD0\x... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://smeshariki.ru/GameMain.aspx#location │ 3 │ -│ http://slovari.yandex.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://audio_video.yandex.ru/mosday.html?item=4#photo446962 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://v102.ru/investate/apartment/?id=137336IseNhcbx3J85GkHSnzgnsPdZUU&where=a... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com.ua/kiev/detail.ru/auth=1..640... │ 3 │ -│ http://video.yandex.ru/personal/offeebe34c7e12944&op_product │ 3 │ -│ http://pogoda.yandex.ru/vlas-moskovskaya │ 3 │ -│ http://video.yandex.ru/extra │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/photo/narod.irr.ru/katering=1&pr=569&s_yers │ 3 │ -│ http://dom.net/provoe-pervouralnaya/ChildGluZ19oZXJlci1kYXRpb25zPU4mbj0zJmlkPTM... │ 3 │ -│ http://trashbox.ru/book │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ https://slovarenok │ 3 │ -│ http://afishers/story │ 3 │ -│ http://love.ru/ru/irk/event=little&category_id=731-643736&mode=1 │ 3 │ -│ http://alpari.yandex.ru/saint-petersburg.irr.ru/cars/page3 │ 3 │ -│ http://auto_ford Mix).mp3.ucoz.ru/v1430497.html%3Fhtml │ 3 │ -│ http://kommersantamina │ 3 │ -│ http://guid=6&pw=2&pv=0&with_video.yandex.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.ru/cgi-bin/click.cgi%3Fsize │ 3 │ -│ http://state_shariki │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://bdsmpeople.ru/show/39932/Itemid,40.0.html_params%3DfsSaHR0cDovL2pzLnNtaT... │ 3 │ -│ http://afisha.yandex.php?partments │ 3 │ -│ http://video.yandex.ru/real-estate/out-of-town/house.ru&pvid=1&distreet_legkovo... │ 3 │ -│ http://loveplanet.ru/Bezli-all │ 3 │ -│ http://omsk/evential/housession%3D240%26rleurl%3D//ad.adriver.ru/marshavskaya-r... │ 3 │ -│ http://radiorecord │ 3 │ -│ http://kinopoisk.ru/registernet Explorer&aV=5.0 (Windows NT 5.1; ru-ru&cE=true&... │ 3 │ -│ http://en.lyrsenses/zamba_zabor_polnitsa-s.narod.irr.ru/bank/otzyvy/12031%2F%23... │ 3 │ -│ http://auto_id=0&color=0&confiscategory_id=3205&bt=7&bn=1&bc=3&ct=1&prr=http:%2... │ 3 │ -│ http://balcon_caddy Club relove │ 3 │ -│ http://video.yandex.ru%2Fkategory_id=9759527418 │ 3 │ -│ http://kinopoisk.ru/spb.pulscen.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://smeshariki.ru/cgi-bin/click.cgi%3Fsid%3D0%26pz │ 3 │ -│ http://video │ 3 │ -│ http://wildberring │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://cxem.net/324487194836848 │ 3 │ -│ http://sunmar.ru/cart&ch=utf-8&sF=11,7,700&aN=Opera&aV=9.80 (Windows │ 3 │ -│ http://afisha.yandex.ru/manga.ru/?rtext=\xD0\xBF\xD1\x80\xD0\xB8\xD0\xB1\xD0\xB... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://smeshariki.ru/furniture.html5 │ 3 │ -│ http://myloveplanet.ru/passenger/kitched_country_id=4312&input │ 3 │ -│ http://wildberries.ru/comment/search │ 3 │ -│ http://klubnich/zrh/ │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http:%2F%2Fwww.bonprix.ru/imagesize │ 3 │ -│ http://saint-petersburg-gorod/goodal │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://un1.adriver.ru/page=30138117749516%252f110916%252fmedicinema/movie_ross ... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_product_brand=4200&lo=h... │ 3 │ -│ http://wildberries.ru/comme%2F2.12.388 Version/1552/page │ 3 │ -│ http://ssl.hurra.com/iframe-owa │ 3 │ -│ http://saint-peter │ 3 │ -│ http://lazarevskoe │ 3 │ -│ http://video.yandex.ru/realty/search/main.aspx?sort=popular │ 3 │ -│ http://whoyougle.com/iframe/iframe_right.ru/spb │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&film/67... │ 3 │ -│ http://jobs-education │ 3 │ -│ http://pogoda.yandex.ua/telefon_shtukaturkey │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://afisha.yandex.ru/sell/resident │ 3 │ -│ http://afisha.yandex.php?p=176d43f96ef32d5bc1272 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://video=0&input_with_video.yandex.ru/page/10/women.aspx │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://pogoda.yandex.ru/real-estate/apartments/73151 │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://bdsmpeople.ru/index.by/ru/page=0&confiscategory_id │ 3 │ -│ http://auto_id=0&color=0&confiscategoriya%2Fzhienskaya-advert25593 │ 3 │ -│ http://video_dvd/suppoll/dleead6718.php?SECTION │ 3 │ -│ http://gotovim-doma.ru/personal/commersant.ru/image=19&pvno=2&engineVolum │ 3 │ -│ http://radioscannerica/film/47018.html?1=1&cid=691390&pvno │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://krasnodar.irr.ru/yaransferapid │ 3 │ -│ http://forum/topics/ingradskazka-lookoformalities/poetry/events/?date&csrf-8200... │ 3 │ -│ http://irr.ru/index.php?showalbum/logizer8/num-1/refremost │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://tyva-5/country=-1&washestvo/den_sidentialAmount │ 3 │ -│ http://rukodel=0&sort=newly&trafkey=27065/2/child.aspx#location │ 3 │ -│ http://zapchastny_fashing_mashes/index.ru │ 3 │ -│ http://myloveplanet.ru/bad_by_sidential/nizhnieiewva88 │ 3 │ -│ http://irr.ru/index.php?showalbum/login-do-500-kv-m-Maya │ 3 │ -│ http://mylove.ru/zoom.php?GID=2&IsOrder │ 3 │ -│ http://arma/frl-4/travel.ru/moscow/details │ 3 │ -│ http://msk/platia-nashing/vanny.diary.ru/otdam_daily │ 3 │ -│ http://kinopoisk.ru/cgi-bin/click.cgi%3Fsid%3D1216 │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/procoolonelopitered/ministrict/3d-probeg-340-... │ 3 │ -│ http://ssl.hurra.com/iframe/iframe-owa.html%26custom%3D%26custom │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://maps#ru_5_ru_1_ru_ru_ru_202_ru_1_ru_ru_ru_ru_2_ru_1743.html?oscsid=36303... │ 3 │ -│ http://direct.yandex.ru%2F&sr │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://chehod/zvenia8312&input_who2=1&input_age1=35&aN=Netscape │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=&is... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/used/KIA-Cee-d... │ 3 │ -│ http://kinopoisk.ru/real-estate=2013-07-2089241607/photo=0&with_exchangeType │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://rsdn.ru/info_all=yes&razdumy-i-trikshop/search&_h_page/1719920 │ 3 │ -│ http://smeshariki.ru/news/2013/peshnye-udivlekanka.ru/l_03_00/bodreamfood.ua │ 3 │ -│ http://forum/topnews/222968695,910112_1164074834-908745 │ 3 │ -│ http://myfashihtzu.html?1=1&cid=65625f313230303&po_yers=2013/07 │ 3 │ -│ http://myloveplants_list-obl.irr.ru%2Fobuv-zhienskaia-modiezhda │ 3 │ -│ http://smeshariki.ru/chipinfo │ 3 │ -│ http://zagranimals-planet.ru/user/31059&Module │ 3 │ -│ http://kuharka=48&modeloveplanet │ 3 │ -│ http://smeshariki.ru/index.ru/main.pl?cmd=show/47555 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://whoyougle.ru/basket&ch=utf-8&sF=11,7,7,7,700 │ 3 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 3 │ -│ http://afisha.yandex.ua/index.ru/recipe │ 3 │ -│ http://video.yandex.ru%2F%2Fwww.bonprix │ 3 │ -│ http://afisha.mail.ru/stars/page │ 3 │ -│ http://bdsmpeople.ru/Web/price │ 3 │ -│ http://video=0&is_hot=0&che_simeis │ 3 │ -│ http://irr.ru/6323%26bn%3D27888895,96772&op_page47 │ 3 │ -│ http://sendflower │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://avtoto.ashx/1001087496197797217530729; Media Centernet_mastersburg │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://votpusk.ru/ │ 3 │ -│ http://auto_operey-v-v-meha.ru/catalog/public │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountry │ 3 │ -│ http://wildberries.ru/page=0&vip │ 3 │ -│ http://forum/topnews/2229362067528195&op_categoriya │ 3 │ -│ http://nizhnieie-bielie-bieriends&fb_source-temno │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 3 │ -│ http://stories.ru/art/MACKLEMORE │ 3 │ -│ http://afisha.yandex.php?link=114735200&brand=23368 │ 3 │ -│ http://afisha.yandex.php?w=3160/transports │ 3 │ -│ http://kurort/SP1399&op │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://pogoda.turizm.ru/communit=0&vip=0&order_by=2 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://smeshariki.ru/Web/price │ 3 │ -│ http://irr.htm?from]=&int[852][to]=10&lastdiscussins/?keyworld │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 3 │ -│ http://zarplata.ru/?p=1290&op_product_price=990348531&schoosOSRquM8gE&where=all... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&top │ 3 │ -│ http://ej.ru/muss/roll_to_audi/mode=replies │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://ur.hh.ru/file/news/2013 \xD0\xB3\xD0\xBE\xD0\xB4\xD0\xB0 │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://radiorecord.ru │ 3 │ -│ http://my.alpari.ru/filmId=yJg89hqV8gE&where=all&film/530/?fromCityCodeForcentr... │ 3 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 │ 3 │ -│ http://gorbus.aspx#localiformalitic │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&text=\x... │ 3 │ -│ http://guid=6&pw=2&pv=0&po_yers=0&with_video │ 3 │ -│ http:%2F%2Fmuzhchine/ru-g-Chelya_v_tsentyabrskii │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://rlsnet.ru/nogin.html5/v12 │ 2 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 2 │ -│ http://edp2.adrive_type=4754679564&pt=b&pd=9&price │ 2 │ -│ http://poisk.ru/news/articles │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 2 │ -│ http://wildberries.ru/editem_no=100¤cy=1#country=&op_proizvodskaya-obuv-z... │ 2 │ -│ http://wildberries.ru/filmId=4920/room=1&lang=all&filmId=DnGbyVQVUXI&wheretomug... │ 2 │ -│ http://smeshariki.ru/product&op │ 2 │ -│ http://forum.php?t=420 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://afisha.yandex.php?id=1620_4_licanel │ 2 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD1\x81\... │ 2 │ -│ http://moscow/detail/Torgovljatory/storii_efferanslyatting │ 2 │ -│ http://mysw.inform%26q%3Dversion.cheltyj │ 2 │ -│ http://wildberrin/foton │ 2 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 2 │ -│ http://sslow_13500000%26rnd%3D278888 │ 2 │ -│ http://wildberries.ru/index.ua │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://smeshariki.ru/kitched_country=-1&target= │ 2 │ -│ http://balance/#21792/guest=\xD1\x89\xD0\xB5\xD0\xBD\xD0\xBA\xD0\xBE \xD0\xB8 \... │ 2 │ -│ http://afisha.yandex.php?s=4d450&pid │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://diary.ru/search?familliklink │ 2 │ -│ http://kaluga/?ext=\xD1\x80\xD0\xB5\xD0\xB9 \xD0\xBD\xD0\xB0 \xD0\xB1\xD1\x80\x... │ 2 │ -│ http://afisha.yandex.php?city=418&view │ 2 │ -│ http://afisha.yandex.ru/?trafkey=54073799 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://ssl.hurra.com.ua/auto_repairs=0&page=10&category_id=detail │ 2 │ -│ http://video.yandex.html?1=1&cid=1&bodystyle=0&color=0 │ 2 │ -│ http://hotel-2-komn-kvarticle/7081272794,921006 │ 2 │ -│ http://svpressages/00012166260001216629%26sliceid%3D0%26ad │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 2 │ -│ http://video.yandex.ru/?win=82a&cid │ 2 │ -│ http://video.yandex.ru/rent/info/messa.ru │ 2 │ -│ http://news/398261_enl.jpg-1 │ 2 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 2 │ -│ http://auto.ria.ua/auto_auto.ria │ 2 │ -│ http://loveplants/740392319 │ 2 │ -│ http://yaroslavl.irr │ 2 │ -│ http://masterlingvo/#I 1 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://ur.hh.ru/page=10&can_be_checked_auto_region=1&rm=1&lang=all&film/18212.1... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://video.yandex.html5/v12 │ 2 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 2 │ -│ http://pogoda.yandex.ru&xdm_e=http://name=\xD0\x91\xD0\xB0\xD0\xBB\xD0\xB5\xD1\... │ 2 │ -│ http://auto_id=3159&input_age2 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://smeshariki.ru/user_id=1&bodystyle=0&vip=&int[12][from=last_auto_id=13143... │ 2 │ -│ http://en.lyrsense.com/iframe-owa.html?query=2&hide_black.asp?TOPIC_ID=914958&o... │ 2 │ -│ http://pogoda.yandex.ru/places/premiery%2Fpage=2&confiscategoriya │ 2 │ -│ http://video=0&wi=1362653061306535333&site_id=&auth │ 2 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=2furii10.html?city=0&hide │ 2 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.117 user 0.520000 sys 0.028000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -┌────────────────┬────────────────┬─────────────┬────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ TraficSourceID │ SearchEngineID │ AdvEngineID │ src │ dst │ pageviews │ -├────────────────┼────────────────┼─────────────┼────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33069 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 24703 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 15817 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 14233 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 │ 6549 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 │ 5257 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 5257 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 3547 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login │ 3538 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik │ 3371 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php │ 3333 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 3309 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 2923 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 2616 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 2441 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php │ 2418 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 2390 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari │ 2367 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 2349 │ -│ 1 │ 0 │ 0 │ https://google.com.ua/url?sa=t&rct │ http://komme%2F27.0.1453.116 │ 2228 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 1920 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://komme%2F27.0.1453.116 │ 1679 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ekburg.irr.ru%2Fpuloveplanet │ 1340 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://aliningrad │ 1270 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1254 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1213 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1193 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1186 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1142 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F27.0.1453.116 │ 1007 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 954 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://aliningrad │ 859 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login │ 821 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 791 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 777 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 757 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 722 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://video.yandex.php │ 702 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 691 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/obucheyelants │ 684 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 632 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 593 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 577 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://smeshariki.ru/obucheyelants │ 566 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.php │ 452 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 445 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBA\xD0\xBE\xD0\xBD\xD1\x82\xD1\x80\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 422 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 421 │ -│ 0 │ 0 │ 0 │ │ http://obninsk/detail │ 407 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 386 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug.ru │ http://irr.ru/index.php │ 374 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru/index │ 360 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 348 │ -│ 2 │ 0 │ 13 │ │ http://komme%2F27.0.1453.116 │ 347 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-2011/43597 │ 293 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 293 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6HS │ 282 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 268 │ -│ 3 │ 72 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 267 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 259 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 257 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari │ 256 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 │ 241 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login │ 237 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 │ 236 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/28435&lr=157 │ http://komme%2F27.0.1453.116 │ 233 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/ru/photo/6936313555&text=\xD1\x85\xD0\xBA \xD0\xBB\xD0\xBE\xD... │ http://komme%2F27.0.1453.116 │ 224 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php │ 223 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cat/dushkirillovyj │ http://komme%2F27.0.1453.116 │ 222 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=359&op_page2/... │ 218 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kriminally-bezhevsk │ 217 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0 │ 212 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 207 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 202 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 199 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://ekburg.irr.ru%2Fpuloveplanet │ 199 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 198 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 197 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 195 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/414526863.xlsx │ 191 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CDMQFjAJ │ http://irr.ru/index.php?showalbum/login-kiriskaya-obl.irr.ru/index.ru/GameMain.... │ 190 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=0&with_photo.ashx/101/4/?cat=6257271 │ 188 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=partments-sale/search?text=\xD1\x81\xD0... │ 187 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://afisha.yandex.ru/index │ 183 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://ekburg.irr.ru%2Fpuloveplanet │ 171 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 168 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 164 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 163 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://ekburg.irr.ru%2Fpuloveplanet │ 160 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 160 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/life.ru/cars/misc/travel.ru/?trafkey=058143&p... │ 155 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 │ 154 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=577&oki=1&oby=&op_s... │ 152 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://wildberrior/uphold │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6HS │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 149 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 146 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustic/meterburg │ 144 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 144 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 144 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E │ 141 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 140 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 136 │ -│ 1 │ 0 │ 0 │ http://kombardighantnie │ http://komme%2F27.0.1453.116 │ 136 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login │ 135 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kysjacevtika │ 132 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://komme%2F27.0.1453.116 Safari │ 124 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 121 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category/stroy/dachines... │ 114 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/?state │ http://komme%2F27.0.1453.116 │ 113 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/GameMain │ http://komme%2F27.0.1453.116 │ 113 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 111 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login │ 110 │ -│ 3 │ 85 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 110 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kardigan │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 107 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosoftwarenok.ru/projects/zhbi.po... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 105 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268143.html?1=1&cid=577&o... │ 104 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 104 │ -│ 0 │ 0 │ 0 │ │ http://tvidi.ru/photo=0&confiscategory_id=0&engineVolumeFrom=&fuelRateFrom=type... │ 103 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 102 │ -│ 3 │ 14 │ 0 │ │ http://komme%2F27.0.1453.116 │ 102 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 100 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 98 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://video.yandex.php │ 97 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 96 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.21150895 │ 96 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?sort=price │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 95 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login │ 95 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.12.388 │ 93 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 93 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 92 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberrior/uphold │ 92 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 91 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-tanks/search=1&ady=62&modeloveplanet.ru │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 90 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.145332.15&he=10&category_id=2740387993 │ 89 │ -│ 1 │ 0 │ 0 │ http://in-the-weightEnd=2351&numphoto=&isExclusiver.ru/alertljus │ http://komme%2F27.0.1453.116 │ 88 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fprodazha_Italja_unit=1&av=1&nm=1&lang=ru │ http://irr.ru/index.ru/show/414526863.xlsx │ 87 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 86 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb1.html?item_no=386703/?bundle=7172&msid=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 86 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 │ 85 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 84 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733582852/ │ 84 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupit/action │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 82 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 82 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?bodystyle │ http://komme%2F27.0.1453.116 │ 81 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F22.14&he=768486 │ 81 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusk.ru/ru/lookaginitial/Prodayu-Dach... │ 80 │ -│ 3 │ 4 │ 0 │ │ http://komme%2F27.0.1453.116 │ 80 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 79 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://aliningrad │ 78 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 78 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login │ 76 │ -│ 1 │ 0 │ 0 │ https://go.1ps.ru/show&showforum │ http://komme%2F27.0.1453.116 │ 76 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandex.ru │ http://komme%2F27.0.1453.116 │ 76 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26221/detail.ru/v1496366&... │ 75 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php │ 75 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 74 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%2F&sr=http://bonprix.ru... │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len80/page/product │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filtr/all/perm.pulscen... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 73 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 73 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F1.7.1364.172 │ 72 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://diary.ru/forum/intries │ 72 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/show/414526863_112 │ 72 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2618561&pp=1059&op_produc... │ 72 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://video.yandex.php │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 71 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 71 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari │ 71 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://aliningrad │ 71 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=product_id=42&Selection.chelov.ru/searc... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/yandsearch?te... │ http://irr.ru/index.php │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/server=sc.chel.ru/main... │ http://irr.ru/index.php │ 70 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification/?year=\xD0\xB1\... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2791954~43.87725656132&op... │ 69 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://komme%2F27.0.1453.116 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/produkty/bleacs/udilis... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 69 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&wi=1366&br... │ 68 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 67 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731&CgID=124jc&where=all... │ 66 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/Yozh/Goodda │ http://komme%2F27.0.1453.116 │ 66 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2725215195&ti=\xD0\x9F\xD... │ 66 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://ekburg.irr.ru%2Fpuloveplanet │ 65 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru/index │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 64 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://afisha.yandex.ru/index │ 64 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 63 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nmg.com%2F4.0 Safari%26clients-sale/search&e... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 63 │ -│ 2 │ 13 │ 13 │ │ http://komme%2F27.0.1453.116 │ 62 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 62 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login │ 62 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://ekburg.irr.ru%2Fpuloveplanet │ 61 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 61 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 61 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26092.html?s_text=\x5C\x5... │ 59 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/thenon-houses/public/g... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27755f32316.30; .NET CLR ... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742/details/?cauth=0&dam... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fautoad/kniga.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26004/?_h=galle/\xD0\xBF\... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert27930555&sob=1&p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2692.html%3Fhtml?period=3... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/world/photo31469:Album... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=leting "HalUXI&where=\xD0\x9F\xD0\xB5\xD1\x8... │ http://irr.ru/index.php?showalbum/lofiver.ru/articles/86121%26uid%3D139750%26ad... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27000,224648804-recept-so... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/koshka.com/ig/iframe-o... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory=cinema.perm.p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/realty/suntime-5/extre... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742357/detskii_gosts.xml... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742304][from]=&int[14670... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694336/photo/suzannason/... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Forum25/top... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268320995,968650f45491882... │ 57 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupit/action │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1375605&ga... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://video.yandex.php │ 56 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga/choice/... │ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_product_... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?showtopic,5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26017/quarius_Moscow/cavi... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2726125413975d77cf&search... │ 55 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 54 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266848223/review_type=pro... │ 53 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 53 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26989%26bt%3Dad.adriver.r... │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/album/login-1800002&pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 52 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0 │ 51 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2698172,93932353064614618... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605085268a9c4d61-9862.49... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19&numphoto/login=A-CL-MS-36575c72937][to]=&int[12822304 │ http://irr.ru/index.ru/widgetchrome%2F&ti=no&dom_v_bordovye-printime.ru │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/investate/comple/make/?page5/&docid=jlMNIrXw... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 50 │ -│ 0 │ 0 │ 0 │ │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 50 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F11.7.1364 │ 50 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 50 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26051.htBeg=6&NightRegist... │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xBE\xD1\x81\xD0\xB8\xD0\xB1... │ 49 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 49 │ -│ 5 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 49 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?group=days=3&text=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27410/photo-2.xhtml&serve... │ 48 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandsearch │ http://komme%2F27.0.1453.116 │ 48 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 48 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusty-i-vkont.at.ua/search=0&userId=0... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266128182&op_seo_entry/de... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?sort=&br... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/intrumen │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://afisha.yandex.ru/index │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumanegenre=33155?analog/kitchen_mini... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik │ 46 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login%2Flick.g.doubleclick.diary.ru/car/kw/3061/us... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734.html_params%3Dfh_loc... │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://afisha.yandex.ru/index │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/GameMain.p1Yo4A │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ua/searchads/jo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605906/frl-2/sportal.ru/... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fshow/lpp/cre.ru/pers... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.asp?search&ev... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logize.ru/msk/events_liver.ru/russert-plies.r... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26099486633033&countpage=... │ 44 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://sslow_135000008&position=search │ 43 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 43 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischet-solik/odezhda-plos... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert277553/38021/66936575776/... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271050&with_photo.kurortm... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 43 │ -│ 1 │ 0 │ 0 │ http://forums/liii-kuler S22oHgBJTngegotavgorod55.ru/filtr[2]=42 │ http://komme%2F27.0.1453.116 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-folders/?end=9a08488... │ http://irr.ru/index.php?showalbum/litamak.irr.ru/catalog/8570/travel.ru/cgi-bin... │ 41 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 41 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 41 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/0001216629/#top_by │ 41 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26040931&s_yers=0&with_vi... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://ekburg.irr.ru%2Fpuloveplanet │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.137508&s_yers=200... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://aliningrad │ 40 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logito8ergo_6470/page_type_id=20872/0/001:12:... │ 40 │ -│ 0 │ 0 │ 0 │ │ http://afisha.mail.ru/dmitrij │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/readar-nashi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27924563724&key=46960/med... │ 40 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://lib.ru/exp?sid=3205&bt=7&bn │ 40 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 39 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credit │ http://komme%2F27.0.1453.116 │ 38 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=8570/page=5&s_yers=2006 │ 38 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2679/detail/5482,935033/2... │ 38 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 38 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login │ 37 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dnevnoy-kv-Samara.irr.ru/searchAutos&marka=4... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://komme%2F27.0.1453.116 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 37 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=theating&page=... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?group │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2784389/room=39695,966681... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/forum/topicID=269&stat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/votkim-rukav-i-kova-ul-adve... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filmId=xFX1UbTNJjxe4yF... │ http://irr.ru/index.php?showalbum/login.asp?razdel7/test/matched=115909d9_dsc07... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wolf-club478561/OPEL | \xD0\xB0\xD0\xB2\xD1\... │ http://irr.ru/index.php?showalbum/login-marka=13&city&custom=0&damages/0001216 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27553&s_yers=0&po_yers/46... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert275226607660?design=6efxq... │ 36 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=show=&fgroup-tab-mara.irinables/yearT... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278171&size=52094&op_seo_... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27330528/photo/6223799%26... │ 36 │ -│ 5 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://smeshariki.ru/obucheyelants │ 36 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36&he │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=post_neu%3D22%26pz%3D0%26rleurl%3D%26xp... │ 35 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/8569 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/index.ru/katego... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vip.ru/book.com/photo/7086/page=0&state=mont... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-cukickdrops_bak_yant4tlGtZgy3BQ&bvm=bv.49478... │ http://irr.ru/index.php?showalbum/login-501-98552&op_uid=1060948/6#f │ 35 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (Windows │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19&m_staraya-obl │ http://irr.ru/index.ru/show/414526863_112 │ 34 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.12.388 │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/promo=C-Tease/rent/dat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 34 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_uid=1... │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/list │ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 34 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/directadvert/kavanga_728x90.html │ http://komme%2F27.0.1453.116 │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 33 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshops.xml?typ │ http://komme%2F27.0.1453.116 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?cPath=default.ru/yandex.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33 │ -│ 1 │ 0 │ 0 │ http://auto.ria.ua/\xD0\x9A\xD0\xBB\xD0\xB8\xD0\xBC\xD0\xB0\xD1\x82\xD1\x8C\xD1... │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 33 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://video.yandex.php │ 33 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 33 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser │ 33 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265748ba3f755671_1162684&... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertif?sle=85000 \xD0\xB4\xD0... │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364 │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/pub │ 32 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zxcvbnm1990&search/jobinmoscow.ru/yandex.by/... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements-obschaya │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fproduct/foto-4/login... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&op_category_id=1174]... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniturers_id=575386110 │ http://irr.ru/index.php?showalbum/login.exit/rem/php/board/10_kruizer_8267049&o... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiepochto_id=93231238][to... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://afisha.yandex.ru/index │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ssl.hurra.com/iframe │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischuk_Kerasivye-klavik/?... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/jobs-education/m_acces... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26791e3a7c543c9bf9ef5c0-9... │ 31 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt │ http://komme%2F27.0.1453.116 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertificia/?letteo_985428161/... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_api.php?cid=41389/rooms/perm.pulscen.ru/sear... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 30 │ -│ 1 │ 0 │ 0 │ http://domchelov.html&lr=55&text=\xD0\xB4\xD1\x80\xD1\x83\xD0\xB3\xD0\xB8\xD0\x... │ http://komme%2F27.0.1453.116 │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoryID=12452929587/... │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 30 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 30 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268630533&site_ot=&price_... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-k608i.trashbox.ru/cars/page5/&docid=577&lr=2... │ http://irr.ru/index.php?showalbum/logii_58247.php?forum.borovichy77/page5 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2716/~3/150160947694,9745... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2696374/#page/tab=com_avt... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/products/?category_id=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 30 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/topic │ 30 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.14&he=768&wi=1920 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605446518%2F&sr=http://e... │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26979553039353936694&ch=u... │ 30 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://smeshariki.ru/obucheyelants │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-carava/fotok-8-steklove.ru │ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26928142&op_products/view... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kupibatikov/?recrete/personal/klimat.ru/work... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 29 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x9A\xD0\xBE\xD1\x80\xD0\xBE\xD1\... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_countpage/1 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertifications[0][150]=60&inp... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/wedditeli_foto-2/__48/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/click.net/paged=696&lo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274229696.0 (Windows)&bL=... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-14132&with_photo=1&srtime-zone.net/catalog/t... │ http://irr.ru/index.php?showalbum/login-sierra%2Fdlia-zhienskaia-moda-zhiensmed │ 29 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704.html%3Fhtml5/v12/?fr... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2664952&xdm_p=1#country=-... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategoriya/zhienskaya-... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734/fotographic_2017666&... │ 29 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9649.html%3... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/small_mercial-shoppicc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27350350394&ch=UTF-8&sF=1... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_seo_entr... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27425.3120%26ntype_id=2&r... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-baza/gadge... │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 28 │ -│ 1 │ 0 │ 0 │ https://directions[0][1541769377921968 │ http://komme%2F1.7.1364.172 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13733097&op_categ... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert272291684/hasimage=0&view... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/mann japanel/?text=\xD1\x81\xD0\xBC\xD0... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269500469412216388/detail... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-my-sovies/bebybum.homestion bonprix.ru/... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikarbox=2&tz=9b81998&po_y... │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?lang=all&fi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-176-kakaya-obl.irr.ru/search?lr=191&ei=N6rrg... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\xA4\... │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 28 │ -│ 3 │ 95 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/anime-2/#album/l... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-eve-nn.ru/cars/passenger/\xD0\x92\xD0\x90\xD... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274236&pv=15&prr=8&sid=35... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/41452626u1zIq0SGLXCI │ 28 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic/8939858-59929477/detailshop │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://ekburg.irr.ru/#lingvo │ 28 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F22.14&he=768486 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678/page=2&marka=84&mode... │ 27 │ -│ 3 │ 2 │ 0 │ │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/search?cl4url=http://yandex.ru/online... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27013.ya.ru/sportalznakom... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert24061.jpg.html%3Fhtml?1=1... │ 27 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031343-sh-898675/sort=pri... │ http://irr.ru/index.php?showalbum/login-5158&jenre/slyudya_metroman.ru/GameMain... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19195/offset=101&district=&city&with_photo=¤cy │ http://irr.ru/img/catalog/53485785/topic/8939850.php?page=3&marka=84&mm │ 27 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 27 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 27 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-oleginnesujeli-i-centum=etarget=search?clid=... │ http://irr.ru/index.php?showalbum/login-kupe.html_params%3Drhost%3Dad.adriver.r... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fhit_list.html?s... │ http://irr.ru/index.php?showalbum/login-irake-myservierk-supierts%2F&ti=\xD0\x9... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27928092_price=\xD0\xBC\x... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/mazda-dievochnye_avari... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-19.xhtml?city_to_one=All&film]/on/orders/810... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/1.19.mobile_photo=6001... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27630/?item_id=0&bodystyl... │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ 1 │ 0 │ 0 │ http://bonprix_ru_11559&lr=12&usln │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lizova_ii_1112_1150&option[price_do=300... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E&wher... │ 26 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detall/seconomicsmovinki │ http://komme%2F27.0.1453.116 │ 26 │ -│ 1 │ 0 │ 0 │ http://yandex.ua/?target=search/tab=user │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://obninsk/detail │ 26 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/loginsk.irr.ru%2Fkategory_id=9204471-650/?_h=... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login-jjjekrju-s-pring=1&price[price=\xD0\xBC... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification.ru/messa.ru/rea... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lejandsearch?text=\xD1\x82\xD0\xB8\xD1\x85\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert21124631/?Search.php │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 26 │ -│ -1 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_category... │ 26 │ -│ 3 │ 197 │ 0 │ │ http://komme%2F27.0.1453.116 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?messa.ru/worlds/page2/7277932582&text=\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 26 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-kupe-20010120652838799.html?n=7148.html... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemey-volvom80Cw&where=a... │ 26 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=607&state=4... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.22 (KHTML, like Gecko) │ 25 │ -│ 1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 25 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/people.ru/cars/passets... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731/?city=0&pvno=2&evlg=... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vivarki/redki_sdelaet-law.ru/produkty/kartgo... │ http://irr.ru/index.php?showalbum/login=driff/en-ru/?p=2#messages/00001216629 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.html&ei=GcXrUYe8F2DAU-RbuRlLMczoJ--5uDCx8 │ http://irr.ru/index.php?showalbum/login-132/#images/0000.h95/\xD1\x82\xD1\x8E\x... │ 25 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert264314953); ru) AppleWebK... │ 25 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://obninsk/detail │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search?text=myaccountry │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguard-payments-sale/se... │ http://irr.ru/index.php?showalbum/login-88i8LaDyEkCVv6-DhRfEDcw==&action │ 25 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/yandex.ru/?f... │ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.ru/film/46351/frl-2/bage │ 25 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605838.html5/v12/?from=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-link=1147373-1&p=1&p=2&text=\xD0\xBC\xD0\xBE... │ http://irr.ru/index.php?showalbum/logisterlingvo/#! │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741559&t=5204/make=Chrys... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694589/detail/555-javata... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=shops.html?id=1&body_type=\xD0\xBF\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781472265&op_seo_entry=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626rleurl%3D//ad.adriver... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685/price_ot=&price=\xD0... │ 24 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://diary.ru/forum/intries │ 24 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-117031&op_category&op_seo_entry=&op_uid │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26685857947301_Zoryatiya/... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27399992.php/board,13.0) ... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671291&fridgets/2012&pri... │ 24 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://video.yandex.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo/7... │ http://irr.ru/index.php?showalbum/login-kupit-plitka-s-korsiety/art/151/n4p/160... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguar.ru/page4/?emain.a... │ http://irr.ru/index.php?showalbum/login=ogabass.ru/filmId=9WOqzzitive&view │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-annative-expressageId=0oVXx21hUXI&where=all&... │ http://irr.ru/index.php?showalbum/login-rybnovlevojj_mamountry=-1&type%3D0 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/intrumen │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kupalnaya_elit=&id=3&clid=9403&lr=1913849 │ http://irr.ru/index.php?showalbum/login-gallebia_zhurnalCall D.R.E.mp3ex.net/ph... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lolacake/tatyaka.html?1=1&cid=577&oki=1 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinglish.rus-l7-p70505-1... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/yandsearch?lr=1... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-yvaken/topic=7702.jpg.html?1=1&cid=577&... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-547552/1/?sma=RAINBOW&op_uid=fdd1b9d2721728&... │ http://irr.ru/index.php?showalbum/login-kapusta-advertika/search/room=1&damage=... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=1&bc=3&ct=1&pr │ 23 │ -│ 1 │ 0 │ 0 │ http://auto_volkswagency=1&text │ http://komme%2F27.0.1453.116 │ 23 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgarials/commend/searchvage-2.html&dt=13734... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-tuniki_510... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263090024189-528619c2077/... │ 23 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/ru/index.ru%26bid │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333572620201709/page/10... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news/radio&clid=198555... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26939.html?page/3/#count=... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/kategory_id=5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2740974%26pz%3D0%26ar_sli... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 23 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/resultatya9176da22f1a521a5853.html&... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26600721376439322%26pz%3D... │ 23 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga_728x90.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://afisha.yandex.ru │ 22 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2697574697.html_param=0&u... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemennoke0070553995f27d6... │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-105vr/ │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=search.tutung/s... │ 22 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/wheel.ceratornaya_solnechka │ http://komme%2F27.0.1453.116 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-10760b31b65633eZateq eb806e887d9f15ccf593280... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2091&local=1&TypeSearch?mail.ru/yandsearch&c... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26058&nnum=s4746835895&op... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-the-ili-tempera-minsk.irr.ru/produkty/bluzki... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodny │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2649.html?pg=menu_29.jpg&... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739693071/pomer... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/v1465][from]=&int[157281 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267830840994,95688781470%... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-business/sale/ru/search?mail.ru/film%2Fbarna... │ http://irr.ru/index.php?showalbum/login-zakon_type=1&fuelRatesTypeSearch │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 22 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27200,55.7654.html_parat-... │ 22 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26601629][from=&fuelRateT... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&l... │ http://irr.ru/introlux_page5/2/page/Renaul.irr.ru/start=1500-rublic/gamemain/Tu... │ 22 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://ekburg.irr.ru%2Fpuloveplanet │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liliate_id=24145602&with_phoney.yandex.ru;ya... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-natashkarola.irr.ru%2Fviewforum/MsgList.html... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26900/technics-technics-t... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&city[7]=23239330794,25826044_Black_lis... │ 21 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?price_till beli.php?cx=015216684_36 │ http://komme%2F27.0.1453.116 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694,978825315f373400/det... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691_ru_17832523.html_par... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1) AppleWebKit%2F537 │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/world/photo/41/62b1dfa450/3/women-clother%26tms%3D%26ev_ltx%3D... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/basketshop.ru/yandsear... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265228.html?1=1&cid=577&o... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/forum%2Fukhov/l... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27025&Select[4][]=32598.h... │ 21 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://wildberrior/uphold │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/houses/passenge... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo44953.116.php?t=4... │ http://irr.ru/index.php?showalbum/login=ko&page=0&view.aspx?group_cod_1s=85&key... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/?from │ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-dance/index.ru/?favorite_off=1&encoding&hl=\... │ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 21 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logino-s-grigerator/page1=&input_age1 │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cars/commeddesk.ru/google_s&12.5.746.59954.30 (KHTML, like/\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 21 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/logii-bin/click.cgi%3Fsid%3D158195,97987231-9... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ramblery/pic/893985650697... │ http://smeshariki.ru/obucheyelants │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26737865/?page/12186/#/ec... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login.html?1=1&input_who1=2&input_who2=1&inpu... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929&from%3D%26custom=0&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dietsik.ru/razdel_id=111,7,700&w=728x90.html... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/club.ru/anime-zone.ru/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-brie... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2186/detail.aspx#location... │ 21 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739353/details.... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kupalnik │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification%2F4.0; XBLWP7; ... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/9-0--navam-zhien... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741465][from]=&input_wit... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_uid=1147... │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login=uzpp2XnEw&bvm=bv.49784469][from │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 20 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-Larki-i-korobeg-1124-95367/guestblowinp... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26219&wi=1280&lo=http://w... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/yandex.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estatic.diary.ru/... │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=58&Selectronics-technics/k... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b706234651&op_uid=1&b... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?er=2&text=\xD0\xBA\xD1\x83\xD0\xBF\xD0\xB8\... │ http://irr.ru/index.php?showalbum/login=razer2.moikrug.ru/recipes/shop.spb │ 20 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://video.yandex.php │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&price │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=NeIdeaCenternet Explorer&aV=5.0 │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertificirovka.ru/noteating-i... │ 20 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate=0&ReturnUr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271161076&st=261&t=170977... │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313163307/?from]=&int[15... │ http://che.ru/produkty_zarubezhei-niepochekhly │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604370757034dea482207549... │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_api.php?id=7262882,9454472&op_product_n... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 20 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/cars │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-karbox=0&top=0&category/1961/#images/00... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26637&model=1705.html?1=1... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumilora481 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt=13733129/room │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-iulyanovskaya-obl/lyubvi?page=166733&eid=485... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27502/detail/?folders/#pa... │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://video.yandex.php │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626u1zIq0SGLXCI&where=al... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=240&u_cd=242035165&input_who2=1... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news_id_72387877055/nu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?sort=popup/casualg... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikulina.ru/real-estate/ou... │ 19 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/filmId=eRaB4pYAXCI │ http://komme%2F27.0.1453.116 │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBB\xD0\xB8\xD0\xBA\xD0\xBE\xD0\xBB\xD0\... │ http://irr.ru/index.php │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.ru/daily │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260540718534/?dateOn=1&bo... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login=perfect.yandex.ru%2Fkategoriya%2F537 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261921&pr=2441185112.html... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26672655462.html?city.ru/... │ 19 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostok.ru/cars │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/check=1&redir=1&lang=a... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671&Destige-all.html_par... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27404895,95923.html%26cus... │ 19 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2638/?from=&raceTo=&power... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313437&pvno=2&evlg=VC,2;... │ http://irr.ru/index.php?showalbum/login-rizova.ru/look/timashirtsevm.cofe │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6HS │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27355f3132366336601&uuid=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166333/price[price_till]=&pri... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F_liveresum... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.php?f=11340.1j1j2.12...5998-103358071/8/page... │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/00007/10/1320006&p... │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto_repai... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://smeshariki.ru/obucheyelants │ 19 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.15751 Safari%2F5.0 (Windows │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2675406828ac956028692581/... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://obninsk/detail │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26968794618/irr.ru/real-e... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2629&lr=65&rstr=-213&text=back=False&withmos... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2702483,631;IC,444041%2F&... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26669E116-1-1550046560013... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?search?text=\xD1\x80\xD1\x83\xD1\x81\xD0\xB... │ http://irr.ru/index.php?showalbum/login-inadlinny-sched_car=0&city=0&price=18&p... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=http://video.yandex.ru/real-estate │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274026181891&rn=52&brand=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/dlia-zhiensk... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-165&jenre]/49047&lr=163&text=samsung ga-moda... │ http://irr.ru/index.php?showalbum/login-2-komn-v-na-92f5-6ccf-fef3-013f9f926a21... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&checked=0&state/room=10002-g-v-program... │ 18 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ru/index │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5 │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?elmt=popular&where=all&film/723/num │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/restate/out-of-town-resh=1&cnt=1373... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2012-recept-Ragu-iz-2-x │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupaljinik-slidi/places/?PAGEN_1=2 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 18 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=deltec&type=citymo=&version%2F4.0 (Linu... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27534.30 (KHTML, like Gec... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kochtinsk.pulscendinskiy-krasnodar.irr.ru │ http://irr.ru/index.php?showalbum/login-kuplyuskij/zapchastorii_state/out │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zzzi.ru/a-shop and blademas/?id=1&marka=0&ma... │ http://irr.ru/index.php?showalbum/login-2003prev=/search/index.ru/meteor.html%3... │ 18 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login.moikrug.ru/catalog/otchenko-ul-advert27... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\... │ 18 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278453&with_exchange=0&ex... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27132069304;IC,1511e8d158... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=367132&pv=2... │ 18 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27071038_114736895,923401... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x9A\... │ 18 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27420780790931.html5/v12/... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-natura.ru/jobs-education=post_137336ID0... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kanza_tab_2499472&lr=1074&sort=popular │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/8018&l=malkogo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/?target=search?... │ http://irr.ru/index.php?showalbum/login-bigutensiysk.irr.ru/regular&bid=3205&bt... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/obuv%2F │ 18 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-21102&text=\xD0\xB1\xD0\xB0\xD0\xB1\xD0\xBA\... │ http://irr.ru/index.php?showalbum/login.mospsy.ru/super-na-kostronics/tv-audio.... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685&numphoto=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626p/ │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertimeshariki.ru/cgi-bin/cli... │ 18 │ -│ 1 │ 0 │ 0 │ https://diary.ru/magnitogorsk │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260908861%26point-peter=&... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sien-zhienskaya-i-kvartir.html?id=1955451-01... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 18 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1453.116 Safari%2F8536 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=yators/?jumping.ru/series.ru/priceup&page=3&... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeSearch?text=\xD0\xB2\xD0\xBA\xD1\x83\xD1... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinger/\xD0\x93\xD0\x90\... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga1788778.137329TVmbFNmbVNaR0YwW$UyNDg4fxS... │ http://irr.ru/index.php?showalbum/login-sienskaia-moda-zhienskikh_rabotjaga.ru/... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929750%26ar_sliceid%3D2... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=36427929&stat... │ 18 │ -│ 4 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691c352005&pvno=2&evlg=V... │ 17 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27422544/3/womens.ru/ufa.... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru%2Ffilm/4858?pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/cars/communalux.ru/dos... │ http://irr.ru/index.php?showalbum/loshaya-obl.irr.ru/film/35562C?analytics/kto-... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271527091/frl-2/bage-2.ht... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.103402.html%3Fhtml │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#7878... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=artira-Lyubyat │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.aspx#location │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=yes&y1=2013-07-07.ya.ru/5_search?text=\... │ 17 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://aliningrad │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.15.html?1=1 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266149_51928/detail.ru/ka... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%26rleurl%3D%26CompPath%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=8&ch=UTF-8&ifr=1&av=1&nm=1&ved=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2670308&po_yers=0&price_o... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_Bolers-device.aspx#commersanta_premiery... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2613168312.71 Safari%2F__... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginyazevskaya%2F5.0 (Windows NT 5.1) Prestorygoo... │ http://irr.ru/index.php?showalbum/login-kupilomatic.aspx#location=rr&days&city&... │ 17 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6HS │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-dlia-zhiensmed.ru/el... │ 17 │ -│ 1 │ 0 │ 0 │ http://bdsmpeople.ru/register2123 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logistereda.ru/photo/70861/6#f │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263161366&bid=2823&numpho... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowthreadreplies=24&ids=74&cu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273363331251798796&op_seo... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313&ussp=mstp&usln=1&inp... │ http://irr.ru/index.php?showalbum/login-3767436/roomootofile/obzor.com/iframe │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26041.htm%3Fsid%3D0%26pz%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27595,9305f3931339322%26n... │ 17 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/stan/index.kz/yandex │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertistia.html?partment/searc... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-search%3Futm_source=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031323532363&win=70&mode=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://diary.ru/forum/intries │ 17 │ -│ 1 │ 0 │ 0 │ http://bibieters_sig=5ea8QizwGSPy0yGcm14ATH74D4Bw&usg=AFQjCNHxMZlonova.ru/p1847... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 17 │ -│ 3 │ 22 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_style... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261636/detail/odnoklassim... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-futbolki-k... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lenfiki-zhienskaia-moda-muzhchin-sitafa ishq... │ http://irr.ru/index.php?showalbum/logical.ru/link=11485-9065383235348%2F&sr=htt... │ 17 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/messar.ru/professig │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 1 │ 0 │ 0 │ http://acase.php?input_who1=2&input_who2=1 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.10 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26618531&t=111246.html?p=... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694&option%2F12.14&he=76... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-irina19910735%2F27.0.1453.116 Safari │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268814453957595,94406/det... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733145625/details... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=Like Gecko) Chrome%2Fpotnik=1&price_do=&curr... │ http://irr.ru/index.php?showalbum/login-the-poxudet_sponsor=&o=6000708498/?caut... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b76faact/less=1&ru=1&... │ 17 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-53584715,1327288/belgium/arch=1&themec.... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=bestwo.diary/details&id=9160/0/posti.ri... │ 17 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/category=cinem rasmusic │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098707209463401090/deta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgas.rQRAX2Ua3IF1iHJcaz4ATv34DABw │ http://irr.ru/index.php?showalbum/login/?page=1024&wi=144438-88-3906740/currenc... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267044005320AA76)&vendor_... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-herokee_tatus=1&rm=18606bAxFEcQT6smBB4W... │ 16 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search?clid=19554786.0.8.0.2.2003558_3497926... │ http://irr.ru/index.php?showalbum/logie-niz-doktops/electronics/4185570c72 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27034195,9520160173/5/wom... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search/cuZXdzJTJGJmxyPTI4fQ,... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26396284046.xhtml%3Fhtml%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-plan... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&pt=b&pd=7&pw=2&page5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate/out-of-tow... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260612d7cf.4082813,74,756... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/page=91552&q=\... │ http://irr.ru/index.php │ 16 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login.aspx?sle=13&s_yers=0&page_type=0&door=0... │ 16 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/movinki... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?&clients-sale&siteurl%3D//ads/search?text... │ http://irr.ru/index.php?showalbum/login-kapusta-advert273786/foto-5/#photo=2817... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnaja-prost-petersburg/detail/?fold... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-krasok.html_params%3Drhost%3Dad.adriver │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661232&streamfood.com/if... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti/redmonds %2F offset=403134333.2201560&c... │ http://irr.ru/index.php?showalbum/login-v40-velopiter.feriod=0&scroll_to_auto.r... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiensmed.ru/dl/\xD0\xB7\x... │ 16 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-57548811426&text=\xD0\xBB\xD1\x83\xD1\x87\xD... │ http://irr.ru/index.php?showalbum/login-coolonellana-Molodilnik.ru/index.php?vi... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/70948/35.htm?id=... │ http://irr.ru/index.php?showalbum/logiy-luchaiev%2F&sr=http:%2F%2Fwww.bonprix c... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166303133334%2F&ei=horia+iudi... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26765.html_params%3Drhost... │ 16 │ -│ 1 │ 0 │ 0 │ http://histore/profile;u= │ http://irr.ru/index.php?showalbum/login-kupe-20013&uuid=1373356164564_1sort/sta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=findjob.ru/planet.ru/index.ru/carinov.h... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2755597.html%3Fhtml?1=1&c... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/showtopic,803813.html&... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_catalog/1250/?itemsg/cd... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert260477/cities/visas/exp?s... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://video.yandex.php │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2728024/detail/508/?instv... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26887301/Zona/simferopol.... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27429&input_bdsm_position... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-153-4b9c4.4331626.htm?size%3D0%26rleurl │ 16 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect │ http://komme%2F27.0.1453.116 │ 16 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://msuzie-shop/premiery-c-38208_2.html │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-marka=23&model&desting?field_town/search/pri... │ http://irr.ru/index.php?showalbum/login-492ea9&show/417582,9626512874":10157628... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/posts&q=\xD0\xB2\xD0\xB5\xD0\xB4\xD1\x83\xD1... │ http://irr.ru/index.php?showalbum/login-mistore/#CATALOG_LIST-s-ovosibirsk/deta... │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/sch/price][min]=41&lr=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27364229586121%26url%3D%2... │ 16 │ -│ 1 │ 0 │ 0 │ http://volgografiyah_27_iyunya_50_let%2FgetId │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-maritkiN │ http://irr.ru/index.php?showalbum/login-ladimir.irr.ru/remiery-telefon.ru │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333%26bid%3D1%26rleurl%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661251-6efa-d61f-fef3-01... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/tv-audi/a7-spec=9894797179698712/p1/8... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new=127201148][fr... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269899458/price/3815&pvno... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5.0 (Win... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=view/5/item3963... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/7427510/detail/5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti%2F%3Ff%3D100%&http://afisha.yandsearch&... │ http://irr.ru/index.php?showalbum/login-6030d.html_params%3Drhost%3D90%26height │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logiste's a di galaxy-s4-story1.aspx#location... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2753][from]=&pricedownloa... │ 16 │ -│ 1 │ 0 │ 0 │ http://yandsearch?lr=2&color │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2735030373434-152495.php?... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new10152954vac │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-gorod/search?p=7&oprnd=9902.jpg&img_url=http... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27222.vk.me/u3166 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691.html?1=1&input_age2/... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makeevka.ru/perm.irr.ru/slingvo/#1\xD0\... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-irinakai/page_num_read2306e4574&ei=Ot_rUfjPG... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26708662307][]=&selection... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313033393633310,932803][... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2010000087378687/pic/8939375f303839303935373... │ http://irr.ru/index.php?showalbum/login-souse=2/path=39_1.html?1=1&cid=577&oki=... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumy.ua/search?text=\xD1\x81\xD0\xB5\... │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/artira │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikal.ru/search/room=1&dam... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2695/19545602cf94d77/repl... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/?strict=5710... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertists/736850/?item_no=737&... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&model=1173... │ 15 │ -│ 1 │ 0 │ 0 │ http://forums/liiiervierk-suppohudeemvmesting/294465&op_categoriya │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=new&cad=rjt&fu=0&input_country_id=228&lr=961... │ http://irr.ru/index.php?showalbum/login-cam.shtml#objdesc=true&uA=Mozilla │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login/?do=ready-mansion%3D0%26url%3D%26bn%3D0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb=1&prepairs=0&city[1 │ http://irr.ru/index.php?showalbum/lounona-muzhchin-kupaljinik-chernyj-odnoe/san... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginsk.eu/ru/Pagesize=13&m1=07 │ http://irr.ru/index.php?showalbum/login-3muda/truction/vacancies/750207190765.h... │ 15 │ -│ 1 │ 0 │ 0 │ http://launcher-searchads/search │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-album/login.2/second... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_produkty/kiev.org/forum... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781308;IC,238208836746/0... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-4433140796851/detail/536180&all=False&i... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26981583/page=1089592445/... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945HPS?analog/r10418/sort=price_do=200&site_off=1 │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 15 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshop/id_art_type=7&s_yers │ http://irr.ru/index.php?showalbum/login │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F1.7.1364.172 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273389; U; Android 4.0.14... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2724/?_h=search?text=\xD0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-goluboj-podserial&dfs=13 │ http://irr.ru/index.php?showalbum/login-haus.html#news/228670,257&pci=3012/frl │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273631253711/foto-6320166... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#67 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266356163f39ea6f8/7a8745&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26605566254/room=1.6&wher... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/washes/built-in_two_ch... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?Topic │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/state=1909644 │ http://afisha.yandex.ua/auto_id=0&with_photo.kurortmag.ru/razdnitsya-vishnury │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://irr.ru/index.ru/show/414526863_112 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?transk.i... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/index.ru/ange=0&s_yers=(3000&static.diary.ru... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678&op_page=60000/curren... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginskikh-viana.irr.ru/GameMain.aspx?sle=1/house.... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-land_search?text=\xD0\xB0\xD0\xBC\xD0\xB1\xD... │ http://irr.ru/6323%26bn%3D27888895,963095425 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/produkty%2Fplatj... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list.html?typ=SMA&anbieter=aleks_evilkos.com │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?page37/?sta... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.1Uv1UfqCA8SonYC4BQ&usg=AFQjCNGB3pBUuKY1jJPP3... │ http://irr.ru/index.php?showalbum/login-tank-sale/search%3Fmode=&page_type │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=alini... │ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?Id=9582 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-tumbov.irr.ru/page1=&input_who1=2&id=29... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2718599/photo=0&is_hot=0&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liloveplanet.ru/futurer533/women.aspx?group-... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2631920&lo=http://person_... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.php?city[1]=700003 │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26761,55.654289600/detail... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27515/418695&st=327-B110Q │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login=vladimir/page_type=0&expand_search?text... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2699364224073532663835386... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?dok=001&rightbox/week&From=0&user │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/frl-4/trailer/view/3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-lential/secondary/details9.html_params%... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page9/#14... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=25127691%2F... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-676216b8af/4fd00fa61b3185631821/page_ty... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/gamemain.asp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1955451&lr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27877437494,943082&stat_t... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2730675595,9292fa-d61f-fe... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_appliazh-pliance/mista-bez-uchaiev/000202&cl... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/ch/floore troubleclick... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienshchin-planet.ru/myma... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27265484158197%26width%3D... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-102-821922237&arrFilter2013/08-iH4AT5zIGQDA │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_type=&freetao.diary.ru/sea... │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 15 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Ford-antent... │ 15 │ -│ 3 │ 123 │ 0 │ │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nashinger/users/search?textilead&353%26ev_pl... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginy_name/3196906481/currency=6465373200&price=\... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2776/?date=0&doorov.irr.r... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert276278-ploschaya_checked_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/877/ru/buildin... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274298799461981/?date=129... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert262997385f32313335781094&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2693_763613.html_params%3... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9584%26pz%3... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://ekburg.irr.ru%2Fpuloveplanet │ 15 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credir=1 │ http://komme%2F27.0.1453.116 │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.10 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=on&input_onliner... │ http://irr.ru/index.php?showalbum/login.html%26custom%3D%26CompPath.2; WOW64; r... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/gosthelp.ru/auto.ria.... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert269257798044.html_partmen... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2651734&pt=b&pd=7&pw=1&pr... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=on&in... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19&numphoto/login=A-CL-MS-smi.org%2F40 (KHTML, like Gecko) Version... │ http://irr.ru/index.ru/widgetchrome%2F&ti=no&dom_sell/pansiyskaya-obl.irr.html%... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga664.1721/deti74.ru/animals-planet.ru │ http://irr.ru/index.php?showalbum/login=K-SR-B-13-9635095,9661/9#f │ 15 │ -└────────────────┴────────────────┴─────────────┴────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.487 user 1.308000 sys 0.100000 -D 07-01' AND (DATE '1970-01-01' + 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; -Run Time: real 0.548 user 0.680000 sys 0.028000 -D D (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND DontCountHits = 0 AND URLHash = 686716256552154761 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Run Time: real 0.139 user 0.392000 sys 0.012000 -D " = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) ORDER BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))); -Run Time: real 0.017 user 0.200000 sys 0.020000 -D -D -D -D SELECT count(*) FROM hits;+ -┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 100000000 │ -└──────────────┘ -Run Time: real 0.012 user 0.180000 sys 0.000000 -D SELECT count(*) FROM hits WHERE AdvEngineID != 0; -d┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 630535 │ -└──────────────┘ -Run Time: real 0.154 user 0.400000 sys 0.052000 -D SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits; -┌──────────────────┬──────────────┬──────────────────────┐ -│ sum(advengineid) │ count_star() │ avg(resolutionwidth) │ -├──────────────────┼──────────────┼──────────────────────┤ -│ 7280824 │ 100000000 │ 1513.48908394 │ -└──────────────────┴──────────────┴──────────────────────┘ -Run Time: real 0.049 user 0.728000 sys 0.032000 -D SELECT sum(UserID) FROM hits;e -┌───────────────────────┐ -│ sum(userid) │ -├───────────────────────┤ -│ 3.230605869407883e+26 │ -└───────────────────────┘ -Run Time: real 0.471 user 0.608000 sys 0.088000 -D SELECT COUNT(DISTINCT UserID) FROM hits; -┌───────────────┐ -│ count(userid) │ -├───────────────┤ -│ 17630976 │ -└───────────────┘ -Run Time: real 8.319 user 10.196000 sys 0.916000 -D SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -o┌─────────────────────┐ -│ count(searchphrase) │ -├─────────────────────┤ -│ 6019589 │ -└─────────────────────┘ -Run Time: real 12.888 user 17.264000 sys 1.072000 -D SELECT min(EventDate), max(EventDate) FROM hits; -o┌────────────────┬────────────────┐ -│ min(eventdate) │ max(eventdate) │ -├────────────────┼────────────────┤ -│ 15888 │ 15917 │ -└────────────────┴────────────────┘ -Run Time: real 0.183 user 0.528000 sys 0.040000 -D SELECT AdvEngineID, count(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC;s -┌─────────────┬──────────────┐ -│ AdvEngineID │ count_star() │ -├─────────────┼──────────────┤ -│ 2 │ 404620 │ -│ 27 │ 113167 │ -│ 13 │ 45633 │ -│ 45 │ 38974 │ -│ 44 │ 9731 │ -│ 3 │ 6896 │ -│ 62 │ 5266 │ -│ 52 │ 3554 │ -│ 50 │ 938 │ -│ 28 │ 836 │ -│ 53 │ 350 │ -│ 25 │ 343 │ -│ 61 │ 158 │ -│ 21 │ 38 │ -│ 42 │ 20 │ -│ 16 │ 7 │ -│ 7 │ 3 │ -│ 22 │ 1 │ -└─────────────┴──────────────┘ -Run Time: real 0.048 user 0.684000 sys 0.000000 -D SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10;) -┌──────────┬─────────┐ -│ RegionID │ u │ -├──────────┼─────────┤ -│ 229 │ 2845673 │ -│ 2 │ 1081016 │ -│ 208 │ 831676 │ -│ 169 │ 604583 │ -│ 184 │ 322661 │ -│ 158 │ 307152 │ -│ 34 │ 299479 │ -│ 55 │ 286525 │ -│ 107 │ 272448 │ -│ 42 │ 243181 │ -└──────────┴─────────┘ -Run Time: real 10.040 user 11.608000 sys 1.272000 -D SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10;i -┌──────────┬──────────────────┬──────────┬──────────────────────┬───────────────┐ -│ RegionID │ sum(advengineid) │ c │ avg(resolutionwidth) │ count(userid) │ -├──────────┼──────────────────┼──────────┼──────────────────────┼───────────────┤ -│ 229 │ 2078084 │ 18296430 │ 1506.0876750819696 │ 2845673 │ -│ 2 │ 441711 │ 6687708 │ 1479.8410618406187 │ 1081016 │ -│ 208 │ 285925 │ 4261945 │ 1285.260504769536 │ 831676 │ -│ 169 │ 100887 │ 3320286 │ 1465.90517142198 │ 604583 │ -│ 32 │ 81498 │ 1843721 │ 1538.0370495318978 │ 216010 │ -│ 34 │ 161779 │ 1792406 │ 1548.364990409539 │ 299479 │ -│ 184 │ 55526 │ 1755223 │ 1506.8102679830426 │ 322661 │ -│ 42 │ 108820 │ 1542771 │ 1587.1074287758845 │ 243181 │ -│ 107 │ 120470 │ 1516722 │ 1548.6039623609336 │ 272448 │ -│ 51 │ 98212 │ 1435598 │ 1579.8864215469791 │ 211505 │ -└──────────┴──────────────────┴──────────┴──────────────────────┴───────────────┘ -Run Time: real 10.689 user 13.532000 sys 1.120000 -D SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(MobilePhoneModel) > 0 GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;e -┌──────────────────┬─────────┐ -│ MobilePhoneModel │ u │ -├──────────────────┼─────────┤ -│ iPad │ 1090347 │ -│ iPhone │ 45758 │ -│ A500 │ 16046 │ -│ N8-00 │ 5565 │ -│ iPho │ 3300 │ -│ ONE TOUCH 6030A │ 2759 │ -│ GT-P7300B │ 1907 │ -│ 3110000 │ 1871 │ -│ GT-I9500 │ 1598 │ -│ eagle75 │ 1492 │ -└──────────────────┴─────────┘ -Run Time: real 2.253 user 6.108000 sys 0.468000 -D SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(MobilePhoneModel) > 0 GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;, -┌─────────────┬──────────────────┬────────┐ -│ MobilePhone │ MobilePhoneModel │ u │ -├─────────────┼──────────────────┼────────┤ -│ 1 │ iPad │ 931038 │ -│ 5 │ iPad │ 48385 │ -│ 6 │ iPad │ 29710 │ -│ 7 │ iPad │ 28391 │ -│ 118 │ A500 │ 16005 │ -│ 6 │ iPhone │ 14516 │ -│ 26 │ iPhone │ 13566 │ -│ 10 │ iPad │ 11433 │ -│ 32 │ iPad │ 9503 │ -│ 13 │ iPad │ 9417 │ -└─────────────┴──────────────────┴────────┘ -Run Time: real 2.528 user 6.544000 sys 0.508000 -D SELECT SearchPhrase, count(*) AS c FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;+ -┌────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchPhrase │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 70263 │ -│ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 34675 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 24579 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 21649 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 19703 │ -│ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 19195 │ -│ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 17284 │ -│ galaxy table │ 16746 │ -│ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 16620 │ -│ \xD1\x81\xD0\xBA\xD0\xBE\xD0\xBB\xD1\x8C\xD0\xBA\xD0\xBE \xD0\xBC\xD1\x8B\xD1\x... │ 12317 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 1.127 user 16.040000 sys 0.016000 -D SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;m -┌────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchPhrase │ u │ -├────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 23673 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 19743 │ -│ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 18394 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 17553 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 14600 │ -│ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 14529 │ -│ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 14198 │ -│ \xD1\x81\xD0\xBA\xD0\xBE\xD0\xBB\xD1\x8C\xD0\xBA\xD0\xBE \xD0\xBC\xD1\x8B\xD1\x... │ 9007 │ -│ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 8792 │ -│ \xD0\xBA\xD0\xBE\xD0\xBC\xD0\xB1\xD0\xB8\xD0\xBD\xD0\xB8\xD1\x80\xD0\xBE\xD0\xB... │ 7572 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 13.811 user 18.612000 sys 1.096000 -D SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;m -┌────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchEngineID │ SearchPhrase │ c │ -├────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ 2 │ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 46258 │ -│ 2 │ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 18871 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 16905 │ -│ 3 │ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 16748 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 14911 │ -│ 2 │ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 13716 │ -│ 2 │ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 13414 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 13105 │ -│ 3 │ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 12815 │ -│ 2 │ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 11946 │ -└────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 1.170 user 16.316000 sys 0.060000 -D SELECT UserID, count(*) FROM hits GROUP BY UserID ORDER BY count(*) DESC LIMIT 10;R -┌─────────────────────┬──────────────┐ -│ UserID │ count_star() │ -├─────────────────────┼──────────────┤ -│ 1313338681122956954 │ 29097 │ -│ 1907779576417363396 │ 25333 │ -│ 2305303682471783379 │ 10611 │ -│ 7982623143712728547 │ 7584 │ -│ 6018350421959114808 │ 6678 │ -│ 7280399273658728997 │ 6411 │ -│ 1090981537032625727 │ 6197 │ -│ 5730251990344211405 │ 6019 │ -│ 835157184735512989 │ 5211 │ -│ 770542365400669095 │ 4906 │ -└─────────────────────┴──────────────┘ -Run Time: real 0.961 user 12.416000 sys 0.028000 -D SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10;l -┌─────────────────────┬──────────────┬──────────────┐ -│ UserID │ SearchPhrase │ count_star() │ -├─────────────────────┼──────────────┼──────────────┤ -│ 1313338681122956954 │ │ 29097 │ -│ 1907779576417363396 │ │ 25333 │ -│ 2305303682471783379 │ │ 10611 │ -│ 7982623143712728547 │ │ 6669 │ -│ 7280399273658728997 │ │ 6408 │ -│ 1090981537032625727 │ │ 6196 │ -│ 5730251990344211405 │ │ 6019 │ -│ 6018350421959114808 │ │ 5990 │ -│ 835157184735512989 │ │ 5209 │ -│ 770542365400669095 │ │ 4906 │ -└─────────────────────┴──────────────┴──────────────┘ -Run Time: real 2.309 user 29.740000 sys 0.256000 -D SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10;h -┌──────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬──────────────┐ -│ UserID │ SearchPhrase │ count_star() │ -├──────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼──────────────┤ -│ 2033505069917754133 │ │ 13 │ -│ 4177603680108851335 │ │ 2 │ -│ 2865865758754579169 │ │ 4 │ -│ 1348434893485989289 │ │ 1 │ -│ 4623542685155091116 │ \xD0\xB0\xD0\xB2\xD1\x82\xD0\xBE\xD1\x80\xD0\xBE\xD0\xB9 \xD0\xBF\xD0\xB5\xD1\x... │ 1 │ -│ 6824890550049353433 │ │ 14 │ -│ 1722975923542031 │ │ 2 │ -│ 1510416351411292943 │ \xD0\xBC\xD0\xB5\xD0\xB9\xD0\xBD\xD1\x8B\xD1\x85 \xD0\xB2 \xD0\xB0\xD0\xB2\xD1\... │ 2 │ -│ 10675348576094566371 │ │ 1 │ -│ 5485742187896741713 │ \xD0\xBF\xD0\xBE\xD0\xB3\xD0\xBE\xD0\xB4\xD0\xB0 │ 3 │ -└──────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴──────────────┘ -Run Time: real 2.013 user 29.824000 sys 0.232000 -D SELECT UserID, extract(minute FROM (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) AS m, SearchPhrase, count(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10;) -┌─────────────────────┬────┬──────────────┬──────────────┐ -│ UserID │ m │ SearchPhrase │ count_star() │ -├─────────────────────┼────┼──────────────┼──────────────┤ -│ 1313338681122956954 │ 31 │ │ 589 │ -│ 1313338681122956954 │ 28 │ │ 578 │ -│ 1313338681122956954 │ 29 │ │ 572 │ -│ 1313338681122956954 │ 33 │ │ 567 │ -│ 1313338681122956954 │ 27 │ │ 557 │ -│ 1313338681122956954 │ 32 │ │ 554 │ -│ 1313338681122956954 │ 30 │ │ 552 │ -│ 1313338681122956954 │ 34 │ │ 546 │ -│ 1313338681122956954 │ 26 │ │ 540 │ -│ 1313338681122956954 │ 10 │ │ 539 │ -└─────────────────────┴────┴──────────────┴──────────────┘ -Run Time: real 4.940 user 52.524000 sys 1.056000 -D SELECT UserID FROM hits WHERE UserID = 12345678901234567890; -Run Time: real 0.258 user 0.256000 sys 0.004000 -D SELECT count(*) FROM hits WHERE URL::TEXT LIKE '%metrika%';e -┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 211 │ -└──────────────┘ -Run Time: real 2.626 user 41.564000 sys 0.000000 -D SELECT SearchPhrase, min(URL), count(*) AS c FROM hits WHERE URL::TEXT LIKE '%metrika%' AND octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;u -┌────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───┐ -│ SearchPhrase │ min(url) │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───┤ -│ \xD0\xBC\xD1\x8B\xD1\x81\xD0\xBB \xD0\xBF\xD0\xB5\xD1\x81\xD0\xB5\xD0\xBD \xD1\... │ http://smeshariki.ru/a-folder=cmarka=15&sort=&sll=36872/metrika/frl-2/bage-all/... │ 1 │ -│ \xD0\xB2\xD0\xB8\xD0\xB4\xD0\xB5\xD0\xBE \xD1\x82\xD0\xB0\xD0\xBA\xD0\xBE\xD0\x... │ http://maximum_438424&pvno=2&evlg=VC,0;VL,628;IC,1653-82676212&op_page=0&door=0... │ 1 │ -│ \xD1\x82\xD0\xB5\xD0\xBA\xD1\x81\xD1\x82\xD1\x8B \xD0\xBF\xD0\xBE\xD0\xB4\xD1\x... │ http://bdsmpeople.ru/index.ru/metrikadeletedAutoSearch │ 1 │ -│ \xD1\x83\xD0\xBD\xD0\xB8\xD0\xB2\xD0\xB5\xD1\x80\xD0\xBC\xD0\xB5\xD0\xB4\xD0\xB... │ http://smeshariki.ru/index.ua/syllanet.ru/busine-tyazin?model=4878/page=10&cate... │ 1 │ -│ \xD0\xB4\xD0\xB8\xD0\xBD\xD0\xB0 \xD0\xB2\xD0\xB0\xD0\xBA\xD0\xB0\xD0\xBD\xD1\x... │ http://ecrn.ru/personal/gost277572,9589&pt=b&pd=8&pw=2&page3/?state=0&damages/0... │ 1 │ -│ \xD1\x82\xD1\x80\xD1\x83\xD0\xB4\xD0\xB0 \xD0\xB4\xD0\xBB\xD1\x8F \xD0\xB2\xD1\... │ http://smeshariki.ru/recipes/sadovka.ru/gorod55.ru/search?text=\xD0\xB8\xD0\xB7... │ 1 │ -│ \xD0\xBA\xD0\xBE\xD1\x88\xD0\xBA\xD1\x83 \xD1\x81\xD0\xB7\xD0\xB0\xD0\xBE \xD1\... │ http://bdsmpeople.ru/index.ru/metrikadeletedAutoSearch │ 1 │ -│ \xD1\x80\xD0\xB5\xD1\x86\xD0\xB5\xD0\xBF\xD1\x82 │ http://smeshariki.ru/index.ua/search/metrikanske-urali-belorussia/28.html?1=1&c... │ 1 │ -│ \xD0\xB4\xD0\xB8\xD0\xBD\xD0\xB0 \xD0\xB2\xD0\xB0\xD0\xBA\xD0\xB0\xD0\xBD\xD1\x... │ http://ecrn.ru/personal/gost277572,9589&pt=b&pd=8&pw=2&page3/?state=0&damages/0... │ 1 │ -│ \xD1\x80\xD0\xB0\xD0\xB1\xD0\xBE\xD0\xB2\xD1\x8B\xD0\xB5 \xD0\xB4\xD0\xBE\xD1\x... │ http://smeshariki.ru/index.ua/auto.ria.ua/change=0&s_yers=0&with_video.yandex.u... │ 1 │ -└────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───┘ -Run Time: real 0.716 user 11.292000 sys 0.000000 -D count(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title::TEXT LIKE '%Яндекс%' AND URL::TEXT NOT LIKE '%.yandex.%' AND octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;( -Run Time: real 3.669 user 19.464000 sys 0.932000 -D SELECT * FROM hits WHERE URL::TEXT LIKE '%metrika%' ORDER BY EventTime LIMIT 10;e -┌─────────────────────┬────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┬────────────┬───────────┬───────────┬────────────┬──────────┬─────────────────────┬──────────────┬─────┬───────────┬────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬─────────┬───────────────────┬─────────────────┬───────────────┬─────────────┬─────────────────┬──────────────────┬─────────────────┬────────────┬────────────┬─────────────┬──────────┬──────────┬────────────────┬────────────────┬──────────────┬──────────────────┬──────────┬─────────────┬──────────────────┬────────┬─────────────┬────────────────┬────────────────┬──────────────┬─────────────┬─────────────┬───────────────────┬────────────────────┬────────────────┬─────────────────┬─────────────────────┬─────────────────────┬─────────────────────┬─────────────────────┬──────────────────────┬─────────────┬────────┬────────────┬─────────────┬─────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┬──────────────┬─────────┬─────────────┬───────────────┬──────────┬──────────┬────────────────┬─────┬─────┬────────┬───────────┬───────────┬────────────┬────────────┬────────────┬───────────────┬─────────────────┬────────────────┬───────────────┬──────────────┬───────────┬────────────┬───────────┬───────────────┬─────────────────────┬───────────────────┬─────────────┬───────────────────────┬──────────────────┬────────────┬──────────────┬───────────────┬─────────────────┬─────────────────────┬────────────────────┬──────────────┬──────────────────┬───────────┬───────────┬─────────────┬────────────┬─────────┬─────────┬──────────┬──────────────────────┬──────────────────────┬──────┐ -│ WatchID │ JavaEnable │ Title │ GoodEvent │ EventTime │ EventDate │ CounterID │ ClientIP │ RegionID │ UserID │ CounterClass │ OS │ UserAgent │ URL │ Referer │ Refresh │ RefererCategoryID │ RefererRegionID │ URLCategoryID │ URLRegionID │ ResolutionWidth │ ResolutionHeight │ ResolutionDepth │ FlashMajor │ FlashMinor │ FlashMinor2 │ NetMajor │ NetMinor │ UserAgentMajor │ UserAgentMinor │ CookieEnable │ JavascriptEnable │ IsMobile │ MobilePhone │ MobilePhoneModel │ Params │ IPNetworkID │ TraficSourceID │ SearchEngineID │ SearchPhrase │ AdvEngineID │ IsArtifical │ WindowClientWidth │ WindowClientHeight │ ClientTimeZone │ ClientEventTime │ SilverlightVersion1 │ SilverlightVersion2 │ SilverlightVersion3 │ SilverlightVersion4 │ PageCharset │ CodeVersion │ IsLink │ IsDownload │ IsNotBounce │ FUniqID │ OriginalURL │ HID │ IsOldCounter │ IsEvent │ IsParameter │ DontCountHits │ WithHash │ HitColor │ LocalEventTime │ Age │ Sex │ Income │ Interests │ Robotness │ RemoteIP │ WindowName │ OpenerName │ HistoryLength │ BrowserLanguage │ BrowserCountry │ SocialNetwork │ SocialAction │ HTTPError │ SendTiming │ DNSTiming │ ConnectTiming │ ResponseStartTiming │ ResponseEndTiming │ FetchTiming │ SocialSourceNetworkID │ SocialSourcePage │ ParamPrice │ ParamOrderID │ ParamCurrency │ ParamCurrencyID │ OpenstatServiceName │ OpenstatCampaignID │ OpenstatAdID │ OpenstatSourceID │ UTMSource │ UTMMedium │ UTMCampaign │ UTMContent │ UTMTerm │ FromTag │ HasGCLID │ RefererHash │ URLHash │ CLID │ -├─────────────────────┼────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┼────────────┼───────────┼───────────┼────────────┼──────────┼─────────────────────┼──────────────┼─────┼───────────┼────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼─────────┼───────────────────┼─────────────────┼───────────────┼─────────────┼─────────────────┼──────────────────┼─────────────────┼────────────┼────────────┼─────────────┼──────────┼──────────┼────────────────┼────────────────┼──────────────┼──────────────────┼──────────┼─────────────┼──────────────────┼────────┼─────────────┼────────────────┼────────────────┼──────────────┼─────────────┼─────────────┼───────────────────┼────────────────────┼────────────────┼─────────────────┼─────────────────────┼─────────────────────┼─────────────────────┼─────────────────────┼──────────────────────┼─────────────┼────────┼────────────┼─────────────┼─────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┼──────────────┼─────────┼─────────────┼───────────────┼──────────┼──────────┼────────────────┼─────┼─────┼────────┼───────────┼───────────┼────────────┼────────────┼────────────┼───────────────┼─────────────────┼────────────────┼───────────────┼──────────────┼───────────┼────────────┼───────────┼───────────────┼─────────────────────┼───────────────────┼─────────────┼───────────────────────┼──────────────────┼────────────┼──────────────┼───────────────┼─────────────────┼─────────────────────┼────────────────────┼──────────────┼──────────────────┼───────────┼───────────┼─────────────┼────────────┼─────────┼─────────┼──────────┼──────────────────────┼──────────────────────┼──────┤ -│ 9008228978173248400 │ 1 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372714203 │ 15888 │ 46429 │ 1741285710 │ 208 │ 727970985383478408 │ 0 │ 2 │ 2 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ http://smeshariki.ru/page%3D%26ev_n%3Dtvor_3_4 │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 1638 │ 1658 │ 37 │ 14 │ 0 │ 700.22 │ 0 │ 0 │ 15 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 2349927 │ -1 │ 0 │ │ 0 │ 0 │ 1844 │ 888 │ 135 │ 1372711075 │ 3 │ 0 │ 29241 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 0 │ │ 926364569 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372755313 │ 31 │ 1 │ 0 │ 0 │ 0 │ 1985697352 │ 13001 │ -1 │ 22 │ vG │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 16989158750233735299 │ 2619784076535420345 │ 0 │ -│ 7319686668886932386 │ 0 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372742048 │ 15888 │ 46429 │ 675534320 │ 208 │ 1167829038705361511 │ 0 │ 45 │ 1 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ http://smeshariki.ru/page=6&sqi=2&ved=0CC0QFjAA │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 280 │ 733 │ 0 │ 0 │ 0 │ │ 0 │ 0 │ 4 │ sO │ 1 │ 1 │ 1 │ 0 │ │ │ 1444853 │ -1 │ 0 │ │ 0 │ 0 │ 997 │ 800 │ 623 │ 1372713898 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 4655941889681510909 │ │ 721190565 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372747193 │ 31 │ 2 │ 3 │ 717 │ 0 │ 2935426383 │ 22084 │ -1 │ 1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 11629579472615090596 │ 2619784076535420345 │ 0 │ -│ 5650132582613869356 │ 0 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372742083 │ 15888 │ 46429 │ 675534320 │ 208 │ 1167829038705361511 │ 0 │ 45 │ 1 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ http://smeshariki.ru/page=6&sqi=2&ved=0CC0QFjAA │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 280 │ 733 │ 0 │ 0 │ 0 │ │ 0 │ 0 │ 4 │ D\xE0 │ 1 │ 1 │ 1 │ 0 │ │ │ 1444853 │ -1 │ 0 │ │ 0 │ 0 │ 997 │ 800 │ 623 │ 1372713960 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 4655941889681510909 │ │ 721255326 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372747245 │ 31 │ 2 │ 3 │ 717 │ 0 │ 2935426383 │ 64225 │ -1 │ 1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 11629579472615090596 │ 2619784076535420345 │ 0 │ -│ 6211862319117980382 │ 0 │ \xD0\x9A\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ 1 │ 1372744794 │ 15888 │ 46429 │ 1384976586 │ 208 │ 45205883125357727 │ 0 │ 8 │ 1 │ http://smeshariki.ru/used/Audi-1993527&stateID=0&metrikauto/bmw_74012099/0/&&pu... │ │ 0 │ 0 │ 0 │ 9911 │ 216 │ 475 │ 368 │ 0 │ 0 │ 0 │ │ 0 │ 0 │ 7 │ D\xE0 │ 1 │ 1 │ 1 │ 0 │ │ │ 3195177 │ 0 │ 0 │ │ 0 │ 0 │ 412 │ 286 │ 135 │ 1372732810 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 7015666733105959947 │ │ 641313121 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372774931 │ 28 │ 1 │ 2 │ 1 │ 0 │ 1889444196 │ 44835 │ -1 │ 0 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 18150585289071012696 │ 2619784076535420345 │ 0 │ -│ 8127216893234548048 │ 0 │ \xD0\xA2\xD0\xBE\xD0\xBB\xD1\x81\xD1\x82\xD1\x83\xD1\x85\xD1\x83, \xD0\xB5\xD1\... │ 1 │ 1372752539 │ 15888 │ 46429 │ 1237806922 │ 208 │ 750085446646698731 │ 0 │ 2 │ 3 │ http://smeshariki.ru/used/99114578622-14406072,9269587/roometrika/?page=7&pw=2&... │ http://smeshariki.ru/page=11956bd3f5ba-bolshiient │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 1081 │ 979 │ 23 │ 15 │ 7 │ 700 │ 0 │ 0 │ 17 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 2914423 │ -1 │ 0 │ │ 0 │ 0 │ 1434 │ 739 │ 623 │ 1372781865 │ 0 │ 0 │ 0 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 5496132257355632424 │ │ 983272850 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372760187 │ 50 │ 2 │ 2 │ 60 │ 18 │ 1113465640 │ 60610 │ -1 │ 29 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 194 │ 1631 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 13570011243001147277 │ 11339053197878713733 │ 0 │ -│ 5016976626924998727 │ 1 │ Toyota \xD1\x80\xD1\x83\xD0\xBA\xD0\xB0\xD0\xBB\xD1\x8B \xD0\xB4\xD0\xBB\xD1\x8... │ 1 │ 1372758784 │ 15888 │ 5645 │ 3888153915 │ 107 │ 4058808124307537573 │ 1 │ 2 │ 88 │ http:%2F%2Fwwww.bonprix.ru&pvid=131&op_products/transformality.pulsceness/01-me... │ http://forum.amur.info/node/12451180167540 │ 0 │ 10868 │ 635 │ 16361 │ 943 │ 1638 │ 777 │ 37 │ 15 │ 7 │ 700 │ 0 │ 0 │ 1 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 2726799 │ -1 │ 0 │ │ 0 │ 0 │ 1509 │ 618 │ 135 │ 1372835656 │ 0 │ 0 │ 0 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 6227398498798751865 │ │ 255959698 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372763371 │ 31 │ 1 │ 3 │ 3555 │ 13 │ 4006630121 │ -1 │ -1 │ -1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 10333931216342739575 │ 10009133566342666602 │ 0 │ -│ 4963765960745323978 │ 0 │ \xD0\x93\xD0\x90\xD0\x97 (ZAZ) 5403J \xE2\x80\x93 \xD0\xA4\xD0\xB8\xD0\xBB\xD1\... │ 1 │ 1372793626 │ 15888 │ 46429 │ 2063319617 │ 23967 │ 5283184799411504286 │ 0 │ 44 │ 2 │ http://smeshariki.ru/userId=0&matched_car=Volkswagency=1&p=8090/roometrika/?pag... │ http://smeshariki.ru/smsarhiv/num/33363634383294&lr=66&v6s=2&bodystyle │ 0 │ 16000 │ 158 │ 9911 │ 216 │ 1996 │ 1781 │ 37 │ 15 │ 7 │ 700 │ 0 │ 0 │ 10 │ nA │ 1 │ 1 │ 0 │ 0 │ │ │ 4110783 │ -1 │ 0 │ │ 0 │ 0 │ 1551 │ 955 │ 291 │ 1372730435 │ 7 │ 1 │ 9577 │ 0 │ windows-1251;charset │ 1601 │ 0 │ 0 │ 0 │ 8904020920948834668 │ │ 883138215 │ 0 │ 0 │ 0 │ 0 │ 0 │ g │ 1372777308 │ 22 │ 2 │ 2 │ 11339 │ 18 │ 2136940575 │ 1653 │ -1 │ 1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 18276268115953212999 │ 9484754903086635093 │ 0 │ -│ 6284148982888572412 │ 1 │ Morskoj port \xE2\x80\x93 \xD0\x9D\xD0\xBE\xD0\xB2\xD0\xBE\xD1\x81\xD1\x82\xD1\... │ 1 │ 1372795222 │ 15889 │ 3922 │ 2460998382 │ 40 │ 1643466856862289966 │ 1 │ 44 │ 7 │ http://moikrug.ru/message-12-kak-pravdorubovnik_metrika-info.php?f=23&prr │ http://diary.ru/GameMain.aspx?d=1412&lr=75&mode=photo/login=igorod.irr.ru/i6102... │ 0 │ 306 │ 29199 │ 92 │ 247 │ 1368 │ 554 │ 37 │ 15 │ 7 │ 800.94 │ 0 │ 0 │ 12 │ D\xE0 │ 1 │ 1 │ 0 │ 0 │ │ │ 1166094 │ 1 │ 0 │ │ 0 │ 0 │ 1253 │ 642 │ 135 │ 1372799147 │ 4 │ 1 │ 16561 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 7061143530822060136 │ │ 699865379 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372810401 │ 31 │ 2 │ 3 │ 6 │ 0 │ 3849445958 │ -1 │ -1 │ -1 │ S0 │ h1 │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 14060834305267311691 │ 1791983733815963315 │ 0 │ -│ 6024648629746505393 │ 0 │ \xD0\x9F\xD0\xBE\xD0\xB3\xD1\x80\xD0\xB5\xD0\xB2\xD0\xB0\xD1\x82\xD0\xB5\xD0\xB... │ 1 │ 1372795271 │ 15889 │ 1200 │ 2932550360 │ 208 │ 1578473929930714515 │ 1 │ 107 │ 82 │ http://afisha.yandex.ru/real-estate/out-of-town/household_app_metrika.ru/wildbe... │ http://ad.adrive_type_id=1959251&stUrl%3Dpopular/kw/306161&lr=1418][to]=&input_... │ 0 │ 15265 │ 19757 │ 8953 │ 32252 │ 958 │ 1871 │ 37 │ 0 │ 0 │ │ 0 │ 0 │ 3 │ D\xE0 │ 1 │ 1 │ 1 │ 6 │ │ │ 4010641 │ -1 │ 0 │ │ 0 │ 0 │ 521 │ 1803 │ -1 │ 1372870507 │ 0 │ 0 │ 0 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 0 │ http://b.kavanga.ru/tags/%FD%ED%FB%E2%F3%E6%E5%E6%ED%E8%ED%F1%FF%ED&ti=%D0%BD%D... │ 120241233 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372842094 │ 0 │ 0 │ 0 │ 0 │ 0 │ 3975911785 │ -1 │ -1 │ -1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 9685011611637290144 │ 5941547189901908071 │ 0 │ -│ 4857420640250996887 │ 1 │ Suzuki Escort 4* (\xD0\xA4\xD1\x80\xD0\xB0\xD0\xBD\xD1\x86\xD1\x8B 3 \xD1\x81\x... │ 1 │ 1372800737 │ 15889 │ 3922 │ 3201307115 │ 190 │ 7335986183190726964 │ 1 │ 44 │ 88 │ http://pova-ul-mir.irr.ru/search.php?gidcar=37408&uuid=1&bc=3&city=0&pv=20&s_ye... │ http://news/238/~6/?cauth=1&av=1&nm=1&ms=1,2/currency=RUR/page_num_id=15366563&... │ 0 │ 306 │ 144 │ 304 │ 132 │ 1368 │ 554 │ 37 │ 15 │ 4 │ 202 │ 0 │ 0 │ 1 │ fi │ 1 │ 1 │ 0 │ 0 │ │ │ 2311071 │ -1 │ 0 │ │ 0 │ 0 │ 1333 │ 924 │ 322 │ 1372840359 │ 4 │ 1 │ 16561 │ 0 │ windows │ 1 │ 0 │ 0 │ 0 │ 7659179697273795837 │ │ 232010762 │ 0 │ 0 │ 0 │ 0 │ 0 │ 5 │ 1372866397 │ 31 │ 2 │ 2 │ 3658 │ 28 │ 2920265313 │ -1 │ -1 │ -1 │ S0 │ \xD0\x0C │ │ │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 0 │ │ NH\x1C │ 0 │ │ │ │ │ │ │ │ │ │ │ 0 │ 8839345929686869081 │ 5449259806403761803 │ 0 │ -└─────────────────────┴────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┴────────────┴───────────┴───────────┴────────────┴──────────┴─────────────────────┴──────────────┴─────┴───────────┴────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴─────────┴───────────────────┴─────────────────┴───────────────┴─────────────┴─────────────────┴──────────────────┴─────────────────┴────────────┴────────────┴─────────────┴──────────┴──────────┴────────────────┴────────────────┴──────────────┴──────────────────┴──────────┴─────────────┴──────────────────┴────────┴─────────────┴────────────────┴────────────────┴──────────────┴─────────────┴─────────────┴───────────────────┴────────────────────┴────────────────┴─────────────────┴─────────────────────┴─────────────────────┴─────────────────────┴─────────────────────┴──────────────────────┴─────────────┴────────┴────────────┴─────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┴──────────────┴─────────┴─────────────┴───────────────┴──────────┴──────────┴────────────────┴─────┴─────┴────────┴───────────┴───────────┴────────────┴────────────┴────────────┴───────────────┴─────────────────┴────────────────┴───────────────┴──────────────┴───────────┴────────────┴───────────┴───────────────┴─────────────────────┴───────────────────┴─────────────┴───────────────────────┴──────────────────┴────────────┴──────────────┴───────────────┴─────────────────┴─────────────────────┴────────────────────┴──────────────┴──────────────────┴───────────┴───────────┴─────────────┴────────────┴─────────┴─────────┴──────────┴──────────────────────┴──────────────────────┴──────┘ -Run Time: real 38.169 user 135.808000 sys 9.856000 -D SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY EventTime LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ \xD1\x81\xD0\xB8\xD0\xBC\xD0\xBF\xD1\x82\xD0\xBE\xD0\xBC\xD1\x8B \xD1\x80\xD0\x... │ -│ galaxy s4 zoom \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC │ -│ \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC \xD0\xBD\xD0\xB5\xD0\xB1\xD0\xBE\xD0\x... │ -│ \xD0\xBD\xD0\xBE\xD1\x87\xD0\xBD\xD0\xBE \xD0\xBA\xD0\xB8\xD1\x82\xD0\xB0\xD1\x... │ -│ \xD0\xB0\xD0\xB2\xD0\xBE\xD0\xBC \xD0\xBA\xD0\xBE\xD0\xBD\xD1\x81\xD1\x82\xD0\x... │ -│ \xD0\xBE\xD1\x82\xD0\xB4\xD1\x8B\xD1\x85\xD0\xB0 \xD1\x87\xD0\xB5\xD0\xBC \xD0\... │ -│ \xD1\x81\xD0\xBA\xD0\xB0\xD1\x87\xD0\xB0\xD1\x82\xD1\x8C \xD1\x87\xD0\xB8\xD1\x... │ -│ \xD0\xB0\xD0\xBD\xD0\xB0\xD0\xBF\xD0\xB0 \xD0\xBE\xD0\xBF\xD0\xB5\xD1\x80\xD0\x... │ -│ \xD1\x81\xD0\xBB\xD0\xBE\xD0\xBD.\xD1\x80\xD1\x83\xD0\xB1., \xD0\xB4. \xD0\xB0.... │ -│ \xD1\x80\xD0\xB0\xD1\x81\xD0\xBF\xD0\xB8\xD1\x81\xD0\xB0\xD0\xBD\xD0\xB8\xD0\xB... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 0.227 user 3.484000 sys 0.080000 -D SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY SearchPhrase LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ ! hektdf gjcgjhn conster │ -│ ! \xD1\x81\xD0\xBA\xD0\xB0\xD1\x80\xD0\xBF │ -│ !(\xD0\xBA\xD0\xB0\xD0\xBA \xD0\xB2\xD0\xBE\xD1\x80\xD0\xBE\xD0\xBD\xD0\xB8 │ -│ !(\xD0\xBF\xD0\xBE \xD0\xB3\xD0\xBE\xD1\x80\xD0\xB8\xD1\x8E \xD0\xB2 \xD1\x8F\x... │ -│ !(\xD1\x81) \xD0\xBF\xD1\x80\xD0\xBE \xD0\xB4\xD0\xBF\xD0\xBE \xD1\x81\xD0\xB5\... │ -│ !(\xD1\x81\xD0\xB0\xD0\xBB\xD0\xBE\xD0\xBD\xD1\x8B \xD0\xBE\xD1\x81\xD1\x82\xD0... │ -│ !(\xD1\x81\xD1\x82\xD0\xB0\xD1\x80\xD1\x82\xD0\xB5\xD1\x80 rav4 \xD1\x82\xD1\x8... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD0\xB4\xD0\xBB\xD1\x8F \xD0\xBC\xD1\x8F\xD1... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD0\xBC\xD0\xB0\xD1\x83\xD1\x81 \xD0\xBA\xD0... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD1\x81\xD0\xB5\xD1\x80\xD0\xB8\xD0\xB8 │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 0.271 user 3.996000 sys 0.092000 -D SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY EventTime, SearchPhrase LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ galaxy s4 zoom \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC │ -│ \xD0\xBD\xD0\xBE\xD1\x87\xD0\xBD\xD0\xBE \xD0\xBA\xD0\xB8\xD1\x82\xD0\xB0\xD1\x... │ -│ \xD1\x81\xD0\xB8\xD0\xBC\xD0\xBF\xD1\x82\xD0\xBE\xD0\xBC\xD1\x8B \xD1\x80\xD0\x... │ -│ \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC \xD0\xBD\xD0\xB5\xD0\xB1\xD0\xBE\xD0\x... │ -│ \xD0\xB0\xD0\xB2\xD0\xBE\xD0\xBC \xD0\xBA\xD0\xBE\xD0\xBD\xD1\x81\xD1\x82\xD0\x... │ -│ \xD0\xB0\xD0\xBD\xD0\xB0\xD0\xBF\xD0\xB0 \xD0\xBE\xD0\xBF\xD0\xB5\xD1\x80\xD0\x... │ -│ \xD0\xB1\xD1\x80\xD0\xB8\xD1\x82\xD0\xB0 \xD0\xB3\xD0\xB0\xD0\xBD\xD0\xB0\xD0\x... │ -│ \xD0\xBA\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ -│ \xD0\xBE\xD1\x82\xD0\xB4\xD1\x8B\xD1\x85\xD0\xB0 \xD1\x87\xD0\xB5\xD0\xBC \xD0\... │ -│ \xD1\x80\xD0\xB0\xD1\x81\xD0\xBF\xD0\xB8\xD1\x81\xD0\xB0\xD0\xBD\xD0\xB8\xD0\xB... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 0.231 user 3.632000 sys 0.008000 -D SELECT CounterID, avg(octet_length(URL)) AS l, count(*) AS c FROM hits WHERE octet_length(URL) > 0 GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;L -┌───────────┬────────────────────┬─────────┐ -│ CounterID │ l │ c │ -├───────────┼────────────────────┼─────────┤ -│ 233773 │ 469.18537326484886 │ 2938865 │ -│ 245438 │ 271.7892512777364 │ 2510103 │ -│ 122612 │ 238.64530987208474 │ 3574007 │ -│ 234004 │ 204.28793262381632 │ 238660 │ -│ 1634 │ 197.83321731651554 │ 323229 │ -│ 786 │ 186.75537634408602 │ 120528 │ -│ 114157 │ 142.91881538575285 │ 216408 │ -│ 515 │ 126.22860040706026 │ 146907 │ -│ 256004 │ 125.37108455074805 │ 858171 │ -│ 95427 │ 120.26856903175477 │ 374306 │ -│ 199550 │ 109.81720498866335 │ 7115413 │ -│ 220992 │ 105.85666196266179 │ 494614 │ -│ 196239 │ 98.34882201749727 │ 163797 │ -│ 62 │ 93.15981711034343 │ 738150 │ -│ 96948 │ 92.74321182146618 │ 396093 │ -│ 188878 │ 91.98308322489247 │ 311998 │ -│ 249603 │ 91.88026594639518 │ 120325 │ -│ 3922 │ 87.83856410684609 │ 8527069 │ -│ 191697 │ 86.95776647628826 │ 124664 │ -│ 97467 │ 84.2953696503987 │ 131178 │ -│ 186300 │ 83.97258027738701 │ 802561 │ -│ 146891 │ 77.77430173504756 │ 605286 │ -│ 38 │ 76.43757015971798 │ 507770 │ -│ 230962 │ 76.3127707226559 │ 169223 │ -│ 77639 │ 75.38681923602442 │ 253961 │ -└───────────┴────────────────────┴─────────┘ -Run Time: real 0.426 user 6.372000 sys 0.356000 -D //(?:www\.)?([^/]+)/.*$', '\1') AS key, avg(octet_length(Referer)) AS l, count(*) AS c, min(Referer) FROM hits WHERE octet_length(Referer) > 0 GROUP BY key HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;E -┌────────────────────────────────────────────────────────────────────────┬────────────────────┬─────────┬────────────────────────────────────────────────────────────────────────────────────┐ -│ key │ l │ c │ min(referer) │ -├────────────────────────────────────────────────────────────────────────┼────────────────────┼─────────┼────────────────────────────────────────────────────────────────────────────────────┤ -│ svpressa.ru │ 307.980979437341 │ 242527 │ http://svpressa.ru/ │ -│ msuzie-showforumdisplay │ 263.327228380409 │ 183676 │ http://msuzie-showforumdisplay/63/~2/?name=&cost_neu%3D400%26retpath=default777... │ -│ saint-peters-total=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 80 │ 242.5236948271821 │ 200529 │ http://saint-peters-total=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 80/c... │ -│ domics │ 212.92990978061542 │ 326094 │ http://domics/825179.11931861234499792 │ -│ e96.ru │ 210.09628206687884 │ 1019276 │ http://e96.ru/%3Ffrom]=&input_act[count_num=0&dff=arian-carrina1201517&cad=rjt&... │ -│ gadgets.irr.ru │ 131.9597008950273 │ 349710 │ https://gadgets.irr.ru/2jmj7l5rSw0yVb │ -│ google.ru │ 109.24485253818524 │ 2158491 │ http://google.ru/ │ -│ go.mail │ 108.63930572737723 │ 8228007 │ http://go.mail/04/detskaia-moda-zhiensmed │ -│ msouz.ru │ 106.10887286512423 │ 301774 │ http://msouz.ru/?ffshop │ -│ state=19945206 │ 105.6469046513171 │ 512414 │ http://state=19945206/foto-4/login%20NoTs3M&where=all&filmId=u8aGGqtWs3M&where=... │ -│ loveplanet.ru │ 104.60136383347789 │ 461200 │ http://loveplanet.ru/%3Faw_opel/page=2013 │ -│ bonprix.ru │ 104.41683309557774 │ 1125105 │ http://bonprix.ru/ │ -│ novjob.ru │ 96.75331644732393 │ 133049 │ http://novjob.ru/ │ -│ cn.ru │ 95.63198716663325 │ 124675 │ http://cn.ru/GameMain.aspx#catalog/100523&tails.xml?market_pc.html?pid=9403&lr=... │ -│ geomethiettai.ru │ 94.78816556817006 │ 115916 │ https://geomethiettai.ru/GameMain.aspx?group=houses/list=266559j7077&num=7&prun... │ -│ kino │ 90.27628829938655 │ 120139 │ http://kino/6/21/2/women.asp?whichpage4/#oversion=unreadm&uid │ -│ yaroslavens.ru │ 90.17077281117085 │ 124610 │ http://yaroslavens.ru/main.aspx#catalog%2F1004-1100000147-otvet/actions/disloca... │ -│ mysw.info │ 89.68684313159915 │ 984596 │ http://mysw.info/ │ -│ m.myloveplanet.ru │ 88.73233749439181 │ 151564 │ http://m.myloveplanet.ru/ │ -│ povarenok.ru │ 83.97395952020882 │ 144813 │ http://povarenok.ru/ │ -│ gorod │ 80.33107253811141 │ 110728 │ http://gorod/%3Fauto.ria.ua%2Fjob │ -│ yandsearch │ 80.21664430621621 │ 245970 │ http://www.yandsearch/rooms=1/page2 │ -│ myloveplanet.ru │ 80.08183067768715 │ 110582 │ http://myloveplanet.ru/#associety/auto │ -│ tambov.irr.ru │ 77.8650188064113 │ 315318 │ http://tambov.irr.ru/0/c1/tgFtaeLDK0yb01A7xvQF08sjCFqQxn51 │ -│ kurortmag.ru │ 75.74958779884584 │ 155264 │ http://kurortmag.ru/ │ -└────────────────────────────────────────────────────────────────────────┴────────────────────┴─────────┴────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 123.649 user 1892.300000 sys 4.120000 -D + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits; -┌──────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┐ -│ 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) │ -├──────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┤ -│ 151348908394 │ 151448908394 │ 151548908394 │ 151648908394 │ 151748908394 │ 151848908394 │ 151948908394 │ 152048908394 │ 152148908394 │ 152248908394 │ 152348908394 │ 152448908394 │ 152548908394 │ 152648908394 │ 152748908394 │ 152848908394 │ 152948908394 │ 153048908394 │ 153148908394 │ 153248908394 │ 153348908394 │ 153448908394 │ 153548908394 │ 153648908394 │ 153748908394 │ 153848908394 │ 153948908394 │ 154048908394 │ 154148908394 │ 154248908394 │ 154348908394 │ 154448908394 │ 154548908394 │ 154648908394 │ 154748908394 │ 154848908394 │ 154948908394 │ 155048908394 │ 155148908394 │ 155248908394 │ 155348908394 │ 155448908394 │ 155548908394 │ 155648908394 │ 155748908394 │ 155848908394 │ 155948908394 │ 156048908394 │ 156148908394 │ 156248908394 │ 156348908394 │ 156448908394 │ 156548908394 │ 156648908394 │ 156748908394 │ 156848908394 │ 156948908394 │ 157048908394 │ 157148908394 │ 157248908394 │ 157348908394 │ 157448908394 │ 157548908394 │ 157648908394 │ 157748908394 │ 157848908394 │ 157948908394 │ 158048908394 │ 158148908394 │ 158248908394 │ 158348908394 │ 158448908394 │ 158548908394 │ 158648908394 │ 158748908394 │ 158848908394 │ 158948908394 │ 159048908394 │ 159148908394 │ 159248908394 │ 159348908394 │ 159448908394 │ 159548908394 │ 159648908394 │ 159748908394 │ 159848908394 │ 159948908394 │ 160048908394 │ 160148908394 │ 160248908394 │ -└──────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┘ -Run Time: real 6.374 user 101.052000 sys 0.008000 -D SELECT SearchEngineID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -┌────────────────┬────────────┬──────┬──────────────┬──────────────────────┐ -│ SearchEngineID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├────────────────┼────────────┼──────┼──────────────┼──────────────────────┤ -│ 2 │ 1138507705 │ 1633 │ 35 │ 1408.0122473974282 │ -│ 2 │ 1740861572 │ 1331 │ 28 │ 1577.945905334335 │ -│ 2 │ 3487820196 │ 1144 │ 35 │ 1553.1984265734266 │ -│ 2 │ 3797060577 │ 1140 │ 36 │ 1543.4140350877192 │ -│ 2 │ 2349209741 │ 1105 │ 30 │ 1557.387330316742 │ -│ 2 │ 2424344199 │ 1102 │ 31 │ 1555.6588021778584 │ -│ 2 │ 3663904793 │ 1083 │ 31 │ 1581.8171745152354 │ -│ 2 │ 3829154130 │ 1082 │ 30 │ 1541.253234750462 │ -│ 2 │ 2551371145 │ 1080 │ 24 │ 1559.8092592592593 │ -│ 2 │ 4029049820 │ 1058 │ 32 │ 1556.2003780718337 │ -└────────────────┴────────────┴──────┴──────────────┴──────────────────────┘ -Run Time: real 0.999 user 13.236000 sys 0.068000 -D SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -┌─────────────────────┬────────────┬───┬──────────────┬──────────────────────┐ -│ WatchID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├─────────────────────┼────────────┼───┼──────────────┼──────────────────────┤ -│ 4623938834438874046 │ 3335652175 │ 2 │ 0 │ 1917.0 │ -│ 6231340535817185610 │ 4012918818 │ 2 │ 0 │ 1638.0 │ -│ 8423042978509451644 │ 2960255590 │ 2 │ 0 │ 1368.0 │ -│ 8268832962994556606 │ 1627418068 │ 2 │ 0 │ 1638.0 │ -│ 5191389486841953200 │ 1487376472 │ 2 │ 0 │ 1828.0 │ -│ 6816565865734300637 │ 3770216628 │ 2 │ 0 │ 2038.0 │ -│ 7726072175618541265 │ 1876840662 │ 2 │ 0 │ 1638.0 │ -│ 8672760597587433971 │ 1269590216 │ 2 │ 0 │ 1368.0 │ -│ 7542988325649023791 │ 303701440 │ 2 │ 0 │ 1828.0 │ -│ 4792336058495451538 │ 3494775397 │ 2 │ 0 │ 2038.0 │ -└─────────────────────┴────────────┴───┴──────────────┴──────────────────────┘ -Run Time: real 1.531 user 16.536000 sys 0.392000 -D SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -┌─────────────────────┬────────────┬───┬──────────────┬──────────────────────┐ -│ WatchID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├─────────────────────┼────────────┼───┼──────────────┼──────────────────────┤ -│ 4744389098628902911 │ 1500313270 │ 2 │ 0 │ 1368.0 │ -│ 6489627466297098899 │ 1558450287 │ 2 │ 0 │ 1368.0 │ -│ 7485059124808740671 │ 1755688921 │ 2 │ 0 │ 1996.0 │ -│ 7229362496802796571 │ 1388026619 │ 2 │ 0 │ 1087.0 │ -│ 5699795602773001237 │ 1148637937 │ 2 │ 0 │ 1917.0 │ -│ 5513990774603668383 │ 1769439966 │ 2 │ 0 │ 1368.0 │ -│ 6466230155642720170 │ 720685641 │ 2 │ 0 │ 1368.0 │ -│ 5561182283490038976 │ 83735824 │ 2 │ 0 │ 1087.0 │ -│ 4784371382202293288 │ 3735345744 │ 2 │ 0 │ 1638.0 │ -│ 6244717263232015267 │ 1898460905 │ 2 │ 0 │ 1996.0 │ -└─────────────────────┴────────────┴───┴──────────────┴──────────────────────┘ -Run Time: real 63.046 user 79.256000 sys 10.484000 -D SELECT URL, count(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬─────────┐ -│ URL │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼─────────┤ -│ http://liver.ru/belgorod/page/1006.j\xD0\xBA\xD0\xB8/\xD0\xB4\xD0\xBE\xD0\xBF_\... │ 3288173 │ -│ http://kinopoisk.ru │ 1625251 │ -│ http://bdsm_po_yers=0&with_video │ 791465 │ -│ http://video.yandex │ 582404 │ -│ http://smeshariki.ru/region │ 514984 │ -│ http://auto_fiat_dlya-bluzki%2F8536.30.18&he=900&with │ 507995 │ -│ http://liver.ru/place_rukodel=365115eb7bbb90 │ 359893 │ -│ http://kinopoisk.ru/vladimir.irr.ru │ 354690 │ -│ http://video.yandex.ru/search/?jenre=50&s_yers │ 318979 │ -│ http://tienskaia-moda │ 289355 │ -└────────────────────────────────────────────────────────────────────────────────────┴─────────┘ -Run Time: real 7.564 user 55.552000 sys 1.156000 -D SELECT 1, URL, count(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -┌───┬────────────────────────────────────────────────────────────────────────────────────┬─────────┐ -│ 1 │ URL │ c │ -├───┼────────────────────────────────────────────────────────────────────────────────────┼─────────┤ -│ 1 │ http://liver.ru/belgorod/page/1006.j\xD0\xBA\xD0\xB8/\xD0\xB4\xD0\xBE\xD0\xBF_\... │ 3288173 │ -│ 1 │ http://kinopoisk.ru │ 1625251 │ -│ 1 │ http://bdsm_po_yers=0&with_video │ 791465 │ -│ 1 │ http://video.yandex │ 582404 │ -│ 1 │ http://smeshariki.ru/region │ 514984 │ -│ 1 │ http://auto_fiat_dlya-bluzki%2F8536.30.18&he=900&with │ 507995 │ -│ 1 │ http://liver.ru/place_rukodel=365115eb7bbb90 │ 359893 │ -│ 1 │ http://kinopoisk.ru/vladimir.irr.ru │ 354690 │ -│ 1 │ http://video.yandex.ru/search/?jenre=50&s_yers │ 318979 │ -│ 1 │ http://tienskaia-moda │ 289355 │ -└───┴────────────────────────────────────────────────────────────────────────────────────┴─────────┘ -Run Time: real 4.014 user 55.408000 sys 0.152000 -D 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; -┌────────────┬──────────────┬──────────────┬──────────────┬───────┐ -│ ClientIP │ clientip - 1 │ clientip - 2 │ clientip - 3 │ c │ -├────────────┼──────────────┼──────────────┼──────────────┼───────┤ -│ 4255045322 │ 4255045321 │ 4255045320 │ 4255045319 │ 47008 │ -│ 2596862839 │ 2596862838 │ 2596862837 │ 2596862836 │ 29121 │ -│ 3119147744 │ 3119147743 │ 3119147742 │ 3119147741 │ 25333 │ -│ 1696638182 │ 1696638181 │ 1696638180 │ 1696638179 │ 20230 │ -│ 1138507705 │ 1138507704 │ 1138507703 │ 1138507702 │ 15778 │ -│ 3367941774 │ 3367941773 │ 3367941772 │ 3367941771 │ 12768 │ -│ 3032827420 │ 3032827419 │ 3032827418 │ 3032827417 │ 11349 │ -│ 1740861572 │ 1740861571 │ 1740861570 │ 1740861569 │ 11315 │ -│ 3487820196 │ 3487820195 │ 3487820194 │ 3487820193 │ 9881 │ -│ 3663904793 │ 3663904792 │ 3663904791 │ 3663904790 │ 9718 │ -└────────────┴──────────────┴──────────────┴──────────────┴───────┘ -Run Time: real 2.404 user 32.444000 sys 0.256000 -D '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(URL) > 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -┌──────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├──────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 102341 │ -│ http://komme%2F27.0.1453.116 │ 51218 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18315 │ -│ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16461 │ -│ http://irr.ru/index.php │ 12577 │ -│ http://irr.ru/index.php?showalbum/login │ 10880 │ -│ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 7627 │ -│ http://irr.ru/index.php?showalbum/login-kupalnik │ 4369 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 4058 │ -│ http://komme%2F27.0.1453.116 Safari │ 3021 │ -└──────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.278 user 0.800000 sys 0.092000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(Title) > 0 GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ Title │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ \xD0\xA2\xD0\xB5\xD1\x81\xD1\x82 (\xD0\xA0\xD0\xBE\xD1\x81\xD1\x81\xD0\xB8\xD1\... │ 122407 │ -│ \xD0\xA8\xD0\xB0\xD1\x80\xD0\xB0\xD1\x80\xD0\xB0\xD0\xB9), \xD0\x92\xD1\x8B\xD0... │ 82935 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA - IRR.ru │ 80958 │ -│ \xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\xB8 New Era H (\xD0\x90\xD1\x81\xD1\x83\xD... │ 39098 │ -│ \xD0\xA2\xD0\xB5\xD0\xBF\xD0\xBB\xD0\xBE\xD1\x81\xD0\xBA\xD1\x83 \xD0\xBD\xD0\x... │ 23123 │ -│ Dave and Hotpoint sport \xE2\x80\x93 \xD1\x81\xD0\xB0\xD0\xBC\xD1\x8B\xD0\xB5 \... │ 14329 │ -│ AUTO.ria.ua \xE2\x84\xA2 - \xD0\x90\xD0\xBF\xD0\xBF\xD0\xB5\xD1\x80 │ 14053 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA (\xD0\xA0\xD0\... │ 13912 │ -│ OWAProfessign), \xD0\xBF\xD1\x80\xD0\xBE\xD0\xB4\xD0\xB0\xD1\x82\xD1\x8C │ 10919 │ -│ \xD0\xA2\xD1\x80\xD1\x83\xD1\x81\xD0\xB8 - \xD0\xA8\xD0\xBE\xD1\x83\xD0\xB1\xD0... │ 10157 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.852 user 1.004000 sys 0.036000 -D AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 7479 │ -│ http://aliningrad │ 4791 │ -│ http://ekburg.irr.ru%2Fpuloveplanet │ 3584 │ -│ http://smeshariki.ru/obucheyelants │ 3064 │ -│ http://video.yandex.php │ 2887 │ -│ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 1084 │ -│ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 891 │ -│ http://afisha.yandex.ru/index │ 855 │ -│ http://sslow_13507.html?aspx?naId=6HS │ 521 │ -│ http://wildberrior/uphold │ 484 │ -│ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 289 │ -│ http://obninsk/detail │ 241 │ -│ http://diary.ru/forum/intries │ 208 │ -│ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 185 │ -│ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 185 │ -│ http://kurort/SINA, ADRIAN │ 157 │ -│ http://afisha.yandex.ru │ 132 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 124 │ -│ http://ssl.hurra.com/iframe │ 123 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 119 │ -│ http://stars-visa.html_params%3Drhost%3Dad.adriver.ru/catalog.php │ 105 │ -│ http://komme%2F27.0.1453.116 │ 83 │ -│ http://pogoda.yandex │ 80 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 79 │ -│ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 73 │ -│ http://video.yandex │ 71 │ -│ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 68 │ -│ http://wildberries.ru/daily │ 68 │ -│ http://smeshariki.ru/ru/index.ru%26bid │ 68 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 64 │ -│ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 64 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn │ 63 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 59 │ -│ http://sslow_135000008&position=search │ 58 │ -│ http:%2F%2Fwwwwww.bonprix.ru/topic │ 53 │ -│ http://sslow_13507.html/articles │ 52 │ -│ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 51 │ -│ http://rsdn.ru/rss.ya.ru/catalog │ 51 │ -│ http://ekburg.irr.ru/#lingvo │ 46 │ -│ http://pogoda.yandex.ru │ 45 │ -│ http://sslow_13500000%26rnd%3D2788881.html │ 44 │ -│ http://maps#ru_5_ru_22106.377648194,975924][to]=&int[14270pa106&op_uid=17759/6#... │ 42 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 37 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 37 │ -│ http://video.yandex.ru/film/46351/frl-2/bage │ 36 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 35 │ -│ http://direct.yandex │ 34 │ -│ http://sslow_13507.html?aspx?naId=3X_3bhLcs3M │ 33 │ -│ http://gotovim-doma │ 32 │ -│ http://che.ru/produkty_zarubezhei-niepochekhly │ 31 │ -│ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 30 │ -│ http://video.yandex.ru/Newsletter │ 29 │ -│ http://sslow_13507.html?aspx?naId=6D8IzMGys3M │ 29 │ -│ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ http://video.yandex.ru │ 28 │ -│ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 28 │ -│ http://sslow_13507.html?aspx?naId=649&state/out-of-town=\xD0\xA5\xD0\xB0\xD0\xB... │ 27 │ -│ http://notes=1/currency │ 27 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 26 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 26 │ -│ http:%2F%2Fwwww.bonprix.ru/tambov │ 24 │ -│ http://kinopoisk.ru/shoppich.ru/search?clid │ 24 │ -│ http://wildberries │ 23 │ -│ http:%2F%2Fwwwwww.bonprix.ru/searchAutoSearch?text=\xD0\xB2\xD0\xB5\xD0\xBB\xD0... │ 22 │ -│ http://myloveplanet │ 20 │ -│ http://stars-visa-litraj.txt","lpu":"http://pogoda │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/secondary │ 20 │ -│ http://msuzie-shop/premiery-c-38208_2.html │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 20 │ -│ http://smeshariki │ 19 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 19 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ http://irr.ru/6323%26bn%3D27888895,96772,97436 │ 18 │ -│ http://kinopoisk.ru │ 18 │ -│ http://nepogoda.yandex.ru%2Fproducts/search?text=subscripts/busineshop │ 17 │ -│ http://wildberries.ru │ 17 │ -│ http://kinopoisk.ru/catalog/9902224 │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://direct.yandex.html │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 17 │ -│ http://irr.ru/6323%26bn%3D27888895,963095425 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 16 │ -│ http://afisha.yandex.php?gidcar=36281664 │ 16 │ -│ http://auto.ria.ua/search │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://direct │ 15 │ -│ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ http://irr.ru/index.php?showalbum/login-kupalnaya-obl │ 15 │ -│ http://kinopoisk.ru/odessya │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 14 │ -│ http://ekburg.irr.irr.ru/maker │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.yandex │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.mail/rnd=0.9788 │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://auto_many_to_auto.ria.ua/igrush43/ │ 13 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 13 │ -│ http://afisha.yandex.ru/cars │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 13 │ -│ http://love.ru/a-myprofi │ 13 │ -│ http: │ 13 │ -│ http:%2F%2Fwww.bonprix │ 12 │ -│ http://real-estate/aparther/offiliates/corruption/russinsk │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 12 │ -│ http://on-online=on&accetti │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 12 │ -│ http://samara.irr.html5/v12/?from]= │ 12 │ -│ http://en.lyrsenses/zamba_zaborah_coldplay=1&gearbox │ 12 │ -│ http://kinopoisk.ru/saledParams │ 12 │ -│ http://zvukovo/hondar/2007&state/renlew/rigma.ru/scribed │ 12 │ -│ http://pogoda.yandex.php?SECTION │ 12 │ -│ http://msk/platia-nashing/vanny.diary.ru/moscow │ 12 │ -│ http://video.yandex.ru/GameMain.E6smreQhiu_hXR4&where=all&film │ 12 │ -│ http://nizhnieiene/p17378705/currency=1&with_photo-ideapadeno │ 11 │ -│ http://news/6483731559676/Unlocknotebooks/m83/800_D_Black_list │ 11 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 11 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty/photo-12/#imag... │ 11 │ -│ http://msk/platia-nashing/vanny.diary.ru/sale/liver │ 11 │ -│ http://video.yandex.ru/GameMain.aspx#location │ 11 │ -│ http://guid=6&pw=6&pv=13 │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 11 │ -│ http://wildberries.ru/rost.html?1 │ 11 │ -│ http://smeshariki.ru/?win=82&stat=1&page/196264&pt │ 11 │ -│ http://pogoda.yandex.ru/catalog/jokers │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 11 │ -│ http://en.lyrsenses/zamba_zabudtrimazok.html?page=12&prr=http://fap1.adrive_typ... │ 11 │ -│ http://bonprix.ru%26bid │ 11 │ -│ http://loveplanet.ru/GameMain │ 10 │ -│ http://bdsmpeople.ru │ 10 │ -│ http://liver.ru/cheboksicily/foto.aspx?sort=newly&trafkey │ 10 │ -│ http://video.yandex.ru&pvid │ 10 │ -│ http:%2F%2F%2Fwwww.bonprix │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://club.ru/spokoiteli/photo37775280000 │ 10 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 10 │ -│ http:%2F%2Fwwww.bonprix │ 10 │ -│ http://auto_id=240&n=13901038 │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 10 │ -│ http://smeshariki.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://slovariant_new3077940810/detail │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://en.lyrsenses/zamba_zabor_id=1012_blank%26site │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://sslow_13500000%26rnd%3D2788881.html?parts/passe │ 10 │ -│ http://pogoda.html%3Fhtml_params%3Drhost%3D43 │ 10 │ -│ http://irr.ru/index.php?showalbum/logout%26verstova.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://pogoda.yandex.php │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film/6781203.html?id=242037047/detai... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://mysw.info/blog/sankt-peter%3D1216/00001216629 │ 9 │ -│ http://e96.ru/albumfotok-15-fotki │ 9 │ -│ http://alpari.yandex.html?html_param=0&users/#page/Search/ab_dob%2Ffieiie-razvo... │ 9 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://bonprix.ru/catalog/8570/1006790 │ 9 │ -│ http://b2b.testered/main/discuss/matched_country=-1&top=0&cityid=1024&wi=1366&o... │ 9 │ -│ http://svpressa.ru/topic=17082630 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://bonprix.ru │ 9 │ -│ http:%2F%2Fwwww.bonprix.ru/filmId=8j5j97LRs3M&where=all&sources │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/#win_13.html_p... │ 9 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/4744089758 │ 9 │ -│ http://nail=Yes&target=search │ 9 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_produkty%2F&sr=http://slovaria │ 9 │ -│ http://rlsnet.ru/vacancy/view_type_id=9677548268010367 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/imagecache/wm/2013&where=all&filmId │ 8 │ -│ https://m.myloveplanet.ru/forum/abrika-kobelenie_nebestsenal │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 8 │ -│ http://zapchast.com/iframe-owa.html?1=1&cid=577&oki=1&op │ 8 │ -│ http://afisha.yandex.php?t=141880517 │ 8 │ -│ http://poisk.ru/price_ot=&price_ot=&price │ 8 │ -│ http://zarplata.ru/velika_all=\xD0\xBE\xD1\x82 │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http:%2F%2Fwwwwww.bonprix.ru/news/222974895&op │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://fore=3&marka=0&top=0 │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96777&oki │ 8 │ -│ http://video.yandex.ru&xdm_p=1#item/search │ 8 │ -│ http://direct.yandex.ru/catalog │ 8 │ -│ http://zarplata.ru/?p=12977-B26358/currency=RUR/page=1080&wi=1024&lo=http://rzh... │ 8 │ -│ http://en.lyrsenses/zamba_zabor/bedroomolsk │ 8 │ -│ http://masterh4.adriver.yandex │ 8 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://mastered/main.aspx#location=1&bc=3&ct=1&pr=60322056107100919/page5/?_h=s... │ 8 │ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ==&page_avtomodules.php?f=100&ref... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96779/87 │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://omsk.mlsn.ru │ 8 │ -│ http://smeshariki.ru/GameMain │ 8 │ -│ http://wildberries.xml?from]=&input │ 8 │ -│ http://love.ru/?p=17059 │ 8 │ -│ http://afisha.yandex.php?gidcar=367108851%2Fr%2F1 │ 8 │ -│ http://che.ru&pvid=13733142835/100/topic,5240556895&ch=UTF-8&sF=11,7,7,0 │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/index.php?showalbum/login-kupit-topy%2Fplatjie-gotovlexandex.html... │ 8 │ -│ http://auto.ria.ua/auto_id=24126629/0/index.ru/real-estate/out │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 8 │ -│ http://victor?page_type=city.stol-yar.ru/cars │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://kinopoisk.ru/ch/feed/letniaiaprice/1609 │ 7 │ -│ http://video.yandex.ru%2F&sr=http://loveplants/65398f55 │ 7 │ -│ http://real-estate/aparts/Aquarevski │ 7 │ -│ http://bonprix.ru/social/product_id │ 7 │ -│ http://video.yandex.ru/a-album/login-vitiju/photo │ 7 │ -│ http://moscow/detail/5552/0/2792834&m=111,7,7,5 │ 7 │ -│ http://irr.ru/index.php?showalbum/login-kapustics?sort=pogoda.yandex.ru%26bt%3D... │ 7 │ -│ http://irr.ru/Registered/main/topnewsru.com/page=6 │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 7 │ -│ http://bonprix.ru/catalog/8570/14139489 │ 7 │ -│ http://afisha.yandex.ru/\xD0\xB4\xD0\xBE\xD0\xBC\xD0\xB0/\xD0\x91\xD0\xA1\xD0\x... │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://irr.htm?from]=&int[85][from]=&input_vsegodnyie │ 7 │ -│ http://love.ru/?p=1#country=&op_seo │ 7 │ -│ http://bdsmpeople.ru/niktory/shtory/308/roomed.ru/p59473682740295 │ 7 │ -│ http:%2F%2Fwwwww.bonprix │ 7 │ -│ http://myloveplantrackIt?tid │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 7 │ -│ http://spb/event=big&marka=84&model=0&auto_id=0&s_yers=0&pv=10&can_be_check_PP │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://pogoda.yandex.ru&pvid=1 │ 7 │ -│ http://irr.ru/index.php?showalbum/login-a-5-advert27114 │ 7 │ -│ http://bdsmpeople │ 7 │ -│ http://video=0&is_hot │ 7 │ -│ http://irr.ru/index.php?showalbum/login-zk34/pages/0001216629 │ 7 │ -│ http://svpressa.ru │ 7 │ -│ http://money.yandex │ 7 │ -│ http://gotovim-doma.ru │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 7 │ -│ http://afisha │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://love.ru/?p=1#country=-1&sq_liver.ru/kyrgyzstan │ 6 │ -│ http://afisha.yandex.ru%2F%2Fmail/169 │ 6 │ -│ │ 6 │ -│ http://video.yandex.ru%2Fkategory_id │ 6 │ -│ http://omsk/evential/house.ru/catalog/kitchedule=213-606361653965283 │ 6 │ -│ http://video.yandex.ru/ekt │ 6 │ -│ http://stars-varenok.ru/16745959680706/800_0.jpeg.html%3Fhtml5/v123593 │ 6 │ -│ http://afisha.yandex.ru/catalog=on │ 6 │ -│ http://bdsmpeople.ru/film/64544.690022.rar.html_params%3Drhost%3D_black_list=0&... │ 6 │ -│ http://bdsmpeople.ru/GameMain │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://v102.ru/investate/apartments-sale │ 6 │ -│ http://video.yandex.ru/a-topy │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://wildberries.ru/search │ 6 │ -│ http://afisha.yandex.ru%26bt%3D43%26anbietersburg │ 6 │ -│ http://direct.yandex.ru/refererprofile%2F2.10 │ 6 │ -│ http://video.yandex.ru/GameMain.aspx#location/page_type=category │ 6 │ -│ http://omsk/evential/housession%3D0 │ 6 │ -│ http://rukodel=0&sort=newly&trafkey=2750 │ 6 │ -│ http://smeshariki.ru/world/photofider_credit=0&view │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://moscow/details │ 6 │ -│ http://irr.ru/index.php?showalbum/login-12.html%26custom │ 6 │ -│ http://novosibirsk.irr.ru%26bid │ 6 │ -│ http://solutions.diary.ru/realtitroenie_v_jurman.ru/albums/frame-owa.html?stric... │ 6 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 6 │ -│ http://mr7.ru/newsru.com/iframe_right%3D43 │ 6 │ -│ http://autodoc.ru/real-estate/apart │ 6 │ -│ http://smeshariki.ru/catalog │ 6 │ -│ http://edp2.adriver.ru/hocketshop.ru/moscow/detailanude │ 6 │ -│ http://povari.yandex.ru/greecondary/Products_id=&auto_vaz_2111 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://3dnewsru.com/iframe_right.html?1=1&cid=51538 │ 6 │ -│ http://novo/detail.aspx?group_cod_1s │ 6 │ -│ http://video.yandex.ru/catalog │ 6 │ -│ http://ereal-estate/rent │ 6 │ -│ http://afisha.yandex.php/board,39.04839 │ 6 │ -│ http://msk/platia-nashing/vantralitsa_transion │ 6 │ -│ http://auto_hyundai_sarator │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://kalininmyclonus1 │ 6 │ -│ http://bdsmpeople.ru/saledParams=rhost%3D43%26bid%3D1 │ 6 │ -│ http://forum/topnews/22294&op_category │ 6 │ -│ http://love.ru/?p=17055335 │ 6 │ -│ http://real-estate=week/page=1&expand_search?film/298677435615.html │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://afisha.yandex.ru/mymail.php │ 6 │ -│ http://video=0&with_exchangeType │ 6 │ -│ http://afisha.yandex.ru/index.ru/recipe │ 6 │ -│ http://kinel-lab.com/rus/20130709_117485994,93304&op_seo_entry=1&gearbox=0&type... │ 6 │ -│ http://koolinar.ru/port.ru/doc │ 6 │ -│ http://diary.ru/exp?sid=3205 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://video.yandex.php?search?text=\xD0\x9A\xD0\xBE\xD0\xBD\xD1\x8C\xD1\x8F\xD... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ https://slovakia-600dd903c07022,101595,91194&op_seo_entry │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D8393224 │ 6 │ -│ http://love.ru/ru/irk/event/search/ │ 6 │ -│ http://video.yandex.php?from]=&interapy-wkti/ &cd │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 5 │ -│ http://kinopoisk.ru/catalog │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://auto_volkswagen/vologdano/il_dlya-dnevka.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://barnaul/details/?cauth │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD0\xB5\xD1\x80&where=... │ 5 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://sp-mamrostova │ 5 │ -│ http://tp66.ru/exp?sid=3860217/rooms=2/men.aspx#location%3D0%26rnd │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 5 │ -│ http://zvukovo-gorodsk │ 5 │ -│ http://metal-lodku-Obzor │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://video.yandex.by/?state_id=&auth=1..630;IC,7711588 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 5 │ -│ http:%2F%2Fwwww.bonprix.ru/?id=2013&where=all&filmId │ 5 │ -│ http://rmnt.ru/search/offilia_Sovetov_living_chamber/?78142 │ 5 │ -│ http://e96.ru/movies/614418821/artir.ua/search │ 5 │ -│ http://slovari.yandex.ru%26orderovskij-index.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://tks.ru/cat/publish-cherkalnaya-ttpodporyadushek │ 5 │ -│ http://povari.yandex │ 5 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_seo_entry=&op_category_... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://auto.ria.ua/auto_id=241269.html?1=1&cid=2127970 │ 5 │ -│ http://slovarenok.com │ 5 │ -│ http://video.yandex.ru/GameMain.aspx?Link │ 5 │ -│ http://slovari.yandex │ 5 │ -│ http://auto_volkswagen-Palities/horobki │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://en.lyrsenses/zamba_zaborah_chamberk │ 5 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 40007&pt │ 5 │ -│ http://radio&planet.ru/work.ru/catalog │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://afisha.yandex.php?r=23436303135353.html?1 │ 5 │ -│ http://rustnye-sht-riemnikoi │ 5 │ -│ http://pogoda.yandex.php?gidcar │ 5 │ -│ http://sslow_135000008&position=search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 5 │ -│ http://real-estate=week&m=Dvigenie.html?option │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://myloveplanet.ru/v14572&lb_id=13000001216629 │ 5 │ -│ http://avtomobile/motory.ru/comp.ru/view.php │ 5 │ -│ http://topnews.ru/GameMain.aspx?group_cod_1s=1983&pt=b&pd=9&pw=0 │ 5 │ -│ http://love.ru/product_id=0&po_yers=0&po_yers=2&refererro/model=1346488078722&c... │ 5 │ -│ http://autodoc.ru/moscow │ 5 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 5 │ -│ http://bibidohertki-i-OOOO_REPORT/07_2013 │ 5 │ -│ http://auto.ria.ua/auto │ 5 │ -│ http://loveplanet.ru/mymail/rudi │ 5 │ -│ http://video.yandex.ru/firms.turizm │ 5 │ -│ http://video.yandex.ru%2FkategoriendflowerTo=&powerTo= │ 5 │ -│ http://nizhnieiewva88/photo/101246465376&cmd=show-to-buchaiev-pugache=51db32a68... │ 5 │ -│ http://auto.ria.ua │ 5 │ -│ http://tks.ru/cat/publish-chemec.ru/search?filmId=CktclMBmUXI │ 5 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D158197%26width │ 5 │ -│ http://direct.yandex.ru/index │ 5 │ -│ http://club.ru/cinema/movies/no-pos │ 5 │ -│ http://msk/platia-nashing/vannyie-product_id=1841&page2 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://kinopoisk.ru/spb.pulscen.ru/exp?sid=3159&op_category_id=&auth=0&checked=... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://holodilnik-rp-ploschaya-obuv/?ci=1280&with_video=0&choosOyg==&op_uid=111... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://slovariant_neu%3D1%26bid%3D1216/0001216629%26bt%3Dad │ 5 │ -│ http://wildberries.ru/real │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://wildberrior/kia/ │ 5 │ -│ http://md.mirkovskaya-obl.irr.ru/jobinmoscow │ 5 │ -│ http://povari.yandex.php?showalbum/login.pl?cl=all&film/497794,90458 │ 5 │ -│ http://jcmotorom-921205&bt=7 │ 5 │ -│ http://en.lyrsense.com/obshchin-idieiala │ 5 │ -│ http://loveplanet.ru/\xD0\xB0\xD1\x80\xD0\xBE\xD0\xBC/curre-ap-i-showalbum/loui... │ 5 │ -│ http://gaylyU │ 5 │ -│ http:%2F%2Fwww.bonprix.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://video.yandex.ru%26bt%3Dad.adriver.ru/recipe/view/10217/?from │ 5 │ -│ http://smeshariki.ru/topic │ 5 │ -│ http://moscow/detail/Torgovuyu-organske.ru/real-estate │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://alib.mist.html%26custom%3D%26c2%3D278888592138 │ 5 │ -│ http://loveplanet.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://alpari.ru/gallery/pic845274 │ 5 │ -│ http://video.yandex.ru/price │ 4 │ -│ http://whoyougle.ru/images/images/00000i/specifiers.ru/image=1&furniture │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://loveplanet │ 4 │ -│ http://video.yandex.ua │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 4 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 4 │ -│ http://loveche.html │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://sslovarenok.ru │ 4 │ -│ http://auto.ria.ua/auto_id=0 │ 4 │ -│ http://afisha.yandex.ru/zoom.php?f=5162613838.html_partments │ 4 │ -│ http://bdsmpeople.ru/Web/Pages=1/feedsmag.ru/~\xD0\xBA\xD0\xBD\xD0\xB8\xD0\xB3\... │ 4 │ -│ http://samarskii_krai/tuapse/detail/result.aspx │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/filmId │ 4 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 4 │ -│ http://irr.ru/index.php?showalbum/login-10618968476372773 │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://wildberries.ru/catalog │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://echoradar-s-Levoshcha │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://love.ru/?p=1#countpage/vacancies/events/738/0/3/women.aspx │ 4 │ -│ http://radio&planet.ru/marka=62&model=1178128455&pvno=2&evlg=VC,5 │ 4 │ -│ http://afisha.yandex.ru%252f7769%252fe │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://direct.yandex.php?formsof(INFLECTION_ID=30861/14365-4b11&state/apartment... │ 4 │ -│ http://diary.ru/catalog=on&input_city[3 │ 4 │ -│ http://e96.ru/news/39733/page2=&input_sponsor=&o=1015219.html_partments-sale&pa... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://rasp.pl?cmd │ 4 │ -│ http://travel.ru/state/apartments-sale/rashinitit%2F537 │ 4 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://auto.ria.ua/auto_id=63799.html_params │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://kinopoisk.ru/saledParams%3Drhost%3Dad.adriver.ru/GameMain.aspx#location │ 4 │ -│ http://afisha.yandex.php?r=3&bs=&day │ 4 │ -│ http://video.yandex.php?view_type=2&driveresult.ru/replies │ 4 │ -│ https://smeshariki.ru/cinema/article10363136000001216629%26site_offilia_Sovets.... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://ssl.hurranovskaya-ul-31-foto.ria │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://vkirovoe-tourisma │ 4 │ -│ http://card/windows)&bL=ru&cE │ 4 │ -│ http://kniga.ru/view=\xD0\xA1\xD1\x82\xD0\xB0\xD0\xB2\xD1\x80\xD0\xBE\xD0\xB2&w... │ 4 │ -│ http://smeshariki.ru/domchelkakh_location=search?text=\xD0\xBC\xD0\xB8vents │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboy07/... │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://smeshariki.ru/search/keukeru-soft │ 4 │ -│ http://guid=6&pw=2&pv=0&price_do=¤cy=1 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://matched_car=373838928155755775482794,9453.116 Safari%2F537.36&he=10&s_ye... │ 4 │ -│ http://zarplata.ru/?p=12977-B26358/hasimages=1/page │ 4 │ -│ http://video.yandex.ru/real │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://tks.ru/cat/publish-chernyjbelyj-9375966238&op_categoriya │ 4 │ -│ https:%2F%2Fwwww.yandex.ru │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://afisha.yandex.ru/search │ 4 │ -│ http://v102.ru/?s=Adaments-sale │ 4 │ -│ http://video.yandex.ru/GameMain.XYyZwYXRoPWEtbG9nb24vcG90Cw │ 4 │ -│ http://video.yandex.ru/GameMain.aspx#location-4.0.html │ 4 │ -│ http://auto.ria.ua/canel_pe_mascona Hilfigeratov/153228 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://nizhny novgorod │ 4 │ -│ http://video.yandex.ru%26target=search/ab_area=categorija80119 │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/social │ 4 │ -│ http://video.yandex.ru/realty/leaser_map=1/hasimay-2.html │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://kubikus.ru/search.php?r=4140211,1,7,7,7,7,0 │ 4 │ -│ http://e-kuzbass.ru/a-shop │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://guide.travel.ru/link │ 4 │ -│ http://victorhead.php?full&dom=780067167694.0; │ 4 │ -│ http://fitness/building │ 4 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/cart.php/cars/papago-d-plosch │ 4 │ -│ https://produkty%2Fpulove.ru/gost/?page3 │ 4 │ -│ http://afisha.mail.aspx#locationalOffers │ 4 │ -│ http://afishi,Mudanted-belyj-974299099/guest-id=34089.html │ 4 │ -│ http://svpress_up.aspx#location[propfilter_pf[PODVAL]=&arrFilter][3463351841195... │ 4 │ -│ http:%2F%2Fwwww.bonprix.ru/real-estate/out-of-town/house │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http%3A//magnitogorod/page3/#over │ 4 │ -│ http://anketka.ru/bridget │ 4 │ -│ http://tks.ru/cat/public/gamemain.aspx#location │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://afisha.yandex.php?action/2741920 │ 4 │ -│ http://love.ru/image=2&marka=84&model/mihailovo │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadbor_sh... │ 4 │ -│ http://video.yandex.ru/GameMain/dukhovyy │ 4 │ -│ http://nepogoda.yandex.ru/search=1&target=search=0&can_be │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://kinopoisk.ru/searchAutoSearch=0&driver.ru/catalog/1/women.aspx#locationp... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://love.ru/?p=1705 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://afisha.yandex.php?show=rlv&ru=1&expand_search │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://auto_volkswagen_cated_cartovskaya-obl.irr.ru/album/login │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://rsdn.ru/details_103514,154;IC │ 4 │ -│ http://auto_s_product_name=\xD0\x9A\xD1\x80\xD1\x83\xD0\xB3\xD0\xBB\xD0\xBE\xD0... │ 4 │ -│ http://pogoda.yandex.ru/cooking_sm5_1148786993ab-417/photo/69363/26#formi.ru/co... │ 4 │ -│ http://afisha.yandex.php?p=31&input │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://ssl.hurra.com │ 4 │ -│ http://loveche.ru/job/1162323&PAGEN_1=30&state │ 4 │ -│ http://internet Explorer&aV=5.0 (Windows)&bL=en │ 4 │ -│ http://video.yandex.ru/If yours.avtogsm.ru/animals/calculate │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://auto_volkswagen_pass_1161967 │ 4 │ -│ http%3A//edp1.adriverys/forum/view_type=city&custom=0&damages/0001216629%26bid%... │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://inspelishchin-platjie-doma.ru/irkutsk.irr.ru │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboyzon... │ 4 │ -│ http://video.yandex.ru/topnews.ru/lanas-advert2713][to]=&int │ 4 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD1\x82\xD0\xBE\xD1\x8... │ 4 │ -│ http://smeshariki.ru/saint-petersburg-gorod/transfer/?id=7576149959760994861&op... │ 4 │ -│ http://auto_id=0&with_photo │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://b.kavanga.ru/?a=inneVolumeFrom │ 4 │ -│ http://en.lyrsenses/zamba_zabudka/photo-1/#page=0&sale/search?text=\xD0\x9C\xD0... │ 4 │ -│ http://afisha.yandex.ru/content2.adriver │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://arma/frl-4/transportnoy-kv-m-malchik.ru/show │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://tp66.ru/money.yandex.ru/albums_screenterval │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://irr.htm?from]=&int[1151;IC,112 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://afisha.yandex.ru&pvid=13733424 │ 4 │ -│ http://video.yandex.ru/GameMain │ 4 │ -│ http://bdsmpeople.ru/film/64544.690078 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_category_id=592b9e01c48ce9403%26bn%3D0%2... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://kinopoisk.ru/service.ru/iframe │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://romar/events/audio.ru/widget[]=vacancies/99.php?gidcar │ 3 │ -│ http://banantikov-nadushenie_orleona_server=sc.cheloveplant_11612/page │ 3 │ -│ http://radioscannerica/filmId=Ba_id=13733568414&city=\xD0\x9C\xD0\xBE\xD1\x81\x... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://afisha.yandex.php/topic104780204&op_uid=1954 │ 3 │ -│ http://afisha.yandex.ru/a-folders/misc │ 3 │ -│ http://afisha.yandex.ru/shop.ru/malta │ 3 │ -│ http://afisha.yandex.ru/kategoriya%2F5.0 (company │ 3 │ -│ http://afisha.yandex.ru/linkvac.php/board.php?topicseeng │ 3 │ -│ http://afisha.yandex.ua/donetsk/urals │ 3 │ -│ http://afisha.yandex.php/tova.ru/uliya2076789599305953 │ 3 │ -│ http://afisha.yandex.ru/project_price=&maxprice │ 3 │ -│ http://irr.kz/realty/lease/3516093&pvno=2&evlg │ 3 │ -│ https://produkty%2Fplatjie-kuzbass.ru/newsru.com/iframe_right=0&auto_ria=0&meta... │ 3 │ -│ http://fuckfind=rent/view/2021/3 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://radiorecord.ru/login-m3w.html?1=1&cid │ 3 │ -│ http://video.yandex.ru%2Fplata.ru/ch │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ https://orenburg/?arrFiltersburg │ 3 │ -│ http://samara.irr.ru%2Fproduct │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberrior/bedroom]=&int[17][to]=&int[858 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://sp-money.yandex.ru │ 3 │ -│ http://kaluga/?ext=\xD0\xB3\xD0\xB5\xD1\x80\xD0\xBE\xD1\x8F\xD1\x82\xD0\xBD\xD1... │ 3 │ -│ http://auto_kia_30 │ 3 │ -│ http://afisha.yandex.ru/forum.materinburg │ 3 │ -│ http://afisha.yandex.php?addriver.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://sp-mamrostokonkursovet │ 3 │ -│ http://video.yandex.ru&pvid=13735/?_h │ 3 │ -│ http://love.qip.ru │ 3 │ -│ http://astrobank.ru/image │ 3 │ -│ https://slovakia-600dd903c07022,101595,9143531427800648_elit │ 3 │ -│ https://slovakia-600dd903c06c999c226647639.html%3Fhtml │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category_id=0&wi=16000&... │ 3 │ -│ http://video.yandex.by/search/?target%3D43%26bid%3D2 │ 3 │ -│ http://love.ru/forum.cofe.ru/forum/view_type=city=790&Selectronics-technik │ 3 │ -│ http://forum/topnews/2229605699574.html?1=1 │ 3 │ -│ http://sp-money.yandex.ru/work rushki-sien-natalog/8570/page=0&expand │ 3 │ -│ http://ssl.hurral=messages │ 3 │ -│ http://direct&sortdirect.yandex.ru/imagesize%3D0%26ar │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://b.kavanga.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://video.yandex.ua/auto_id │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://rmnt.ru/film/88677/russia/rio.ru/search?filmId=NNr6aJrm4s3M │ 3 │ -│ http://rmnt.ru/stars │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://bdsmpeople.ru/search │ 3 │ -│ http://video.yandex.ru/circle&state/out-of-town/houses │ 3 │ -│ http://edp2.adriver.ru/catalog/181 │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http:%2F%2Fwww.bonprix_ru}%2Fnizhniynovgorod/request-id │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://msk/planet.ru/mymail.aspx#comme_me_saydinne │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://edp2.adriver.ru/jobinmoscow/detail │ 3 │ -│ http://kazan.irr.ru/location │ 3 │ -│ http://nigma.ru/product&op_category_name=\xD0\x91\xD0\xB8\xD0\xBA\xD0\xB8\xD0\x... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://card/windows NT 6.1) AppleWebKit%2F5 │ 3 │ -│ http://tks.ru/cat/publish-chin-play.php?categoriya%2Fzhiensmed │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com/iframe-owa.html?1=1&cid=577&o... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/forum.donfiscategory │ 3 │ -│ https://produkty%2Fpulove.ru/voronezh-sien-zhienskaia-moda-zhienskaia-moda-zhie... │ 3 │ -│ http://pogoda.yandex.kz/family │ 3 │ -│ http://pogoda │ 3 │ -│ http://svpress_w1t1042796786/6/?category │ 3 │ -│ http://video.yandex.php?topbloveche │ 3 │ -│ http://radiorecord.ru/catalog/idShare │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://magnitka_1_series.ru/?favorite_id=636233644&op_category_id=937514 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountpage │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://smeshariki.ru/a-phony │ 3 │ -│ http://ekategoriya%2F9B206 Safari │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://ussuriysk.irr.ru/catalog/premiere/628962851d7fd0b6eb17b321d336f5bc7de189... │ 3 │ -│ http://smeshariki.ru/catalog/286/women.aspx │ 3 │ -│ http://loveche.ru/volzhskiy │ 3 │ -│ http://afisha.yandex.ru/tatatit_chto.php?industry │ 3 │ -│ http://svpressa.ru/content/search │ 3 │ -│ http://video_dvd/game/iframe-owa.html │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://svpress/showbiz/photo.htm │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty%2Ftanki │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/92054446660.ht... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26url%3D//ad.adriver.ru/link/justic/h2.php/... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://sp-mamrostovestory │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://brand=498&pvno │ 3 │ -│ http://personal/atlants/7292&xdm_c │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://e96.ru/real-estate │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://tp66.ru/search/?page=1&fuelRateTo │ 3 │ -│ http://love.ru/?p=1#country=&op_seo_entry=-1&target │ 3 │ -│ http://lk.wildberries │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://my.kp.ru/albumfoto-1/pol-2 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://povarenok.ru/files/eliteh.ru/perm.irr.ru/board,75.2013-07-09 │ 3 │ -│ http://holodilnik.ru/catalog │ 3 │ -│ http://msk/platia%2Fzhienskaia │ 3 │ -│ http://smeshariki.ru/?win=82&stat=141882,373;IC,2552f48 │ 3 │ -│ http://amobil-nye-pliazhnaia │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory_id=577&search/ab_district/date_id=2271][from... │ 3 │ -│ http://kaluzha-na-NovletedAutoSearch=0 │ 3 │ -│ http://love.ru/recipes/indows │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://str_ob.html?1=1&choosO8gPJSs3M&where=all&filmId=mAyiC7y6M2mGV2GoA9hFoN3q... │ 3 │ -│ http://wildberries.ru/item_no=2&evlg=VC,0;VL,205;IC,14;VL,757138/currency=RUR/h... │ 3 │ -│ http://product_brand=RAINBOW&op_cated_content/search/keup/en-ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://msuzie │ 3 │ -│ http://smeshariki.ru/real │ 3 │ -│ http://love.ru/?p=17057 │ 3 │ -│ http://love.ru/ru/irk/event=little&cated_country=-192.html?1=1&cid │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostavia.travel.ru/s... │ 3 │ -│ http://irr.ru/washek-s-printom │ 3 │ -│ http://afisha.yandex.php?app=membered │ 3 │ -│ http://afisha.yandex.php?ELEMENT_ID │ 3 │ -│ http://afisha.yandex.ru/lesyach-hotels │ 3 │ -│ http://3dnews.ru/?p=12636464/5#f │ 3 │ -│ http://afisha.yandex.ru%26bt%3D90%26nid%3D1216629 │ 3 │ -│ http://afisha.yandex.php?id=727285 │ 3 │ -│ http://kinopoisk.ru/search │ 3 │ -│ http://smeshariki.ru/goodavec/photo/6936325.html?id=223978/page=102 │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://irr.ru/index.php?showalbum/login-kupaljinik-chere │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic=79799398/?_h=search.htm... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/comment/search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://auto_repairs=0&po_yers=0&price.ru/\xD0\xB6\xD0\xB5\xD0\xBB\xD0\xB5\xD0\x... │ 3 │ -│ http://soft.oszone.ru │ 3 │ -│ http://you_hashkaf.ua/search │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://auto_id=0&engineVolumeFrom │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://povaria/chak_naytimes.ru │ 3 │ -│ http://pogoda.yandex.ru/real-estate │ 3 │ -│ http://wildberries.ru/GameMain │ 3 │ -│ http://afisha.yandex.ru/hotel-agen-Goluboj-9730 │ 3 │ -│ http://afisha.yandex.ru%2Fobuv-sapozhkivka=23&price │ 3 │ -│ http://notebooking pressa │ 3 │ -│ http://video_dvd/ratesTypeSearch?text=\xD1\x87\xD0\xB5\xD0\xBB\xD0\xBE\xD0\xB2\... │ 3 │ -│ http://auto_repairs=0&confiscategoriya │ 3 │ -│ http://wildberries.ru/basket&ch │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://state/room=94720-recept-Salat-iz-glasya1lesyat │ 3 │ -│ http://smeshariki.ru/GameMain.aspx#location │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD0\xB2\... │ 3 │ -│ http://li.ru/filmId=XpzlPj8P8gE&where=all&text=\xD1\x81\xD0\xBA\xD0\xB0\xD1\x87... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://kinopoisk.ru/sessulyanovka.ru/photosessid=3205&bt │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://dom.net/provoe-pervouralnaya/ChildGluZ19oZXJlci1kYXRpb25zPU4mbj0zJmlkPTM... │ 3 │ -│ http://video.yandex.ru/extra │ 3 │ -│ http://slovari.yandex.ru │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/photo/narod.irr.ru/katering=1&pr=569&s_yers │ 3 │ -│ http://love.ru/ru/irk/event=little&category_id=731-643736&mode=1 │ 3 │ -│ http://kommersantamina │ 3 │ -│ http://afishers/story │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com.ua/kiev/detail.ru/auth=1..640... │ 3 │ -│ http://pogoda.yandex.ru/vlas-moskovskaya │ 3 │ -│ http://audio_video.yandex.ru/mosday.html?item=4#photo446962 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://video.yandex.ru/personal/offeebe34c7e12944&op_product │ 3 │ -│ http://v102.ru/investate/apartment/?id=137336IseNhcbx3J85GkHSnzgnsPdZUU&where=a... │ 3 │ -│ https://slovarenok │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://auto_ford Mix).mp3.ucoz.ru/v1430497.html%3Fhtml │ 3 │ -│ http://trashbox.ru/book │ 3 │ -│ http://guid=6&pw=2&pv=0&with_video.yandex.ru │ 3 │ -│ http://alpari.yandex.ru/saint-petersburg.irr.ru/cars/page3 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://radiorecord │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://en.lyrsenses/zamba_zabor_polnitsa-s.narod.irr.ru/bank/otzyvy/12031%2F%23... │ 3 │ -│ http://state_shariki │ 3 │ -│ http://bdsmpeople.ru/show/39932/Itemid,40.0.html_params%3DfsSaHR0cDovL2pzLnNtaT... │ 3 │ -│ http://afisha.yandex.php?partments │ 3 │ -│ http://wildberring │ 3 │ -│ http://video │ 3 │ -│ http://balcon_caddy Club relove │ 3 │ -│ http://smeshariki.ru/cgi-bin/click.cgi%3Fsid%3D0%26pz │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://kinopoisk.ru/registernet Explorer&aV=5.0 (Windows NT 5.1; ru-ru&cE=true&... │ 3 │ -│ http://video.yandex.ru%2Fkategory_id=9759527418 │ 3 │ -│ http://video.yandex.ru/real-estate/out-of-town/house.ru&pvid=1&distreet_legkovo... │ 3 │ -│ http://omsk/evential/housession%3D240%26rleurl%3D//ad.adriver.ru/marshavskaya-r... │ 3 │ -│ http://wildberries.ru/cgi-bin/click.cgi%3Fsize │ 3 │ -│ http://loveplanet.ru/Bezli-all │ 3 │ -│ http://auto_id=0&color=0&confiscategory_id=3205&bt=7&bn=1&bc=3&ct=1&prr=http:%2... │ 3 │ -│ http://kinopoisk.ru/spb.pulscen.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://ssl.hurra.com/iframe-owa │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http:%2F%2Fwww.bonprix.ru/imagesize │ 3 │ -│ http://saint-peter │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://cxem.net/324487194836848 │ 3 │ -│ http://video.yandex.ru/realty/search/main.aspx?sort=popular │ 3 │ -│ http://un1.adriver.ru/page=30138117749516%252f110916%252fmedicinema/movie_ross ... │ 3 │ -│ http://smeshariki.ru/furniture.html5 │ 3 │ -│ http://lazarevskoe │ 3 │ -│ http://whoyougle.com/iframe/iframe_right.ru/spb │ 3 │ -│ http://myloveplanet.ru/passenger/kitched_country_id=4312&input │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_product_brand=4200&lo=h... │ 3 │ -│ http://klubnich/zrh/ │ 3 │ -│ http://afisha.yandex.ru/manga.ru/?rtext=\xD0\xBF\xD1\x80\xD0\xB8\xD0\xB1\xD0\xB... │ 3 │ -│ http://saint-petersburg-gorod/goodal │ 3 │ -│ http://wildberries.ru/comment/search │ 3 │ -│ http://sunmar.ru/cart&ch=utf-8&sF=11,7,700&aN=Opera&aV=9.80 (Windows │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://wildberries.ru/comme%2F2.12.388 Version/1552/page │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&film/67... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://jobs-education │ 3 │ -│ http://pogoda.yandex.ua/telefon_shtukaturkey │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://krasnodar.irr.ru/yaransferapid │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://irr.ru/index.php?showalbum/logizer8/num-1/refremost │ 3 │ -│ http://radioscannerica/film/47018.html?1=1&cid=691390&pvno │ 3 │ -│ http://forum/topics/ingradskazka-lookoformalities/poetry/events/?date&csrf-8200... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://afisha.yandex.ru/sell/resident │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://video_dvd/suppoll/dleead6718.php?SECTION │ 3 │ -│ http://afisha.yandex.php?p=176d43f96ef32d5bc1272 │ 3 │ -│ http://bdsmpeople.ru/index.by/ru/page=0&confiscategory_id │ 3 │ -│ http://gotovim-doma.ru/personal/commersant.ru/image=19&pvno=2&engineVolum │ 3 │ -│ http://video=0&input_with_video.yandex.ru/page/10/women.aspx │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://auto_id=0&color=0&confiscategoriya%2Fzhienskaya-advert25593 │ 3 │ -│ http://pogoda.yandex.ru/real-estate/apartments/73151 │ 3 │ -│ http://kinopoisk.ru/cgi-bin/click.cgi%3Fsid%3D1216 │ 3 │ -│ http://tyva-5/country=-1&washestvo/den_sidentialAmount │ 3 │ -│ http://rukodel=0&sort=newly&trafkey=27065/2/child.aspx#location │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://mylove.ru/zoom.php?GID=2&IsOrder │ 3 │ -│ http://rsdn.ru/info_all=yes&razdumy-i-trikshop/search&_h_page/1719920 │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=&is... │ 3 │ -│ http://irr.ru/index.php?showalbum/login-do-500-kv-m-Maya │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/procoolonelopitered/ministrict/3d-probeg-340-... │ 3 │ -│ http://ssl.hurra.com/iframe/iframe-owa.html%26custom%3D%26custom │ 3 │ -│ http://myloveplanet.ru/bad_by_sidential/nizhnieiewva88 │ 3 │ -│ http://kinopoisk.ru/real-estate=2013-07-2089241607/photo=0&with_exchangeType │ 3 │ -│ http://chehod/zvenia8312&input_who2=1&input_age1=35&aN=Netscape │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://direct.yandex.ru%2F&sr │ 3 │ -│ http://zapchastny_fashing_mashes/index.ru │ 3 │ -│ http://smeshariki.ru/news/2013/peshnye-udivlekanka.ru/l_03_00/bodreamfood.ua │ 3 │ -│ http://msk/platia-nashing/vanny.diary.ru/otdam_daily │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/used/KIA-Cee-d... │ 3 │ -│ http://arma/frl-4/travel.ru/moscow/details │ 3 │ -│ http://maps#ru_5_ru_1_ru_ru_ru_202_ru_1_ru_ru_ru_ru_2_ru_1743.html?oscsid=36303... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://votpusk.ru/ │ 3 │ -│ http://kuharka=48&modeloveplanet │ 3 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 3 │ -│ http://afisha.mail.ru/stars/page │ 3 │ -│ http://avtoto.ashx/1001087496197797217530729; Media Centernet_mastersburg │ 3 │ -│ http://whoyougle.ru/basket&ch=utf-8&sF=11,7,7,7,700 │ 3 │ -│ http://irr.ru/6323%26bn%3D27888895,96772&op_page47 │ 3 │ -│ http://myloveplants_list-obl.irr.ru%2Fobuv-zhienskaia-modiezhda │ 3 │ -│ http://sendflower │ 3 │ -│ http://smeshariki.ru/index.ru/main.pl?cmd=show/47555 │ 3 │ -│ http://afisha.yandex.ua/index.ru/recipe │ 3 │ -│ http://video=0&is_hot=0&che_simeis │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://bdsmpeople.ru/Web/price │ 3 │ -│ http://smeshariki.ru/chipinfo │ 3 │ -│ http://video.yandex.ru%2F%2Fwww.bonprix │ 3 │ -│ http://myfashihtzu.html?1=1&cid=65625f313230303&po_yers=2013/07 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://auto_operey-v-v-meha.ru/catalog/public │ 3 │ -│ http://forum/topnews/222968695,910112_1164074834-908745 │ 3 │ -│ http://zagranimals-planet.ru/user/31059&Module │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountry │ 3 │ -│ http://kurort/SP1399&op │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stories.ru/art/MACKLEMORE │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 3 │ -│ http://irr.htm?from]=&int[852][to]=10&lastdiscussins/?keyworld │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.php?w=3160/transports │ 3 │ -│ http://afisha.yandex.php?link=114735200&brand=23368 │ 3 │ -│ http://pogoda.turizm.ru/communit=0&vip=0&order_by=2 │ 3 │ -│ http://wildberries.ru/page=0&vip │ 3 │ -│ http://smeshariki.ru/Web/price │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://nizhnieie-bielie-bieriends&fb_source-temno │ 3 │ -│ http://forum/topnews/2229362067528195&op_categoriya │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://zarplata.ru/?p=1290&op_product_price=990348531&schoosOSRquM8gE&where=all... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http:%2F%2Fmuzhchine/ru-g-Chelya_v_tsentyabrskii │ 3 │ -│ http://radiorecord.ru │ 3 │ -│ http://my.alpari.ru/filmId=yJg89hqV8gE&where=all&film/530/?fromCityCodeForcentr... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&top │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 │ 3 │ -│ http://ur.hh.ru/file/news/2013 \xD0\xB3\xD0\xBE\xD0\xB4\xD0\xB0 │ 3 │ -│ http://gorbus.aspx#localiformalitic │ 3 │ -│ http://ej.ru/muss/roll_to_audi/mode=replies │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 3 │ -│ http://guid=6&pw=2&pv=0&po_yers=0&with_video │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&text=\x... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 2 │ -│ http://video.yandex.ru/rent/info/messa.ru │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://ssl.hurra.com.ua/auto_repairs=0&page=10&category_id=detail │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://ur.hh.ru/page=10&can_be_checked_auto_region=1&rm=1&lang=all&film/18212.1... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://pogoda.yandex.php?t=14555667679/?PAGEN_3=5&s_yers=0&numphoto/62232/7#f │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://rlsnet.ru/nogin.html5/v12 │ 2 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 2 │ -│ http://edp2.adrive_type=4754679564&pt=b&pd=9&price │ 2 │ -│ http://wildberries.ru/editem_no=100¤cy=1#country=&op_proizvodskaya-obuv-z... │ 2 │ -│ http://wildberries.ru/filmId=4920/room=1&lang=all&filmId=DnGbyVQVUXI&wheretomug... │ 2 │ -│ http://pogoda.yandex.ru/jobinmoscow │ 2 │ -│ http://auto.ria.ua/auto_auto.ria │ 2 │ -│ http://afisha.mail.ru/cheva.ru/reportby │ 2 │ -│ http://pogoda.yandex.ru/images/0001216629/0 │ 2 │ -│ http://kaluga/?ext=\xD1\x80\xD0\xB5\xD0\xB9 \xD0\xBD\xD0\xB0 \xD0\xB1\xD1\x80\x... │ 2 │ -│ http://masterlingvo/#I 1 │ 2 │ -│ http:%2F%2Fwww.bonprix.ru/search=0&deletedAutos&marketing │ 2 │ -│ http://pogoda.yandex.ru/places/premiery%2Fpage=2&confiscategoriya │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://smeshariki.ru/kitched_country=-1&target= │ 2 │ -│ http://balance/#21792/guest=\xD1\x89\xD0\xB5\xD0\xBD\xD0\xBA\xD0\xBE \xD0\xB8 \... │ 2 │ -│ http://sslow_13500000%26rnd%3D278888 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://poisk.ru/news/articles │ 2 │ -│ http://wildberrin/foton │ 2 │ -│ http://yartb.html?city=55&TopicID=2&IsOrderedProduct │ 2 │ -│ http://video=0&wi=1362653061306535333&site_id=&auth │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 2 │ -│ http://en.lyrsense.com/iframe-owa.html?query=2&hide_black.asp?TOPIC_ID=914958&o... │ 2 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 2 │ -│ http://smeshariki.ru/user_id=1&bodystyle=0&vip=&int[12][from=last_auto_id=13143... │ 2 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=2furii10.html?city=0&hide │ 2 │ -│ http://omsk.spb.ru/cars/art/MVEF2WkMweE5qVTVJOHN - Gl\xC3\xB6\xC3\xB6ckler │ 2 │ -│ http://forum.php?t=420 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://smeshariki.ru/tashkinsk │ 2 │ -│ http://pogoda.yandex.ru/chat/ivan │ 2 │ -│ http://video.yandex.html5/v12 │ 2 │ -│ http://loveplants/740392319 │ 2 │ -│ http://yaroslavl.irr │ 2 │ -│ http://kinopoisk │ 2 │ -│ http://mysw.info/node/21544 │ 2 │ -│ http://smeshariki.ru/product&op │ 2 │ -│ http://video=0&input_state │ 2 │ -│ http://board=11.ua.150.html%3Fhtml │ 2 │ -│ http://slovarenok.ru/chapochki │ 2 │ -│ http://pogoda.yandex.ru/real │ 2 │ -│ https://diary/2013-07-05/101090/currency │ 2 │ -│ http://krasnyj-chastory │ 2 │ -│ http://kinopoisk.ru/saint │ 2 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.139 user 0.404000 sys 0.020000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -┌────────────────┬────────────────┬─────────────┬────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ TraficSourceID │ SearchEngineID │ AdvEngineID │ src │ dst │ pageviews │ -├────────────────┼────────────────┼─────────────┼────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33069 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 24703 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 15817 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 14233 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 │ 6549 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 │ 5257 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 5257 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 3547 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login │ 3538 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik │ 3371 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php │ 3333 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 3309 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 2923 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 2616 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 2441 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php │ 2418 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 2390 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari │ 2367 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 2349 │ -│ 1 │ 0 │ 0 │ https://google.com.ua/url?sa=t&rct │ http://komme%2F27.0.1453.116 │ 2228 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 1920 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://komme%2F27.0.1453.116 │ 1679 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ekburg.irr.ru%2Fpuloveplanet │ 1340 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://aliningrad │ 1270 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1254 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1213 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1193 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1186 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1142 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F27.0.1453.116 │ 1007 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 954 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://aliningrad │ 859 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login │ 821 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 791 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 777 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 757 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 722 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://video.yandex.php │ 702 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 691 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/obucheyelants │ 684 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 632 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 593 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 577 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://smeshariki.ru/obucheyelants │ 566 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.php │ 452 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 445 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBA\xD0\xBE\xD0\xBD\xD1\x82\xD1\x80\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 422 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 421 │ -│ 0 │ 0 │ 0 │ │ http://obninsk/detail │ 407 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 386 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug.ru │ http://irr.ru/index.php │ 374 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru/index │ 360 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 348 │ -│ 2 │ 0 │ 13 │ │ http://komme%2F27.0.1453.116 │ 347 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-2011/43597 │ 293 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 293 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6HS │ 282 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 268 │ -│ 3 │ 72 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 267 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 259 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 257 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari │ 256 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 │ 241 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login │ 237 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 │ 236 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/28435&lr=157 │ http://komme%2F27.0.1453.116 │ 233 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/ru/photo/6936313555&text=\xD1\x85\xD0\xBA \xD0\xBB\xD0\xBE\xD... │ http://komme%2F27.0.1453.116 │ 224 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php │ 223 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cat/dushkirillovyj │ http://komme%2F27.0.1453.116 │ 222 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=359&op_page2/... │ 218 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kriminally-bezhevsk │ 217 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0 │ 212 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 207 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 202 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 199 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://ekburg.irr.ru%2Fpuloveplanet │ 199 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 198 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 197 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 195 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/414526863.xlsx │ 191 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CDMQFjAJ │ http://irr.ru/index.php?showalbum/login-kiriskaya-obl.irr.ru/index.ru/GameMain.... │ 190 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=0&with_photo.ashx/101/4/?cat=6257271 │ 188 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=partments-sale/search?text=\xD1\x81\xD0... │ 187 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://afisha.yandex.ru/index │ 183 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://ekburg.irr.ru%2Fpuloveplanet │ 171 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 168 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 164 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 163 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://ekburg.irr.ru%2Fpuloveplanet │ 160 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 160 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/life.ru/cars/misc/travel.ru/?trafkey=058143&p... │ 155 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 │ 154 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=577&oki=1&oby=&op_s... │ 152 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://wildberrior/uphold │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6HS │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 149 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 146 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustic/meterburg │ 144 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 144 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 144 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E │ 141 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 140 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 136 │ -│ 1 │ 0 │ 0 │ http://kombardighantnie │ http://komme%2F27.0.1453.116 │ 136 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login │ 135 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kysjacevtika │ 132 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://komme%2F27.0.1453.116 Safari │ 124 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 121 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category/stroy/dachines... │ 114 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/?state │ http://komme%2F27.0.1453.116 │ 113 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/GameMain │ http://komme%2F27.0.1453.116 │ 113 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 111 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login │ 110 │ -│ 3 │ 85 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 110 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kardigan │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 107 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosoftwarenok.ru/projects/zhbi.po... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 105 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268143.html?1=1&cid=577&o... │ 104 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 104 │ -│ 0 │ 0 │ 0 │ │ http://tvidi.ru/photo=0&confiscategory_id=0&engineVolumeFrom=&fuelRateFrom=type... │ 103 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 102 │ -│ 3 │ 14 │ 0 │ │ http://komme%2F27.0.1453.116 │ 102 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 100 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 98 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://video.yandex.php │ 97 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 96 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.21150895 │ 96 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?sort=price │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 95 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login │ 95 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.12.388 │ 93 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 93 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 92 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberrior/uphold │ 92 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 91 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-tanks/search=1&ady=62&modeloveplanet.ru │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 90 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.145332.15&he=10&category_id=2740387993 │ 89 │ -│ 1 │ 0 │ 0 │ http://in-the-weightEnd=2351&numphoto=&isExclusiver.ru/alertljus │ http://komme%2F27.0.1453.116 │ 88 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fprodazha_Italja_unit=1&av=1&nm=1&lang=ru │ http://irr.ru/index.ru/show/414526863.xlsx │ 87 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 86 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb1.html?item_no=386703/?bundle=7172&msid=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 86 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 │ 85 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 84 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733582852/ │ 84 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupit/action │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 82 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 82 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?bodystyle │ http://komme%2F27.0.1453.116 │ 81 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F22.14&he=768486 │ 81 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusk.ru/ru/lookaginitial/Prodayu-Dach... │ 80 │ -│ 3 │ 4 │ 0 │ │ http://komme%2F27.0.1453.116 │ 80 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 79 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://aliningrad │ 78 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 78 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login │ 76 │ -│ 1 │ 0 │ 0 │ https://go.1ps.ru/show&showforum │ http://komme%2F27.0.1453.116 │ 76 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandex.ru │ http://komme%2F27.0.1453.116 │ 76 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26221/detail.ru/v1496366&... │ 75 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php │ 75 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 74 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%2F&sr=http://bonprix.ru... │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len80/page/product │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filtr/all/perm.pulscen... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 73 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 73 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://diary.ru/forum/intries │ 72 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F1.7.1364.172 │ 72 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/show/414526863_112 │ 72 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2618561&pp=1059&op_produc... │ 72 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://video.yandex.php │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 71 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 71 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari │ 71 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://aliningrad │ 71 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=product_id=42&Selection.chelov.ru/searc... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/yandsearch?te... │ http://irr.ru/index.php │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/server=sc.chel.ru/main... │ http://irr.ru/index.php │ 70 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification/?year=\xD0\xB1\... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2791954~43.87725656132&op... │ 69 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://komme%2F27.0.1453.116 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/produkty/bleacs/udilis... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 69 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&wi=1366&br... │ 68 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 67 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731&CgID=124jc&where=all... │ 66 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/Yozh/Goodda │ http://komme%2F27.0.1453.116 │ 66 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2725215195&ti=\xD0\x9F\xD... │ 66 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://ekburg.irr.ru%2Fpuloveplanet │ 65 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru/index │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 64 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://afisha.yandex.ru/index │ 64 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 63 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nmg.com%2F4.0 Safari%26clients-sale/search&e... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 63 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 62 │ -│ 2 │ 13 │ 13 │ │ http://komme%2F27.0.1453.116 │ 62 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login │ 62 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://ekburg.irr.ru%2Fpuloveplanet │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 61 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 61 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26092.html?s_text=\x5C\x5... │ 59 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/thenon-houses/public/g... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27755f32316.30; .NET CLR ... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742/details/?cauth=0&dam... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fautoad/kniga.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26004/?_h=galle/\xD0\xBF\... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert27930555&sob=1&p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2692.html%3Fhtml?period=3... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/world/photo31469:Album... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=leting "HalUXI&where=\xD0\x9F\xD0\xB5\xD1\x8... │ http://irr.ru/index.php?showalbum/lofiver.ru/articles/86121%26uid%3D139750%26ad... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27000,224648804-recept-so... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/koshka.com/ig/iframe-o... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory=cinema.perm.p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/realty/suntime-5/extre... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742304][from]=&int[14670... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742357/detskii_gosts.xml... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694336/photo/suzannason/... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Forum25/top... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268320995,968650f45491882... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 57 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupit/action │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1375605&ga... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://video.yandex.php │ 56 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga/choice/... │ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_product_... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?showtopic,5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26017/quarius_Moscow/cavi... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2726125413975d77cf&search... │ 55 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 54 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266848223/review_type=pro... │ 53 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 53 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26989%26bt%3Dad.adriver.r... │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/album/login-1800002&pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 52 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0 │ 51 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2698172,93932353064614618... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605085268a9c4d61-9862.49... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19&numphoto/login=A-CL-MS-36575c72937][to]=&int[12822304 │ http://irr.ru/index.ru/widgetchrome%2F&ti=no&dom_v_bordovye-printime.ru │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/investate/comple/make/?page5/&docid=jlMNIrXw... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 50 │ -│ 0 │ 0 │ 0 │ │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 50 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F11.7.1364 │ 50 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 50 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26051.htBeg=6&NightRegist... │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xBE\xD1\x81\xD0\xB8\xD0\xB1... │ 49 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 49 │ -│ 5 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 49 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?group=days=3&text=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27410/photo-2.xhtml&serve... │ 48 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandsearch │ http://komme%2F27.0.1453.116 │ 48 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 48 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusty-i-vkont.at.ua/search=0&userId=0... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266128182&op_seo_entry/de... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?sort=&br... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/intrumen │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://afisha.yandex.ru/index │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumanegenre=33155?analog/kitchen_mini... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login%2Flick.g.doubleclick.diary.ru/car/kw/3061/us... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://afisha.yandex.ru/index │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734.html_params%3Dfh_loc... │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/GameMain.p1Yo4A │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ua/searchads/jo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605906/frl-2/sportal.ru/... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fshow/lpp/cre.ru/pers... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.asp?search&ev... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logize.ru/msk/events_liver.ru/russert-plies.r... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26099486633033&countpage=... │ 44 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://sslow_135000008&position=search │ 43 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 43 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischet-solik/odezhda-plos... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert277553/38021/66936575776/... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271050&with_photo.kurortm... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 43 │ -│ 1 │ 0 │ 0 │ http://forums/liii-kuler S22oHgBJTngegotavgorod55.ru/filtr[2]=42 │ http://komme%2F27.0.1453.116 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-folders/?end=9a08488... │ http://irr.ru/index.php?showalbum/litamak.irr.ru/catalog/8570/travel.ru/cgi-bin... │ 41 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 41 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 41 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/0001216629/#top_by │ 41 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26040931&s_yers=0&with_vi... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://ekburg.irr.ru%2Fpuloveplanet │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.137508&s_yers=200... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://aliningrad │ 40 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logito8ergo_6470/page_type_id=20872/0/001:12:... │ 40 │ -│ 0 │ 0 │ 0 │ │ http://afisha.mail.ru/dmitrij │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/readar-nashi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27924563724&key=46960/med... │ 40 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://lib.ru/exp?sid=3205&bt=7&bn │ 40 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 39 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credit │ http://komme%2F27.0.1453.116 │ 38 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=8570/page=5&s_yers=2006 │ 38 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2679/detail/5482,935033/2... │ 38 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 38 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login │ 37 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dnevnoy-kv-Samara.irr.ru/searchAutos&marka=4... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 37 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://komme%2F27.0.1453.116 │ 37 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=theating&page=... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2784389/room=39695,966681... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?group │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/forum/topicID=269&stat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/votkim-rukav-i-kova-ul-adve... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wolf-club478561/OPEL | \xD0\xB0\xD0\xB2\xD1\... │ http://irr.ru/index.php?showalbum/login-marka=13&city&custom=0&damages/0001216 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filmId=xFX1UbTNJjxe4yF... │ http://irr.ru/index.php?showalbum/login.asp?razdel7/test/matched=115909d9_dsc07... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27553&s_yers=0&po_yers/46... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert275226607660?design=6efxq... │ 36 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=show=&fgroup-tab-mara.irinables/yearT... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278171&size=52094&op_seo_... │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27330528/photo/6223799%26... │ 36 │ -│ 5 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://smeshariki.ru/obucheyelants │ 36 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36&he │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=post_neu%3D22%26pz%3D0%26rleurl%3D%26xp... │ 35 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/8569 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/index.ru/katego... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vip.ru/book.com/photo/7086/page=0&state=mont... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-cukickdrops_bak_yant4tlGtZgy3BQ&bvm=bv.49478... │ http://irr.ru/index.php?showalbum/login-501-98552&op_uid=1060948/6#f │ 35 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (Windows │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19&m_staraya-obl │ http://irr.ru/index.ru/show/414526863_112 │ 34 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.12.388 │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 34 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_uid=1... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/promo=C-Tease/rent/dat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/list │ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 34 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/directadvert/kavanga_728x90.html │ http://komme%2F27.0.1453.116 │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 33 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshops.xml?typ │ http://komme%2F27.0.1453.116 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?cPath=default.ru/yandex.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33 │ -│ 1 │ 0 │ 0 │ http://auto.ria.ua/\xD0\x9A\xD0\xBB\xD0\xB8\xD0\xBC\xD0\xB0\xD1\x82\xD1\x8C\xD1... │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 33 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://video.yandex.php │ 33 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 33 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser │ 33 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265748ba3f755671_1162684&... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertif?sle=85000 \xD0\xB4\xD0... │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/pub │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364 │ 32 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zxcvbnm1990&search/jobinmoscow.ru/yandex.by/... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements-obschaya │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fproduct/foto-4/login... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&op_category_id=1174]... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniturers_id=575386110 │ http://irr.ru/index.php?showalbum/login.exit/rem/php/board/10_kruizer_8267049&o... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiepochto_id=93231238][to... │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://afisha.yandex.ru/index │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ssl.hurra.com/iframe │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischuk_Kerasivye-klavik/?... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/jobs-education/m_acces... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26791e3a7c543c9bf9ef5c0-9... │ 31 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt │ http://komme%2F27.0.1453.116 │ 31 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertificia/?letteo_985428161/... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_api.php?cid=41389/rooms/perm.pulscen.ru/sear... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 30 │ -│ 1 │ 0 │ 0 │ http://domchelov.html&lr=55&text=\xD0\xB4\xD1\x80\xD1\x83\xD0\xB3\xD0\xB8\xD0\x... │ http://komme%2F27.0.1453.116 │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoryID=12452929587/... │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 30 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 30 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268630533&site_ot=&price_... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-k608i.trashbox.ru/cars/page5/&docid=577&lr=2... │ http://irr.ru/index.php?showalbum/logii_58247.php?forum.borovichy77/page5 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2716/~3/150160947694,9745... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2696374/#page/tab=com_avt... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/products/?category_id=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 30 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/topic │ 30 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.14&he=768&wi=1920 │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26979553039353936694&ch=u... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605446518%2F&sr=http://e... │ 30 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://smeshariki.ru/obucheyelants │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-carava/fotok-8-steklove.ru │ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26928142&op_products/view... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kupibatikov/?recrete/personal/klimat.ru/work... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 29 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x9A\xD0\xBE\xD1\x80\xD0\xBE\xD1\... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertifications[0][150]=60&inp... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/wedditeli_foto-2/__48/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_countpage/1 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/click.net/paged=696&lo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274229696.0 (Windows)&bL=... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-14132&with_photo=1&srtime-zone.net/catalog/t... │ http://irr.ru/index.php?showalbum/login-sierra%2Fdlia-zhienskaia-moda-zhiensmed │ 29 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704.html%3Fhtml5/v12/?fr... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2664952&xdm_p=1#country=-... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734/fotographic_2017666&... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategoriya/zhienskaya-... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9649.html%3... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/small_mercial-shoppicc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27350350394&ch=UTF-8&sF=1... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_seo_entr... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-baza/gadge... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27425.3120%26ntype_id=2&r... │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 28 │ -│ 1 │ 0 │ 0 │ https://directions[0][1541769377921968 │ http://komme%2F1.7.1364.172 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert272291684/hasimage=0&view... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13733097&op_categ... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/mann japanel/?text=\xD1\x81\xD0\xBC\xD0... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269500469412216388/detail... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-my-sovies/bebybum.homestion bonprix.ru/... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikarbox=2&tz=9b81998&po_y... │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?lang=all&fi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-176-kakaya-obl.irr.ru/search?lr=191&ei=N6rrg... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 28 │ -│ 3 │ 95 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\xA4\... │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/anime-2/#album/l... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-eve-nn.ru/cars/passenger/\xD0\x92\xD0\x90\xD... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274236&pv=15&prr=8&sid=35... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/41452626u1zIq0SGLXCI │ 28 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic/8939858-59929477/detailshop │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://ekburg.irr.ru/#lingvo │ 28 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F22.14&he=768486 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678/page=2&marka=84&mode... │ 27 │ -│ 3 │ 2 │ 0 │ │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/search?cl4url=http://yandex.ru/online... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27013.ya.ru/sportalznakom... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert24061.jpg.html%3Fhtml?1=1... │ 27 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031343-sh-898675/sort=pri... │ http://irr.ru/index.php?showalbum/login-5158&jenre/slyudya_metroman.ru/GameMain... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19195/offset=101&district=&city&with_photo=¤cy │ http://irr.ru/img/catalog/53485785/topic/8939850.php?page=3&marka=84&mm │ 27 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 27 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 27 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-oleginnesujeli-i-centum=etarget=search?clid=... │ http://irr.ru/index.php?showalbum/login-kupe.html_params%3Drhost%3Dad.adriver.r... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fhit_list.html?s... │ http://irr.ru/index.php?showalbum/login-irake-myservierk-supierts%2F&ti=\xD0\x9... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27928092_price=\xD0\xBC\x... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/mazda-dievochnye_avari... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-19.xhtml?city_to_one=All&film]/on/orders/810... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/1.19.mobile_photo=6001... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27630/?item_id=0&bodystyl... │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ 1 │ 0 │ 0 │ http://bonprix_ru_11559&lr=12&usln │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lizova_ii_1112_1150&option[price_do=300... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E&wher... │ 26 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 26 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detall/seconomicsmovinki │ http://komme%2F27.0.1453.116 │ 26 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://obninsk/detail │ 26 │ -│ 1 │ 0 │ 0 │ http://yandex.ua/?target=search/tab=user │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/loginsk.irr.ru%2Fkategory_id=9204471-650/?_h=... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login-jjjekrju-s-pring=1&price[price=\xD0\xBC... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification.ru/messa.ru/rea... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lejandsearch?text=\xD1\x82\xD0\xB8\xD1\x85\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert21124631/?Search.php │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 26 │ -│ -1 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_category... │ 26 │ -│ 3 │ 197 │ 0 │ │ http://komme%2F27.0.1453.116 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?messa.ru/worlds/page2/7277932582&text=\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 26 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-kupe-20010120652838799.html?n=7148.html... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemey-volvom80Cw&where=a... │ 26 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=607&state=4... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.22 (KHTML, like Gecko) │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 25 │ -│ 1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 25 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/people.ru/cars/passets... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731/?city=0&pvno=2&evlg=... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vivarki/redki_sdelaet-law.ru/produkty/kartgo... │ http://irr.ru/index.php?showalbum/login=driff/en-ru/?p=2#messages/00001216629 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.html&ei=GcXrUYe8F2DAU-RbuRlLMczoJ--5uDCx8 │ http://irr.ru/index.php?showalbum/login-132/#images/0000.h95/\xD1\x82\xD1\x8E\x... │ 25 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert264314953); ru) AppleWebK... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search?text=myaccountry │ 25 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://obninsk/detail │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguard-payments-sale/se... │ http://irr.ru/index.php?showalbum/login-88i8LaDyEkCVv6-DhRfEDcw==&action │ 25 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/yandex.ru/?f... │ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.ru/film/46351/frl-2/bage │ 25 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605838.html5/v12/?from=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741559&t=5204/make=Chrys... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-link=1147373-1&p=1&p=2&text=\xD0\xBC\xD0\xBE... │ http://irr.ru/index.php?showalbum/logisterlingvo/#! │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694589/detail/555-javata... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=shops.html?id=1&body_type=\xD0\xBF\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781472265&op_seo_entry=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626rleurl%3D//ad.adriver... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685/price_ot=&price=\xD0... │ 24 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://diary.ru/forum/intries │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27399992.php/board,13.0) ... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671291&fridgets/2012&pri... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26685857947301_Zoryatiya/... │ 24 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-117031&op_category&op_seo_entry=&op_uid │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo/7... │ http://irr.ru/index.php?showalbum/login-kupit-plitka-s-korsiety/art/151/n4p/160... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguar.ru/page4/?emain.a... │ http://irr.ru/index.php?showalbum/login=ogabass.ru/filmId=9WOqzzitive&view │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-annative-expressageId=0oVXx21hUXI&where=all&... │ http://irr.ru/index.php?showalbum/login-rybnovlevojj_mamountry=-1&type%3D0 │ 24 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://video.yandex.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/intrumen │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kupalnaya_elit=&id=3&clid=9403&lr=1913849 │ http://irr.ru/index.php?showalbum/login-gallebia_zhurnalCall D.R.E.mp3ex.net/ph... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lolacake/tatyaka.html?1=1&cid=577&oki=1 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinglish.rus-l7-p70505-1... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/yandsearch?lr=1... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-yvaken/topic=7702.jpg.html?1=1&cid=577&... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=1&bc=3&ct=1&pr │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-547552/1/?sma=RAINBOW&op_uid=fdd1b9d2721728&... │ http://irr.ru/index.php?showalbum/login-kapusta-advertika/search/room=1&damage=... │ 23 │ -│ 1 │ 0 │ 0 │ http://auto_volkswagency=1&text │ http://komme%2F27.0.1453.116 │ 23 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgarials/commend/searchvage-2.html&dt=13734... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-tuniki_510... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263090024189-528619c2077/... │ 23 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/ru/index.ru%26bid │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333572620201709/page/10... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26939.html?page/3/#count=... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news/radio&clid=198555... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/kategory_id=5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2740974%26pz%3D0%26ar_sli... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 23 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/resultatya9176da22f1a521a5853.html&... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26600721376439322%26pz%3D... │ 23 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga_728x90.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://afisha.yandex.ru │ 22 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemennoke0070553995f27d6... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2697574697.html_param=0&u... │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-105vr/ │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=search.tutung/s... │ 22 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/wheel.ceratornaya_solnechka │ http://komme%2F27.0.1453.116 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-10760b31b65633eZateq eb806e887d9f15ccf593280... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2091&local=1&TypeSearch?mail.ru/yandsearch&c... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26058&nnum=s4746835895&op... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-the-ili-tempera-minsk.irr.ru/produkty/bluzki... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739693071/pomer... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodny │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2649.html?pg=menu_29.jpg&... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/v1465][from]=&int[157281 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267830840994,95688781470%... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-business/sale/ru/search?mail.ru/film%2Fbarna... │ http://irr.ru/index.php?showalbum/login-zakon_type=1&fuelRatesTypeSearch │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 22 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27200,55.7654.html_parat-... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&l... │ http://irr.ru/introlux_page5/2/page/Renaul.irr.ru/start=1500-rublic/gamemain/Tu... │ 22 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26601629][from=&fuelRateT... │ 22 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://ekburg.irr.ru%2Fpuloveplanet │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liliate_id=24145602&with_phoney.yandex.ru;ya... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-natashkarola.irr.ru%2Fviewforum/MsgList.html... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26900/technics-technics-t... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&city[7]=23239330794,25826044_Black_lis... │ 21 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?price_till beli.php?cx=015216684_36 │ http://komme%2F27.0.1453.116 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691_ru_17832523.html_par... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694,978825315f373400/det... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1) AppleWebKit%2F537 │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/world/photo/41/62b1dfa450/3/women-clother%26tms%3D%26ev_ltx%3D... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/basketshop.ru/yandsear... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/forum%2Fukhov/l... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265228.html?1=1&cid=577&o... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27025&Select[4][]=32598.h... │ 21 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://wildberrior/uphold │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/?from │ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/houses/passenge... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo44953.116.php?t=4... │ http://irr.ru/index.php?showalbum/login=ko&page=0&view.aspx?group_cod_1s=85&key... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-dance/index.ru/?favorite_off=1&encoding&hl=\... │ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logino-s-grigerator/page1=&input_age1 │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cars/commeddesk.ru/google_s&12.5.746.59954.30 (KHTML, like/\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26737865/?page/12186/#/ec... │ 21 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ramblery/pic/893985650697... │ http://smeshariki.ru/obucheyelants │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login.html?1=1&input_who1=2&input_who2=1&inpu... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/logii-bin/click.cgi%3Fsid%3D158195,97987231-9... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/club.ru/anime-zone.ru/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-brie... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dietsik.ru/razdel_id=111,7,700&w=728x90.html... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929&from%3D%26custom=0&... │ 21 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2186/detail.aspx#location... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739353/details.... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kupalnik │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification%2F4.0; XBLWP7; ... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/9-0--navam-zhien... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741465][from]=&input_wit... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_uid=1147... │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login=uzpp2XnEw&bvm=bv.49784469][from │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 20 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-Larki-i-korobeg-1124-95367/guestblowinp... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26219&wi=1280&lo=http://w... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estatic.diary.ru/... │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=58&Selectronics-technics/k... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/yandex.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?er=2&text=\xD0\xBA\xD1\x83\xD0\xBF\xD0\xB8\... │ http://irr.ru/index.php?showalbum/login=razer2.moikrug.ru/recipes/shop.spb │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b706234651&op_uid=1&b... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&price │ 20 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://video.yandex.php │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=NeIdeaCenternet Explorer&aV=5.0 │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertificirovka.ru/noteating-i... │ 20 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271161076&st=261&t=170977... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate=0&ReturnUr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313163307/?from]=&int[15... │ http://che.ru/produkty_zarubezhei-niepochekhly │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604370757034dea482207549... │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_api.php?id=7262882,9454472&op_product_n... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 20 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/cars │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26637&model=1705.html?1=1... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-karbox=0&top=0&category/1961/#images/00... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumilora481 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt=13733129/room │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-iulyanovskaya-obl/lyubvi?page=166733&eid=485... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27502/detail/?folders/#pa... │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://video.yandex.php │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626u1zIq0SGLXCI&where=al... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=240&u_cd=242035165&input_who2=1... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news_id_72387877055/nu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikulina.ru/real-estate/ou... │ 19 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?sort=popup/casualg... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/filmId=eRaB4pYAXCI │ http://komme%2F27.0.1453.116 │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.ru/daily │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBB\xD0\xB8\xD0\xBA\xD0\xBE\xD0\xBB\xD0\... │ http://irr.ru/index.php │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login=perfect.yandex.ru%2Fkategoriya%2F537 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260540718534/?dateOn=1&bo... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26672655462.html?city.ru/... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261921&pr=2441185112.html... │ 19 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostok.ru/cars │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671&Destige-all.html_par... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/check=1&redir=1&lang=a... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27404895,95923.html%26cus... │ 19 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2638/?from=&raceTo=&power... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313437&pvno=2&evlg=VC,2;... │ http://irr.ru/index.php?showalbum/login-rizova.ru/look/timashirtsevm.cofe │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6HS │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166333/price[price_till]=&pri... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F_liveresum... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.php?f=11340.1j1j2.12...5998-103358071/8/page... │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/00007/10/1320006&p... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27355f3132366336601&uuid=... │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto_repai... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://smeshariki.ru/obucheyelants │ 19 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.15751 Safari%2F5.0 (Windows │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2675406828ac956028692581/... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26968794618/irr.ru/real-e... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://obninsk/detail │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2702483,631;IC,444041%2F&... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2629&lr=65&rstr=-213&text=back=False&withmos... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=http://video.yandex.ru/real-estate │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26669E116-1-1550046560013... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?search?text=\xD1\x80\xD1\x83\xD1\x81\xD0\xB... │ http://irr.ru/index.php?showalbum/login-inadlinny-sched_car=0&city=0&price=18&p... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/dlia-zhiensk... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274026181891&rn=52&brand=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-165&jenre]/49047&lr=163&text=samsung ga-moda... │ http://irr.ru/index.php?showalbum/login-2-komn-v-na-92f5-6ccf-fef3-013f9f926a21... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&checked=0&state/room=10002-g-v-program... │ 18 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ru/index │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5 │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?elmt=popular&where=all&film/723/num │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/restate/out-of-town-resh=1&cnt=1373... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2012-recept-Ragu-iz-2-x │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupaljinik-slidi/places/?PAGEN_1=2 │ 18 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27534.30 (KHTML, like Gec... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=deltec&type=citymo=&version%2F4.0 (Linu... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kochtinsk.pulscendinskiy-krasnodar.irr.ru │ http://irr.ru/index.php?showalbum/login-kuplyuskij/zapchastorii_state/out │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zzzi.ru/a-shop and blademas/?id=1&marka=0&ma... │ http://irr.ru/index.php?showalbum/login-2003prev=/search/index.ru/meteor.html%3... │ 18 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login.moikrug.ru/catalog/otchenko-ul-advert27... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\... │ 18 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278453&with_exchange=0&ex... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27132069304;IC,1511e8d158... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=367132&pv=2... │ 18 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27071038_114736895,923401... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x9A\... │ 18 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kanza_tab_2499472&lr=1074&sort=popular │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/?target=search?... │ http://irr.ru/index.php?showalbum/login-bigutensiysk.irr.ru/regular&bid=3205&bt... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27420780790931.html5/v12/... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/8018&l=malkogo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-natura.ru/jobs-education=post_137336ID0... │ 18 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-21102&text=\xD0\xB1\xD0\xB0\xD0\xB1\xD0\xBA\... │ http://irr.ru/index.php?showalbum/login.mospsy.ru/super-na-kostronics/tv-audio.... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/obuv%2F │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685&numphoto=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626p/ │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertimeshariki.ru/cgi-bin/cli... │ 18 │ -│ 1 │ 0 │ 0 │ https://diary.ru/magnitogorsk │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sien-zhienskaya-i-kvartir.html?id=1955451-01... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260908861%26point-peter=&... │ 18 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1453.116 Safari%2F8536 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeSearch?text=\xD0\xB2\xD0\xBA\xD1\x83\xD1... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=yators/?jumping.ru/series.ru/priceup&page=3&... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga1788778.137329TVmbFNmbVNaR0YwW$UyNDg4fxS... │ http://irr.ru/index.php?showalbum/login-sienskaia-moda-zhienskikh_rabotjaga.ru/... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=36427929&stat... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinger/\xD0\x93\xD0\x90\... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929750%26ar_sliceid%3D2... │ 18 │ -│ 4 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691c352005&pvno=2&evlg=V... │ 17 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/cars/communalux.ru/dos... │ http://irr.ru/index.php?showalbum/loshaya-obl.irr.ru/film/35562C?analytics/kto-... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271527091/frl-2/bage-2.ht... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru%2Ffilm/4858?pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27422544/3/womens.ru/ufa.... │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#7878... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.103402.html%3Fhtml │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=artira-Lyubyat │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.aspx#location │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=yes&y1=2013-07-07.ya.ru/5_search?text=\... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.15.html?1=1 │ 17 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://aliningrad │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266149_51928/detail.ru/ka... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%26rleurl%3D%26CompPath%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_Bolers-device.aspx#commersanta_premiery... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2670308&po_yers=0&price_o... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2613168312.71 Safari%2F__... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=8&ch=UTF-8&ifr=1&av=1&nm=1&ved=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-dlia-zhiensmed.ru/el... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 17 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6HS │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginyazevskaya%2F5.0 (Windows NT 5.1) Prestorygoo... │ http://irr.ru/index.php?showalbum/login-kupilomatic.aspx#location=rr&days&city&... │ 17 │ -│ 1 │ 0 │ 0 │ http://bdsmpeople.ru/register2123 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logistereda.ru/photo/70861/6#f │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263161366&bid=2823&numpho... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26041.htm%3Fsid%3D0%26pz%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowthreadreplies=24&ids=74&cu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273363331251798796&op_seo... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27595,9305f3931339322%26n... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313&ussp=mstp&usln=1&inp... │ http://irr.ru/index.php?showalbum/login-3767436/roomootofile/obzor.com/iframe │ 17 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/stan/index.kz/yandex │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-search%3Futm_source=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertistia.html?partment/searc... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031323532363&win=70&mode=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 1 │ 0 │ 0 │ http://bibieters_sig=5ea8QizwGSPy0yGcm14ATH74D4Bw&usg=AFQjCNHxMZlonova.ru/p1847... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://diary.ru/forum/intries │ 17 │ -│ 3 │ 22 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_style... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lenfiki-zhienskaia-moda-muzhchin-sitafa ishq... │ http://irr.ru/index.php?showalbum/logical.ru/link=11485-9065383235348%2F&sr=htt... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-futbolki-k... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261636/detail/odnoklassim... │ 17 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/messar.ru/professig │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 1 │ 0 │ 0 │ http://acase.php?input_who1=2&input_who2=1 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.10 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268814453957595,94406/det... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-irina19910735%2F27.0.1453.116 Safari │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=Like Gecko) Chrome%2Fpotnik=1&price_do=&curr... │ http://irr.ru/index.php?showalbum/login-the-poxudet_sponsor=&o=6000708498/?caut... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733145625/details... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26618531&t=111246.html?p=... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694&option%2F12.14&he=76... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b76faact/less=1&ru=1&... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=bestwo.diary/details&id=9160/0/posti.ri... │ 17 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-53584715,1327288/belgium/arch=1&themec.... │ 17 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/category=cinem rasmusic │ http://komme%2F27.0.1453.116 │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-herokee_tatus=1&rm=18606bAxFEcQT6smBB4W... │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267044005320AA76)&vendor_... │ 16 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search?clid=19554786.0.8.0.2.2003558_3497926... │ http://irr.ru/index.php?showalbum/logie-niz-doktops/electronics/4185570c72 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098707209463401090/deta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27034195,9520160173/5/wom... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgas.rQRAX2Ua3IF1iHJcaz4ATv34DABw │ http://irr.ru/index.php?showalbum/login/?page=1024&wi=144438-88-3906740/currenc... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search/cuZXdzJTJGJmxyPTI4fQ,... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&pt=b&pd=7&pw=2&page5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-plan... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26396284046.xhtml%3Fhtml%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate/out-of-tow... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260612d7cf.4082813,74,756... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login.aspx?sle=13&s_yers=0&page_type=0&door=0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnaja-prost-petersburg/detail/?fold... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?&clients-sale&siteurl%3D//ads/search?text... │ http://irr.ru/index.php?showalbum/login-kapusta-advert273786/foto-5/#photo=2817... │ 16 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/movinki... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/page=91552&q=\... │ http://irr.ru/index.php │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-krasok.html_params%3Drhost%3Dad.adriver │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti/redmonds %2F offset=403134333.2201560&c... │ http://irr.ru/index.php?showalbum/login-v40-velopiter.feriod=0&scroll_to_auto.r... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661232&streamfood.com/if... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiensmed.ru/dl/\xD0\xB7\x... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166303133334%2F&ei=horia+iudi... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26765.html_params%3Drhost... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-57548811426&text=\xD0\xBB\xD1\x83\xD1\x87\xD... │ http://irr.ru/index.php?showalbum/login-coolonellana-Molodilnik.ru/index.php?vi... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/70948/35.htm?id=... │ http://irr.ru/index.php?showalbum/logiy-luchaiev%2F&sr=http:%2F%2Fwww.bonprix c... │ 16 │ -│ 1 │ 0 │ 0 │ http://histore/profile;u= │ http://irr.ru/index.php?showalbum/login-kupe-20013&uuid=1373356164564_1sort/sta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/showtopic,803813.html&... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_catalog/1250/?itemsg/cd... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=findjob.ru/planet.ru/index.ru/carinov.h... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2755597.html%3Fhtml?1=1&c... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert260477/cities/visas/exp?s... │ 16 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://video.yandex.php │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26887301/Zona/simferopol.... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2728024/detail/508/?instv... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27429&input_bdsm_position... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-153-4b9c4.4331626.htm?size%3D0%26rleurl │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/posts&q=\xD0\xB2\xD0\xB5\xD0\xB4\xD1\x83\xD1... │ http://irr.ru/index.php?showalbum/login-mistore/#CATALOG_LIST-s-ovosibirsk/deta... │ 16 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://msuzie-shop/premiery-c-38208_2.html │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 16 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-marka=23&model&desting?field_town/search/pri... │ http://irr.ru/index.php?showalbum/login-492ea9&show/417582,9626512874":10157628... │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27364229586121%26url%3D%2... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/sch/price][min]=41&lr=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ 1 │ 0 │ 0 │ http://volgografiyah_27_iyunya_50_let%2FgetId │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333%26bid%3D1%26rleurl%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new=127201148][fr... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661251-6efa-d61f-fef3-01... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-maritkiN │ http://irr.ru/index.php?showalbum/login-ladimir.irr.ru/remiery-telefon.ru │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/tv-audi/a7-spec=9894797179698712/p1/8... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logiste's a di galaxy-s4-story1.aspx#location... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5.0 (Win... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti%2F%3Ff%3D100%&http://afisha.yandsearch&... │ http://irr.ru/index.php?showalbum/login-6030d.html_params%3Drhost%3D90%26height │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/7427510/detail/5... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269899458/price/3815&pvno... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=view/5/item3963... │ 16 │ -│ 1 │ 0 │ 0 │ http://yandsearch?lr=2&color │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new10152954vac │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-gorod/search?p=7&oprnd=9902.jpg&img_url=http... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27222.vk.me/u3166 │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2735030373434-152495.php?... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2753][from]=&pricedownloa... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makeevka.ru/perm.irr.ru/slingvo/#1\xD0\... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691.html?1=1&input_age2/... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2010000087378687/pic/8939375f303839303935373... │ http://irr.ru/index.php?showalbum/login-souse=2/path=39_1.html?1=1&cid=577&oki=... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-irinakai/page_num_read2306e4574&ei=Ot_rUfjPG... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26708662307][]=&selection... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313033393633310,932803][... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumy.ua/search?text=\xD1\x81\xD0\xB5\... │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2695/19545602cf94d77/repl... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/artira │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/?strict=5710... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertists/736850/?item_no=737&... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikal.ru/search/room=1&dam... │ 16 │ -│ 1 │ 0 │ 0 │ http://forums/liiiervierk-suppohudeemvmesting/294465&op_categoriya │ http://komme%2F27.0.1453.116 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&model=1173... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=new&cad=rjt&fu=0&input_country_id=228&lr=961... │ http://irr.ru/index.php?showalbum/login-cam.shtml#objdesc=true&uA=Mozilla │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login/?do=ready-mansion%3D0%26url%3D%26bn%3D0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb=1&prepairs=0&city[1 │ http://irr.ru/index.php?showalbum/lounona-muzhchin-kupaljinik-chernyj-odnoe/san... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-4433140796851/detail/536180&all=False&i... │ 15 │ -│ 1 │ 0 │ 0 │ http://launcher-searchads/search │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26981583/page=1089592445/... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_produkty/kiev.org/forum... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginsk.eu/ru/Pagesize=13&m1=07 │ http://irr.ru/index.php?showalbum/login-3muda/truction/vacancies/750207190765.h... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-album/login.2/second... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781308;IC,238208836746/0... │ 15 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F1.7.1364.172 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945HPS?analog/r10418/sort=price_do=200&site_off=1 │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 15 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshop/id_art_type=7&s_yers │ http://irr.ru/index.php?showalbum/login │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273631253711/foto-6320166... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2724/?_h=search?text=\xD0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273389; U; Android 4.0.14... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-goluboj-podserial&dfs=13 │ http://irr.ru/index.php?showalbum/login-haus.html#news/228670,257&pci=3012/frl │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#67 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266356163f39ea6f8/7a8745&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/washes/built-in_two_ch... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26605566254/room=1.6&wher... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?transk.i... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://irr.ru/index.ru/show/414526863_112 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?Topic │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?page37/?sta... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/produkty%2Fplatj... │ 15 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/state=1909644 │ http://afisha.yandex.ua/auto_id=0&with_photo.kurortmag.ru/razdnitsya-vishnury │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list.html?typ=SMA&anbieter=aleks_evilkos.com │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/index.ru/ange=0&s_yers=(3000&static.diary.ru... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-land_search?text=\xD0\xB0\xD0\xBC\xD0\xB1\xD... │ http://irr.ru/6323%26bn%3D27888895,963095425 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginskikh-viana.irr.ru/GameMain.aspx?sle=1/house.... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678&op_page=60000/curren... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?Id=9582 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=alini... │ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liloveplanet.ru/futurer533/women.aspx?group-... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2631920&lo=http://person_... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-tumbov.irr.ru/page1=&input_who1=2&id=29... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.1Uv1UfqCA8SonYC4BQ&usg=AFQjCNGB3pBUuKY1jJPP3... │ http://irr.ru/index.php?showalbum/login-tank-sale/search%3Fmode=&page_type │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2718599/photo=0&is_hot=0&... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.php?city[1]=700003 │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26761,55.654289600/detail... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2699364224073532663835386... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login=vladimir/page_type=0&expand_search?text... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27515/418695&st=327-B110Q │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?dok=001&rightbox/week&From=0&user │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/frl-4/trailer/view/3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-lential/secondary/details9.html_params%... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/gamemain.asp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page9/#14... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienshchin-planet.ru/myma... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=25127691%2F... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27265484158197%26width%3D... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27877437494,943082&stat_t... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2730675595,9292fa-d61f-fe... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-676216b8af/4fd00fa61b3185631821/page_ty... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_appliazh-pliance/mista-bez-uchaiev/000202&cl... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/ch/floore troubleclick... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1955451&lr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 3 │ 123 │ 0 │ │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Ford-antent... │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginy_name/3196906481/currency=6465373200&price=\... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2776/?date=0&doorov.irr.r... │ 15 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nashinger/users/search?textilead&353%26ev_pl... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-102-821922237&arrFilter2013/08-iH4AT5zIGQDA │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_type=&freetao.diary.ru/sea... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/877/ru/buildin... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert262997385f32313335781094&... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9584%26pz%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2693_763613.html_params%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274298799461981/?date=129... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert276278-ploschaya_checked_... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.10 │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://ekburg.irr.ru%2Fpuloveplanet │ 15 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credir=1 │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/gosthelp.ru/auto.ria.... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=on&input_onliner... │ http://irr.ru/index.php?showalbum/login.html%26custom%3D%26CompPath.2; WOW64; r... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=on&in... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga664.1721/deti74.ru/animals-planet.ru │ http://irr.ru/index.php?showalbum/login=K-SR-B-13-9635095,9661/9#f │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search?text=\xD1\x84\x... │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/doc.ru/belgorod.irr.ru... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19&numphoto/login=A-CL-MS-smi.org%2F40 (KHTML, like Gecko) Version... │ http://irr.ru/index.ru/widgetchrome%2F&ti=no&dom_sell/pansiyskaya-obl.irr.html%... │ 15 │ -└────────────────┴────────────────┴─────────────┴────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.986 user 1.912000 sys 0.104000 -D 07-01' AND (DATE '1970-01-01' + 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; -Run Time: real 0.861 user 0.860000 sys 0.044000 -D D (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND DontCountHits = 0 AND URLHash = 686716256552154761 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Run Time: real 1.168 user 1.208000 sys 0.096000 -D " = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) ORDER BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))); -Run Time: real 0.171 user 0.248000 sys 0.020000 -D -D CREATE INDEX counter_id_idx ON hits (CounterID) -> ; -Run Time: real 18.194 user 16.256000 sys 0.092000 -D D (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND DontCountHits = 0 AND URLHash = 686716256552154761 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Run Time: real 0.022 user 0.296000 sys 0.000000 -D -D -D -D '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(URL) > 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -┌──────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├──────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 102341 │ -│ http://komme%2F27.0.1453.116 │ 51218 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18315 │ -│ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16461 │ -│ http://irr.ru/index.php │ 12577 │ -│ http://irr.ru/index.php?showalbum/login │ 10880 │ -│ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 7627 │ -│ http://irr.ru/index.php?showalbum/login-kupalnik │ 4369 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 4058 │ -│ http://komme%2F27.0.1453.116 Safari │ 3021 │ -└──────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.068 user 0.844000 sys 0.000000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(Title) > 0 GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ Title │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ \xD0\xA2\xD0\xB5\xD1\x81\xD1\x82 (\xD0\xA0\xD0\xBE\xD1\x81\xD1\x81\xD0\xB8\xD1\... │ 122407 │ -│ \xD0\xA8\xD0\xB0\xD1\x80\xD0\xB0\xD1\x80\xD0\xB0\xD0\xB9), \xD0\x92\xD1\x8B\xD0... │ 82935 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA - IRR.ru │ 80958 │ -│ \xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\xB8 New Era H (\xD0\x90\xD1\x81\xD1\x83\xD... │ 39098 │ -│ \xD0\xA2\xD0\xB5\xD0\xBF\xD0\xBB\xD0\xBE\xD1\x81\xD0\xBA\xD1\x83 \xD0\xBD\xD0\x... │ 23123 │ -│ Dave and Hotpoint sport \xE2\x80\x93 \xD1\x81\xD0\xB0\xD0\xBC\xD1\x8B\xD0\xB5 \... │ 14329 │ -│ AUTO.ria.ua \xE2\x84\xA2 - \xD0\x90\xD0\xBF\xD0\xBF\xD0\xB5\xD1\x80 │ 14053 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA (\xD0\xA0\xD0\... │ 13912 │ -│ OWAProfessign), \xD0\xBF\xD1\x80\xD0\xBE\xD0\xB4\xD0\xB0\xD1\x82\xD1\x8C │ 10919 │ -│ \xD0\xA2\xD1\x80\xD1\x83\xD1\x81\xD0\xB8 - \xD0\xA8\xD0\xBE\xD1\x83\xD0\xB1\xD0... │ 10157 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.045 user 0.584000 sys 0.000000 -D AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 7479 │ -│ http://aliningrad │ 4791 │ -│ http://ekburg.irr.ru%2Fpuloveplanet │ 3584 │ -│ http://smeshariki.ru/obucheyelants │ 3064 │ -│ http://video.yandex.php │ 2887 │ -│ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 1084 │ -│ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 891 │ -│ http://afisha.yandex.ru/index │ 855 │ -│ http://sslow_13507.html?aspx?naId=6HS │ 521 │ -│ http://wildberrior/uphold │ 484 │ -│ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 289 │ -│ http://obninsk/detail │ 241 │ -│ http://diary.ru/forum/intries │ 208 │ -│ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 185 │ -│ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 185 │ -│ http://kurort/SINA, ADRIAN │ 157 │ -│ http://afisha.yandex.ru │ 132 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 124 │ -│ http://ssl.hurra.com/iframe │ 123 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 119 │ -│ http://stars-visa.html_params%3Drhost%3Dad.adriver.ru/catalog.php │ 105 │ -│ http://komme%2F27.0.1453.116 │ 83 │ -│ http://pogoda.yandex │ 80 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 79 │ -│ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 73 │ -│ http://video.yandex │ 71 │ -│ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 68 │ -│ http://wildberries.ru/daily │ 68 │ -│ http://smeshariki.ru/ru/index.ru%26bid │ 68 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 64 │ -│ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 64 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn │ 63 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 59 │ -│ http://sslow_135000008&position=search │ 58 │ -│ http:%2F%2Fwwwwww.bonprix.ru/topic │ 53 │ -│ http://sslow_13507.html/articles │ 52 │ -│ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 51 │ -│ http://rsdn.ru/rss.ya.ru/catalog │ 51 │ -│ http://ekburg.irr.ru/#lingvo │ 46 │ -│ http://pogoda.yandex.ru │ 45 │ -│ http://sslow_13500000%26rnd%3D2788881.html │ 44 │ -│ http://maps#ru_5_ru_22106.377648194,975924][to]=&int[14270pa106&op_uid=17759/6#... │ 42 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 37 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 37 │ -│ http://video.yandex.ru/film/46351/frl-2/bage │ 36 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 35 │ -│ http://direct.yandex │ 34 │ -│ http://sslow_13507.html?aspx?naId=3X_3bhLcs3M │ 33 │ -│ http://gotovim-doma │ 32 │ -│ http://che.ru/produkty_zarubezhei-niepochekhly │ 31 │ -│ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 30 │ -│ http://video.yandex.ru/Newsletter │ 29 │ -│ http://sslow_13507.html?aspx?naId=6D8IzMGys3M │ 29 │ -│ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ http://video.yandex.ru │ 28 │ -│ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 28 │ -│ http://sslow_13507.html?aspx?naId=649&state/out-of-town=\xD0\xA5\xD0\xB0\xD0\xB... │ 27 │ -│ http://notes=1/currency │ 27 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 26 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 26 │ -│ http:%2F%2Fwwww.bonprix.ru/tambov │ 24 │ -│ http://kinopoisk.ru/shoppich.ru/search?clid │ 24 │ -│ http://wildberries │ 23 │ -│ http:%2F%2Fwwwwww.bonprix.ru/searchAutoSearch?text=\xD0\xB2\xD0\xB5\xD0\xBB\xD0... │ 22 │ -│ http://myloveplanet │ 20 │ -│ http://stars-visa-litraj.txt","lpu":"http://pogoda │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/secondary │ 20 │ -│ http://msuzie-shop/premiery-c-38208_2.html │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 20 │ -│ http://smeshariki │ 19 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 19 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ http://irr.ru/6323%26bn%3D27888895,96772,97436 │ 18 │ -│ http://kinopoisk.ru │ 18 │ -│ http://nepogoda.yandex.ru%2Fproducts/search?text=subscripts/busineshop │ 17 │ -│ http://wildberries.ru │ 17 │ -│ http://kinopoisk.ru/catalog/9902224 │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://direct.yandex.html │ 17 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 17 │ -│ http://irr.ru/6323%26bn%3D27888895,963095425 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 16 │ -│ http://afisha.yandex.php?gidcar=36281664 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://auto.ria.ua/search │ 15 │ -│ http://direct │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ http://irr.ru/index.php?showalbum/login-kupalnaya-obl │ 15 │ -│ http://kinopoisk.ru/odessya │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 14 │ -│ http://ekburg.irr.irr.ru/maker │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.yandex │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.mail/rnd=0.9788 │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://auto_many_to_auto.ria.ua/igrush43/ │ 13 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 13 │ -│ http://afisha.yandex.ru/cars │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 13 │ -│ http://love.ru/a-myprofi │ 13 │ -│ http: │ 13 │ -│ http:%2F%2Fwww.bonprix │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://real-estate/aparther/offiliates/corruption/russinsk │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 12 │ -│ http://on-online=on&accetti │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 12 │ -│ http://samara.irr.html5/v12/?from]= │ 12 │ -│ http://kinopoisk.ru/saledParams │ 12 │ -│ http://en.lyrsenses/zamba_zaborah_coldplay=1&gearbox │ 12 │ -│ http://zvukovo/hondar/2007&state/renlew/rigma.ru/scribed │ 12 │ -│ http://pogoda.yandex.php?SECTION │ 12 │ -│ http://msk/platia-nashing/vanny.diary.ru/moscow │ 12 │ -│ http://video.yandex.ru/GameMain.E6smreQhiu_hXR4&where=all&film │ 12 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 11 │ -│ http://nizhnieiene/p17378705/currency=1&with_photo-ideapadeno │ 11 │ -│ http://news/6483731559676/Unlocknotebooks/m83/800_D_Black_list │ 11 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty/photo-12/#imag... │ 11 │ -│ http://msk/platia-nashing/vanny.diary.ru/sale/liver │ 11 │ -│ http://video.yandex.ru/GameMain.aspx#location │ 11 │ -│ http://guid=6&pw=6&pv=13 │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 11 │ -│ http://wildberries.ru/rost.html?1 │ 11 │ -│ http://smeshariki.ru/?win=82&stat=1&page/196264&pt │ 11 │ -│ http://pogoda.yandex.ru/catalog/jokers │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 11 │ -│ http://en.lyrsenses/zamba_zabudtrimazok.html?page=12&prr=http://fap1.adrive_typ... │ 11 │ -│ http://bonprix.ru%26bid │ 11 │ -│ http://loveplanet.ru/GameMain │ 10 │ -│ http://bdsmpeople.ru │ 10 │ -│ http://liver.ru/cheboksicily/foto.aspx?sort=newly&trafkey │ 10 │ -│ http://video.yandex.ru&pvid │ 10 │ -│ http:%2F%2F%2Fwwww.bonprix │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 10 │ -│ http://club.ru/spokoiteli/photo37775280000 │ 10 │ -│ http:%2F%2Fwwww.bonprix │ 10 │ -│ http://auto_id=240&n=13901038 │ 10 │ -│ http://smeshariki.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://slovariant_new3077940810/detail │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://en.lyrsenses/zamba_zabor_id=1012_blank%26site │ 10 │ -│ http://sslow_13500000%26rnd%3D2788881.html?parts/passe │ 10 │ -│ http://pogoda.html%3Fhtml_params%3Drhost%3D43 │ 10 │ -│ http://irr.ru/index.php?showalbum/logout%26verstova.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://pogoda.yandex.php │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film/6781203.html?id=242037047/detai... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://mysw.info/blog/sankt-peter%3D1216/00001216629 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://e96.ru/albumfotok-15-fotki │ 9 │ -│ http://alpari.yandex.html?html_param=0&users/#page/Search/ab_dob%2Ffieiie-razvo... │ 9 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://bonprix.ru/catalog/8570/1006790 │ 9 │ -│ http://b2b.testered/main/discuss/matched_country=-1&top=0&cityid=1024&wi=1366&o... │ 9 │ -│ http://svpressa.ru/topic=17082630 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://bonprix.ru │ 9 │ -│ http:%2F%2Fwwww.bonprix.ru/filmId=8j5j97LRs3M&where=all&sources │ 9 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/#win_13.html_p... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/4744089758 │ 9 │ -│ http://nail=Yes&target=search │ 9 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_produkty%2F&sr=http://slovaria │ 9 │ -│ http://rlsnet.ru/vacancy/view_type_id=9677548268010367 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ https://m.myloveplanet.ru/forum/abrika-kobelenie_nebestsenal │ 8 │ -│ http://irr.ru/imagecache/wm/2013&where=all&filmId │ 8 │ -│ http://poisk.ru/price_ot=&price_ot=&price │ 8 │ -│ http://zapchast.com/iframe-owa.html?1=1&cid=577&oki=1&op │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 8 │ -│ http://afisha.yandex.php?t=141880517 │ 8 │ -│ http://zarplata.ru/velika_all=\xD0\xBE\xD1\x82 │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http:%2F%2Fwwwwww.bonprix.ru/news/222974895&op │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://fore=3&marka=0&top=0 │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96777&oki │ 8 │ -│ http://video.yandex.ru&xdm_p=1#item/search │ 8 │ -│ http://direct.yandex.ru/catalog │ 8 │ -│ http://zarplata.ru/?p=12977-B26358/currency=RUR/page=1080&wi=1024&lo=http://rzh... │ 8 │ -│ http://en.lyrsenses/zamba_zabor/bedroomolsk │ 8 │ -│ http://masterh4.adriver.yandex │ 8 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ==&page_avtomodules.php?f=100&ref... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://mastered/main.aspx#location=1&bc=3&ct=1&pr=60322056107100919/page5/?_h=s... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://omsk.mlsn.ru │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96779/87 │ 8 │ -│ http://smeshariki.ru/GameMain │ 8 │ -│ http://afisha.yandex.php?gidcar=367108851%2Fr%2F1 │ 8 │ -│ http://che.ru&pvid=13733142835/100/topic,5240556895&ch=UTF-8&sF=11,7,7,0 │ 8 │ -│ http://love.ru/?p=17059 │ 8 │ -│ http://wildberries.xml?from]=&input │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/index.php?showalbum/login-kupit-topy%2Fplatjie-gotovlexandex.html... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://auto.ria.ua/auto_id=24126629/0/index.ru/real-estate/out │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 8 │ -│ http://victor?page_type=city.stol-yar.ru/cars │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://kinopoisk.ru/ch/feed/letniaiaprice/1609 │ 7 │ -│ http://video.yandex.ru%2F&sr=http://loveplants/65398f55 │ 7 │ -│ http://real-estate/aparts/Aquarevski │ 7 │ -│ http://bonprix.ru/social/product_id │ 7 │ -│ http://video.yandex.ru/a-album/login-vitiju/photo │ 7 │ -│ http://moscow/detail/5552/0/2792834&m=111,7,7,5 │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 7 │ -│ http://irr.ru/Registered/main/topnewsru.com/page=6 │ 7 │ -│ http://irr.ru/index.php?showalbum/login-kapustics?sort=pogoda.yandex.ru%26bt%3D... │ 7 │ -│ http://bonprix.ru/catalog/8570/14139489 │ 7 │ -│ http://afisha.yandex.ru/\xD0\xB4\xD0\xBE\xD0\xBC\xD0\xB0/\xD0\x91\xD0\xA1\xD0\x... │ 7 │ -│ http://irr.htm?from]=&int[85][from]=&input_vsegodnyie │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://love.ru/?p=1#country=&op_seo │ 7 │ -│ http://bdsmpeople.ru/niktory/shtory/308/roomed.ru/p59473682740295 │ 7 │ -│ http:%2F%2Fwwwww.bonprix │ 7 │ -│ http://myloveplantrackIt?tid │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 7 │ -│ http://spb/event=big&marka=84&model=0&auto_id=0&s_yers=0&pv=10&can_be_check_PP │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://pogoda.yandex.ru&pvid=1 │ 7 │ -│ http://irr.ru/index.php?showalbum/login-a-5-advert27114 │ 7 │ -│ http://bdsmpeople │ 7 │ -│ http://video=0&is_hot │ 7 │ -│ http://irr.ru/index.php?showalbum/login-zk34/pages/0001216629 │ 7 │ -│ http://svpressa.ru │ 7 │ -│ http://money.yandex │ 7 │ -│ http://gotovim-doma.ru │ 7 │ -│ http://afisha │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 7 │ -│ http://video.yandex.ru%2Fkategory_id │ 6 │ -│ │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://afisha.yandex.ru%2F%2Fmail/169 │ 6 │ -│ http://omsk/evential/house.ru/catalog/kitchedule=213-606361653965283 │ 6 │ -│ http://video.yandex.ru/ekt │ 6 │ -│ http://love.ru/?p=1#country=-1&sq_liver.ru/kyrgyzstan │ 6 │ -│ http://stars-varenok.ru/16745959680706/800_0.jpeg.html%3Fhtml5/v123593 │ 6 │ -│ http://afisha.yandex.ru/catalog=on │ 6 │ -│ http://bdsmpeople.ru/film/64544.690022.rar.html_params%3Drhost%3D_black_list=0&... │ 6 │ -│ http://bdsmpeople.ru/GameMain │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://v102.ru/investate/apartments-sale │ 6 │ -│ http://video.yandex.ru/GameMain.aspx#location/page_type=category │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://wildberries.ru/search │ 6 │ -│ http://afisha.yandex.ru%26bt%3D43%26anbietersburg │ 6 │ -│ http://omsk/evential/housession%3D0 │ 6 │ -│ http://video.yandex.ru/a-topy │ 6 │ -│ http://direct.yandex.ru/refererprofile%2F2.10 │ 6 │ -│ http://moscow/details │ 6 │ -│ http://rukodel=0&sort=newly&trafkey=2750 │ 6 │ -│ http://smeshariki.ru/world/photofider_credit=0&view │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://novosibirsk.irr.ru%26bid │ 6 │ -│ http://irr.ru/index.php?showalbum/login-12.html%26custom │ 6 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 6 │ -│ http://solutions.diary.ru/realtitroenie_v_jurman.ru/albums/frame-owa.html?stric... │ 6 │ -│ http://mr7.ru/newsru.com/iframe_right%3D43 │ 6 │ -│ http://autodoc.ru/real-estate/apart │ 6 │ -│ http://smeshariki.ru/catalog │ 6 │ -│ http://edp2.adriver.ru/hocketshop.ru/moscow/detailanude │ 6 │ -│ http://povari.yandex.ru/greecondary/Products_id=&auto_vaz_2111 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://msk/platia-nashing/vantralitsa_transion │ 6 │ -│ http://video.yandex.ru/catalog │ 6 │ -│ http://ereal-estate/rent │ 6 │ -│ http://3dnewsru.com/iframe_right.html?1=1&cid=51538 │ 6 │ -│ http://afisha.yandex.php/board,39.04839 │ 6 │ -│ http://novo/detail.aspx?group_cod_1s │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://kalininmyclonus1 │ 6 │ -│ http://auto_hyundai_sarator │ 6 │ -│ http://bdsmpeople.ru/saledParams=rhost%3D43%26bid%3D1 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://forum/topnews/22294&op_category │ 6 │ -│ http://love.ru/?p=17055335 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://afisha.yandex.ru/mymail.php │ 6 │ -│ http://real-estate=week/page=1&expand_search?film/298677435615.html │ 6 │ -│ http://koolinar.ru/port.ru/doc │ 6 │ -│ http://afisha.yandex.ru/index.ru/recipe │ 6 │ -│ http://video=0&with_exchangeType │ 6 │ -│ http://kinel-lab.com/rus/20130709_117485994,93304&op_seo_entry=1&gearbox=0&type... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ https://slovakia-600dd903c07022,101595,91194&op_seo_entry │ 6 │ -│ http://diary.ru/exp?sid=3205 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://video.yandex.php?search?text=\xD0\x9A\xD0\xBE\xD0\xBD\xD1\x8C\xD1\x8F\xD... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://video.yandex.php?from]=&interapy-wkti/ &cd │ 6 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D8393224 │ 6 │ -│ http://love.ru/ru/irk/event/search/ │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://kinopoisk.ru/catalog │ 5 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 5 │ -│ http://auto_volkswagen/vologdano/il_dlya-dnevka.ru │ 5 │ -│ http://barnaul/details/?cauth │ 5 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD0\xB5\xD1\x80&where=... │ 5 │ -│ http://metal-lodku-Obzor │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 5 │ -│ http://sp-mamrostova │ 5 │ -│ http://tp66.ru/exp?sid=3860217/rooms=2/men.aspx#location%3D0%26rnd │ 5 │ -│ http://zvukovo-gorodsk │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 5 │ -│ http://rmnt.ru/search/offilia_Sovetov_living_chamber/?78142 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http:%2F%2Fwwww.bonprix.ru/?id=2013&where=all&filmId │ 5 │ -│ http://video.yandex.by/?state_id=&auth=1..630;IC,7711588 │ 5 │ -│ http://e96.ru/movies/614418821/artir.ua/search │ 5 │ -│ http://slovari.yandex.ru%26orderovskij-index.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://povari.yandex │ 5 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_seo_entry=&op_category_... │ 5 │ -│ http://tks.ru/cat/publish-cherkalnaya-ttpodporyadushek │ 5 │ -│ http://auto.ria.ua/auto_id=241269.html?1=1&cid=2127970 │ 5 │ -│ http://video.yandex.ru/GameMain.aspx?Link │ 5 │ -│ http://slovarenok.com │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://slovari.yandex │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://auto_volkswagen-Palities/horobki │ 5 │ -│ http://afisha.yandex.php?r=23436303135353.html?1 │ 5 │ -│ http://rustnye-sht-riemnikoi │ 5 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 40007&pt │ 5 │ -│ http://radio&planet.ru/work.ru/catalog │ 5 │ -│ http://en.lyrsenses/zamba_zaborah_chamberk │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://myloveplanet.ru/v14572&lb_id=13000001216629 │ 5 │ -│ http://sslow_135000008&position=search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 5 │ -│ http://pogoda.yandex.php?gidcar │ 5 │ -│ http://bibidohertki-i-OOOO_REPORT/07_2013 │ 5 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 5 │ -│ http://real-estate=week&m=Dvigenie.html?option │ 5 │ -│ http://autodoc.ru/moscow │ 5 │ -│ http://love.ru/product_id=0&po_yers=0&po_yers=2&refererro/model=1346488078722&c... │ 5 │ -│ http://topnews.ru/GameMain.aspx?group_cod_1s=1983&pt=b&pd=9&pw=0 │ 5 │ -│ http://avtomobile/motory.ru/comp.ru/view.php │ 5 │ -│ http://video.yandex.ru/firms.turizm │ 5 │ -│ http://video.yandex.ru%2FkategoriendflowerTo=&powerTo= │ 5 │ -│ http://loveplanet.ru/mymail/rudi │ 5 │ -│ http://auto.ria.ua/auto │ 5 │ -│ http://msk/platia-nashing/vannyie-product_id=1841&page2 │ 5 │ -│ http://auto.ria.ua │ 5 │ -│ http://nizhnieiewva88/photo/101246465376&cmd=show-to-buchaiev-pugache=51db32a68... │ 5 │ -│ http://club.ru/cinema/movies/no-pos │ 5 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D158197%26width │ 5 │ -│ http://direct.yandex.ru/index │ 5 │ -│ http://tks.ru/cat/publish-chemec.ru/search?filmId=CktclMBmUXI │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://kinopoisk.ru/spb.pulscen.ru/exp?sid=3159&op_category_id=&auth=0&checked=... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://holodilnik-rp-ploschaya-obuv/?ci=1280&with_video=0&choosOyg==&op_uid=111... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://wildberries.ru/real │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://slovariant_neu%3D1%26bid%3D1216/0001216629%26bt%3Dad │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://wildberrior/kia/ │ 5 │ -│ http://md.mirkovskaya-obl.irr.ru/jobinmoscow │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://jcmotorom-921205&bt=7 │ 5 │ -│ http://povari.yandex.php?showalbum/login.pl?cl=all&film/497794,90458 │ 5 │ -│ http://en.lyrsense.com/obshchin-idieiala │ 5 │ -│ http://loveplanet.ru/\xD0\xB0\xD1\x80\xD0\xBE\xD0\xBC/curre-ap-i-showalbum/loui... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://moscow/detail/Torgovuyu-organske.ru/real-estate │ 5 │ -│ http://smeshariki.ru/topic │ 5 │ -│ http:%2F%2Fwww.bonprix.ru │ 5 │ -│ http://gaylyU │ 5 │ -│ http://video.yandex.ru%26bt%3Dad.adriver.ru/recipe/view/10217/?from │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 5 │ -│ http://loveplanet.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://alpari.ru/gallery/pic845274 │ 5 │ -│ http://alib.mist.html%26custom%3D%26c2%3D278888592138 │ 5 │ -│ http://loveplanet │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://whoyougle.ru/images/images/00000i/specifiers.ru/image=1&furniture │ 4 │ -│ http://video.yandex.ua │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 4 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 4 │ -│ http://video.yandex.ru/price │ 4 │ -│ http://sslovarenok.ru │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/filmId │ 4 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 4 │ -│ http://afisha.yandex.ru/zoom.php?f=5162613838.html_partments │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://auto.ria.ua/auto_id=0 │ 4 │ -│ http://irr.ru/index.php?showalbum/login-10618968476372773 │ 4 │ -│ http://samarskii_krai/tuapse/detail/result.aspx │ 4 │ -│ http://loveche.html │ 4 │ -│ http://bdsmpeople.ru/Web/Pages=1/feedsmag.ru/~\xD0\xBA\xD0\xBD\xD0\xB8\xD0\xB3\... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://wildberries.ru/catalog │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://e96.ru/news/39733/page2=&input_sponsor=&o=1015219.html_partments-sale&pa... │ 4 │ -│ http://afisha.yandex.ru%252f7769%252fe │ 4 │ -│ http://echoradar-s-Levoshcha │ 4 │ -│ http://diary.ru/catalog=on&input_city[3 │ 4 │ -│ http://love.ru/?p=1#countpage/vacancies/events/738/0/3/women.aspx │ 4 │ -│ http://direct.yandex.php?formsof(INFLECTION_ID=30861/14365-4b11&state/apartment... │ 4 │ -│ http://radio&planet.ru/marka=62&model=1178128455&pvno=2&evlg=VC,5 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://rasp.pl?cmd │ 4 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://travel.ru/state/apartments-sale/rashinitit%2F537 │ 4 │ -│ http://auto.ria.ua/auto_id=63799.html_params │ 4 │ -│ http://video.yandex.php?view_type=2&driveresult.ru/replies │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://afisha.yandex.php?r=3&bs=&day │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://kinopoisk.ru/saledParams%3Drhost%3Dad.adriver.ru/GameMain.aspx#location │ 4 │ -│ http://ssl.hurranovskaya-ul-31-foto.ria │ 4 │ -│ https://smeshariki.ru/cinema/article10363136000001216629%26site_offilia_Sovets.... │ 4 │ -│ http://vkirovoe-tourisma │ 4 │ -│ http://smeshariki.ru/domchelkakh_location=search?text=\xD0\xBC\xD0\xB8vents │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboy07/... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://guid=6&pw=2&pv=0&price_do=¤cy=1 │ 4 │ -│ http://smeshariki.ru/search/keukeru-soft │ 4 │ -│ http://card/windows)&bL=ru&cE │ 4 │ -│ http://kniga.ru/view=\xD0\xA1\xD1\x82\xD0\xB0\xD0\xB2\xD1\x80\xD0\xBE\xD0\xB2&w... │ 4 │ -│ http://afisha.yandex.ru/search │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://zarplata.ru/?p=12977-B26358/hasimages=1/page │ 4 │ -│ http://video.yandex.ru/GameMain.XYyZwYXRoPWEtbG9nb24vcG90Cw │ 4 │ -│ http://video.yandex.ru/real │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://v102.ru/?s=Adaments-sale │ 4 │ -│ http://matched_car=373838928155755775482794,9453.116 Safari%2F537.36&he=10&s_ye... │ 4 │ -│ https:%2F%2Fwwww.yandex.ru │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/GameMain.aspx#location-4.0.html │ 4 │ -│ http://tks.ru/cat/publish-chernyjbelyj-9375966238&op_categoriya │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/social │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://video.yandex.ru/realty/leaser_map=1/hasimay-2.html │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://video.yandex.ru%26target=search/ab_area=categorija80119 │ 4 │ -│ http://nizhny novgorod │ 4 │ -│ http://auto.ria.ua/canel_pe_mascona Hilfigeratov/153228 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://e-kuzbass.ru/a-shop │ 4 │ -│ http://victorhead.php?full&dom=780067167694.0; │ 4 │ -│ http://guide.travel.ru/link │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://kubikus.ru/search.php?r=4140211,1,7,7,7,7,0 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/cart.php/cars/papago-d-plosch │ 4 │ -│ http://afishi,Mudanted-belyj-974299099/guest-id=34089.html │ 4 │ -│ http://fitness/building │ 4 │ -│ http://svpress_up.aspx#location[propfilter_pf[PODVAL]=&arrFilter][3463351841195... │ 4 │ -│ http:%2F%2Fwwww.bonprix.ru/real-estate/out-of-town/house │ 4 │ -│ https://produkty%2Fpulove.ru/gost/?page3 │ 4 │ -│ http://afisha.mail.aspx#locationalOffers │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://anketka.ru/bridget │ 4 │ -│ http%3A//magnitogorod/page3/#over │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadbor_sh... │ 4 │ -│ http://video.yandex.ru/GameMain/dukhovyy │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://afisha.yandex.php?action/2741920 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://love.ru/image=2&marka=84&model/mihailovo │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://tks.ru/cat/public/gamemain.aspx#location │ 4 │ -│ http://nepogoda.yandex.ru/search=1&target=search=0&can_be │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://kinopoisk.ru/searchAutoSearch=0&driver.ru/catalog/1/women.aspx#locationp... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://afisha.yandex.php?show=rlv&ru=1&expand_search │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://love.ru/?p=1705 │ 4 │ -│ http://ssl.hurra.com │ 4 │ -│ http://auto_volkswagen_cated_cartovskaya-obl.irr.ru/album/login │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://afisha.yandex.php?p=31&input │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://loveche.ru/job/1162323&PAGEN_1=30&state │ 4 │ -│ http://video.yandex.ru/If yours.avtogsm.ru/animals/calculate │ 4 │ -│ http://internet Explorer&aV=5.0 (Windows)&bL=en │ 4 │ -│ http://rsdn.ru/details_103514,154;IC │ 4 │ -│ http://pogoda.yandex.ru/cooking_sm5_1148786993ab-417/photo/69363/26#formi.ru/co... │ 4 │ -│ http://auto_s_product_name=\xD0\x9A\xD1\x80\xD1\x83\xD0\xB3\xD0\xBB\xD0\xBE\xD0... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboyzon... │ 4 │ -│ http://video.yandex.ru/topnews.ru/lanas-advert2713][to]=&int │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://auto_volkswagen_pass_1161967 │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD1\x82\xD0\xBE\xD1\x8... │ 4 │ -│ http%3A//edp1.adriverys/forum/view_type=city&custom=0&damages/0001216629%26bid%... │ 4 │ -│ http://inspelishchin-platjie-doma.ru/irkutsk.irr.ru │ 4 │ -│ http://smeshariki.ru/saint-petersburg-gorod/transfer/?id=7576149959760994861&op... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://afisha.yandex.ru/content2.adriver │ 4 │ -│ http://en.lyrsenses/zamba_zabudka/photo-1/#page=0&sale/search?text=\xD0\x9C\xD0... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://auto_id=0&with_photo │ 4 │ -│ http://b.kavanga.ru/?a=inneVolumeFrom │ 4 │ -│ http://arma/frl-4/transportnoy-kv-m-malchik.ru/show │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://tp66.ru/money.yandex.ru/albums_screenterval │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://irr.htm?from]=&int[1151;IC,112 │ 4 │ -│ http://afisha.yandex.ru&pvid=13733424 │ 4 │ -│ http://bdsmpeople.ru/film/64544.690078 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_category_id=592b9e01c48ce9403%26bn%3D0%2... │ 4 │ -│ http://video.yandex.ru/GameMain │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ https://orenburg/?arrFiltersburg │ 3 │ -│ http://romar/events/audio.ru/widget[]=vacancies/99.php?gidcar │ 3 │ -│ http://afisha.yandex.php/topic104780204&op_uid=1954 │ 3 │ -│ http://afisha.yandex.php/tova.ru/uliya2076789599305953 │ 3 │ -│ http://afisha.yandex.ua/donetsk/urals │ 3 │ -│ http://afisha.yandex.ru/a-folders/misc │ 3 │ -│ http://afisha.yandex.ru/shop.ru/malta │ 3 │ -│ http://afisha.yandex.ru/project_price=&maxprice │ 3 │ -│ http://afisha.yandex.ru/kategoriya%2F5.0 (company │ 3 │ -│ http://afisha.yandex.ru/linkvac.php/board.php?topicseeng │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://samara.irr.ru%2Fproduct │ 3 │ -│ http://radiorecord.ru/login-m3w.html?1=1&cid │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://kinopoisk.ru/service.ru/iframe │ 3 │ -│ http://video.yandex.ru%2Fplata.ru/ch │ 3 │ -│ http://radioscannerica/filmId=Ba_id=13733568414&city=\xD0\x9C\xD0\xBE\xD1\x81\x... │ 3 │ -│ http://banantikov-nadushenie_orleona_server=sc.cheloveplant_11612/page │ 3 │ -│ http://irr.kz/realty/lease/3516093&pvno=2&evlg │ 3 │ -│ https://produkty%2Fplatjie-kuzbass.ru/newsru.com/iframe_right=0&auto_ria=0&meta... │ 3 │ -│ http://fuckfind=rent/view/2021/3 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ https://slovakia-600dd903c07022,101595,9143531427800648_elit │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://love.qip.ru │ 3 │ -│ http://auto_kia_30 │ 3 │ -│ http://kaluga/?ext=\xD0\xB3\xD0\xB5\xD1\x80\xD0\xBE\xD1\x8F\xD1\x82\xD0\xBD\xD1... │ 3 │ -│ http://sp-mamrostokonkursovet │ 3 │ -│ http://afisha.yandex.ru/forum.materinburg │ 3 │ -│ http://afisha.yandex.php?addriver.ru │ 3 │ -│ https://slovakia-600dd903c06c999c226647639.html%3Fhtml │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category_id=0&wi=16000&... │ 3 │ -│ http://sp-money.yandex.ru │ 3 │ -│ http://sp-money.yandex.ru/work rushki-sien-natalog/8570/page=0&expand │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://ssl.hurral=messages │ 3 │ -│ http://astrobank.ru/image │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://video.yandex.ru&pvid=13735/?_h │ 3 │ -│ http://love.ru/forum.cofe.ru/forum/view_type=city=790&Selectronics-technik │ 3 │ -│ http://wildberrior/bedroom]=&int[17][to]=&int[858 │ 3 │ -│ http://direct&sortdirect.yandex.ru/imagesize%3D0%26ar │ 3 │ -│ http://video.yandex.by/search/?target%3D43%26bid%3D2 │ 3 │ -│ http://forum/topnews/2229605699574.html?1=1 │ 3 │ -│ http://rmnt.ru/stars │ 3 │ -│ http://b.kavanga.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://video.yandex.ua/auto_id │ 3 │ -│ http://bdsmpeople.ru/search │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http:%2F%2Fwww.bonprix_ru}%2Fnizhniynovgorod/request-id │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://edp2.adriver.ru/catalog/181 │ 3 │ -│ http://rmnt.ru/film/88677/russia/rio.ru/search?filmId=NNr6aJrm4s3M │ 3 │ -│ http://video.yandex.ru/circle&state/out-of-town/houses │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film.ru │ 3 │ -│ http://msk/planet.ru/mymail.aspx#comme_me_saydinne │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://edp2.adriver.ru/jobinmoscow/detail │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://svpress_w1t1042796786/6/?category │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://pogoda │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/forum.donfiscategory │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://pogoda.yandex.kz/family │ 3 │ -│ http://kazan.irr.ru/location │ 3 │ -│ http://video.yandex.php?topbloveche │ 3 │ -│ http://nigma.ru/product&op_category_name=\xD0\x91\xD0\xB8\xD0\xBA\xD0\xB8\xD0\x... │ 3 │ -│ http://radiorecord.ru/catalog/idShare │ 3 │ -│ https://produkty%2Fpulove.ru/voronezh-sien-zhienskaia-moda-zhienskaia-moda-zhie... │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com/iframe-owa.html?1=1&cid=577&o... │ 3 │ -│ http://tks.ru/cat/publish-chin-play.php?categoriya%2Fzhiensmed │ 3 │ -│ http://card/windows NT 6.1) AppleWebKit%2F5 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://magnitka_1_series.ru/?favorite_id=636233644&op_category_id=937514 │ 3 │ -│ http://svpress/showbiz/photo.htm │ 3 │ -│ http://loveche.ru/volzhskiy │ 3 │ -│ http://ussuriysk.irr.ru/catalog/premiere/628962851d7fd0b6eb17b321d336f5bc7de189... │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountpage │ 3 │ -│ http://afisha.yandex.ru/tatatit_chto.php?industry │ 3 │ -│ http://smeshariki.ru/a-phony │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26url%3D//ad.adriver.ru/link/justic/h2.php/... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty%2Ftanki │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/92054446660.ht... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://ekategoriya%2F9B206 Safari │ 3 │ -│ http://smeshariki.ru/catalog/286/women.aspx │ 3 │ -│ http://svpressa.ru/content/search │ 3 │ -│ http://video_dvd/game/iframe-owa.html │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://holodilnik.ru/catalog │ 3 │ -│ http://msk/platia%2Fzhienskaia │ 3 │ -│ http://lk.wildberries │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://tp66.ru/search/?page=1&fuelRateTo │ 3 │ -│ http://e96.ru/real-estate │ 3 │ -│ http://my.kp.ru/albumfoto-1/pol-2 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://personal/atlants/7292&xdm_c │ 3 │ -│ http://povarenok.ru/files/eliteh.ru/perm.irr.ru/board,75.2013-07-09 │ 3 │ -│ http://brand=498&pvno │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://sp-mamrostovestory │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://amobil-nye-pliazhnaia │ 3 │ -│ http://smeshariki.ru/?win=82&stat=141882,373;IC,2552f48 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://love.ru/?p=1#country=&op_seo_entry=-1&target │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://irr.ru/washek-s-printom │ 3 │ -│ http://irr.ru/index.php?showalbum/login-kupaljinik-chere │ 3 │ -│ http://smeshariki.ru/goodavec/photo/6936325.html?id=223978/page=102 │ 3 │ -│ http://str_ob.html?1=1&choosO8gPJSs3M&where=all&filmId=mAyiC7y6M2mGV2GoA9hFoN3q... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberries.ru/item_no=2&evlg=VC,0;VL,205;IC,14;VL,757138/currency=RUR/h... │ 3 │ -│ http://msuzie │ 3 │ -│ http://smeshariki.ru/real │ 3 │ -│ http://kinopoisk.ru/search │ 3 │ -│ http://afisha.yandex.php?app=membered │ 3 │ -│ http://afisha.yandex.php?ELEMENT_ID │ 3 │ -│ http://afisha.yandex.ru/lesyach-hotels │ 3 │ -│ http://3dnews.ru/?p=12636464/5#f │ 3 │ -│ http://afisha.yandex.php?id=727285 │ 3 │ -│ http://afisha.yandex.ru%26bt%3D90%26nid%3D1216629 │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://kaluzha-na-NovletedAutoSearch=0 │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic=79799398/?_h=search.htm... │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory_id=577&search/ab_district/date_id=2271][from... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://love.ru/?p=17057 │ 3 │ -│ http://love.ru/ru/irk/event=little&cated_country=-192.html?1=1&cid │ 3 │ -│ http://product_brand=RAINBOW&op_cated_content/search/keup/en-ru │ 3 │ -│ http://love.ru/recipes/indows │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostavia.travel.ru/s... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://kinopoisk.ru/sessulyanovka.ru/photosessid=3205&bt │ 3 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD0\xB2\... │ 3 │ -│ http://li.ru/filmId=XpzlPj8P8gE&where=all&text=\xD1\x81\xD0\xBA\xD0\xB0\xD1\x87... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://you_hashkaf.ua/search │ 3 │ -│ http://auto_repairs=0&confiscategoriya │ 3 │ -│ http://wildberries.ru/GameMain │ 3 │ -│ http://povaria/chak_naytimes.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://afisha.yandex.ru/comment/search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 3 │ -│ http://afisha.yandex.ru%2Fobuv-sapozhkivka=23&price │ 3 │ -│ http://afisha.yandex.ru/hotel-agen-Goluboj-9730 │ 3 │ -│ http://notebooking pressa │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://smeshariki.ru/GameMain.aspx#location │ 3 │ -│ http://soft.oszone.ru │ 3 │ -│ http://pogoda.yandex.ru/real-estate │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://auto_repairs=0&po_yers=0&price.ru/\xD0\xB6\xD0\xB5\xD0\xBB\xD0\xB5\xD0\x... │ 3 │ -│ http://wildberries.ru/basket&ch │ 3 │ -│ http://state/room=94720-recept-Salat-iz-glasya1lesyat │ 3 │ -│ http://video_dvd/ratesTypeSearch?text=\xD1\x87\xD0\xB5\xD0\xBB\xD0\xBE\xD0\xB2\... │ 3 │ -│ http://auto_id=0&engineVolumeFrom │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://pogoda.yandex.ru/vlas-moskovskaya │ 3 │ -│ http://video.yandex.ru/personal/offeebe34c7e12944&op_product │ 3 │ -│ http://trashbox.ru/book │ 3 │ -│ http://slovari.yandex.ru │ 3 │ -│ http://kommersantamina │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://afishers/story │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ https://slovarenok │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://guid=6&pw=2&pv=0&with_video.yandex.ru │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/photo/narod.irr.ru/katering=1&pr=569&s_yers │ 3 │ -│ http://audio_video.yandex.ru/mosday.html?item=4#photo446962 │ 3 │ -│ http://auto_ford Mix).mp3.ucoz.ru/v1430497.html%3Fhtml │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://alpari.yandex.ru/saint-petersburg.irr.ru/cars/page3 │ 3 │ -│ http://video.yandex.ru/extra │ 3 │ -│ http://love.ru/ru/irk/event=little&category_id=731-643736&mode=1 │ 3 │ -│ http://dom.net/provoe-pervouralnaya/ChildGluZ19oZXJlci1kYXRpb25zPU4mbj0zJmlkPTM... │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com.ua/kiev/detail.ru/auth=1..640... │ 3 │ -│ http://v102.ru/investate/apartment/?id=137336IseNhcbx3J85GkHSnzgnsPdZUU&where=a... │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://kinopoisk.ru/registernet Explorer&aV=5.0 (Windows NT 5.1; ru-ru&cE=true&... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://video.yandex.ru%2Fkategory_id=9759527418 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberring │ 3 │ -│ http://video │ 3 │ -│ http://radiorecord │ 3 │ -│ http://balcon_caddy Club relove │ 3 │ -│ http://afisha.yandex.php?partments │ 3 │ -│ http://omsk/evential/housession%3D240%26rleurl%3D//ad.adriver.ru/marshavskaya-r... │ 3 │ -│ http://en.lyrsenses/zamba_zabor_polnitsa-s.narod.irr.ru/bank/otzyvy/12031%2F%23... │ 3 │ -│ http://auto_id=0&color=0&confiscategory_id=3205&bt=7&bn=1&bc=3&ct=1&prr=http:%2... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://state_shariki │ 3 │ -│ http://loveplanet.ru/Bezli-all │ 3 │ -│ http://smeshariki.ru/cgi-bin/click.cgi%3Fsid%3D0%26pz │ 3 │ -│ http://video.yandex.ru/real-estate/out-of-town/house.ru&pvid=1&distreet_legkovo... │ 3 │ -│ http://wildberries.ru/cgi-bin/click.cgi%3Fsize │ 3 │ -│ http://bdsmpeople.ru/show/39932/Itemid,40.0.html_params%3DfsSaHR0cDovL2pzLnNtaT... │ 3 │ -│ http://kinopoisk.ru/spb.pulscen.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://smeshariki.ru/furniture.html5 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/manga.ru/?rtext=\xD0\xBF\xD1\x80\xD0\xB8\xD0\xB1\xD0\xB... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://saint-peter │ 3 │ -│ http://lazarevskoe │ 3 │ -│ http://whoyougle.com/iframe/iframe_right.ru/spb │ 3 │ -│ http:%2F%2Fwww.bonprix.ru/imagesize │ 3 │ -│ http://video.yandex.ru/realty/search/main.aspx?sort=popular │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_product_brand=4200&lo=h... │ 3 │ -│ http://ssl.hurra.com/iframe-owa │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&film/67... │ 3 │ -│ http://klubnich/zrh/ │ 3 │ -│ http://un1.adriver.ru/page=30138117749516%252f110916%252fmedicinema/movie_ross ... │ 3 │ -│ http://wildberries.ru/comment/search │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://cxem.net/324487194836848 │ 3 │ -│ http://saint-petersburg-gorod/goodal │ 3 │ -│ http://sunmar.ru/cart&ch=utf-8&sF=11,7,700&aN=Opera&aV=9.80 (Windows │ 3 │ -│ http://myloveplanet.ru/passenger/kitched_country_id=4312&input │ 3 │ -│ http://wildberries.ru/comme%2F2.12.388 Version/1552/page │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://pogoda.yandex.ua/telefon_shtukaturkey │ 3 │ -│ http://bdsmpeople.ru/index.by/ru/page=0&confiscategory_id │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://gotovim-doma.ru/personal/commersant.ru/image=19&pvno=2&engineVolum │ 3 │ -│ http://krasnodar.irr.ru/yaransferapid │ 3 │ -│ http://jobs-education │ 3 │ -│ http://irr.ru/index.php?showalbum/logizer8/num-1/refremost │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://afisha.yandex.ru/sell/resident │ 3 │ -│ http://afisha.yandex.php?p=176d43f96ef32d5bc1272 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://auto_id=0&color=0&confiscategoriya%2Fzhienskaya-advert25593 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://forum/topics/ingradskazka-lookoformalities/poetry/events/?date&csrf-8200... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://radioscannerica/film/47018.html?1=1&cid=691390&pvno │ 3 │ -│ http://video=0&input_with_video.yandex.ru/page/10/women.aspx │ 3 │ -│ http://pogoda.yandex.ru/real-estate/apartments/73151 │ 3 │ -│ http://video_dvd/suppoll/dleead6718.php?SECTION │ 3 │ -│ http://msk/platia-nashing/vanny.diary.ru/otdam_daily │ 3 │ -│ http://direct.yandex.ru%2F&sr │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=&is... │ 3 │ -│ http://irr.ru/index.php?showalbum/login-do-500-kv-m-Maya │ 3 │ -│ http://kinopoisk.ru/real-estate=2013-07-2089241607/photo=0&with_exchangeType │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/used/KIA-Cee-d... │ 3 │ -│ http://tyva-5/country=-1&washestvo/den_sidentialAmount │ 3 │ -│ http://rukodel=0&sort=newly&trafkey=27065/2/child.aspx#location │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://maps#ru_5_ru_1_ru_ru_ru_202_ru_1_ru_ru_ru_ru_2_ru_1743.html?oscsid=36303... │ 3 │ -│ http://zapchastny_fashing_mashes/index.ru │ 3 │ -│ http://kinopoisk.ru/cgi-bin/click.cgi%3Fsid%3D1216 │ 3 │ -│ http://mylove.ru/zoom.php?GID=2&IsOrder │ 3 │ -│ http://smeshariki.ru/news/2013/peshnye-udivlekanka.ru/l_03_00/bodreamfood.ua │ 3 │ -│ http://arma/frl-4/travel.ru/moscow/details │ 3 │ -│ http://rsdn.ru/info_all=yes&razdumy-i-trikshop/search&_h_page/1719920 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://chehod/zvenia8312&input_who2=1&input_age1=35&aN=Netscape │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory │ 3 │ -│ http://myloveplanet.ru/bad_by_sidential/nizhnieiewva88 │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/procoolonelopitered/ministrict/3d-probeg-340-... │ 3 │ -│ http://ssl.hurra.com/iframe/iframe-owa.html%26custom%3D%26custom │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://avtoto.ashx/1001087496197797217530729; Media Centernet_mastersburg │ 3 │ -│ http://zagranimals-planet.ru/user/31059&Module │ 3 │ -│ http://kuharka=48&modeloveplanet │ 3 │ -│ http://whoyougle.ru/basket&ch=utf-8&sF=11,7,7,7,700 │ 3 │ -│ http://sendflower │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://bdsmpeople.ru/Web/price │ 3 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 3 │ -│ http://afisha.yandex.ua/index.ru/recipe │ 3 │ -│ http://votpusk.ru/ │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://auto_operey-v-v-meha.ru/catalog/public │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://smeshariki.ru/index.ru/main.pl?cmd=show/47555 │ 3 │ -│ http://afisha.mail.ru/stars/page │ 3 │ -│ http://irr.ru/6323%26bn%3D27888895,96772&op_page47 │ 3 │ -│ http://video.yandex.ru%2F%2Fwww.bonprix │ 3 │ -│ http://myfashihtzu.html?1=1&cid=65625f313230303&po_yers=2013/07 │ 3 │ -│ http://forum/topnews/222968695,910112_1164074834-908745 │ 3 │ -│ http://video=0&is_hot=0&che_simeis │ 3 │ -│ http://smeshariki.ru/chipinfo │ 3 │ -│ http://myloveplants_list-obl.irr.ru%2Fobuv-zhienskaia-modiezhda │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://pogoda.turizm.ru/communit=0&vip=0&order_by=2 │ 3 │ -│ http://kurort/SP1399&op │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://stories.ru/art/MACKLEMORE │ 3 │ -│ http://irr.htm?from]=&int[852][to]=10&lastdiscussins/?keyworld │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountry │ 3 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 3 │ -│ http://afisha.yandex.php?w=3160/transports │ 3 │ -│ http://afisha.yandex.php?link=114735200&brand=23368 │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://wildberries.ru/page=0&vip │ 3 │ -│ http://smeshariki.ru/Web/price │ 3 │ -│ http://nizhnieie-bielie-bieriends&fb_source-temno │ 3 │ -│ http://forum/topnews/2229362067528195&op_categoriya │ 3 │ -│ http:%2F%2Fmuzhchine/ru-g-Chelya_v_tsentyabrskii │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://zarplata.ru/?p=1290&op_product_price=990348531&schoosOSRquM8gE&where=all... │ 3 │ -│ http://radiorecord.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://guid=6&pw=2&pv=0&po_yers=0&with_video │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&text=\x... │ 3 │ -│ http://ur.hh.ru/file/news/2013 \xD0\xB3\xD0\xBE\xD0\xB4\xD0\xB0 │ 3 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 │ 3 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&top │ 3 │ -│ http://my.alpari.ru/filmId=yJg89hqV8gE&where=all&film/530/?fromCityCodeForcentr... │ 3 │ -│ http://gorbus.aspx#localiformalitic │ 3 │ -│ http://ej.ru/muss/roll_to_audi/mode=replies │ 3 │ -│ http://pogoda.yandex.php?t=14555667679/?PAGEN_3=5&s_yers=0&numphoto/62232/7#f │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD1\x81\... │ 2 │ -│ http://pogoda.yandex.ru&xdm_e=http://name=\xD0\x91\xD0\xB0\xD0\xBB\xD0\xB5\xD1\... │ 2 │ -│ http://auto_id=3159&input_age2 │ 2 │ -│ http://pogoda.yandex.ru/chat/ivan │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://sslow_13500000%26rnd%3D278888 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://moscow/detail/Torgovljatory/storii_efferanslyatting │ 2 │ -│ http://kinopoisk │ 2 │ -│ http://video=0&input_state │ 2 │ -│ http://poisk.ru/news/articles │ 2 │ -│ http://forum.php?t=420 │ 2 │ -│ http://mysw.info/node/21544 │ 2 │ -│ http://smeshariki.ru/tashkinsk │ 2 │ -│ http://smeshariki.ru/product&op │ 2 │ -│ http://board=11.ua.150.html%3Fhtml │ 2 │ -│ http://afisha.mail.ru/cheva.ru/reportby │ 2 │ -│ http://pogoda.yandex.ru/jobinmoscow │ 2 │ -│ https://diary/2013-07-05/101090/currency │ 2 │ -│ http://slovarenok.ru/chapochki │ 2 │ -│ http://kinopoisk.ru/saint │ 2 │ -│ http://krasnyj-chastory │ 2 │ -│ http://pogoda.yandex.ru/real │ 2 │ -│ http://3dnews.ru/msk/events │ 2 │ -│ http://yartb.html?city=55&TopicID=2&IsOrderedProduct │ 2 │ -│ http://yaroslavl.irr │ 2 │ -│ http://pogoda.yandex.php?showalbum │ 2 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 2 │ -│ http://wildberries.ru/editem_no=100¤cy=1#country=&op_proizvodskaya-obuv-z... │ 2 │ -│ http://wildberries.ru/filmId=4920/room=1&lang=all&filmId=DnGbyVQVUXI&wheretomug... │ 2 │ -│ http://wildberrin/foton │ 2 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 2 │ -│ http://omsk.spb.ru/filmId=RUb3Hf2m0Cw&where=all&text │ 2 │ -│ http://afisha.yandex.php?s=4d450&pid │ 2 │ -│ http://afisha.yandex.php?city=418&view │ 2 │ -│ http://afisha.yandex.ru/?trafkey=54073799 │ 2 │ -│ http://kaluga/?ext=\xD1\x80\xD0\xB5\xD0\xB9 \xD0\xBD\xD0\xB0 \xD0\xB1\xD1\x80\x... │ 2 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 2 │ -│ http://news/398261_enl.jpg-1 │ 2 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 2 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 2 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://pogoda.yandex.ru/images/0001216629/0 │ 2 │ -│ http://rlsnet.ru/nogin.html5/v12 │ 2 │ -│ http://guid=6&pw=2&pv=0&price=690&s_yers=1916 │ 2 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=2furii10.html?city=0&hide │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://ur.hh.ru/page=10&can_be_checked_auto_region=1&rm=1&lang=all&film/18212.1... │ 2 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.041 user 0.380000 sys 0.004000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -┌────────────────┬────────────────┬─────────────┬────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ TraficSourceID │ SearchEngineID │ AdvEngineID │ src │ dst │ pageviews │ -├────────────────┼────────────────┼─────────────┼────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33069 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 24703 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 15817 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 14233 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 │ 6549 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 │ 5257 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 5257 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 3547 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login │ 3538 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik │ 3371 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php │ 3333 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 3309 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 2923 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 2616 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 2441 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php │ 2418 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 2390 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari │ 2367 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 2349 │ -│ 1 │ 0 │ 0 │ https://google.com.ua/url?sa=t&rct │ http://komme%2F27.0.1453.116 │ 2228 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 1920 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://komme%2F27.0.1453.116 │ 1679 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ekburg.irr.ru%2Fpuloveplanet │ 1340 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://aliningrad │ 1270 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1254 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1213 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1193 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1186 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1142 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F27.0.1453.116 │ 1007 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 954 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://aliningrad │ 859 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login │ 821 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 791 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 777 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 757 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 722 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://video.yandex.php │ 702 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 691 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/obucheyelants │ 684 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 632 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 593 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 577 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://smeshariki.ru/obucheyelants │ 566 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.php │ 452 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 445 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBA\xD0\xBE\xD0\xBD\xD1\x82\xD1\x80\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 422 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 421 │ -│ 0 │ 0 │ 0 │ │ http://obninsk/detail │ 407 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 386 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug.ru │ http://irr.ru/index.php │ 374 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru/index │ 360 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 348 │ -│ 2 │ 0 │ 13 │ │ http://komme%2F27.0.1453.116 │ 347 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-2011/43597 │ 293 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 293 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6HS │ 282 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 268 │ -│ 3 │ 72 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 267 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 259 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 257 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari │ 256 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 │ 241 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login │ 237 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 │ 236 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/28435&lr=157 │ http://komme%2F27.0.1453.116 │ 233 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/ru/photo/6936313555&text=\xD1\x85\xD0\xBA \xD0\xBB\xD0\xBE\xD... │ http://komme%2F27.0.1453.116 │ 224 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php │ 223 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cat/dushkirillovyj │ http://komme%2F27.0.1453.116 │ 222 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=359&op_page2/... │ 218 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kriminally-bezhevsk │ 217 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0 │ 212 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 207 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 202 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 199 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://ekburg.irr.ru%2Fpuloveplanet │ 199 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 198 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 197 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 195 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/414526863.xlsx │ 191 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CDMQFjAJ │ http://irr.ru/index.php?showalbum/login-kiriskaya-obl.irr.ru/index.ru/GameMain.... │ 190 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=0&with_photo.ashx/101/4/?cat=6257271 │ 188 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=partments-sale/search?text=\xD1\x81\xD0... │ 187 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://afisha.yandex.ru/index │ 183 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://ekburg.irr.ru%2Fpuloveplanet │ 171 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 168 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 164 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 163 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://ekburg.irr.ru%2Fpuloveplanet │ 160 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 160 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/life.ru/cars/misc/travel.ru/?trafkey=058143&p... │ 155 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 │ 154 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=577&oki=1&oby=&op_s... │ 152 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://wildberrior/uphold │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6HS │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 149 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 146 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustic/meterburg │ 144 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 144 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 144 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E │ 141 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 140 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 136 │ -│ 1 │ 0 │ 0 │ http://kombardighantnie │ http://komme%2F27.0.1453.116 │ 136 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login │ 135 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kysjacevtika │ 132 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://komme%2F27.0.1453.116 Safari │ 124 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 121 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category/stroy/dachines... │ 114 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/?state │ http://komme%2F27.0.1453.116 │ 113 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/GameMain │ http://komme%2F27.0.1453.116 │ 113 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 111 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login │ 110 │ -│ 3 │ 85 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 110 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kardigan │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 107 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosoftwarenok.ru/projects/zhbi.po... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 105 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268143.html?1=1&cid=577&o... │ 104 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 104 │ -│ 0 │ 0 │ 0 │ │ http://tvidi.ru/photo=0&confiscategory_id=0&engineVolumeFrom=&fuelRateFrom=type... │ 103 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 102 │ -│ 3 │ 14 │ 0 │ │ http://komme%2F27.0.1453.116 │ 102 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 100 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 98 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://video.yandex.php │ 97 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 96 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.21150895 │ 96 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?sort=price │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 95 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login │ 95 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.12.388 │ 93 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 93 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 92 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberrior/uphold │ 92 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 91 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-tanks/search=1&ady=62&modeloveplanet.ru │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 90 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.145332.15&he=10&category_id=2740387993 │ 89 │ -│ 1 │ 0 │ 0 │ http://in-the-weightEnd=2351&numphoto=&isExclusiver.ru/alertljus │ http://komme%2F27.0.1453.116 │ 88 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fprodazha_Italja_unit=1&av=1&nm=1&lang=ru │ http://irr.ru/index.ru/show/414526863.xlsx │ 87 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 86 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb1.html?item_no=386703/?bundle=7172&msid=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 86 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 │ 85 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 84 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733582852/ │ 84 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupit/action │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 82 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 82 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?bodystyle │ http://komme%2F27.0.1453.116 │ 81 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F22.14&he=768486 │ 81 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusk.ru/ru/lookaginitial/Prodayu-Dach... │ 80 │ -│ 3 │ 4 │ 0 │ │ http://komme%2F27.0.1453.116 │ 80 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 79 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://aliningrad │ 78 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 78 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login │ 76 │ -│ 1 │ 0 │ 0 │ https://go.1ps.ru/show&showforum │ http://komme%2F27.0.1453.116 │ 76 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandex.ru │ http://komme%2F27.0.1453.116 │ 76 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26221/detail.ru/v1496366&... │ 75 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php │ 75 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 74 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%2F&sr=http://bonprix.ru... │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len80/page/product │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filtr/all/perm.pulscen... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 73 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 73 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F1.7.1364.172 │ 72 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://diary.ru/forum/intries │ 72 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/show/414526863_112 │ 72 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2618561&pp=1059&op_produc... │ 72 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://video.yandex.php │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 71 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 71 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari │ 71 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://aliningrad │ 71 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=product_id=42&Selection.chelov.ru/searc... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/yandsearch?te... │ http://irr.ru/index.php │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/server=sc.chel.ru/main... │ http://irr.ru/index.php │ 70 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification/?year=\xD0\xB1\... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2791954~43.87725656132&op... │ 69 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://komme%2F27.0.1453.116 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/produkty/bleacs/udilis... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 69 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&wi=1366&br... │ 68 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 67 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731&CgID=124jc&where=all... │ 66 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/Yozh/Goodda │ http://komme%2F27.0.1453.116 │ 66 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2725215195&ti=\xD0\x9F\xD... │ 66 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://ekburg.irr.ru%2Fpuloveplanet │ 65 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru/index │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 64 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://afisha.yandex.ru/index │ 64 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 63 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nmg.com%2F4.0 Safari%26clients-sale/search&e... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 63 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 62 │ -│ 2 │ 13 │ 13 │ │ http://komme%2F27.0.1453.116 │ 62 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login │ 62 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://ekburg.irr.ru%2Fpuloveplanet │ 61 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 61 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 61 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26092.html?s_text=\x5C\x5... │ 59 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/thenon-houses/public/g... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27755f32316.30; .NET CLR ... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742/details/?cauth=0&dam... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fautoad/kniga.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26004/?_h=galle/\xD0\xBF\... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert27930555&sob=1&p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2692.html%3Fhtml?period=3... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/world/photo31469:Album... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=leting "HalUXI&where=\xD0\x9F\xD0\xB5\xD1\x8... │ http://irr.ru/index.php?showalbum/lofiver.ru/articles/86121%26uid%3D139750%26ad... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27000,224648804-recept-so... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/koshka.com/ig/iframe-o... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/realty/suntime-5/extre... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory=cinema.perm.p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742357/detskii_gosts.xml... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742304][from]=&int[14670... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694336/photo/suzannason/... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Forum25/top... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268320995,968650f45491882... │ 57 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupit/action │ 57 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1375605&ga... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://video.yandex.php │ 56 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga/choice/... │ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_product_... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?showtopic,5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26017/quarius_Moscow/cavi... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2726125413975d77cf&search... │ 55 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 54 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266848223/review_type=pro... │ 53 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 53 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26989%26bt%3Dad.adriver.r... │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/album/login-1800002&pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 52 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0 │ 51 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2698172,93932353064614618... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605085268a9c4d61-9862.49... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19&numphoto/login=A-CL-MS-36575c72937][to]=&int[12822304 │ http://irr.ru/index.ru/widgetchrome%2F&ti=no&dom_v_bordovye-printime.ru │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/investate/comple/make/?page5/&docid=jlMNIrXw... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 50 │ -│ 0 │ 0 │ 0 │ │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 50 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F11.7.1364 │ 50 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 50 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26051.htBeg=6&NightRegist... │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xBE\xD1\x81\xD0\xB8\xD0\xB1... │ 49 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 49 │ -│ 5 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 49 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?group=days=3&text=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27410/photo-2.xhtml&serve... │ 48 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandsearch │ http://komme%2F27.0.1453.116 │ 48 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 48 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusty-i-vkont.at.ua/search=0&userId=0... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266128182&op_seo_entry/de... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?sort=&br... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/intrumen │ 46 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://afisha.yandex.ru/index │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumanegenre=33155?analog/kitchen_mini... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik │ 46 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login%2Flick.g.doubleclick.diary.ru/car/kw/3061/us... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734.html_params%3Dfh_loc... │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://afisha.yandex.ru/index │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/GameMain.p1Yo4A │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ua/searchads/jo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605906/frl-2/sportal.ru/... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fshow/lpp/cre.ru/pers... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.asp?search&ev... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logize.ru/msk/events_liver.ru/russert-plies.r... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26099486633033&countpage=... │ 44 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 43 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://sslow_135000008&position=search │ 43 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischet-solik/odezhda-plos... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert277553/38021/66936575776/... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271050&with_photo.kurortm... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 43 │ -│ 1 │ 0 │ 0 │ http://forums/liii-kuler S22oHgBJTngegotavgorod55.ru/filtr[2]=42 │ http://komme%2F27.0.1453.116 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-folders/?end=9a08488... │ http://irr.ru/index.php?showalbum/litamak.irr.ru/catalog/8570/travel.ru/cgi-bin... │ 41 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 41 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 41 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/0001216629/#top_by │ 41 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26040931&s_yers=0&with_vi... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://ekburg.irr.ru%2Fpuloveplanet │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.137508&s_yers=200... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://aliningrad │ 40 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logito8ergo_6470/page_type_id=20872/0/001:12:... │ 40 │ -│ 0 │ 0 │ 0 │ │ http://afisha.mail.ru/dmitrij │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/readar-nashi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27924563724&key=46960/med... │ 40 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://lib.ru/exp?sid=3205&bt=7&bn │ 40 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 39 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credit │ http://komme%2F27.0.1453.116 │ 38 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=8570/page=5&s_yers=2006 │ 38 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2679/detail/5482,935033/2... │ 38 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 38 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login │ 37 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dnevnoy-kv-Samara.irr.ru/searchAutos&marka=4... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 37 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://komme%2F27.0.1453.116 │ 37 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=theating&page=... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?group │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2784389/room=39695,966681... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/forum/topicID=269&stat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/votkim-rukav-i-kova-ul-adve... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filmId=xFX1UbTNJjxe4yF... │ http://irr.ru/index.php?showalbum/login.asp?razdel7/test/matched=115909d9_dsc07... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wolf-club478561/OPEL | \xD0\xB0\xD0\xB2\xD1\... │ http://irr.ru/index.php?showalbum/login-marka=13&city&custom=0&damages/0001216 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27553&s_yers=0&po_yers/46... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert275226607660?design=6efxq... │ 36 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=show=&fgroup-tab-mara.irinables/yearT... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278171&size=52094&op_seo_... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27330528/photo/6223799%26... │ 36 │ -│ 5 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 36 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36&he │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://smeshariki.ru/obucheyelants │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=post_neu%3D22%26pz%3D0%26rleurl%3D%26xp... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 35 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/8569 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vip.ru/book.com/photo/7086/page=0&state=mont... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/index.ru/katego... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-cukickdrops_bak_yant4tlGtZgy3BQ&bvm=bv.49478... │ http://irr.ru/index.php?showalbum/login-501-98552&op_uid=1060948/6#f │ 35 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (Windows │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19&m_staraya-obl │ http://irr.ru/index.ru/show/414526863_112 │ 34 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.12.388 │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 34 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_uid=1... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/promo=C-Tease/rent/dat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/list │ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 34 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/directadvert/kavanga_728x90.html │ http://komme%2F27.0.1453.116 │ 34 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshops.xml?typ │ http://komme%2F27.0.1453.116 │ 33 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?cPath=default.ru/yandex.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33 │ -│ 1 │ 0 │ 0 │ http://auto.ria.ua/\xD0\x9A\xD0\xBB\xD0\xB8\xD0\xBC\xD0\xB0\xD1\x82\xD1\x8C\xD1... │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 33 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://video.yandex.php │ 33 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 33 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser │ 33 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265748ba3f755671_1162684&... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertif?sle=85000 \xD0\xB4\xD0... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/pub │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364 │ 32 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zxcvbnm1990&search/jobinmoscow.ru/yandex.by/... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements-obschaya │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fproduct/foto-4/login... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&op_category_id=1174]... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniturers_id=575386110 │ http://irr.ru/index.php?showalbum/login.exit/rem/php/board/10_kruizer_8267049&o... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiepochto_id=93231238][to... │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://afisha.yandex.ru/index │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ssl.hurra.com/iframe │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischuk_Kerasivye-klavik/?... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/jobs-education/m_acces... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26791e3a7c543c9bf9ef5c0-9... │ 31 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt │ http://komme%2F27.0.1453.116 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertificia/?letteo_985428161/... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_api.php?cid=41389/rooms/perm.pulscen.ru/sear... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 30 │ -│ 1 │ 0 │ 0 │ http://domchelov.html&lr=55&text=\xD0\xB4\xD1\x80\xD1\x83\xD0\xB3\xD0\xB8\xD0\x... │ http://komme%2F27.0.1453.116 │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoryID=12452929587/... │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 30 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 30 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268630533&site_ot=&price_... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-k608i.trashbox.ru/cars/page5/&docid=577&lr=2... │ http://irr.ru/index.php?showalbum/logii_58247.php?forum.borovichy77/page5 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2716/~3/150160947694,9745... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/products/?category_id=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2696374/#page/tab=com_avt... │ 30 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/topic │ 30 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.14&he=768&wi=1920 │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26979553039353936694&ch=u... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605446518%2F&sr=http://e... │ 30 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://smeshariki.ru/obucheyelants │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-carava/fotok-8-steklove.ru │ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26928142&op_products/view... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kupibatikov/?recrete/personal/klimat.ru/work... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/wedditeli_foto-2/__48/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertifications[0][150]=60&inp... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_countpage/1 │ 29 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x9A\xD0\xBE\xD1\x80\xD0\xBE\xD1\... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/click.net/paged=696&lo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274229696.0 (Windows)&bL=... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-14132&with_photo=1&srtime-zone.net/catalog/t... │ http://irr.ru/index.php?showalbum/login-sierra%2Fdlia-zhienskaia-moda-zhiensmed │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 29 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2664952&xdm_p=1#country=-... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704.html%3Fhtml5/v12/?fr... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734/fotographic_2017666&... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategoriya/zhienskaya-... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9649.html%3... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/small_mercial-shoppicc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27350350394&ch=UTF-8&sF=1... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_seo_entr... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-baza/gadge... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27425.3120%26ntype_id=2&r... │ 28 │ -│ 1 │ 0 │ 0 │ https://directions[0][1541769377921968 │ http://komme%2F1.7.1364.172 │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13733097&op_categ... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert272291684/hasimage=0&view... │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269500469412216388/detail... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/mann japanel/?text=\xD1\x81\xD0\xBC\xD0... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-my-sovies/bebybum.homestion bonprix.ru/... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikarbox=2&tz=9b81998&po_y... │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-176-kakaya-obl.irr.ru/search?lr=191&ei=N6rrg... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?lang=all&fi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\xA4\... │ 28 │ -│ 3 │ 95 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/anime-2/#album/l... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-eve-nn.ru/cars/passenger/\xD0\x92\xD0\x90\xD... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274236&pv=15&prr=8&sid=35... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/41452626u1zIq0SGLXCI │ 28 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic/8939858-59929477/detailshop │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F22.14&he=768486 │ 28 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://ekburg.irr.ru/#lingvo │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678/page=2&marka=84&mode... │ 27 │ -│ 3 │ 2 │ 0 │ │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/search?cl4url=http://yandex.ru/online... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27013.ya.ru/sportalznakom... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert24061.jpg.html%3Fhtml?1=1... │ 27 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031343-sh-898675/sort=pri... │ http://irr.ru/index.php?showalbum/login-5158&jenre/slyudya_metroman.ru/GameMain... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19195/offset=101&district=&city&with_photo=¤cy │ http://irr.ru/img/catalog/53485785/topic/8939850.php?page=3&marka=84&mm │ 27 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 27 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 27 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fhit_list.html?s... │ http://irr.ru/index.php?showalbum/login-irake-myservierk-supierts%2F&ti=\xD0\x9... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-oleginnesujeli-i-centum=etarget=search?clid=... │ http://irr.ru/index.php?showalbum/login-kupe.html_params%3Drhost%3Dad.adriver.r... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27928092_price=\xD0\xBC\x... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/mazda-dievochnye_avari... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-19.xhtml?city_to_one=All&film]/on/orders/810... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/1.19.mobile_photo=6001... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27630/?item_id=0&bodystyl... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ 1 │ 0 │ 0 │ http://bonprix_ru_11559&lr=12&usln │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lizova_ii_1112_1150&option[price_do=300... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E&wher... │ 26 │ -│ 1 │ 0 │ 0 │ http://yandex.ua/?target=search/tab=user │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detall/seconomicsmovinki │ http://komme%2F27.0.1453.116 │ 26 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 26 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://obninsk/detail │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/loginsk.irr.ru%2Fkategory_id=9204471-650/?_h=... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login-jjjekrju-s-pring=1&price[price=\xD0\xBC... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification.ru/messa.ru/rea... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lejandsearch?text=\xD1\x82\xD0\xB8\xD1\x85\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert21124631/?Search.php │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 26 │ -│ -1 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_category... │ 26 │ -│ 3 │ 197 │ 0 │ │ http://komme%2F27.0.1453.116 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?messa.ru/worlds/page2/7277932582&text=\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 26 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-kupe-20010120652838799.html?n=7148.html... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemey-volvom80Cw&where=a... │ 26 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=607&state=4... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.22 (KHTML, like Gecko) │ 25 │ -│ 1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 25 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/people.ru/cars/passets... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731/?city=0&pvno=2&evlg=... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vivarki/redki_sdelaet-law.ru/produkty/kartgo... │ http://irr.ru/index.php?showalbum/login=driff/en-ru/?p=2#messages/00001216629 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 25 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert264314953); ru) AppleWebK... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.html&ei=GcXrUYe8F2DAU-RbuRlLMczoJ--5uDCx8 │ http://irr.ru/index.php?showalbum/login-132/#images/0000.h95/\xD1\x82\xD1\x8E\x... │ 25 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://obninsk/detail │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search?text=myaccountry │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguard-payments-sale/se... │ http://irr.ru/index.php?showalbum/login-88i8LaDyEkCVv6-DhRfEDcw==&action │ 25 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/yandex.ru/?f... │ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.ru/film/46351/frl-2/bage │ 25 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605838.html5/v12/?from=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-link=1147373-1&p=1&p=2&text=\xD0\xBC\xD0\xBE... │ http://irr.ru/index.php?showalbum/logisterlingvo/#! │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741559&t=5204/make=Chrys... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694589/detail/555-javata... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=shops.html?id=1&body_type=\xD0\xBF\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781472265&op_seo_entry=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626rleurl%3D//ad.adriver... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685/price_ot=&price=\xD0... │ 24 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://diary.ru/forum/intries │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26685857947301_Zoryatiya/... │ 24 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-117031&op_category&op_seo_entry=&op_uid │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671291&fridgets/2012&pri... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27399992.php/board,13.0) ... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-annative-expressageId=0oVXx21hUXI&where=all&... │ http://irr.ru/index.php?showalbum/login-rybnovlevojj_mamountry=-1&type%3D0 │ 24 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://video.yandex.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguar.ru/page4/?emain.a... │ http://irr.ru/index.php?showalbum/login=ogabass.ru/filmId=9WOqzzitive&view │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo/7... │ http://irr.ru/index.php?showalbum/login-kupit-plitka-s-korsiety/art/151/n4p/160... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/intrumen │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kupalnaya_elit=&id=3&clid=9403&lr=1913849 │ http://irr.ru/index.php?showalbum/login-gallebia_zhurnalCall D.R.E.mp3ex.net/ph... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lolacake/tatyaka.html?1=1&cid=577&oki=1 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinglish.rus-l7-p70505-1... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/yandsearch?lr=1... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-yvaken/topic=7702.jpg.html?1=1&cid=577&... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-547552/1/?sma=RAINBOW&op_uid=fdd1b9d2721728&... │ http://irr.ru/index.php?showalbum/login-kapusta-advertika/search/room=1&damage=... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=1&bc=3&ct=1&pr │ 23 │ -│ 1 │ 0 │ 0 │ http://auto_volkswagency=1&text │ http://komme%2F27.0.1453.116 │ 23 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgarials/commend/searchvage-2.html&dt=13734... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-tuniki_510... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263090024189-528619c2077/... │ 23 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/ru/index.ru%26bid │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333572620201709/page/10... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news/radio&clid=198555... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26939.html?page/3/#count=... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2740974%26pz%3D0%26ar_sli... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/kategory_id=5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 23 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/resultatya9176da22f1a521a5853.html&... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26600721376439322%26pz%3D... │ 23 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 23 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga_728x90.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://afisha.yandex.ru │ 22 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemennoke0070553995f27d6... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2697574697.html_param=0&u... │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-105vr/ │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=search.tutung/s... │ 22 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/wheel.ceratornaya_solnechka │ http://komme%2F27.0.1453.116 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2091&local=1&TypeSearch?mail.ru/yandsearch&c... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26058&nnum=s4746835895&op... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-10760b31b65633eZateq eb806e887d9f15ccf593280... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-the-ili-tempera-minsk.irr.ru/produkty/bluzki... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739693071/pomer... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodny │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2649.html?pg=menu_29.jpg&... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/v1465][from]=&int[157281 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267830840994,95688781470%... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-business/sale/ru/search?mail.ru/film%2Fbarna... │ http://irr.ru/index.php?showalbum/login-zakon_type=1&fuelRatesTypeSearch │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 22 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27200,55.7654.html_parat-... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&l... │ http://irr.ru/introlux_page5/2/page/Renaul.irr.ru/start=1500-rublic/gamemain/Tu... │ 22 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26601629][from=&fuelRateT... │ 22 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://ekburg.irr.ru%2Fpuloveplanet │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-natashkarola.irr.ru%2Fviewforum/MsgList.html... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26900/technics-technics-t... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liliate_id=24145602&with_phoney.yandex.ru;ya... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&city[7]=23239330794,25826044_Black_lis... │ 21 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?price_till beli.php?cx=015216684_36 │ http://komme%2F27.0.1453.116 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694,978825315f373400/det... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691_ru_17832523.html_par... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1) AppleWebKit%2F537 │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/world/photo/41/62b1dfa450/3/women-clother%26tms%3D%26ev_ltx%3D... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/basketshop.ru/yandsear... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/forum%2Fukhov/l... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265228.html?1=1&cid=577&o... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27025&Select[4][]=32598.h... │ 21 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://wildberrior/uphold │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/houses/passenge... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo44953.116.php?t=4... │ http://irr.ru/index.php?showalbum/login=ko&page=0&view.aspx?group_cod_1s=85&key... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/?from │ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-dance/index.ru/?favorite_off=1&encoding&hl=\... │ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logino-s-grigerator/page1=&input_age1 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/logii-bin/click.cgi%3Fsid%3D158195,97987231-9... │ 21 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cars/commeddesk.ru/google_s&12.5.746.59954.30 (KHTML, like/\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26737865/?page/12186/#/ec... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ramblery/pic/893985650697... │ http://smeshariki.ru/obucheyelants │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login.html?1=1&input_who1=2&input_who2=1&inpu... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/club.ru/anime-zone.ru/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dietsik.ru/razdel_id=111,7,700&w=728x90.html... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-brie... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929&from%3D%26custom=0&... │ 21 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2186/detail.aspx#location... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739353/details.... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kupalnik │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/9-0--navam-zhien... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741465][from]=&input_wit... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification%2F4.0; XBLWP7; ... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_uid=1147... │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login=uzpp2XnEw&bvm=bv.49784469][from │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-Larki-i-korobeg-1124-95367/guestblowinp... │ 20 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26219&wi=1280&lo=http://w... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/yandex.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estatic.diary.ru/... │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=58&Selectronics-technics/k... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?er=2&text=\xD0\xBA\xD1\x83\xD0\xBF\xD0\xB8\... │ http://irr.ru/index.php?showalbum/login=razer2.moikrug.ru/recipes/shop.spb │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b706234651&op_uid=1&b... │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://video.yandex.php │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&price │ 20 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=NeIdeaCenternet Explorer&aV=5.0 │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertificirovka.ru/noteating-i... │ 20 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271161076&st=261&t=170977... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate=0&ReturnUr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313163307/?from]=&int[15... │ http://che.ru/produkty_zarubezhei-niepochekhly │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604370757034dea482207549... │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_api.php?id=7262882,9454472&op_product_n... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 20 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/cars │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26637&model=1705.html?1=1... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-karbox=0&top=0&category/1961/#images/00... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumilora481 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt=13733129/room │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-iulyanovskaya-obl/lyubvi?page=166733&eid=485... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27502/detail/?folders/#pa... │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://video.yandex.php │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626u1zIq0SGLXCI&where=al... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=240&u_cd=242035165&input_who2=1... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikulina.ru/real-estate/ou... │ 19 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?sort=popup/casualg... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news_id_72387877055/nu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/filmId=eRaB4pYAXCI │ http://komme%2F27.0.1453.116 │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBB\xD0\xB8\xD0\xBA\xD0\xBE\xD0\xBB\xD0\... │ http://irr.ru/index.php │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.ru/daily │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login=perfect.yandex.ru%2Fkategoriya%2F537 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260540718534/?dateOn=1&bo... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261921&pr=2441185112.html... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26672655462.html?city.ru/... │ 19 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostok.ru/cars │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27404895,95923.html%26cus... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/check=1&redir=1&lang=a... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671&Destige-all.html_par... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313437&pvno=2&evlg=VC,2;... │ http://irr.ru/index.php?showalbum/login-rizova.ru/look/timashirtsevm.cofe │ 19 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2638/?from=&raceTo=&power... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6HS │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.php?f=11340.1j1j2.12...5998-103358071/8/page... │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/00007/10/1320006&p... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27355f3132366336601&uuid=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166333/price[price_till]=&pri... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F_liveresum... │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto_repai... │ 19 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.15751 Safari%2F5.0 (Windows │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://smeshariki.ru/obucheyelants │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2675406828ac956028692581/... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://obninsk/detail │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26968794618/irr.ru/real-e... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2629&lr=65&rstr=-213&text=back=False&withmos... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2702483,631;IC,444041%2F&... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?search?text=\xD1\x80\xD1\x83\xD1\x81\xD0\xB... │ http://irr.ru/index.php?showalbum/login-inadlinny-sched_car=0&city=0&price=18&p... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26669E116-1-1550046560013... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=http://video.yandex.ru/real-estate │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274026181891&rn=52&brand=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/dlia-zhiensk... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&checked=0&state/room=10002-g-v-program... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-165&jenre]/49047&lr=163&text=samsung ga-moda... │ http://irr.ru/index.php?showalbum/login-2-komn-v-na-92f5-6ccf-fef3-013f9f926a21... │ 18 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ru/index │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5 │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?elmt=popular&where=all&film/723/num │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/restate/out-of-town-resh=1&cnt=1373... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2012-recept-Ragu-iz-2-x │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupaljinik-slidi/places/?PAGEN_1=2 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=deltec&type=citymo=&version%2F4.0 (Linu... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27534.30 (KHTML, like Gec... │ 18 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kochtinsk.pulscendinskiy-krasnodar.irr.ru │ http://irr.ru/index.php?showalbum/login-kuplyuskij/zapchastorii_state/out │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zzzi.ru/a-shop and blademas/?id=1&marka=0&ma... │ http://irr.ru/index.php?showalbum/login-2003prev=/search/index.ru/meteor.html%3... │ 18 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login.moikrug.ru/catalog/otchenko-ul-advert27... │ 18 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27132069304;IC,1511e8d158... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278453&with_exchange=0&ex... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=367132&pv=2... │ 18 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27071038_114736895,923401... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x9A\... │ 18 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27420780790931.html5/v12/... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/8018&l=malkogo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/?target=search?... │ http://irr.ru/index.php?showalbum/login-bigutensiysk.irr.ru/regular&bid=3205&bt... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kanza_tab_2499472&lr=1074&sort=popular │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-natura.ru/jobs-education=post_137336ID0... │ 18 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/obuv%2F │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-21102&text=\xD0\xB1\xD0\xB0\xD0\xB1\xD0\xBA\... │ http://irr.ru/index.php?showalbum/login.mospsy.ru/super-na-kostronics/tv-audio.... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685&numphoto=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertimeshariki.ru/cgi-bin/cli... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626p/ │ 18 │ -│ 1 │ 0 │ 0 │ https://diary.ru/magnitogorsk │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sien-zhienskaya-i-kvartir.html?id=1955451-01... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260908861%26point-peter=&... │ 18 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1453.116 Safari%2F8536 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeSearch?text=\xD0\xB2\xD0\xBA\xD1\x83\xD1... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=yators/?jumping.ru/series.ru/priceup&page=3&... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinger/\xD0\x93\xD0\x90\... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga1788778.137329TVmbFNmbVNaR0YwW$UyNDg4fxS... │ http://irr.ru/index.php?showalbum/login-sienskaia-moda-zhienskikh_rabotjaga.ru/... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=36427929&stat... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929750%26ar_sliceid%3D2... │ 18 │ -│ 4 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691c352005&pvno=2&evlg=V... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27422544/3/womens.ru/ufa.... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/cars/communalux.ru/dos... │ http://irr.ru/index.php?showalbum/loshaya-obl.irr.ru/film/35562C?analytics/kto-... │ 17 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru%2Ffilm/4858?pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271527091/frl-2/bage-2.ht... │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#7878... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.103402.html%3Fhtml │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=artira-Lyubyat │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.aspx#location │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=yes&y1=2013-07-07.ya.ru/5_search?text=\... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.15.html?1=1 │ 17 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://aliningrad │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266149_51928/detail.ru/ka... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%26rleurl%3D%26CompPath%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2670308&po_yers=0&price_o... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2613168312.71 Safari%2F__... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=8&ch=UTF-8&ifr=1&av=1&nm=1&ved=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_Bolers-device.aspx#commersanta_premiery... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 17 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6HS │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginyazevskaya%2F5.0 (Windows NT 5.1) Prestorygoo... │ http://irr.ru/index.php?showalbum/login-kupilomatic.aspx#location=rr&days&city&... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-dlia-zhiensmed.ru/el... │ 17 │ -│ 1 │ 0 │ 0 │ http://bdsmpeople.ru/register2123 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logistereda.ru/photo/70861/6#f │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263161366&bid=2823&numpho... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowthreadreplies=24&ids=74&cu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313&ussp=mstp&usln=1&inp... │ http://irr.ru/index.php?showalbum/login-3767436/roomootofile/obzor.com/iframe │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27595,9305f3931339322%26n... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273363331251798796&op_seo... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26041.htm%3Fsid%3D0%26pz%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031323532363&win=70&mode=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertistia.html?partment/searc... │ 17 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/stan/index.kz/yandex │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-search%3Futm_source=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 3 │ 22 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ 1 │ 0 │ 0 │ http://bibieters_sig=5ea8QizwGSPy0yGcm14ATH74D4Bw&usg=AFQjCNHxMZlonova.ru/p1847... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://diary.ru/forum/intries │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_style... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-futbolki-k... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lenfiki-zhienskaia-moda-muzhchin-sitafa ishq... │ http://irr.ru/index.php?showalbum/logical.ru/link=11485-9065383235348%2F&sr=htt... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261636/detail/odnoklassim... │ 17 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/messar.ru/professig │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 1 │ 0 │ 0 │ http://acase.php?input_who1=2&input_who2=1 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.10 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694&option%2F12.14&he=76... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26618531&t=111246.html?p=... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733145625/details... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-irina19910735%2F27.0.1453.116 Safari │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268814453957595,94406/det... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=Like Gecko) Chrome%2Fpotnik=1&price_do=&curr... │ http://irr.ru/index.php?showalbum/login-the-poxudet_sponsor=&o=6000708498/?caut... │ 17 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-53584715,1327288/belgium/arch=1&themec.... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=bestwo.diary/details&id=9160/0/posti.ri... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b76faact/less=1&ru=1&... │ 17 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/category=cinem rasmusic │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-herokee_tatus=1&rm=18606bAxFEcQT6smBB4W... │ 16 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search?clid=19554786.0.8.0.2.2003558_3497926... │ http://irr.ru/index.php?showalbum/logie-niz-doktops/electronics/4185570c72 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267044005320AA76)&vendor_... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098707209463401090/deta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgas.rQRAX2Ua3IF1iHJcaz4ATv34DABw │ http://irr.ru/index.php?showalbum/login/?page=1024&wi=144438-88-3906740/currenc... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27034195,9520160173/5/wom... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search/cuZXdzJTJGJmxyPTI4fQ,... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&pt=b&pd=7&pw=2&page5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-plan... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26396284046.xhtml%3Fhtml%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate/out-of-tow... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260612d7cf.4082813,74,756... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login.aspx?sle=13&s_yers=0&page_type=0&door=0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?&clients-sale&siteurl%3D//ads/search?text... │ http://irr.ru/index.php?showalbum/login-kapusta-advert273786/foto-5/#photo=2817... │ 16 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/movinki... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/page=91552&q=\... │ http://irr.ru/index.php │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnaja-prost-petersburg/detail/?fold... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-krasok.html_params%3Drhost%3Dad.adriver │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiensmed.ru/dl/\xD0\xB7\x... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti/redmonds %2F offset=403134333.2201560&c... │ http://irr.ru/index.php?showalbum/login-v40-velopiter.feriod=0&scroll_to_auto.r... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661232&streamfood.com/if... │ 16 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166303133334%2F&ei=horia+iudi... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-57548811426&text=\xD0\xBB\xD1\x83\xD1\x87\xD... │ http://irr.ru/index.php?showalbum/login-coolonellana-Molodilnik.ru/index.php?vi... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26765.html_params%3Drhost... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/70948/35.htm?id=... │ http://irr.ru/index.php?showalbum/logiy-luchaiev%2F&sr=http:%2F%2Fwww.bonprix c... │ 16 │ -│ 1 │ 0 │ 0 │ http://histore/profile;u= │ http://irr.ru/index.php?showalbum/login-kupe-20013&uuid=1373356164564_1sort/sta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2755597.html%3Fhtml?1=1&c... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=findjob.ru/planet.ru/index.ru/carinov.h... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/showtopic,803813.html&... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_catalog/1250/?itemsg/cd... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert260477/cities/visas/exp?s... │ 16 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://video.yandex.php │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27429&input_bdsm_position... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-153-4b9c4.4331626.htm?size%3D0%26rleurl │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26887301/Zona/simferopol.... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2728024/detail/508/?instv... │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 16 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect │ http://komme%2F27.0.1453.116 │ 16 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://msuzie-shop/premiery-c-38208_2.html │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/posts&q=\xD0\xB2\xD0\xB5\xD0\xB4\xD1\x83\xD1... │ http://irr.ru/index.php?showalbum/login-mistore/#CATALOG_LIST-s-ovosibirsk/deta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-marka=23&model&desting?field_town/search/pri... │ http://irr.ru/index.php?showalbum/login-492ea9&show/417582,9626512874":10157628... │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27364229586121%26url%3D%2... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/sch/price][min]=41&lr=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ 1 │ 0 │ 0 │ http://volgografiyah_27_iyunya_50_let%2FgetId │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661251-6efa-d61f-fef3-01... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-maritkiN │ http://irr.ru/index.php?showalbum/login-ladimir.irr.ru/remiery-telefon.ru │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new=127201148][fr... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333%26bid%3D1%26rleurl%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/tv-audi/a7-spec=9894797179698712/p1/8... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=view/5/item3963... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logiste's a di galaxy-s4-story1.aspx#location... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5.0 (Win... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti%2F%3Ff%3D100%&http://afisha.yandsearch&... │ http://irr.ru/index.php?showalbum/login-6030d.html_params%3Drhost%3D90%26height │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/7427510/detail/5... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269899458/price/3815&pvno... │ 16 │ -│ 1 │ 0 │ 0 │ http://yandsearch?lr=2&color │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-gorod/search?p=7&oprnd=9902.jpg&img_url=http... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27222.vk.me/u3166 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2753][from]=&pricedownloa... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new10152954vac │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2735030373434-152495.php?... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691.html?1=1&input_age2/... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makeevka.ru/perm.irr.ru/slingvo/#1\xD0\... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-irinakai/page_num_read2306e4574&ei=Ot_rUfjPG... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2010000087378687/pic/8939375f303839303935373... │ http://irr.ru/index.php?showalbum/login-souse=2/path=39_1.html?1=1&cid=577&oki=... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313033393633310,932803][... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26708662307][]=&selection... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/artira │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikal.ru/search/room=1&dam... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumy.ua/search?text=\xD1\x81\xD0\xB5\... │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/?strict=5710... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertists/736850/?item_no=737&... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2695/19545602cf94d77/repl... │ 16 │ -│ 1 │ 0 │ 0 │ http://forums/liiiervierk-suppohudeemvmesting/294465&op_categoriya │ http://komme%2F27.0.1453.116 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&model=1173... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login/?do=ready-mansion%3D0%26url%3D%26bn%3D0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=new&cad=rjt&fu=0&input_country_id=228&lr=961... │ http://irr.ru/index.php?showalbum/login-cam.shtml#objdesc=true&uA=Mozilla │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb=1&prepairs=0&city[1 │ http://irr.ru/index.php?showalbum/lounona-muzhchin-kupaljinik-chernyj-odnoe/san... │ 15 │ -│ 1 │ 0 │ 0 │ http://launcher-searchads/search │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-album/login.2/second... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginsk.eu/ru/Pagesize=13&m1=07 │ http://irr.ru/index.php?showalbum/login-3muda/truction/vacancies/750207190765.h... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_produkty/kiev.org/forum... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26981583/page=1089592445/... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781308;IC,238208836746/0... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-4433140796851/detail/536180&all=False&i... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273389; U; Android 4.0.14... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945HPS?analog/r10418/sort=price_do=200&site_off=1 │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 15 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshop/id_art_type=7&s_yers │ http://irr.ru/index.php?showalbum/login │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F1.7.1364.172 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273631253711/foto-6320166... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2724/?_h=search?text=\xD0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-goluboj-podserial&dfs=13 │ http://irr.ru/index.php?showalbum/login-haus.html#news/228670,257&pci=3012/frl │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#67 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/washes/built-in_two_ch... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266356163f39ea6f8/7a8745&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26605566254/room=1.6&wher... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?Topic │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://irr.ru/index.ru/show/414526863_112 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list.html?typ=SMA&anbieter=aleks_evilkos.com │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/index.ru/ange=0&s_yers=(3000&static.diary.ru... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/state=1909644 │ http://afisha.yandex.ua/auto_id=0&with_photo.kurortmag.ru/razdnitsya-vishnury │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginskikh-viana.irr.ru/GameMain.aspx?sle=1/house.... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?transk.i... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/produkty%2Fplatj... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?page37/?sta... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-land_search?text=\xD0\xB0\xD0\xBC\xD0\xB1\xD... │ http://irr.ru/6323%26bn%3D27888895,963095425 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678&op_page=60000/curren... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liloveplanet.ru/futurer533/women.aspx?group-... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2631920&lo=http://person_... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?Id=9582 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.1Uv1UfqCA8SonYC4BQ&usg=AFQjCNGB3pBUuKY1jJPP3... │ http://irr.ru/index.php?showalbum/login-tank-sale/search%3Fmode=&page_type │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=alini... │ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-tumbov.irr.ru/page1=&input_who1=2&id=29... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2718599/photo=0&is_hot=0&... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.php?city[1]=700003 │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26761,55.654289600/detail... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27515/418695&st=327-B110Q │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2699364224073532663835386... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login=vladimir/page_type=0&expand_search?text... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?dok=001&rightbox/week&From=0&user │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/frl-4/trailer/view/3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-lential/secondary/details9.html_params%... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2730675595,9292fa-d61f-fe... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page9/#14... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-676216b8af/4fd00fa61b3185631821/page_ty... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienshchin-planet.ru/myma... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=25127691%2F... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27265484158197%26width%3D... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27877437494,943082&stat_t... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1955451&lr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/gamemain.asp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_appliazh-pliance/mista-bez-uchaiev/000202&cl... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/ch/floore troubleclick... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Ford-antent... │ 15 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 3 │ 123 │ 0 │ │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nashinger/users/search?textilead&353%26ev_pl... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginy_name/3196906481/currency=6465373200&price=\... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2776/?date=0&doorov.irr.r... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-102-821922237&arrFilter2013/08-iH4AT5zIGQDA │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_type=&freetao.diary.ru/sea... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2693_763613.html_params%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9584%26pz%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274298799461981/?date=129... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert262997385f32313335781094&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/877/ru/buildin... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert276278-ploschaya_checked_... │ 15 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credir=1 │ http://komme%2F27.0.1453.116 │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://ekburg.irr.ru%2Fpuloveplanet │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=on&input_onliner... │ http://irr.ru/index.php?showalbum/login.html%26custom%3D%26CompPath.2; WOW64; r... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/gosthelp.ru/auto.ria.... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.10 │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=on&in... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga664.1721/deti74.ru/animals-planet.ru │ http://irr.ru/index.php?showalbum/login=K-SR-B-13-9635095,9661/9#f │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert269257798044.html_partmen... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/litlip/saint-peter557932E-8C62-4917%26ad%3D64... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2651734&pt=b&pd=7&pw=1&pr... │ 15 │ -└────────────────┴────────────────┴─────────────┴────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.127 user 1.216000 sys 0.004000 -D 07-01' AND (DATE '1970-01-01' + 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; -Run Time: real 0.014 user 0.180000 sys 0.000000 -D D (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND DontCountHits = 0 AND URLHash = 686716256552154761 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Run Time: real 0.018 user 0.240000 sys 0.000000 -D " = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) ORDER BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))); -Run Time: real 0.012 user 0.156000 sys 0.000000 -D -D -D -D '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(URL) > 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -┌──────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├──────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 102341 │ -│ http://komme%2F27.0.1453.116 │ 51218 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18315 │ -│ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16461 │ -│ http://irr.ru/index.php │ 12577 │ -│ http://irr.ru/index.php?showalbum/login │ 10880 │ -│ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 7627 │ -│ http://irr.ru/index.php?showalbum/login-kupalnik │ 4369 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 4058 │ -│ http://komme%2F27.0.1453.116 Safari │ 3021 │ -└──────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.065 user 0.800000 sys 0.008000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(Title) > 0 GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ Title │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ \xD0\xA2\xD0\xB5\xD1\x81\xD1\x82 (\xD0\xA0\xD0\xBE\xD1\x81\xD1\x81\xD0\xB8\xD1\... │ 122407 │ -│ \xD0\xA8\xD0\xB0\xD1\x80\xD0\xB0\xD1\x80\xD0\xB0\xD0\xB9), \xD0\x92\xD1\x8B\xD0... │ 82935 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA - IRR.ru │ 80958 │ -│ \xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\xB8 New Era H (\xD0\x90\xD1\x81\xD1\x83\xD... │ 39098 │ -│ \xD0\xA2\xD0\xB5\xD0\xBF\xD0\xBB\xD0\xBE\xD1\x81\xD0\xBA\xD1\x83 \xD0\xBD\xD0\x... │ 23123 │ -│ Dave and Hotpoint sport \xE2\x80\x93 \xD1\x81\xD0\xB0\xD0\xBC\xD1\x8B\xD0\xB5 \... │ 14329 │ -│ AUTO.ria.ua \xE2\x84\xA2 - \xD0\x90\xD0\xBF\xD0\xBF\xD0\xB5\xD1\x80 │ 14053 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA (\xD0\xA0\xD0\... │ 13912 │ -│ OWAProfessign), \xD0\xBF\xD1\x80\xD0\xBE\xD0\xB4\xD0\xB0\xD1\x82\xD1\x8C │ 10919 │ -│ \xD0\xA2\xD1\x80\xD1\x83\xD1\x81\xD0\xB8 - \xD0\xA8\xD0\xBE\xD1\x83\xD0\xB1\xD0... │ 10157 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.046 user 0.580000 sys 0.004000 -D AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 7479 │ -│ http://aliningrad │ 4791 │ -│ http://ekburg.irr.ru%2Fpuloveplanet │ 3584 │ -│ http://smeshariki.ru/obucheyelants │ 3064 │ -│ http://video.yandex.php │ 2887 │ -│ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 1084 │ -│ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 891 │ -│ http://afisha.yandex.ru/index │ 855 │ -│ http://sslow_13507.html?aspx?naId=6HS │ 521 │ -│ http://wildberrior/uphold │ 484 │ -│ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 289 │ -│ http://obninsk/detail │ 241 │ -│ http://diary.ru/forum/intries │ 208 │ -│ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 185 │ -│ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 185 │ -│ http://kurort/SINA, ADRIAN │ 157 │ -│ http://afisha.yandex.ru │ 132 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 124 │ -│ http://ssl.hurra.com/iframe │ 123 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 119 │ -│ http://stars-visa.html_params%3Drhost%3Dad.adriver.ru/catalog.php │ 105 │ -│ http://komme%2F27.0.1453.116 │ 83 │ -│ http://pogoda.yandex │ 80 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 79 │ -│ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 73 │ -│ http://video.yandex │ 71 │ -│ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 68 │ -│ http://wildberries.ru/daily │ 68 │ -│ http://smeshariki.ru/ru/index.ru%26bid │ 68 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 64 │ -│ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 64 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn │ 63 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 59 │ -│ http://sslow_135000008&position=search │ 58 │ -│ http:%2F%2Fwwwwww.bonprix.ru/topic │ 53 │ -│ http://sslow_13507.html/articles │ 52 │ -│ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 51 │ -│ http://rsdn.ru/rss.ya.ru/catalog │ 51 │ -│ http://ekburg.irr.ru/#lingvo │ 46 │ -│ http://pogoda.yandex.ru │ 45 │ -│ http://sslow_13500000%26rnd%3D2788881.html │ 44 │ -│ http://maps#ru_5_ru_22106.377648194,975924][to]=&int[14270pa106&op_uid=17759/6#... │ 42 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 37 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 37 │ -│ http://video.yandex.ru/film/46351/frl-2/bage │ 36 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 35 │ -│ http://direct.yandex │ 34 │ -│ http://sslow_13507.html?aspx?naId=3X_3bhLcs3M │ 33 │ -│ http://gotovim-doma │ 32 │ -│ http://che.ru/produkty_zarubezhei-niepochekhly │ 31 │ -│ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 30 │ -│ http://video.yandex.ru/Newsletter │ 29 │ -│ http://sslow_13507.html?aspx?naId=6D8IzMGys3M │ 29 │ -│ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ http://video.yandex.ru │ 28 │ -│ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 28 │ -│ http://notes=1/currency │ 27 │ -│ http://sslow_13507.html?aspx?naId=649&state/out-of-town=\xD0\xA5\xD0\xB0\xD0\xB... │ 27 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 26 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 26 │ -│ http:%2F%2Fwwww.bonprix.ru/tambov │ 24 │ -│ http://kinopoisk.ru/shoppich.ru/search?clid │ 24 │ -│ http://wildberries │ 23 │ -│ http:%2F%2Fwwwwww.bonprix.ru/searchAutoSearch?text=\xD0\xB2\xD0\xB5\xD0\xBB\xD0... │ 22 │ -│ http://myloveplanet │ 20 │ -│ http://stars-visa-litraj.txt","lpu":"http://pogoda │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/secondary │ 20 │ -│ http://msuzie-shop/premiery-c-38208_2.html │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 20 │ -│ http://smeshariki │ 19 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 19 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ http://irr.ru/6323%26bn%3D27888895,96772,97436 │ 18 │ -│ http://kinopoisk.ru │ 18 │ -│ http://nepogoda.yandex.ru%2Fproducts/search?text=subscripts/busineshop │ 17 │ -│ http://wildberries.ru │ 17 │ -│ http://kinopoisk.ru/catalog/9902224 │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://direct.yandex.html │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 17 │ -│ http://irr.ru/6323%26bn%3D27888895,963095425 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 16 │ -│ http://afisha.yandex.php?gidcar=36281664 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://auto.ria.ua/search │ 15 │ -│ http://direct │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ http://irr.ru/index.php?showalbum/login-kupalnaya-obl │ 15 │ -│ http://kinopoisk.ru/odessya │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 14 │ -│ http://ekburg.irr.irr.ru/maker │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.yandex │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.mail/rnd=0.9788 │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://auto_many_to_auto.ria.ua/igrush43/ │ 13 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 13 │ -│ http://afisha.yandex.ru/cars │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 13 │ -│ http://love.ru/a-myprofi │ 13 │ -│ http: │ 13 │ -│ http:%2F%2Fwww.bonprix │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://real-estate/aparther/offiliates/corruption/russinsk │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://on-online=on&accetti │ 12 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 12 │ -│ http://samara.irr.html5/v12/?from]= │ 12 │ -│ http://kinopoisk.ru/saledParams │ 12 │ -│ http://en.lyrsenses/zamba_zaborah_coldplay=1&gearbox │ 12 │ -│ http://zvukovo/hondar/2007&state/renlew/rigma.ru/scribed │ 12 │ -│ http://pogoda.yandex.php?SECTION │ 12 │ -│ http://msk/platia-nashing/vanny.diary.ru/moscow │ 12 │ -│ http://video.yandex.ru/GameMain.E6smreQhiu_hXR4&where=all&film │ 12 │ -│ http://news/6483731559676/Unlocknotebooks/m83/800_D_Black_list │ 11 │ -│ http://nizhnieiene/p17378705/currency=1&with_photo-ideapadeno │ 11 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 11 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty/photo-12/#imag... │ 11 │ -│ http://msk/platia-nashing/vanny.diary.ru/sale/liver │ 11 │ -│ http://video.yandex.ru/GameMain.aspx#location │ 11 │ -│ http://guid=6&pw=6&pv=13 │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 11 │ -│ http://wildberries.ru/rost.html?1 │ 11 │ -│ http://smeshariki.ru/?win=82&stat=1&page/196264&pt │ 11 │ -│ http://pogoda.yandex.ru/catalog/jokers │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 11 │ -│ http://en.lyrsenses/zamba_zabudtrimazok.html?page=12&prr=http://fap1.adrive_typ... │ 11 │ -│ http://bonprix.ru%26bid │ 11 │ -│ http://loveplanet.ru/GameMain │ 10 │ -│ http://bdsmpeople.ru │ 10 │ -│ http://video.yandex.ru&pvid │ 10 │ -│ http://liver.ru/cheboksicily/foto.aspx?sort=newly&trafkey │ 10 │ -│ http:%2F%2F%2Fwwww.bonprix │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://club.ru/spokoiteli/photo37775280000 │ 10 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 10 │ -│ http:%2F%2Fwwww.bonprix │ 10 │ -│ http://auto_id=240&n=13901038 │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 10 │ -│ http://smeshariki.ru │ 10 │ -│ http://slovariant_new3077940810/detail │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://en.lyrsenses/zamba_zabor_id=1012_blank%26site │ 10 │ -│ http://sslow_13500000%26rnd%3D2788881.html?parts/passe │ 10 │ -│ http://pogoda.html%3Fhtml_params%3Drhost%3D43 │ 10 │ -│ http://irr.ru/index.php?showalbum/logout%26verstova.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://pogoda.yandex.php │ 9 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film/6781203.html?id=242037047/detai... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://mysw.info/blog/sankt-peter%3D1216/00001216629 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://e96.ru/albumfotok-15-fotki │ 9 │ -│ http://alpari.yandex.html?html_param=0&users/#page/Search/ab_dob%2Ffieiie-razvo... │ 9 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://bonprix.ru/catalog/8570/1006790 │ 9 │ -│ http://b2b.testered/main/discuss/matched_country=-1&top=0&cityid=1024&wi=1366&o... │ 9 │ -│ http://svpressa.ru/topic=17082630 │ 9 │ -│ http://bonprix.ru │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http:%2F%2Fwwww.bonprix.ru/filmId=8j5j97LRs3M&where=all&sources │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/#win_13.html_p... │ 9 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/4744089758 │ 9 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_produkty%2F&sr=http://slovaria │ 9 │ -│ http://nail=Yes&target=search │ 9 │ -│ http://rlsnet.ru/vacancy/view_type_id=9677548268010367 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/imagecache/wm/2013&where=all&filmId │ 8 │ -│ https://m.myloveplanet.ru/forum/abrika-kobelenie_nebestsenal │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 8 │ -│ http://zapchast.com/iframe-owa.html?1=1&cid=577&oki=1&op │ 8 │ -│ http://poisk.ru/price_ot=&price_ot=&price │ 8 │ -│ http://afisha.yandex.php?t=141880517 │ 8 │ -│ http://zarplata.ru/velika_all=\xD0\xBE\xD1\x82 │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http:%2F%2Fwwwwww.bonprix.ru/news/222974895&op │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://fore=3&marka=0&top=0 │ 8 │ -│ http://direct.yandex.ru/catalog │ 8 │ -│ http://video.yandex.ru&xdm_p=1#item/search │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96777&oki │ 8 │ -│ http://zarplata.ru/?p=12977-B26358/currency=RUR/page=1080&wi=1024&lo=http://rzh... │ 8 │ -│ http://en.lyrsenses/zamba_zabor/bedroomolsk │ 8 │ -│ http://masterh4.adriver.yandex │ 8 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ==&page_avtomodules.php?f=100&ref... │ 8 │ -│ http://mastered/main.aspx#location=1&bc=3&ct=1&pr=60322056107100919/page5/?_h=s... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://omsk.mlsn.ru │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96779/87 │ 8 │ -│ http://smeshariki.ru/GameMain │ 8 │ -│ http://che.ru&pvid=13733142835/100/topic,5240556895&ch=UTF-8&sF=11,7,7,0 │ 8 │ -│ http://afisha.yandex.php?gidcar=367108851%2Fr%2F1 │ 8 │ -│ http://love.ru/?p=17059 │ 8 │ -│ http://wildberries.xml?from]=&input │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/index.php?showalbum/login-kupit-topy%2Fplatjie-gotovlexandex.html... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 8 │ -│ http://auto.ria.ua/auto_id=24126629/0/index.ru/real-estate/out │ 8 │ -│ http://victor?page_type=city.stol-yar.ru/cars │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://video.yandex.ru%2F&sr=http://loveplants/65398f55 │ 7 │ -│ http://kinopoisk.ru/ch/feed/letniaiaprice/1609 │ 7 │ -│ http://real-estate/aparts/Aquarevski │ 7 │ -│ http://video.yandex.ru/a-album/login-vitiju/photo │ 7 │ -│ http://bonprix.ru/social/product_id │ 7 │ -│ http://moscow/detail/5552/0/2792834&m=111,7,7,5 │ 7 │ -│ http://irr.ru/Registered/main/topnewsru.com/page=6 │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 7 │ -│ http://irr.ru/index.php?showalbum/login-kapustics?sort=pogoda.yandex.ru%26bt%3D... │ 7 │ -│ http://bonprix.ru/catalog/8570/14139489 │ 7 │ -│ http://afisha.yandex.ru/\xD0\xB4\xD0\xBE\xD0\xBC\xD0\xB0/\xD0\x91\xD0\xA1\xD0\x... │ 7 │ -│ http://irr.htm?from]=&int[85][from]=&input_vsegodnyie │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://love.ru/?p=1#country=&op_seo │ 7 │ -│ http://bdsmpeople.ru/niktory/shtory/308/roomed.ru/p59473682740295 │ 7 │ -│ http:%2F%2Fwwwww.bonprix │ 7 │ -│ http://myloveplantrackIt?tid │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 7 │ -│ http://spb/event=big&marka=84&model=0&auto_id=0&s_yers=0&pv=10&can_be_check_PP │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://pogoda.yandex.ru&pvid=1 │ 7 │ -│ http://irr.ru/index.php?showalbum/login-a-5-advert27114 │ 7 │ -│ http://video=0&is_hot │ 7 │ -│ http://bdsmpeople │ 7 │ -│ http://irr.ru/index.php?showalbum/login-zk34/pages/0001216629 │ 7 │ -│ http://svpressa.ru │ 7 │ -│ http://money.yandex │ 7 │ -│ http://gotovim-doma.ru │ 7 │ -│ http://afisha │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 7 │ -│ http://video.yandex.ru%2Fkategory_id │ 6 │ -│ │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://afisha.yandex.ru%2F%2Fmail/169 │ 6 │ -│ http://love.ru/?p=1#country=-1&sq_liver.ru/kyrgyzstan │ 6 │ -│ http://omsk/evential/house.ru/catalog/kitchedule=213-606361653965283 │ 6 │ -│ http://video.yandex.ru/ekt │ 6 │ -│ http://stars-varenok.ru/16745959680706/800_0.jpeg.html%3Fhtml5/v123593 │ 6 │ -│ http://bdsmpeople.ru/film/64544.690022.rar.html_params%3Drhost%3D_black_list=0&... │ 6 │ -│ http://afisha.yandex.ru/catalog=on │ 6 │ -│ http://bdsmpeople.ru/GameMain │ 6 │ -│ http://v102.ru/investate/apartments-sale │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://video.yandex.ru/GameMain.aspx#location/page_type=category │ 6 │ -│ http://wildberries.ru/search │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://afisha.yandex.ru%26bt%3D43%26anbietersburg │ 6 │ -│ http://video.yandex.ru/a-topy │ 6 │ -│ http://omsk/evential/housession%3D0 │ 6 │ -│ http://direct.yandex.ru/refererprofile%2F2.10 │ 6 │ -│ http://moscow/details │ 6 │ -│ http://rukodel=0&sort=newly&trafkey=2750 │ 6 │ -│ http://smeshariki.ru/world/photofider_credit=0&view │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://irr.ru/index.php?showalbum/login-12.html%26custom │ 6 │ -│ http://novosibirsk.irr.ru%26bid │ 6 │ -│ http://solutions.diary.ru/realtitroenie_v_jurman.ru/albums/frame-owa.html?stric... │ 6 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 6 │ -│ http://mr7.ru/newsru.com/iframe_right%3D43 │ 6 │ -│ http://autodoc.ru/real-estate/apart │ 6 │ -│ http://smeshariki.ru/catalog │ 6 │ -│ http://edp2.adriver.ru/hocketshop.ru/moscow/detailanude │ 6 │ -│ http://povari.yandex.ru/greecondary/Products_id=&auto_vaz_2111 │ 6 │ -│ http://msk/platia-nashing/vantralitsa_transion │ 6 │ -│ http://3dnewsru.com/iframe_right.html?1=1&cid=51538 │ 6 │ -│ http://video.yandex.ru/catalog │ 6 │ -│ http://ereal-estate/rent │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://afisha.yandex.php/board,39.04839 │ 6 │ -│ http://novo/detail.aspx?group_cod_1s │ 6 │ -│ http://auto_hyundai_sarator │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://kalininmyclonus1 │ 6 │ -│ http://bdsmpeople.ru/saledParams=rhost%3D43%26bid%3D1 │ 6 │ -│ http://forum/topnews/22294&op_category │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://love.ru/?p=17055335 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://afisha.yandex.ru/mymail.php │ 6 │ -│ http://real-estate=week/page=1&expand_search?film/298677435615.html │ 6 │ -│ http://koolinar.ru/port.ru/doc │ 6 │ -│ http://video=0&with_exchangeType │ 6 │ -│ http://afisha.yandex.ru/index.ru/recipe │ 6 │ -│ http://kinel-lab.com/rus/20130709_117485994,93304&op_seo_entry=1&gearbox=0&type... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ https://slovakia-600dd903c07022,101595,91194&op_seo_entry │ 6 │ -│ http://diary.ru/exp?sid=3205 │ 6 │ -│ http://video.yandex.php?search?text=\xD0\x9A\xD0\xBE\xD0\xBD\xD1\x8C\xD1\x8F\xD... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://video.yandex.php?from]=&interapy-wkti/ &cd │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D8393224 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://love.ru/ru/irk/event/search/ │ 6 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://kinopoisk.ru/catalog │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://auto_volkswagen/vologdano/il_dlya-dnevka.ru │ 5 │ -│ http://barnaul/details/?cauth │ 5 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD0\xB5\xD1\x80&where=... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 5 │ -│ http://metal-lodku-Obzor │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 5 │ -│ http://sp-mamrostova │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://tp66.ru/exp?sid=3860217/rooms=2/men.aspx#location%3D0%26rnd │ 5 │ -│ http://zvukovo-gorodsk │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://video.yandex.by/?state_id=&auth=1..630;IC,7711588 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http:%2F%2Fwwww.bonprix.ru/?id=2013&where=all&filmId │ 5 │ -│ http://rmnt.ru/search/offilia_Sovetov_living_chamber/?78142 │ 5 │ -│ http://e96.ru/movies/614418821/artir.ua/search │ 5 │ -│ http://slovari.yandex.ru%26orderovskij-index.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://povari.yandex │ 5 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_seo_entry=&op_category_... │ 5 │ -│ http://tks.ru/cat/publish-cherkalnaya-ttpodporyadushek │ 5 │ -│ http://video.yandex.ru/GameMain.aspx?Link │ 5 │ -│ http://slovarenok.com │ 5 │ -│ http://auto.ria.ua/auto_id=241269.html?1=1&cid=2127970 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://slovari.yandex │ 5 │ -│ http://auto_volkswagen-Palities/horobki │ 5 │ -│ http://afisha.yandex.php?r=23436303135353.html?1 │ 5 │ -│ http://radio&planet.ru/work.ru/catalog │ 5 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 40007&pt │ 5 │ -│ http://en.lyrsenses/zamba_zaborah_chamberk │ 5 │ -│ http://rustnye-sht-riemnikoi │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://myloveplanet.ru/v14572&lb_id=13000001216629 │ 5 │ -│ http://sslow_135000008&position=search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 5 │ -│ http://pogoda.yandex.php?gidcar │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://bibidohertki-i-OOOO_REPORT/07_2013 │ 5 │ -│ http://autodoc.ru/moscow │ 5 │ -│ http://real-estate=week&m=Dvigenie.html?option │ 5 │ -│ http://avtomobile/motory.ru/comp.ru/view.php │ 5 │ -│ http://topnews.ru/GameMain.aspx?group_cod_1s=1983&pt=b&pd=9&pw=0 │ 5 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 5 │ -│ http://love.ru/product_id=0&po_yers=0&po_yers=2&refererro/model=1346488078722&c... │ 5 │ -│ http://video.yandex.ru/firms.turizm │ 5 │ -│ http://video.yandex.ru%2FkategoriendflowerTo=&powerTo= │ 5 │ -│ http://loveplanet.ru/mymail/rudi │ 5 │ -│ http://auto.ria.ua/auto │ 5 │ -│ http://msk/platia-nashing/vannyie-product_id=1841&page2 │ 5 │ -│ http://auto.ria.ua │ 5 │ -│ http://nizhnieiewva88/photo/101246465376&cmd=show-to-buchaiev-pugache=51db32a68... │ 5 │ -│ http://club.ru/cinema/movies/no-pos │ 5 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D158197%26width │ 5 │ -│ http://tks.ru/cat/publish-chemec.ru/search?filmId=CktclMBmUXI │ 5 │ -│ http://direct.yandex.ru/index │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://kinopoisk.ru/spb.pulscen.ru/exp?sid=3159&op_category_id=&auth=0&checked=... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://holodilnik-rp-ploschaya-obuv/?ci=1280&with_video=0&choosOyg==&op_uid=111... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://wildberries.ru/real │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://slovariant_neu%3D1%26bid%3D1216/0001216629%26bt%3Dad │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://wildberrior/kia/ │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://md.mirkovskaya-obl.irr.ru/jobinmoscow │ 5 │ -│ http://jcmotorom-921205&bt=7 │ 5 │ -│ http://loveplanet.ru/\xD0\xB0\xD1\x80\xD0\xBE\xD0\xBC/curre-ap-i-showalbum/loui... │ 5 │ -│ http://povari.yandex.php?showalbum/login.pl?cl=all&film/497794,90458 │ 5 │ -│ http://en.lyrsense.com/obshchin-idieiala │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://video.yandex.ru%26bt%3Dad.adriver.ru/recipe/view/10217/?from │ 5 │ -│ http://gaylyU │ 5 │ -│ http://moscow/detail/Torgovuyu-organske.ru/real-estate │ 5 │ -│ http://smeshariki.ru/topic │ 5 │ -│ http:%2F%2Fwww.bonprix.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://loveplanet.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://alpari.ru/gallery/pic845274 │ 5 │ -│ http://alib.mist.html%26custom%3D%26c2%3D278888592138 │ 5 │ -│ http://loveplanet │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.ua │ 4 │ -│ http://whoyougle.ru/images/images/00000i/specifiers.ru/image=1&furniture │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 4 │ -│ http://video.yandex.ru/price │ 4 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/filmId │ 4 │ -│ http://samarskii_krai/tuapse/detail/result.aspx │ 4 │ -│ http://sslovarenok.ru │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://auto.ria.ua/auto_id=0 │ 4 │ -│ http://afisha.yandex.ru/zoom.php?f=5162613838.html_partments │ 4 │ -│ http://bdsmpeople.ru/Web/Pages=1/feedsmag.ru/~\xD0\xBA\xD0\xBD\xD0\xB8\xD0\xB3\... │ 4 │ -│ http://loveche.html │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 4 │ -│ http://irr.ru/index.php?showalbum/login-10618968476372773 │ 4 │ -│ http://wildberries.ru/catalog │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://afisha.yandex.ru%252f7769%252fe │ 4 │ -│ http://echoradar-s-Levoshcha │ 4 │ -│ http://radio&planet.ru/marka=62&model=1178128455&pvno=2&evlg=VC,5 │ 4 │ -│ http://love.ru/?p=1#countpage/vacancies/events/738/0/3/women.aspx │ 4 │ -│ http://diary.ru/catalog=on&input_city[3 │ 4 │ -│ http://direct.yandex.php?formsof(INFLECTION_ID=30861/14365-4b11&state/apartment... │ 4 │ -│ http://e96.ru/news/39733/page2=&input_sponsor=&o=1015219.html_partments-sale&pa... │ 4 │ -│ http://rasp.pl?cmd │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://travel.ru/state/apartments-sale/rashinitit%2F537 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.php?view_type=2&driveresult.ru/replies │ 4 │ -│ http://auto.ria.ua/auto_id=63799.html_params │ 4 │ -│ http://afisha.yandex.php?r=3&bs=&day │ 4 │ -│ http://ssl.hurranovskaya-ul-31-foto.ria │ 4 │ -│ http://kinopoisk.ru/saledParams%3Drhost%3Dad.adriver.ru/GameMain.aspx#location │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ https://smeshariki.ru/cinema/article10363136000001216629%26site_offilia_Sovets.... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboy07/... │ 4 │ -│ http://vkirovoe-tourisma │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://smeshariki.ru/domchelkakh_location=search?text=\xD0\xBC\xD0\xB8vents │ 4 │ -│ http://guid=6&pw=2&pv=0&price_do=¤cy=1 │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://card/windows)&bL=ru&cE │ 4 │ -│ http://kniga.ru/view=\xD0\xA1\xD1\x82\xD0\xB0\xD0\xB2\xD1\x80\xD0\xBE\xD0\xB2&w... │ 4 │ -│ http://smeshariki.ru/search/keukeru-soft │ 4 │ -│ http://video.yandex.ru/GameMain.XYyZwYXRoPWEtbG9nb24vcG90Cw │ 4 │ -│ http://zarplata.ru/?p=12977-B26358/hasimages=1/page │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://matched_car=373838928155755775482794,9453.116 Safari%2F537.36&he=10&s_ye... │ 4 │ -│ http://video.yandex.ru/real │ 4 │ -│ http://afisha.yandex.ru/search │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://v102.ru/?s=Adaments-sale │ 4 │ -│ http://video.yandex.ru/GameMain.aspx#location-4.0.html │ 4 │ -│ https:%2F%2Fwwww.yandex.ru │ 4 │ -│ http://tks.ru/cat/publish-chernyjbelyj-9375966238&op_categoriya │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/social │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/realty/leaser_map=1/hasimay-2.html │ 4 │ -│ http://auto.ria.ua/canel_pe_mascona Hilfigeratov/153228 │ 4 │ -│ http://nizhny novgorod │ 4 │ -│ http://video.yandex.ru%26target=search/ab_area=categorija80119 │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://kubikus.ru/search.php?r=4140211,1,7,7,7,7,0 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://e-kuzbass.ru/a-shop │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://victorhead.php?full&dom=780067167694.0; │ 4 │ -│ http://guide.travel.ru/link │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/cart.php/cars/papago-d-plosch │ 4 │ -│ http://afishi,Mudanted-belyj-974299099/guest-id=34089.html │ 4 │ -│ http://fitness/building │ 4 │ -│ http://svpress_up.aspx#location[propfilter_pf[PODVAL]=&arrFilter][3463351841195... │ 4 │ -│ https://produkty%2Fpulove.ru/gost/?page3 │ 4 │ -│ http:%2F%2Fwwww.bonprix.ru/real-estate/out-of-town/house │ 4 │ -│ http://afisha.mail.aspx#locationalOffers │ 4 │ -│ http://video.yandex.ru/GameMain/dukhovyy │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadbor_sh... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http%3A//magnitogorod/page3/#over │ 4 │ -│ http://anketka.ru/bridget │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://afisha.yandex.php?action/2741920 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://love.ru/image=2&marka=84&model/mihailovo │ 4 │ -│ http://tks.ru/cat/public/gamemain.aspx#location │ 4 │ -│ http://nepogoda.yandex.ru/search=1&target=search=0&can_be │ 4 │ -│ http://kinopoisk.ru/searchAutoSearch=0&driver.ru/catalog/1/women.aspx#locationp... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://afisha.yandex.php?show=rlv&ru=1&expand_search │ 4 │ -│ http://love.ru/?p=1705 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://ssl.hurra.com │ 4 │ -│ http://auto_volkswagen_cated_cartovskaya-obl.irr.ru/album/login │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.ru/If yours.avtogsm.ru/animals/calculate │ 4 │ -│ http://afisha.yandex.php?p=31&input │ 4 │ -│ http://loveche.ru/job/1162323&PAGEN_1=30&state │ 4 │ -│ http://internet Explorer&aV=5.0 (Windows)&bL=en │ 4 │ -│ http://pogoda.yandex.ru/cooking_sm5_1148786993ab-417/photo/69363/26#formi.ru/co... │ 4 │ -│ http://rsdn.ru/details_103514,154;IC │ 4 │ -│ http://auto_s_product_name=\xD0\x9A\xD1\x80\xD1\x83\xD0\xB3\xD0\xBB\xD0\xBE\xD0... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboyzon... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://video.yandex.ru/topnews.ru/lanas-advert2713][to]=&int │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://auto_volkswagen_pass_1161967 │ 4 │ -│ http%3A//edp1.adriverys/forum/view_type=city&custom=0&damages/0001216629%26bid%... │ 4 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD1\x82\xD0\xBE\xD1\x8... │ 4 │ -│ http://inspelishchin-platjie-doma.ru/irkutsk.irr.ru │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://smeshariki.ru/saint-petersburg-gorod/transfer/?id=7576149959760994861&op... │ 4 │ -│ http://en.lyrsenses/zamba_zabudka/photo-1/#page=0&sale/search?text=\xD0\x9C\xD0... │ 4 │ -│ http://afisha.yandex.ru/content2.adriver │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://auto_id=0&with_photo │ 4 │ -│ http://arma/frl-4/transportnoy-kv-m-malchik.ru/show │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://b.kavanga.ru/?a=inneVolumeFrom │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://irr.htm?from]=&int[1151;IC,112 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_category_id=592b9e01c48ce9403%26bn%3D0%2... │ 4 │ -│ http://tp66.ru/money.yandex.ru/albums_screenterval │ 4 │ -│ http://bdsmpeople.ru/film/64544.690078 │ 4 │ -│ http://afisha.yandex.ru&pvid=13733424 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://video.yandex.ru/GameMain │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ https://orenburg/?arrFiltersburg │ 3 │ -│ http://kinopoisk.ru/service.ru/iframe │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://samara.irr.ru%2Fproduct │ 3 │ -│ http://radiorecord.ru/login-m3w.html?1=1&cid │ 3 │ -│ http://romar/events/audio.ru/widget[]=vacancies/99.php?gidcar │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://afisha.yandex.php/topic104780204&op_uid=1954 │ 3 │ -│ http://afisha.yandex.php/tova.ru/uliya2076789599305953 │ 3 │ -│ http://afisha.yandex.ua/donetsk/urals │ 3 │ -│ http://afisha.yandex.ru/a-folders/misc │ 3 │ -│ http://afisha.yandex.ru/shop.ru/malta │ 3 │ -│ http://afisha.yandex.ru/project_price=&maxprice │ 3 │ -│ http://afisha.yandex.ru/kategoriya%2F5.0 (company │ 3 │ -│ http://afisha.yandex.ru/linkvac.php/board.php?topicseeng │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://irr.kz/realty/lease/3516093&pvno=2&evlg │ 3 │ -│ https://produkty%2Fplatjie-kuzbass.ru/newsru.com/iframe_right=0&auto_ria=0&meta... │ 3 │ -│ http://fuckfind=rent/view/2021/3 │ 3 │ -│ http://banantikov-nadushenie_orleona_server=sc.cheloveplant_11612/page │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://radioscannerica/filmId=Ba_id=13733568414&city=\xD0\x9C\xD0\xBE\xD1\x81\x... │ 3 │ -│ http://video.yandex.ru%2Fplata.ru/ch │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://kaluga/?ext=\xD0\xB3\xD0\xB5\xD1\x80\xD0\xBE\xD1\x8F\xD1\x82\xD0\xBD\xD1... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://love.ru/forum.cofe.ru/forum/view_type=city=790&Selectronics-technik │ 3 │ -│ http://astrobank.ru/image │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://sp-money.yandex.ru │ 3 │ -│ http://love.qip.ru │ 3 │ -│ http://auto_kia_30 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://sp-mamrostokonkursovet │ 3 │ -│ https://slovakia-600dd903c06c999c226647639.html%3Fhtml │ 3 │ -│ https://slovakia-600dd903c07022,101595,9143531427800648_elit │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category_id=0&wi=16000&... │ 3 │ -│ http://sp-money.yandex.ru/work rushki-sien-natalog/8570/page=0&expand │ 3 │ -│ http://video.yandex.ru&pvid=13735/?_h │ 3 │ -│ http://afisha.yandex.ru/forum.materinburg │ 3 │ -│ http://afisha.yandex.php?addriver.ru │ 3 │ -│ http://forum/topnews/2229605699574.html?1=1 │ 3 │ -│ http://video.yandex.by/search/?target%3D43%26bid%3D2 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberrior/bedroom]=&int[17][to]=&int[858 │ 3 │ -│ http://direct&sortdirect.yandex.ru/imagesize%3D0%26ar │ 3 │ -│ http://ssl.hurral=messages │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://rmnt.ru/stars │ 3 │ -│ http://video.yandex.ua/auto_id │ 3 │ -│ http://bdsmpeople.ru/search │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://b.kavanga.ru │ 3 │ -│ http://rmnt.ru/film/88677/russia/rio.ru/search?filmId=NNr6aJrm4s3M │ 3 │ -│ http:%2F%2Fwww.bonprix_ru}%2Fnizhniynovgorod/request-id │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://edp2.adriver.ru/catalog/181 │ 3 │ -│ http://video.yandex.ru/circle&state/out-of-town/houses │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film.ru │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://edp2.adriver.ru/jobinmoscow/detail │ 3 │ -│ http://svpress_w1t1042796786/6/?category │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://pogoda │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://msk/planet.ru/mymail.aspx#comme_me_saydinne │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/forum.donfiscategory │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://pogoda.yandex.kz/family │ 3 │ -│ http://kazan.irr.ru/location │ 3 │ -│ https://produkty%2Fpulove.ru/voronezh-sien-zhienskaia-moda-zhienskaia-moda-zhie... │ 3 │ -│ http://video.yandex.php?topbloveche │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com/iframe-owa.html?1=1&cid=577&o... │ 3 │ -│ http://tks.ru/cat/publish-chin-play.php?categoriya%2Fzhiensmed │ 3 │ -│ http://card/windows NT 6.1) AppleWebKit%2F5 │ 3 │ -│ http://radiorecord.ru/catalog/idShare │ 3 │ -│ http://nigma.ru/product&op_category_name=\xD0\x91\xD0\xB8\xD0\xBA\xD0\xB8\xD0\x... │ 3 │ -│ http://ussuriysk.irr.ru/catalog/premiere/628962851d7fd0b6eb17b321d336f5bc7de189... │ 3 │ -│ http://magnitka_1_series.ru/?favorite_id=636233644&op_category_id=937514 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://loveche.ru/volzhskiy │ 3 │ -│ http://smeshariki.ru/a-phony │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty%2Ftanki │ 3 │ -│ http://svpress/showbiz/photo.htm │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/92054446660.ht... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://afisha.yandex.ru/tatatit_chto.php?industry │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountpage │ 3 │ -│ http://smeshariki.ru/catalog/286/women.aspx │ 3 │ -│ http://svpressa.ru/content/search │ 3 │ -│ http://video_dvd/game/iframe-owa.html │ 3 │ -│ http://ekategoriya%2F9B206 Safari │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26url%3D//ad.adriver.ru/link/justic/h2.php/... │ 3 │ -│ http://msk/platia%2Fzhienskaia │ 3 │ -│ http://lk.wildberries │ 3 │ -│ http://holodilnik.ru/catalog │ 3 │ -│ http://tp66.ru/search/?page=1&fuelRateTo │ 3 │ -│ http://e96.ru/real-estate │ 3 │ -│ http://my.kp.ru/albumfoto-1/pol-2 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://amobil-nye-pliazhnaia │ 3 │ -│ http://personal/atlants/7292&xdm_c │ 3 │ -│ http://povarenok.ru/files/eliteh.ru/perm.irr.ru/board,75.2013-07-09 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://smeshariki.ru/?win=82&stat=141882,373;IC,2552f48 │ 3 │ -│ http://brand=498&pvno │ 3 │ -│ http://love.ru/?p=1#country=&op_seo_entry=-1&target │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://sp-mamrostovestory │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://irr.ru/washek-s-printom │ 3 │ -│ http://smeshariki.ru/real │ 3 │ -│ http://smeshariki.ru/goodavec/photo/6936325.html?id=223978/page=102 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://irr.ru/index.php?showalbum/login-kupaljinik-chere │ 3 │ -│ http://msuzie │ 3 │ -│ http://kinopoisk.ru/search │ 3 │ -│ http://wildberries.ru/item_no=2&evlg=VC,0;VL,205;IC,14;VL,757138/currency=RUR/h... │ 3 │ -│ http://3dnews.ru/?p=12636464/5#f │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic=79799398/?_h=search.htm... │ 3 │ -│ http://afisha.yandex.php?app=membered │ 3 │ -│ http://afisha.yandex.php?ELEMENT_ID │ 3 │ -│ http://afisha.yandex.ru/lesyach-hotels │ 3 │ -│ http://afisha.yandex.ru%26bt%3D90%26nid%3D1216629 │ 3 │ -│ http://afisha.yandex.php?id=727285 │ 3 │ -│ http://str_ob.html?1=1&choosO8gPJSs3M&where=all&filmId=mAyiC7y6M2mGV2GoA9hFoN3q... │ 3 │ -│ http://kaluzha-na-NovletedAutoSearch=0 │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostavia.travel.ru/s... │ 3 │ -│ http://product_brand=RAINBOW&op_cated_content/search/keup/en-ru │ 3 │ -│ http://love.ru/ru/irk/event=little&cated_country=-192.html?1=1&cid │ 3 │ -│ http://love.ru/?p=17057 │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://love.ru/recipes/indows │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory_id=577&search/ab_district/date_id=2271][from... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://kinopoisk.ru/sessulyanovka.ru/photosessid=3205&bt │ 3 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD0\xB2\... │ 3 │ -│ http://li.ru/filmId=XpzlPj8P8gE&where=all&text=\xD1\x81\xD0\xBA\xD0\xB0\xD1\x87... │ 3 │ -│ http://you_hashkaf.ua/search │ 3 │ -│ http://wildberries.ru/GameMain │ 3 │ -│ http://povaria/chak_naytimes.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://auto_repairs=0&confiscategoriya │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://afisha.yandex.ru/comment/search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://smeshariki.ru/GameMain.aspx#location │ 3 │ -│ http://afisha.yandex.ru%2Fobuv-sapozhkivka=23&price │ 3 │ -│ http://afisha.yandex.ru/hotel-agen-Goluboj-9730 │ 3 │ -│ http://notebooking pressa │ 3 │ -│ http://soft.oszone.ru │ 3 │ -│ http://pogoda.yandex.ru/real-estate │ 3 │ -│ http://video_dvd/ratesTypeSearch?text=\xD1\x87\xD0\xB5\xD0\xBB\xD0\xBE\xD0\xB2\... │ 3 │ -│ http://auto_repairs=0&po_yers=0&price.ru/\xD0\xB6\xD0\xB5\xD0\xBB\xD0\xB5\xD0\x... │ 3 │ -│ http://auto_id=0&engineVolumeFrom │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://wildberries.ru/basket&ch │ 3 │ -│ http://state/room=94720-recept-Salat-iz-glasya1lesyat │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://video.yandex.ru/personal/offeebe34c7e12944&op_product │ 3 │ -│ http://trashbox.ru/book │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://slovari.yandex.ru │ 3 │ -│ http://kommersantamina │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://audio_video.yandex.ru/mosday.html?item=4#photo446962 │ 3 │ -│ http://auto_ford Mix).mp3.ucoz.ru/v1430497.html%3Fhtml │ 3 │ -│ http://v102.ru/investate/apartment/?id=137336IseNhcbx3J85GkHSnzgnsPdZUU&where=a... │ 3 │ -│ http://afishers/story │ 3 │ -│ https://slovarenok │ 3 │ -│ http://pogoda.yandex.ru/vlas-moskovskaya │ 3 │ -│ http://guid=6&pw=2&pv=0&with_video.yandex.ru │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/photo/narod.irr.ru/katering=1&pr=569&s_yers │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com.ua/kiev/detail.ru/auth=1..640... │ 3 │ -│ http://video.yandex.ru/extra │ 3 │ -│ http://love.ru/ru/irk/event=little&category_id=731-643736&mode=1 │ 3 │ -│ http://dom.net/provoe-pervouralnaya/ChildGluZ19oZXJlci1kYXRpb25zPU4mbj0zJmlkPTM... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://alpari.yandex.ru/saint-petersburg.irr.ru/cars/page3 │ 3 │ -│ http://video.yandex.ru%2Fkategory_id=9759527418 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://video │ 3 │ -│ http://radiorecord │ 3 │ -│ http://wildberring │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://balcon_caddy Club relove │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://kinopoisk.ru/registernet Explorer&aV=5.0 (Windows NT 5.1; ru-ru&cE=true&... │ 3 │ -│ http://state_shariki │ 3 │ -│ http://en.lyrsenses/zamba_zabor_polnitsa-s.narod.irr.ru/bank/otzyvy/12031%2F%23... │ 3 │ -│ http://auto_id=0&color=0&confiscategory_id=3205&bt=7&bn=1&bc=3&ct=1&prr=http:%2... │ 3 │ -│ http://afisha.yandex.php?partments │ 3 │ -│ http://wildberries.ru/cgi-bin/click.cgi%3Fsize │ 3 │ -│ http://video.yandex.ru/real-estate/out-of-town/house.ru&pvid=1&distreet_legkovo... │ 3 │ -│ http://omsk/evential/housession%3D240%26rleurl%3D//ad.adriver.ru/marshavskaya-r... │ 3 │ -│ http://smeshariki.ru/cgi-bin/click.cgi%3Fsid%3D0%26pz │ 3 │ -│ http://kinopoisk.ru/spb.pulscen.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://loveplanet.ru/Bezli-all │ 3 │ -│ http://bdsmpeople.ru/show/39932/Itemid,40.0.html_params%3DfsSaHR0cDovL2pzLnNtaT... │ 3 │ -│ http://video.yandex.ru/realty/search/main.aspx?sort=popular │ 3 │ -│ http://afisha.yandex.ru/manga.ru/?rtext=\xD0\xBF\xD1\x80\xD0\xB8\xD0\xB1\xD0\xB... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://saint-peter │ 3 │ -│ http://whoyougle.com/iframe/iframe_right.ru/spb │ 3 │ -│ http://lazarevskoe │ 3 │ -│ http:%2F%2Fwww.bonprix.ru/imagesize │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://un1.adriver.ru/page=30138117749516%252f110916%252fmedicinema/movie_ross ... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_product_brand=4200&lo=h... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://klubnich/zrh/ │ 3 │ -│ http://smeshariki.ru/furniture.html5 │ 3 │ -│ http://wildberries.ru/comment/search │ 3 │ -│ http://ssl.hurra.com/iframe-owa │ 3 │ -│ http://myloveplanet.ru/passenger/kitched_country_id=4312&input │ 3 │ -│ http://cxem.net/324487194836848 │ 3 │ -│ http://saint-petersburg-gorod/goodal │ 3 │ -│ http://sunmar.ru/cart&ch=utf-8&sF=11,7,700&aN=Opera&aV=9.80 (Windows │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://wildberries.ru/comme%2F2.12.388 Version/1552/page │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&film/67... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://irr.ru/index.php?showalbum/logizer8/num-1/refremost │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://pogoda.yandex.ru/real-estate/apartments/73151 │ 3 │ -│ http://gotovim-doma.ru/personal/commersant.ru/image=19&pvno=2&engineVolum │ 3 │ -│ http://pogoda.yandex.ua/telefon_shtukaturkey │ 3 │ -│ http://krasnodar.irr.ru/yaransferapid │ 3 │ -│ http://jobs-education │ 3 │ -│ http://bdsmpeople.ru/index.by/ru/page=0&confiscategory_id │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://auto_id=0&color=0&confiscategoriya%2Fzhienskaya-advert25593 │ 3 │ -│ http://afisha.yandex.ru/sell/resident │ 3 │ -│ http://afisha.yandex.php?p=176d43f96ef32d5bc1272 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://forum/topics/ingradskazka-lookoformalities/poetry/events/?date&csrf-8200... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://video_dvd/suppoll/dleead6718.php?SECTION │ 3 │ -│ http://video=0&input_with_video.yandex.ru/page/10/women.aspx │ 3 │ -│ http://radioscannerica/film/47018.html?1=1&cid=691390&pvno │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://msk/platia-nashing/vanny.diary.ru/otdam_daily │ 3 │ -│ http://direct.yandex.ru%2F&sr │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=&is... │ 3 │ -│ http://kinopoisk.ru/cgi-bin/click.cgi%3Fsid%3D1216 │ 3 │ -│ http://irr.ru/index.php?showalbum/login-do-500-kv-m-Maya │ 3 │ -│ http://kinopoisk.ru/real-estate=2013-07-2089241607/photo=0&with_exchangeType │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/used/KIA-Cee-d... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://maps#ru_5_ru_1_ru_ru_ru_202_ru_1_ru_ru_ru_ru_2_ru_1743.html?oscsid=36303... │ 3 │ -│ http://rukodel=0&sort=newly&trafkey=27065/2/child.aspx#location │ 3 │ -│ http://zapchastny_fashing_mashes/index.ru │ 3 │ -│ http://myloveplanet.ru/bad_by_sidential/nizhnieiewva88 │ 3 │ -│ http://tyva-5/country=-1&washestvo/den_sidentialAmount │ 3 │ -│ http://chehod/zvenia8312&input_who2=1&input_age1=35&aN=Netscape │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory │ 3 │ -│ http://arma/frl-4/travel.ru/moscow/details │ 3 │ -│ http://rsdn.ru/info_all=yes&razdumy-i-trikshop/search&_h_page/1719920 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/procoolonelopitered/ministrict/3d-probeg-340-... │ 3 │ -│ http://ssl.hurra.com/iframe/iframe-owa.html%26custom%3D%26custom │ 3 │ -│ http://smeshariki.ru/news/2013/peshnye-udivlekanka.ru/l_03_00/bodreamfood.ua │ 3 │ -│ http://mylove.ru/zoom.php?GID=2&IsOrder │ 3 │ -│ http://zagranimals-planet.ru/user/31059&Module │ 3 │ -│ http://kuharka=48&modeloveplanet │ 3 │ -│ http://avtoto.ashx/1001087496197797217530729; Media Centernet_mastersburg │ 3 │ -│ http://sendflower │ 3 │ -│ http://whoyougle.ru/basket&ch=utf-8&sF=11,7,7,7,700 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://bdsmpeople.ru/Web/price │ 3 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://smeshariki.ru/index.ru/main.pl?cmd=show/47555 │ 3 │ -│ http://afisha.yandex.ua/index.ru/recipe │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://auto_operey-v-v-meha.ru/catalog/public │ 3 │ -│ http://votpusk.ru/ │ 3 │ -│ http://myloveplants_list-obl.irr.ru%2Fobuv-zhienskaia-modiezhda │ 3 │ -│ http://video=0&is_hot=0&che_simeis │ 3 │ -│ http://forum/topnews/222968695,910112_1164074834-908745 │ 3 │ -│ http://myfashihtzu.html?1=1&cid=65625f313230303&po_yers=2013/07 │ 3 │ -│ http://video.yandex.ru%2F%2Fwww.bonprix │ 3 │ -│ http://irr.ru/6323%26bn%3D27888895,96772&op_page47 │ 3 │ -│ http://afisha.mail.ru/stars/page │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://smeshariki.ru/chipinfo │ 3 │ -│ http://kurort/SP1399&op │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://irr.htm?from]=&int[852][to]=10&lastdiscussins/?keyworld │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stories.ru/art/MACKLEMORE │ 3 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 3 │ -│ http://pogoda.turizm.ru/communit=0&vip=0&order_by=2 │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.php?w=3160/transports │ 3 │ -│ http://afisha.yandex.php?link=114735200&brand=23368 │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountry │ 3 │ -│ http://wildberries.ru/page=0&vip │ 3 │ -│ http://nizhnieie-bielie-bieriends&fb_source-temno │ 3 │ -│ http://forum/topnews/2229362067528195&op_categoriya │ 3 │ -│ http://smeshariki.ru/Web/price │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://zarplata.ru/?p=1290&op_product_price=990348531&schoosOSRquM8gE&where=all... │ 3 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&top │ 3 │ -│ http://radiorecord.ru │ 3 │ -│ http:%2F%2Fmuzhchine/ru-g-Chelya_v_tsentyabrskii │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://ej.ru/muss/roll_to_audi/mode=replies │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://ur.hh.ru/file/news/2013 \xD0\xB3\xD0\xBE\xD0\xB4\xD0\xB0 │ 3 │ -│ http://guid=6&pw=2&pv=0&po_yers=0&with_video │ 3 │ -│ http://gorbus.aspx#localiformalitic │ 3 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 │ 3 │ -│ http://my.alpari.ru/filmId=yJg89hqV8gE&where=all&film/530/?fromCityCodeForcentr... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&text=\x... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://pogoda.yandex.ru/chat/ivan │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://yartb.html?city=55&TopicID=2&IsOrderedProduct │ 2 │ -│ http://afisha.yandex.ru/?trafkey=54073799 │ 2 │ -│ http://news/398261_enl.jpg-1 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://poisk.ru/news/articles │ 2 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD1\x81\... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 2 │ -│ http://video.yandex.ru/rent/info/messa.ru │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://ssl.hurra.com.ua/auto_repairs=0&page=10&category_id=detail │ 2 │ -│ http://moscow/detail/Torgovljatory/storii_efferanslyatting │ 2 │ -│ http://kinopoisk │ 2 │ -│ http://video=0&input_state │ 2 │ -│ http://forum.php?t=420 │ 2 │ -│ http://mysw.info/node/21544 │ 2 │ -│ http://smeshariki.ru/tashkinsk │ 2 │ -│ http://smeshariki.ru/product&op │ 2 │ -│ http://yaroslavl.irr │ 2 │ -│ http://board=11.ua.150.html%3Fhtml │ 2 │ -│ http://pogoda.yandex.ru/jobinmoscow │ 2 │ -│ http://afisha.mail.ru/cheva.ru/reportby │ 2 │ -│ https://diary/2013-07-05/101090/currency │ 2 │ -│ http://slovarenok.ru/chapochki │ 2 │ -│ http://pogoda.yandex.php?showalbum │ 2 │ -│ http://kinopoisk.ru/saint │ 2 │ -│ http://krasnyj-chastory │ 2 │ -│ http://pogoda.yandex.ru/real │ 2 │ -│ http://3dnews.ru/msk/events │ 2 │ -│ http://sslow_13500000%26rnd%3D278888 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 2 │ -│ http://wildberrin/foton │ 2 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 2 │ -│ http://diary.ru/search?familliklink │ 2 │ -│ http://omsk.spb.ru/filmId=RUb3Hf2m0Cw&where=all&text │ 2 │ -│ http:%2F%2Fwww.bonprix.ru/search=0&deletedAutos&marketing │ 2 │ -│ http://kaluga/?ext=\xD1\x80\xD0\xB5\xD0\xB9 \xD0\xBD\xD0\xB0 \xD0\xB1\xD1\x80\x... │ 2 │ -│ http://pogoda.yandex.php?t=14555667679/?PAGEN_3=5&s_yers=0&numphoto/62232/7#f │ 2 │ -│ http://pogoda.yandex.ru&xdm_e=http://name=\xD0\x91\xD0\xB0\xD0\xBB\xD0\xB5\xD1\... │ 2 │ -│ http://auto_id=3159&input_age2 │ 2 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 2 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://guid=6&pw=2&pv=0&price=690&s_yers=1916 │ 2 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=2furii10.html?city=0&hide │ 2 │ -│ http://afisha.yandex.php?s=4d450&pid │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://afisha.yandex.php?city=418&view │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://wildberries.ru/editem_no=100¤cy=1#country=&op_proizvodskaya-obuv-z... │ 2 │ -│ http://wildberries.ru/filmId=4920/room=1&lang=all&filmId=DnGbyVQVUXI&wheretomug... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.040 user 0.372000 sys 0.008000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -┌────────────────┬────────────────┬─────────────┬────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ TraficSourceID │ SearchEngineID │ AdvEngineID │ src │ dst │ pageviews │ -├────────────────┼────────────────┼─────────────┼────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33069 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 24703 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 15817 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 14233 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 │ 6549 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 │ 5257 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 5257 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 3547 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login │ 3538 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik │ 3371 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php │ 3333 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 3309 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 2923 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 2616 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 2441 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php │ 2418 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 2390 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari │ 2367 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 2349 │ -│ 1 │ 0 │ 0 │ https://google.com.ua/url?sa=t&rct │ http://komme%2F27.0.1453.116 │ 2228 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 1920 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://komme%2F27.0.1453.116 │ 1679 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ekburg.irr.ru%2Fpuloveplanet │ 1340 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://aliningrad │ 1270 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1254 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1213 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1193 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1186 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1142 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F27.0.1453.116 │ 1007 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 954 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://aliningrad │ 859 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login │ 821 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 791 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 777 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 757 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 722 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://video.yandex.php │ 702 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 691 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/obucheyelants │ 684 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 632 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 593 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 577 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://smeshariki.ru/obucheyelants │ 566 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.php │ 452 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 445 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBA\xD0\xBE\xD0\xBD\xD1\x82\xD1\x80\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 422 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 421 │ -│ 0 │ 0 │ 0 │ │ http://obninsk/detail │ 407 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 386 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug.ru │ http://irr.ru/index.php │ 374 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru/index │ 360 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 348 │ -│ 2 │ 0 │ 13 │ │ http://komme%2F27.0.1453.116 │ 347 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-2011/43597 │ 293 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 293 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6HS │ 282 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 268 │ -│ 3 │ 72 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 267 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 259 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 257 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari │ 256 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 │ 241 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login │ 237 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 │ 236 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/28435&lr=157 │ http://komme%2F27.0.1453.116 │ 233 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/ru/photo/6936313555&text=\xD1\x85\xD0\xBA \xD0\xBB\xD0\xBE\xD... │ http://komme%2F27.0.1453.116 │ 224 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php │ 223 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cat/dushkirillovyj │ http://komme%2F27.0.1453.116 │ 222 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=359&op_page2/... │ 218 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kriminally-bezhevsk │ 217 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0 │ 212 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 207 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 202 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 199 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://ekburg.irr.ru%2Fpuloveplanet │ 199 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 198 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 197 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 195 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/414526863.xlsx │ 191 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CDMQFjAJ │ http://irr.ru/index.php?showalbum/login-kiriskaya-obl.irr.ru/index.ru/GameMain.... │ 190 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=0&with_photo.ashx/101/4/?cat=6257271 │ 188 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=partments-sale/search?text=\xD1\x81\xD0... │ 187 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://afisha.yandex.ru/index │ 183 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://ekburg.irr.ru%2Fpuloveplanet │ 171 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 168 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 164 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 163 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://ekburg.irr.ru%2Fpuloveplanet │ 160 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 160 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/life.ru/cars/misc/travel.ru/?trafkey=058143&p... │ 155 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 │ 154 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=577&oki=1&oby=&op_s... │ 152 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://wildberrior/uphold │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6HS │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 149 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 146 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustic/meterburg │ 144 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 144 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 144 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E │ 141 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 140 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 136 │ -│ 1 │ 0 │ 0 │ http://kombardighantnie │ http://komme%2F27.0.1453.116 │ 136 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login │ 135 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kysjacevtika │ 132 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://komme%2F27.0.1453.116 Safari │ 124 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 121 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category/stroy/dachines... │ 114 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/?state │ http://komme%2F27.0.1453.116 │ 113 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/GameMain │ http://komme%2F27.0.1453.116 │ 113 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 111 │ -│ 3 │ 85 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 110 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login │ 110 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kardigan │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 107 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosoftwarenok.ru/projects/zhbi.po... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 105 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268143.html?1=1&cid=577&o... │ 104 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 104 │ -│ 0 │ 0 │ 0 │ │ http://tvidi.ru/photo=0&confiscategory_id=0&engineVolumeFrom=&fuelRateFrom=type... │ 103 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 102 │ -│ 3 │ 14 │ 0 │ │ http://komme%2F27.0.1453.116 │ 102 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 100 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 98 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://video.yandex.php │ 97 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 96 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.21150895 │ 96 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?sort=price │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 95 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login │ 95 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.12.388 │ 93 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 93 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 92 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberrior/uphold │ 92 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 91 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-tanks/search=1&ady=62&modeloveplanet.ru │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 90 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.145332.15&he=10&category_id=2740387993 │ 89 │ -│ 1 │ 0 │ 0 │ http://in-the-weightEnd=2351&numphoto=&isExclusiver.ru/alertljus │ http://komme%2F27.0.1453.116 │ 88 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fprodazha_Italja_unit=1&av=1&nm=1&lang=ru │ http://irr.ru/index.ru/show/414526863.xlsx │ 87 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 86 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb1.html?item_no=386703/?bundle=7172&msid=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 86 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 │ 85 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 84 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733582852/ │ 84 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupit/action │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 82 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 82 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?bodystyle │ http://komme%2F27.0.1453.116 │ 81 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F22.14&he=768486 │ 81 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusk.ru/ru/lookaginitial/Prodayu-Dach... │ 80 │ -│ 3 │ 4 │ 0 │ │ http://komme%2F27.0.1453.116 │ 80 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 79 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://aliningrad │ 78 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 78 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login │ 76 │ -│ 1 │ 0 │ 0 │ https://go.1ps.ru/show&showforum │ http://komme%2F27.0.1453.116 │ 76 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandex.ru │ http://komme%2F27.0.1453.116 │ 76 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26221/detail.ru/v1496366&... │ 75 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php │ 75 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 74 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%2F&sr=http://bonprix.ru... │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len80/page/product │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filtr/all/perm.pulscen... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 73 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 73 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F1.7.1364.172 │ 72 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://diary.ru/forum/intries │ 72 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/show/414526863_112 │ 72 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2618561&pp=1059&op_produc... │ 72 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://video.yandex.php │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 71 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 71 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari │ 71 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://aliningrad │ 71 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=product_id=42&Selection.chelov.ru/searc... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/yandsearch?te... │ http://irr.ru/index.php │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/server=sc.chel.ru/main... │ http://irr.ru/index.php │ 70 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification/?year=\xD0\xB1\... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2791954~43.87725656132&op... │ 69 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://komme%2F27.0.1453.116 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/produkty/bleacs/udilis... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 69 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&wi=1366&br... │ 68 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 67 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731&CgID=124jc&where=all... │ 66 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/Yozh/Goodda │ http://komme%2F27.0.1453.116 │ 66 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2725215195&ti=\xD0\x9F\xD... │ 66 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://ekburg.irr.ru%2Fpuloveplanet │ 65 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru/index │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 64 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://afisha.yandex.ru/index │ 64 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 63 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nmg.com%2F4.0 Safari%26clients-sale/search&e... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 63 │ -│ 2 │ 13 │ 13 │ │ http://komme%2F27.0.1453.116 │ 62 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 62 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login │ 62 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://ekburg.irr.ru%2Fpuloveplanet │ 61 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 61 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26092.html?s_text=\x5C\x5... │ 59 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/thenon-houses/public/g... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27755f32316.30; .NET CLR ... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742/details/?cauth=0&dam... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fautoad/kniga.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26004/?_h=galle/\xD0\xBF\... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert27930555&sob=1&p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2692.html%3Fhtml?period=3... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/world/photo31469:Album... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=leting "HalUXI&where=\xD0\x9F\xD0\xB5\xD1\x8... │ http://irr.ru/index.php?showalbum/lofiver.ru/articles/86121%26uid%3D139750%26ad... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/koshka.com/ig/iframe-o... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27000,224648804-recept-so... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory=cinema.perm.p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/realty/suntime-5/extre... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742357/detskii_gosts.xml... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742304][from]=&int[14670... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694336/photo/suzannason/... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Forum25/top... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268320995,968650f45491882... │ 57 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupit/action │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1375605&ga... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://video.yandex.php │ 56 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga/choice/... │ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_product_... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?showtopic,5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26017/quarius_Moscow/cavi... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2726125413975d77cf&search... │ 55 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 54 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266848223/review_type=pro... │ 53 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 53 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26989%26bt%3Dad.adriver.r... │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/album/login-1800002&pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 52 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0 │ 51 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2698172,93932353064614618... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605085268a9c4d61-9862.49... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19&numphoto/login=A-CL-MS-36575c72937][to]=&int[12822304 │ http://irr.ru/index.ru/widgetchrome%2F&ti=no&dom_v_bordovye-printime.ru │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/investate/comple/make/?page5/&docid=jlMNIrXw... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 50 │ -│ 0 │ 0 │ 0 │ │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 50 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F11.7.1364 │ 50 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 50 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26051.htBeg=6&NightRegist... │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xBE\xD1\x81\xD0\xB8\xD0\xB1... │ 49 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 49 │ -│ 5 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 49 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?group=days=3&text=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27410/photo-2.xhtml&serve... │ 48 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandsearch │ http://komme%2F27.0.1453.116 │ 48 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 48 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusty-i-vkont.at.ua/search=0&userId=0... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266128182&op_seo_entry/de... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?sort=&br... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/intrumen │ 46 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://afisha.yandex.ru/index │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumanegenre=33155?analog/kitchen_mini... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login%2Flick.g.doubleclick.diary.ru/car/kw/3061/us... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://afisha.yandex.ru/index │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734.html_params%3Dfh_loc... │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/GameMain.p1Yo4A │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ua/searchads/jo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605906/frl-2/sportal.ru/... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fshow/lpp/cre.ru/pers... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.asp?search&ev... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logize.ru/msk/events_liver.ru/russert-plies.r... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26099486633033&countpage=... │ 44 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://sslow_135000008&position=search │ 43 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 43 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischet-solik/odezhda-plos... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert277553/38021/66936575776/... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271050&with_photo.kurortm... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 43 │ -│ 1 │ 0 │ 0 │ http://forums/liii-kuler S22oHgBJTngegotavgorod55.ru/filtr[2]=42 │ http://komme%2F27.0.1453.116 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-folders/?end=9a08488... │ http://irr.ru/index.php?showalbum/litamak.irr.ru/catalog/8570/travel.ru/cgi-bin... │ 41 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 41 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 41 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/0001216629/#top_by │ 41 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26040931&s_yers=0&with_vi... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://ekburg.irr.ru%2Fpuloveplanet │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.137508&s_yers=200... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://aliningrad │ 40 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logito8ergo_6470/page_type_id=20872/0/001:12:... │ 40 │ -│ 0 │ 0 │ 0 │ │ http://afisha.mail.ru/dmitrij │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/readar-nashi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27924563724&key=46960/med... │ 40 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://lib.ru/exp?sid=3205&bt=7&bn │ 40 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 39 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credit │ http://komme%2F27.0.1453.116 │ 38 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=8570/page=5&s_yers=2006 │ 38 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2679/detail/5482,935033/2... │ 38 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 38 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login │ 37 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dnevnoy-kv-Samara.irr.ru/searchAutos&marka=4... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 37 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://komme%2F27.0.1453.116 │ 37 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=theating&page=... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?group │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2784389/room=39695,966681... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/forum/topicID=269&stat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/votkim-rukav-i-kova-ul-adve... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filmId=xFX1UbTNJjxe4yF... │ http://irr.ru/index.php?showalbum/login.asp?razdel7/test/matched=115909d9_dsc07... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wolf-club478561/OPEL | \xD0\xB0\xD0\xB2\xD1\... │ http://irr.ru/index.php?showalbum/login-marka=13&city&custom=0&damages/0001216 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27553&s_yers=0&po_yers/46... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert275226607660?design=6efxq... │ 36 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=show=&fgroup-tab-mara.irinables/yearT... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278171&size=52094&op_seo_... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27330528/photo/6223799%26... │ 36 │ -│ 5 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://smeshariki.ru/obucheyelants │ 36 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36&he │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=post_neu%3D22%26pz%3D0%26rleurl%3D%26xp... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 35 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/8569 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/index.ru/katego... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vip.ru/book.com/photo/7086/page=0&state=mont... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-cukickdrops_bak_yant4tlGtZgy3BQ&bvm=bv.49478... │ http://irr.ru/index.php?showalbum/login-501-98552&op_uid=1060948/6#f │ 35 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (Windows │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19&m_staraya-obl │ http://irr.ru/index.ru/show/414526863_112 │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 34 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.12.388 │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 34 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_uid=1... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/promo=C-Tease/rent/dat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/list │ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 34 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/directadvert/kavanga_728x90.html │ http://komme%2F27.0.1453.116 │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 33 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshops.xml?typ │ http://komme%2F27.0.1453.116 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?cPath=default.ru/yandex.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33 │ -│ 1 │ 0 │ 0 │ http://auto.ria.ua/\xD0\x9A\xD0\xBB\xD0\xB8\xD0\xBC\xD0\xB0\xD1\x82\xD1\x8C\xD1... │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 33 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://video.yandex.php │ 33 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 33 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser │ 33 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265748ba3f755671_1162684&... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertif?sle=85000 \xD0\xB4\xD0... │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364 │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/pub │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 32 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zxcvbnm1990&search/jobinmoscow.ru/yandex.by/... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements-obschaya │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fproduct/foto-4/login... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&op_category_id=1174]... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniturers_id=575386110 │ http://irr.ru/index.php?showalbum/login.exit/rem/php/board/10_kruizer_8267049&o... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiepochto_id=93231238][to... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://afisha.yandex.ru/index │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ssl.hurra.com/iframe │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischuk_Kerasivye-klavik/?... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/jobs-education/m_acces... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26791e3a7c543c9bf9ef5c0-9... │ 31 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt │ http://komme%2F27.0.1453.116 │ 31 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertificia/?letteo_985428161/... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_api.php?cid=41389/rooms/perm.pulscen.ru/sear... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 30 │ -│ 1 │ 0 │ 0 │ http://domchelov.html&lr=55&text=\xD0\xB4\xD1\x80\xD1\x83\xD0\xB3\xD0\xB8\xD0\x... │ http://komme%2F27.0.1453.116 │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoryID=12452929587/... │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 30 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 30 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268630533&site_ot=&price_... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-k608i.trashbox.ru/cars/page5/&docid=577&lr=2... │ http://irr.ru/index.php?showalbum/logii_58247.php?forum.borovichy77/page5 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2716/~3/150160947694,9745... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/products/?category_id=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2696374/#page/tab=com_avt... │ 30 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/topic │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605446518%2F&sr=http://e... │ 30 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.14&he=768&wi=1920 │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26979553039353936694&ch=u... │ 30 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://smeshariki.ru/obucheyelants │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-carava/fotok-8-steklove.ru │ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26928142&op_products/view... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kupibatikov/?recrete/personal/klimat.ru/work... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 29 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x9A\xD0\xBE\xD1\x80\xD0\xBE\xD1\... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/wedditeli_foto-2/__48/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertifications[0][150]=60&inp... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_countpage/1 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/click.net/paged=696&lo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274229696.0 (Windows)&bL=... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-14132&with_photo=1&srtime-zone.net/catalog/t... │ http://irr.ru/index.php?showalbum/login-sierra%2Fdlia-zhienskaia-moda-zhiensmed │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 29 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2664952&xdm_p=1#country=-... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704.html%3Fhtml5/v12/?fr... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734/fotographic_2017666&... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategoriya/zhienskaya-... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9649.html%3... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27350350394&ch=UTF-8&sF=1... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/small_mercial-shoppicc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_seo_entr... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-baza/gadge... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27425.3120%26ntype_id=2&r... │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 28 │ -│ 1 │ 0 │ 0 │ https://directions[0][1541769377921968 │ http://komme%2F1.7.1364.172 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13733097&op_categ... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert272291684/hasimage=0&view... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/mann japanel/?text=\xD1\x81\xD0\xBC\xD0... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269500469412216388/detail... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-my-sovies/bebybum.homestion bonprix.ru/... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikarbox=2&tz=9b81998&po_y... │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?lang=all&fi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-176-kakaya-obl.irr.ru/search?lr=191&ei=N6rrg... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\xA4\... │ 28 │ -│ 3 │ 95 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/anime-2/#album/l... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-eve-nn.ru/cars/passenger/\xD0\x92\xD0\x90\xD... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274236&pv=15&prr=8&sid=35... │ 28 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic/8939858-59929477/detailshop │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/41452626u1zIq0SGLXCI │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F22.14&he=768486 │ 28 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://ekburg.irr.ru/#lingvo │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678/page=2&marka=84&mode... │ 27 │ -│ 3 │ 2 │ 0 │ │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/search?cl4url=http://yandex.ru/online... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27013.ya.ru/sportalznakom... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert24061.jpg.html%3Fhtml?1=1... │ 27 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031343-sh-898675/sort=pri... │ http://irr.ru/index.php?showalbum/login-5158&jenre/slyudya_metroman.ru/GameMain... │ 27 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19195/offset=101&district=&city&with_photo=¤cy │ http://irr.ru/img/catalog/53485785/topic/8939850.php?page=3&marka=84&mm │ 27 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://obninsk/detail │ 27 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-oleginnesujeli-i-centum=etarget=search?clid=... │ http://irr.ru/index.php?showalbum/login-kupe.html_params%3Drhost%3Dad.adriver.r... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fhit_list.html?s... │ http://irr.ru/index.php?showalbum/login-irake-myservierk-supierts%2F&ti=\xD0\x9... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27928092_price=\xD0\xBC\x... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/mazda-dievochnye_avari... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-19.xhtml?city_to_one=All&film]/on/orders/810... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/1.19.mobile_photo=6001... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27630/?item_id=0&bodystyl... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ 1 │ 0 │ 0 │ http://bonprix_ru_11559&lr=12&usln │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lizova_ii_1112_1150&option[price_do=300... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E&wher... │ 26 │ -│ 1 │ 0 │ 0 │ http://yandex.ua/?target=search/tab=user │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://obninsk/detail │ 26 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detall/seconomicsmovinki │ http://komme%2F27.0.1453.116 │ 26 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/loginsk.irr.ru%2Fkategory_id=9204471-650/?_h=... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login-jjjekrju-s-pring=1&price[price=\xD0\xBC... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification.ru/messa.ru/rea... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lejandsearch?text=\xD1\x82\xD0\xB8\xD1\x85\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert21124631/?Search.php │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 26 │ -│ -1 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 26 │ -│ 3 │ 197 │ 0 │ │ http://komme%2F27.0.1453.116 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_category... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?messa.ru/worlds/page2/7277932582&text=\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 26 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-kupe-20010120652838799.html?n=7148.html... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemey-volvom80Cw&where=a... │ 26 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=607&state=4... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.22 (KHTML, like Gecko) │ 25 │ -│ 1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 25 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/people.ru/cars/passets... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731/?city=0&pvno=2&evlg=... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vivarki/redki_sdelaet-law.ru/produkty/kartgo... │ http://irr.ru/index.php?showalbum/login=driff/en-ru/?p=2#messages/00001216629 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.html&ei=GcXrUYe8F2DAU-RbuRlLMczoJ--5uDCx8 │ http://irr.ru/index.php?showalbum/login-132/#images/0000.h95/\xD1\x82\xD1\x8E\x... │ 25 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert264314953); ru) AppleWebK... │ 25 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://obninsk/detail │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search?text=myaccountry │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguard-payments-sale/se... │ http://irr.ru/index.php?showalbum/login-88i8LaDyEkCVv6-DhRfEDcw==&action │ 25 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/yandex.ru/?f... │ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 25 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.ru/film/46351/frl-2/bage │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605838.html5/v12/?from=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-link=1147373-1&p=1&p=2&text=\xD0\xBC\xD0\xBE... │ http://irr.ru/index.php?showalbum/logisterlingvo/#! │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694589/detail/555-javata... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741559&t=5204/make=Chrys... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=shops.html?id=1&body_type=\xD0\xBF\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781472265&op_seo_entry=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626rleurl%3D//ad.adriver... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685/price_ot=&price=\xD0... │ 24 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://diary.ru/forum/intries │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26685857947301_Zoryatiya/... │ 24 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-117031&op_category&op_seo_entry=&op_uid │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27399992.php/board,13.0) ... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671291&fridgets/2012&pri... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguar.ru/page4/?emain.a... │ http://irr.ru/index.php?showalbum/login=ogabass.ru/filmId=9WOqzzitive&view │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo/7... │ http://irr.ru/index.php?showalbum/login-kupit-plitka-s-korsiety/art/151/n4p/160... │ 24 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://video.yandex.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-annative-expressageId=0oVXx21hUXI&where=all&... │ http://irr.ru/index.php?showalbum/login-rybnovlevojj_mamountry=-1&type%3D0 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/intrumen │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kupalnaya_elit=&id=3&clid=9403&lr=1913849 │ http://irr.ru/index.php?showalbum/login-gallebia_zhurnalCall D.R.E.mp3ex.net/ph... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lolacake/tatyaka.html?1=1&cid=577&oki=1 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinglish.rus-l7-p70505-1... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/yandsearch?lr=1... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-yvaken/topic=7702.jpg.html?1=1&cid=577&... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=1&bc=3&ct=1&pr │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-547552/1/?sma=RAINBOW&op_uid=fdd1b9d2721728&... │ http://irr.ru/index.php?showalbum/login-kapusta-advertika/search/room=1&damage=... │ 23 │ -│ 1 │ 0 │ 0 │ http://auto_volkswagency=1&text │ http://komme%2F27.0.1453.116 │ 23 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgarials/commend/searchvage-2.html&dt=13734... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-tuniki_510... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263090024189-528619c2077/... │ 23 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/ru/index.ru%26bid │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333572620201709/page/10... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news/radio&clid=198555... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26939.html?page/3/#count=... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/kategory_id=5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2740974%26pz%3D0%26ar_sli... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 23 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/resultatya9176da22f1a521a5853.html&... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26600721376439322%26pz%3D... │ 23 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga_728x90.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://afisha.yandex.ru │ 22 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2697574697.html_param=0&u... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemennoke0070553995f27d6... │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-105vr/ │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=search.tutung/s... │ 22 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/wheel.ceratornaya_solnechka │ http://komme%2F27.0.1453.116 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-10760b31b65633eZateq eb806e887d9f15ccf593280... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2091&local=1&TypeSearch?mail.ru/yandsearch&c... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26058&nnum=s4746835895&op... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-the-ili-tempera-minsk.irr.ru/produkty/bluzki... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodny │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739693071/pomer... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2649.html?pg=menu_29.jpg&... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/v1465][from]=&int[157281 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267830840994,95688781470%... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-business/sale/ru/search?mail.ru/film%2Fbarna... │ http://irr.ru/index.php?showalbum/login-zakon_type=1&fuelRatesTypeSearch │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 22 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27200,55.7654.html_parat-... │ 22 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26601629][from=&fuelRateT... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&l... │ http://irr.ru/introlux_page5/2/page/Renaul.irr.ru/start=1500-rublic/gamemain/Tu... │ 22 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://ekburg.irr.ru%2Fpuloveplanet │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liliate_id=24145602&with_phoney.yandex.ru;ya... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-natashkarola.irr.ru%2Fviewforum/MsgList.html... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26900/technics-technics-t... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&city[7]=23239330794,25826044_Black_lis... │ 21 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?price_till beli.php?cx=015216684_36 │ http://komme%2F27.0.1453.116 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694,978825315f373400/det... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691_ru_17832523.html_par... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1) AppleWebKit%2F537 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/world/photo/41/62b1dfa450/3/women-clother%26tms%3D%26ev_ltx%3D... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/forum%2Fukhov/l... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27025&Select[4][]=32598.h... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/basketshop.ru/yandsear... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265228.html?1=1&cid=577&o... │ 21 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://wildberrior/uphold │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo44953.116.php?t=4... │ http://irr.ru/index.php?showalbum/login=ko&page=0&view.aspx?group_cod_1s=85&key... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/?from │ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/houses/passenge... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-dance/index.ru/?favorite_off=1&encoding&hl=\... │ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logino-s-grigerator/page1=&input_age1 │ 21 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cars/commeddesk.ru/google_s&12.5.746.59954.30 (KHTML, like/\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/logii-bin/click.cgi%3Fsid%3D158195,97987231-9... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26737865/?page/12186/#/ec... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ramblery/pic/893985650697... │ http://smeshariki.ru/obucheyelants │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login.html?1=1&input_who1=2&input_who2=1&inpu... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/club.ru/anime-zone.ru/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929&from%3D%26custom=0&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dietsik.ru/razdel_id=111,7,700&w=728x90.html... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-brie... │ 21 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739353/details.... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2186/detail.aspx#location... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification%2F4.0; XBLWP7; ... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kupalnik │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/9-0--navam-zhien... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741465][from]=&input_wit... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_uid=1147... │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login=uzpp2XnEw&bvm=bv.49784469][from │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 20 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-Larki-i-korobeg-1124-95367/guestblowinp... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26219&wi=1280&lo=http://w... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estatic.diary.ru/... │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=58&Selectronics-technics/k... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/yandex.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b706234651&op_uid=1&b... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?er=2&text=\xD0\xBA\xD1\x83\xD0\xBF\xD0\xB8\... │ http://irr.ru/index.php?showalbum/login=razer2.moikrug.ru/recipes/shop.spb │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&price │ 20 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://video.yandex.php │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=NeIdeaCenternet Explorer&aV=5.0 │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertificirovka.ru/noteating-i... │ 20 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271161076&st=261&t=170977... │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313163307/?from]=&int[15... │ http://che.ru/produkty_zarubezhei-niepochekhly │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate=0&ReturnUr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604370757034dea482207549... │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_api.php?id=7262882,9454472&op_product_n... │ 20 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/cars │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-karbox=0&top=0&category/1961/#images/00... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26637&model=1705.html?1=1... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumilora481 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-iulyanovskaya-obl/lyubvi?page=166733&eid=485... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27502/detail/?folders/#pa... │ 20 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt=13733129/room │ http://komme%2F27.0.1453.116 │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://video.yandex.php │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626u1zIq0SGLXCI&where=al... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=240&u_cd=242035165&input_who2=1... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news_id_72387877055/nu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?sort=popup/casualg... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikulina.ru/real-estate/ou... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 19 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/filmId=eRaB4pYAXCI │ http://komme%2F27.0.1453.116 │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.ru/daily │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBB\xD0\xB8\xD0\xBA\xD0\xBE\xD0\xBB\xD0\... │ http://irr.ru/index.php │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260540718534/?dateOn=1&bo... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login=perfect.yandex.ru%2Fkategoriya%2F537 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261921&pr=2441185112.html... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26672655462.html?city.ru/... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 19 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostok.ru/cars │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/check=1&redir=1&lang=a... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671&Destige-all.html_par... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27404895,95923.html%26cus... │ 19 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2638/?from=&raceTo=&power... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313437&pvno=2&evlg=VC,2;... │ http://irr.ru/index.php?showalbum/login-rizova.ru/look/timashirtsevm.cofe │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6HS │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166333/price[price_till]=&pri... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F_liveresum... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27355f3132366336601&uuid=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.php?f=11340.1j1j2.12...5998-103358071/8/page... │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/00007/10/1320006&p... │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto_repai... │ 19 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.15751 Safari%2F5.0 (Windows │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://smeshariki.ru/obucheyelants │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2675406828ac956028692581/... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26968794618/irr.ru/real-e... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://obninsk/detail │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2629&lr=65&rstr=-213&text=back=False&withmos... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2702483,631;IC,444041%2F&... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?search?text=\xD1\x80\xD1\x83\xD1\x81\xD0\xB... │ http://irr.ru/index.php?showalbum/login-inadlinny-sched_car=0&city=0&price=18&p... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=http://video.yandex.ru/real-estate │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26669E116-1-1550046560013... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274026181891&rn=52&brand=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/dlia-zhiensk... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-165&jenre]/49047&lr=163&text=samsung ga-moda... │ http://irr.ru/index.php?showalbum/login-2-komn-v-na-92f5-6ccf-fef3-013f9f926a21... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&checked=0&state/room=10002-g-v-program... │ 18 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ru/index │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5 │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?elmt=popular&where=all&film/723/num │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/restate/out-of-town-resh=1&cnt=1373... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2012-recept-Ragu-iz-2-x │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupaljinik-slidi/places/?PAGEN_1=2 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=deltec&type=citymo=&version%2F4.0 (Linu... │ 18 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27534.30 (KHTML, like Gec... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kochtinsk.pulscendinskiy-krasnodar.irr.ru │ http://irr.ru/index.php?showalbum/login-kuplyuskij/zapchastorii_state/out │ 18 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login.moikrug.ru/catalog/otchenko-ul-advert27... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zzzi.ru/a-shop and blademas/?id=1&marka=0&ma... │ http://irr.ru/index.php?showalbum/login-2003prev=/search/index.ru/meteor.html%3... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\... │ 18 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278453&with_exchange=0&ex... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=367132&pv=2... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27132069304;IC,1511e8d158... │ 18 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27071038_114736895,923401... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x9A\... │ 18 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/8018&l=malkogo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kanza_tab_2499472&lr=1074&sort=popular │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/?target=search?... │ http://irr.ru/index.php?showalbum/login-bigutensiysk.irr.ru/regular&bid=3205&bt... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27420780790931.html5/v12/... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-natura.ru/jobs-education=post_137336ID0... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-21102&text=\xD0\xB1\xD0\xB0\xD0\xB1\xD0\xBA\... │ http://irr.ru/index.php?showalbum/login.mospsy.ru/super-na-kostronics/tv-audio.... │ 18 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/obuv%2F │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685&numphoto=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertimeshariki.ru/cgi-bin/cli... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626p/ │ 18 │ -│ 1 │ 0 │ 0 │ https://diary.ru/magnitogorsk │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260908861%26point-peter=&... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sien-zhienskaya-i-kvartir.html?id=1955451-01... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1453.116 Safari%2F8536 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeSearch?text=\xD0\xB2\xD0\xBA\xD1\x83\xD1... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=yators/?jumping.ru/series.ru/priceup&page=3&... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga1788778.137329TVmbFNmbVNaR0YwW$UyNDg4fxS... │ http://irr.ru/index.php?showalbum/login-sienskaia-moda-zhienskikh_rabotjaga.ru/... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=36427929&stat... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinger/\xD0\x93\xD0\x90\... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929750%26ar_sliceid%3D2... │ 18 │ -│ 4 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691c352005&pvno=2&evlg=V... │ 17 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27422544/3/womens.ru/ufa.... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/cars/communalux.ru/dos... │ http://irr.ru/index.php?showalbum/loshaya-obl.irr.ru/film/35562C?analytics/kto-... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru%2Ffilm/4858?pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271527091/frl-2/bage-2.ht... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.103402.html%3Fhtml │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#7878... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=artira-Lyubyat │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.aspx#location │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=yes&y1=2013-07-07.ya.ru/5_search?text=\... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.15.html?1=1 │ 17 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://aliningrad │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266149_51928/detail.ru/ka... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%26rleurl%3D%26CompPath%... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2613168312.71 Safari%2F__... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2670308&po_yers=0&price_o... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=8&ch=UTF-8&ifr=1&av=1&nm=1&ved=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_Bolers-device.aspx#commersanta_premiery... │ 17 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6HS │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-dlia-zhiensmed.ru/el... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginyazevskaya%2F5.0 (Windows NT 5.1) Prestorygoo... │ http://irr.ru/index.php?showalbum/login-kupilomatic.aspx#location=rr&days&city&... │ 17 │ -│ 1 │ 0 │ 0 │ http://bdsmpeople.ru/register2123 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logistereda.ru/photo/70861/6#f │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263161366&bid=2823&numpho... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26041.htm%3Fsid%3D0%26pz%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowthreadreplies=24&ids=74&cu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27595,9305f3931339322%26n... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313&ussp=mstp&usln=1&inp... │ http://irr.ru/index.php?showalbum/login-3767436/roomootofile/obzor.com/iframe │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273363331251798796&op_seo... │ 17 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/stan/index.kz/yandex │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-search%3Futm_source=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031323532363&win=70&mode=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertistia.html?partment/searc... │ 17 │ -│ 3 │ 22 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ 1 │ 0 │ 0 │ http://bibieters_sig=5ea8QizwGSPy0yGcm14ATH74D4Bw&usg=AFQjCNHxMZlonova.ru/p1847... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://diary.ru/forum/intries │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_style... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261636/detail/odnoklassim... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lenfiki-zhienskaia-moda-muzhchin-sitafa ishq... │ http://irr.ru/index.php?showalbum/logical.ru/link=11485-9065383235348%2F&sr=htt... │ 17 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/messar.ru/professig │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-futbolki-k... │ 17 │ -│ 1 │ 0 │ 0 │ http://acase.php?input_who1=2&input_who2=1 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.10 │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733145625/details... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694&option%2F12.14&he=76... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-irina19910735%2F27.0.1453.116 Safari │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268814453957595,94406/det... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26618531&t=111246.html?p=... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=Like Gecko) Chrome%2Fpotnik=1&price_do=&curr... │ http://irr.ru/index.php?showalbum/login-the-poxudet_sponsor=&o=6000708498/?caut... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b76faact/less=1&ru=1&... │ 17 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-53584715,1327288/belgium/arch=1&themec.... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=bestwo.diary/details&id=9160/0/posti.ri... │ 17 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 16 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/category=cinem rasmusic │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098707209463401090/deta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267044005320AA76)&vendor_... │ 16 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search?clid=19554786.0.8.0.2.2003558_3497926... │ http://irr.ru/index.php?showalbum/logie-niz-doktops/electronics/4185570c72 │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-herokee_tatus=1&rm=18606bAxFEcQT6smBB4W... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27034195,9520160173/5/wom... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgas.rQRAX2Ua3IF1iHJcaz4ATv34DABw │ http://irr.ru/index.php?showalbum/login/?page=1024&wi=144438-88-3906740/currenc... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search/cuZXdzJTJGJmxyPTI4fQ,... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&pt=b&pd=7&pw=2&page5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-plan... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26396284046.xhtml%3Fhtml%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate/out-of-tow... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260612d7cf.4082813,74,756... │ 16 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login.aspx?sle=13&s_yers=0&page_type=0&door=0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/page=91552&q=\... │ http://irr.ru/index.php │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-krasok.html_params%3Drhost%3Dad.adriver │ 16 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/movinki... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnaja-prost-petersburg/detail/?fold... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?&clients-sale&siteurl%3D//ads/search?text... │ http://irr.ru/index.php?showalbum/login-kapusta-advert273786/foto-5/#photo=2817... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661232&streamfood.com/if... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiensmed.ru/dl/\xD0\xB7\x... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti/redmonds %2F offset=403134333.2201560&c... │ http://irr.ru/index.php?showalbum/login-v40-velopiter.feriod=0&scroll_to_auto.r... │ 16 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-57548811426&text=\xD0\xBB\xD1\x83\xD1\x87\xD... │ http://irr.ru/index.php?showalbum/login-coolonellana-Molodilnik.ru/index.php?vi... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166303133334%2F&ei=horia+iudi... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26765.html_params%3Drhost... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/70948/35.htm?id=... │ http://irr.ru/index.php?showalbum/logiy-luchaiev%2F&sr=http:%2F%2Fwww.bonprix c... │ 16 │ -│ 1 │ 0 │ 0 │ http://histore/profile;u= │ http://irr.ru/index.php?showalbum/login-kupe-20013&uuid=1373356164564_1sort/sta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=findjob.ru/planet.ru/index.ru/carinov.h... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2755597.html%3Fhtml?1=1&c... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/showtopic,803813.html&... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_catalog/1250/?itemsg/cd... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert260477/cities/visas/exp?s... │ 16 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://video.yandex.php │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27429&input_bdsm_position... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2728024/detail/508/?instv... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26887301/Zona/simferopol.... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-153-4b9c4.4331626.htm?size%3D0%26rleurl │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 16 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect │ http://komme%2F27.0.1453.116 │ 16 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://msuzie-shop/premiery-c-38208_2.html │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/posts&q=\xD0\xB2\xD0\xB5\xD0\xB4\xD1\x83\xD1... │ http://irr.ru/index.php?showalbum/login-mistore/#CATALOG_LIST-s-ovosibirsk/deta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-marka=23&model&desting?field_town/search/pri... │ http://irr.ru/index.php?showalbum/login-492ea9&show/417582,9626512874":10157628... │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/sch/price][min]=41&lr=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27364229586121%26url%3D%2... │ 16 │ -│ 1 │ 0 │ 0 │ http://volgografiyah_27_iyunya_50_let%2FgetId │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/tv-audi/a7-spec=9894797179698712/p1/8... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-maritkiN │ http://irr.ru/index.php?showalbum/login-ladimir.irr.ru/remiery-telefon.ru │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333%26bid%3D1%26rleurl%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661251-6efa-d61f-fef3-01... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new=127201148][fr... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/7427510/detail/5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti%2F%3Ff%3D100%&http://afisha.yandsearch&... │ http://irr.ru/index.php?showalbum/login-6030d.html_params%3Drhost%3D90%26height │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=view/5/item3963... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269899458/price/3815&pvno... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5.0 (Win... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logiste's a di galaxy-s4-story1.aspx#location... │ 16 │ -│ 1 │ 0 │ 0 │ http://yandsearch?lr=2&color │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-gorod/search?p=7&oprnd=9902.jpg&img_url=http... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27222.vk.me/u3166 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2753][from]=&pricedownloa... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new10152954vac │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2735030373434-152495.php?... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makeevka.ru/perm.irr.ru/slingvo/#1\xD0\... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313033393633310,932803][... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691.html?1=1&input_age2/... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26708662307][]=&selection... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-irinakai/page_num_read2306e4574&ei=Ot_rUfjPG... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2010000087378687/pic/8939375f303839303935373... │ http://irr.ru/index.php?showalbum/login-souse=2/path=39_1.html?1=1&cid=577&oki=... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumy.ua/search?text=\xD1\x81\xD0\xB5\... │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/?strict=5710... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/artira │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertists/736850/?item_no=737&... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2695/19545602cf94d77/repl... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikal.ru/search/room=1&dam... │ 16 │ -│ 1 │ 0 │ 0 │ http://forums/liiiervierk-suppohudeemvmesting/294465&op_categoriya │ http://komme%2F27.0.1453.116 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&model=1173... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=new&cad=rjt&fu=0&input_country_id=228&lr=961... │ http://irr.ru/index.php?showalbum/login-cam.shtml#objdesc=true&uA=Mozilla │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login/?do=ready-mansion%3D0%26url%3D%26bn%3D0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb=1&prepairs=0&city[1 │ http://irr.ru/index.php?showalbum/lounona-muzhchin-kupaljinik-chernyj-odnoe/san... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-4433140796851/detail/536180&all=False&i... │ 15 │ -│ 1 │ 0 │ 0 │ http://launcher-searchads/search │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781308;IC,238208836746/0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginsk.eu/ru/Pagesize=13&m1=07 │ http://irr.ru/index.php?showalbum/login-3muda/truction/vacancies/750207190765.h... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-album/login.2/second... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_produkty/kiev.org/forum... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26981583/page=1089592445/... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945HPS?analog/r10418/sort=price_do=200&site_off=1 │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F1.7.1364.172 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273389; U; Android 4.0.14... │ 15 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshop/id_art_type=7&s_yers │ http://irr.ru/index.php?showalbum/login │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273631253711/foto-6320166... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2724/?_h=search?text=\xD0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-goluboj-podserial&dfs=13 │ http://irr.ru/index.php?showalbum/login-haus.html#news/228670,257&pci=3012/frl │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#67 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266356163f39ea6f8/7a8745&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26605566254/room=1.6&wher... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/washes/built-in_two_ch... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?page37/?sta... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/state=1909644 │ http://afisha.yandex.ua/auto_id=0&with_photo.kurortmag.ru/razdnitsya-vishnury │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?Topic │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?transk.i... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginskikh-viana.irr.ru/GameMain.aspx?sle=1/house.... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://irr.ru/index.ru/show/414526863_112 │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-land_search?text=\xD0\xB0\xD0\xBC\xD0\xB1\xD... │ http://irr.ru/6323%26bn%3D27888895,963095425 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678&op_page=60000/curren... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list.html?typ=SMA&anbieter=aleks_evilkos.com │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/produkty%2Fplatj... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/index.ru/ange=0&s_yers=(3000&static.diary.ru... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2718599/photo=0&is_hot=0&... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?Id=9582 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-tumbov.irr.ru/page1=&input_who1=2&id=29... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=alini... │ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liloveplanet.ru/futurer533/women.aspx?group-... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2631920&lo=http://person_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.1Uv1UfqCA8SonYC4BQ&usg=AFQjCNGB3pBUuKY1jJPP3... │ http://irr.ru/index.php?showalbum/login-tank-sale/search%3Fmode=&page_type │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2699364224073532663835386... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.php?city[1]=700003 │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26761,55.654289600/detail... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27515/418695&st=327-B110Q │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login=vladimir/page_type=0&expand_search?text... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?dok=001&rightbox/week&From=0&user │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/frl-4/trailer/view/3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-lential/secondary/details9.html_params%... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page9/#14... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/gamemain.asp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-676216b8af/4fd00fa61b3185631821/page_ty... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=25127691%2F... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2730675595,9292fa-d61f-fe... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27877437494,943082&stat_t... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27265484158197%26width%3D... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienshchin-planet.ru/myma... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1955451&lr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/ch/floore troubleclick... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_appliazh-pliance/mista-bez-uchaiev/000202&cl... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Ford-antent... │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 15 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 15 │ -│ 3 │ 123 │ 0 │ │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginy_name/3196906481/currency=6465373200&price=\... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2776/?date=0&doorov.irr.r... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-102-821922237&arrFilter2013/08-iH4AT5zIGQDA │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_type=&freetao.diary.ru/sea... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nashinger/users/search?textilead&353%26ev_pl... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert262997385f32313335781094&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2693_763613.html_params%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/877/ru/buildin... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert276278-ploschaya_checked_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274298799461981/?date=129... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9584%26pz%3... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.10 │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 15 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credir=1 │ http://komme%2F27.0.1453.116 │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://ekburg.irr.ru%2Fpuloveplanet │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=on&input_onliner... │ http://irr.ru/index.php?showalbum/login.html%26custom%3D%26CompPath.2; WOW64; r... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/gosthelp.ru/auto.ria.... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=on&in... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga664.1721/deti74.ru/animals-planet.ru │ http://irr.ru/index.php?showalbum/login=K-SR-B-13-9635095,9661/9#f │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/litlip/saint-peter557932E-8C62-4917%26ad%3D64... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2651734&pt=b&pd=7&pw=1&pr... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search?text=\xD1\x84\x... │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 15 │ -└────────────────┴────────────────┴─────────────┴────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.126 user 1.188000 sys 0.004000 -D 07-01' AND (DATE '1970-01-01' + 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; -Run Time: real 0.014 user 0.188000 sys 0.004000 -D D (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND DontCountHits = 0 AND URLHash = 686716256552154761 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Run Time: real 0.019 user 0.264000 sys 0.000000 -D " = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) ORDER BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))); -Run Time: real 0.012 user 0.164000 sys 0.000000 -D -D -D -D -D '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(URL) > 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -┌──────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├──────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 102341 │ -│ http://komme%2F27.0.1453.116 │ 51218 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18315 │ -│ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16461 │ -│ http://irr.ru/index.php │ 12577 │ -│ http://irr.ru/index.php?showalbum/login │ 10880 │ -│ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 7627 │ -│ http://irr.ru/index.php?showalbum/login-kupalnik │ 4369 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 4058 │ -│ http://komme%2F27.0.1453.116 Safari │ 3021 │ -└──────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.064 user 0.792000 sys 0.000000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(Title) > 0 GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ Title │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ \xD0\xA2\xD0\xB5\xD1\x81\xD1\x82 (\xD0\xA0\xD0\xBE\xD1\x81\xD1\x81\xD0\xB8\xD1\... │ 122407 │ -│ \xD0\xA8\xD0\xB0\xD1\x80\xD0\xB0\xD1\x80\xD0\xB0\xD0\xB9), \xD0\x92\xD1\x8B\xD0... │ 82935 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA - IRR.ru │ 80958 │ -│ \xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\xB8 New Era H (\xD0\x90\xD1\x81\xD1\x83\xD... │ 39098 │ -│ \xD0\xA2\xD0\xB5\xD0\xBF\xD0\xBB\xD0\xBE\xD1\x81\xD0\xBA\xD1\x83 \xD0\xBD\xD0\x... │ 23123 │ -│ Dave and Hotpoint sport \xE2\x80\x93 \xD1\x81\xD0\xB0\xD0\xBC\xD1\x8B\xD0\xB5 \... │ 14329 │ -│ AUTO.ria.ua \xE2\x84\xA2 - \xD0\x90\xD0\xBF\xD0\xBF\xD0\xB5\xD1\x80 │ 14053 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA (\xD0\xA0\xD0\... │ 13912 │ -│ OWAProfessign), \xD0\xBF\xD1\x80\xD0\xBE\xD0\xB4\xD0\xB0\xD1\x82\xD1\x8C │ 10919 │ -│ \xD0\xA2\xD1\x80\xD1\x83\xD1\x81\xD0\xB8 - \xD0\xA8\xD0\xBE\xD1\x83\xD0\xB1\xD0... │ 10157 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.045 user 0.576000 sys 0.004000 -D AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 7479 │ -│ http://aliningrad │ 4791 │ -│ http://ekburg.irr.ru%2Fpuloveplanet │ 3584 │ -│ http://smeshariki.ru/obucheyelants │ 3064 │ -│ http://video.yandex.php │ 2887 │ -│ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 1084 │ -│ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 891 │ -│ http://afisha.yandex.ru/index │ 855 │ -│ http://sslow_13507.html?aspx?naId=6HS │ 521 │ -│ http://wildberrior/uphold │ 484 │ -│ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 289 │ -│ http://obninsk/detail │ 241 │ -│ http://diary.ru/forum/intries │ 208 │ -│ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 185 │ -│ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 185 │ -│ http://kurort/SINA, ADRIAN │ 157 │ -│ http://afisha.yandex.ru │ 132 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 124 │ -│ http://ssl.hurra.com/iframe │ 123 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 119 │ -│ http://stars-visa.html_params%3Drhost%3Dad.adriver.ru/catalog.php │ 105 │ -│ http://komme%2F27.0.1453.116 │ 83 │ -│ http://pogoda.yandex │ 80 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 79 │ -│ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 73 │ -│ http://video.yandex │ 71 │ -│ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 68 │ -│ http://wildberries.ru/daily │ 68 │ -│ http://smeshariki.ru/ru/index.ru%26bid │ 68 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 64 │ -│ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 64 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn │ 63 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 59 │ -│ http://sslow_135000008&position=search │ 58 │ -│ http:%2F%2Fwwwwww.bonprix.ru/topic │ 53 │ -│ http://sslow_13507.html/articles │ 52 │ -│ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 51 │ -│ http://rsdn.ru/rss.ya.ru/catalog │ 51 │ -│ http://ekburg.irr.ru/#lingvo │ 46 │ -│ http://pogoda.yandex.ru │ 45 │ -│ http://sslow_13500000%26rnd%3D2788881.html │ 44 │ -│ http://maps#ru_5_ru_22106.377648194,975924][to]=&int[14270pa106&op_uid=17759/6#... │ 42 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 37 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 37 │ -│ http://video.yandex.ru/film/46351/frl-2/bage │ 36 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 35 │ -│ http://direct.yandex │ 34 │ -│ http://sslow_13507.html?aspx?naId=3X_3bhLcs3M │ 33 │ -│ http://gotovim-doma │ 32 │ -│ http://che.ru/produkty_zarubezhei-niepochekhly │ 31 │ -│ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 30 │ -│ http://video.yandex.ru/Newsletter │ 29 │ -│ http://sslow_13507.html?aspx?naId=6D8IzMGys3M │ 29 │ -│ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ http://video.yandex.ru │ 28 │ -│ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 28 │ -│ http://notes=1/currency │ 27 │ -│ http://sslow_13507.html?aspx?naId=649&state/out-of-town=\xD0\xA5\xD0\xB0\xD0\xB... │ 27 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 26 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 26 │ -│ http:%2F%2Fwwww.bonprix.ru/tambov │ 24 │ -│ http://kinopoisk.ru/shoppich.ru/search?clid │ 24 │ -│ http://wildberries │ 23 │ -│ http:%2F%2Fwwwwww.bonprix.ru/searchAutoSearch?text=\xD0\xB2\xD0\xB5\xD0\xBB\xD0... │ 22 │ -│ http://myloveplanet │ 20 │ -│ http://stars-visa-litraj.txt","lpu":"http://pogoda │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/secondary │ 20 │ -│ http://msuzie-shop/premiery-c-38208_2.html │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://smeshariki │ 19 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 19 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ http://irr.ru/6323%26bn%3D27888895,96772,97436 │ 18 │ -│ http://kinopoisk.ru │ 18 │ -│ http://nepogoda.yandex.ru%2Fproducts/search?text=subscripts/busineshop │ 17 │ -│ http://wildberries.ru │ 17 │ -│ http://kinopoisk.ru/catalog/9902224 │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://direct.yandex.html │ 17 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 17 │ -│ http://irr.ru/6323%26bn%3D27888895,963095425 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 16 │ -│ http://afisha.yandex.php?gidcar=36281664 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://auto.ria.ua/search │ 15 │ -│ http://direct │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ http://irr.ru/index.php?showalbum/login-kupalnaya-obl │ 15 │ -│ http://kinopoisk.ru/odessya │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 14 │ -│ http://ekburg.irr.irr.ru/maker │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.yandex │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.mail/rnd=0.9788 │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://auto_many_to_auto.ria.ua/igrush43/ │ 13 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 13 │ -│ http://afisha.yandex.ru/cars │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 13 │ -│ http://love.ru/a-myprofi │ 13 │ -│ http: │ 13 │ -│ http:%2F%2Fwww.bonprix │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://real-estate/aparther/offiliates/corruption/russinsk │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://on-online=on&accetti │ 12 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 12 │ -│ http://samara.irr.html5/v12/?from]= │ 12 │ -│ http://en.lyrsenses/zamba_zaborah_coldplay=1&gearbox │ 12 │ -│ http://kinopoisk.ru/saledParams │ 12 │ -│ http://zvukovo/hondar/2007&state/renlew/rigma.ru/scribed │ 12 │ -│ http://pogoda.yandex.php?SECTION │ 12 │ -│ http://msk/platia-nashing/vanny.diary.ru/moscow │ 12 │ -│ http://video.yandex.ru/GameMain.E6smreQhiu_hXR4&where=all&film │ 12 │ -│ http://nizhnieiene/p17378705/currency=1&with_photo-ideapadeno │ 11 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 11 │ -│ http://news/6483731559676/Unlocknotebooks/m83/800_D_Black_list │ 11 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty/photo-12/#imag... │ 11 │ -│ http://msk/platia-nashing/vanny.diary.ru/sale/liver │ 11 │ -│ http://video.yandex.ru/GameMain.aspx#location │ 11 │ -│ http://guid=6&pw=6&pv=13 │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 11 │ -│ http://wildberries.ru/rost.html?1 │ 11 │ -│ http://smeshariki.ru/?win=82&stat=1&page/196264&pt │ 11 │ -│ http://pogoda.yandex.ru/catalog/jokers │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 11 │ -│ http://en.lyrsenses/zamba_zabudtrimazok.html?page=12&prr=http://fap1.adrive_typ... │ 11 │ -│ http://bonprix.ru%26bid │ 11 │ -│ http://loveplanet.ru/GameMain │ 10 │ -│ http://bdsmpeople.ru │ 10 │ -│ http://liver.ru/cheboksicily/foto.aspx?sort=newly&trafkey │ 10 │ -│ http://video.yandex.ru&pvid │ 10 │ -│ http:%2F%2F%2Fwwww.bonprix │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://club.ru/spokoiteli/photo37775280000 │ 10 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 10 │ -│ http:%2F%2Fwwww.bonprix │ 10 │ -│ http://auto_id=240&n=13901038 │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://smeshariki.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 10 │ -│ http://slovariant_new3077940810/detail │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://en.lyrsenses/zamba_zabor_id=1012_blank%26site │ 10 │ -│ http://sslow_13500000%26rnd%3D2788881.html?parts/passe │ 10 │ -│ http://pogoda.html%3Fhtml_params%3Drhost%3D43 │ 10 │ -│ http://irr.ru/index.php?showalbum/logout%26verstova.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://pogoda.yandex.php │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film/6781203.html?id=242037047/detai... │ 9 │ -│ http://mysw.info/blog/sankt-peter%3D1216/00001216629 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://e96.ru/albumfotok-15-fotki │ 9 │ -│ http://alpari.yandex.html?html_param=0&users/#page/Search/ab_dob%2Ffieiie-razvo... │ 9 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://bonprix.ru/catalog/8570/1006790 │ 9 │ -│ http://b2b.testered/main/discuss/matched_country=-1&top=0&cityid=1024&wi=1366&o... │ 9 │ -│ http://svpressa.ru/topic=17082630 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://bonprix.ru │ 9 │ -│ http:%2F%2Fwwww.bonprix.ru/filmId=8j5j97LRs3M&where=all&sources │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/#win_13.html_p... │ 9 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/4744089758 │ 9 │ -│ http://nail=Yes&target=search │ 9 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_produkty%2F&sr=http://slovaria │ 9 │ -│ http://rlsnet.ru/vacancy/view_type_id=9677548268010367 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ https://m.myloveplanet.ru/forum/abrika-kobelenie_nebestsenal │ 8 │ -│ http://irr.ru/imagecache/wm/2013&where=all&filmId │ 8 │ -│ http://poisk.ru/price_ot=&price_ot=&price │ 8 │ -│ http://zapchast.com/iframe-owa.html?1=1&cid=577&oki=1&op │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 8 │ -│ http://afisha.yandex.php?t=141880517 │ 8 │ -│ http://zarplata.ru/velika_all=\xD0\xBE\xD1\x82 │ 8 │ -│ http:%2F%2Fwwwwww.bonprix.ru/news/222974895&op │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://fore=3&marka=0&top=0 │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://direct.yandex.ru/catalog │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96777&oki │ 8 │ -│ http://video.yandex.ru&xdm_p=1#item/search │ 8 │ -│ http://en.lyrsenses/zamba_zabor/bedroomolsk │ 8 │ -│ http://zarplata.ru/?p=12977-B26358/currency=RUR/page=1080&wi=1024&lo=http://rzh... │ 8 │ -│ http://masterh4.adriver.yandex │ 8 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ==&page_avtomodules.php?f=100&ref... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://mastered/main.aspx#location=1&bc=3&ct=1&pr=60322056107100919/page5/?_h=s... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://omsk.mlsn.ru │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96779/87 │ 8 │ -│ http://che.ru&pvid=13733142835/100/topic,5240556895&ch=UTF-8&sF=11,7,7,0 │ 8 │ -│ http://smeshariki.ru/GameMain │ 8 │ -│ http://afisha.yandex.php?gidcar=367108851%2Fr%2F1 │ 8 │ -│ http://wildberries.xml?from]=&input │ 8 │ -│ http://love.ru/?p=17059 │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/index.php?showalbum/login-kupit-topy%2Fplatjie-gotovlexandex.html... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 8 │ -│ http://auto.ria.ua/auto_id=24126629/0/index.ru/real-estate/out │ 8 │ -│ http://victor?page_type=city.stol-yar.ru/cars │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://video.yandex.ru%2F&sr=http://loveplants/65398f55 │ 7 │ -│ http://kinopoisk.ru/ch/feed/letniaiaprice/1609 │ 7 │ -│ http://real-estate/aparts/Aquarevski │ 7 │ -│ http://video.yandex.ru/a-album/login-vitiju/photo │ 7 │ -│ http://bonprix.ru/social/product_id │ 7 │ -│ http://moscow/detail/5552/0/2792834&m=111,7,7,5 │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 7 │ -│ http://irr.ru/Registered/main/topnewsru.com/page=6 │ 7 │ -│ http://irr.ru/index.php?showalbum/login-kapustics?sort=pogoda.yandex.ru%26bt%3D... │ 7 │ -│ http://bonprix.ru/catalog/8570/14139489 │ 7 │ -│ http://afisha.yandex.ru/\xD0\xB4\xD0\xBE\xD0\xBC\xD0\xB0/\xD0\x91\xD0\xA1\xD0\x... │ 7 │ -│ http://irr.htm?from]=&int[85][from]=&input_vsegodnyie │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://love.ru/?p=1#country=&op_seo │ 7 │ -│ http://bdsmpeople.ru/niktory/shtory/308/roomed.ru/p59473682740295 │ 7 │ -│ http:%2F%2Fwwwww.bonprix │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 7 │ -│ http://myloveplantrackIt?tid │ 7 │ -│ http://spb/event=big&marka=84&model=0&auto_id=0&s_yers=0&pv=10&can_be_check_PP │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://pogoda.yandex.ru&pvid=1 │ 7 │ -│ http://irr.ru/index.php?showalbum/login-a-5-advert27114 │ 7 │ -│ http://video=0&is_hot │ 7 │ -│ http://bdsmpeople │ 7 │ -│ http://irr.ru/index.php?showalbum/login-zk34/pages/0001216629 │ 7 │ -│ http://svpressa.ru │ 7 │ -│ http://money.yandex │ 7 │ -│ http://gotovim-doma.ru │ 7 │ -│ http://afisha │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://video.yandex.ru/ekt │ 6 │ -│ http://omsk/evential/house.ru/catalog/kitchedule=213-606361653965283 │ 6 │ -│ http://video.yandex.ru%2Fkategory_id │ 6 │ -│ │ 6 │ -│ http://afisha.yandex.ru%2F%2Fmail/169 │ 6 │ -│ http://love.ru/?p=1#country=-1&sq_liver.ru/kyrgyzstan │ 6 │ -│ http://bdsmpeople.ru/film/64544.690022.rar.html_params%3Drhost%3D_black_list=0&... │ 6 │ -│ http://stars-varenok.ru/16745959680706/800_0.jpeg.html%3Fhtml5/v123593 │ 6 │ -│ http://afisha.yandex.ru/catalog=on │ 6 │ -│ http://bdsmpeople.ru/GameMain │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://v102.ru/investate/apartments-sale │ 6 │ -│ http://wildberries.ru/search │ 6 │ -│ http://omsk/evential/housession%3D0 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://video.yandex.ru/GameMain.aspx#location/page_type=category │ 6 │ -│ http://afisha.yandex.ru%26bt%3D43%26anbietersburg │ 6 │ -│ http://direct.yandex.ru/refererprofile%2F2.10 │ 6 │ -│ http://video.yandex.ru/a-topy │ 6 │ -│ http://rukodel=0&sort=newly&trafkey=2750 │ 6 │ -│ http://moscow/details │ 6 │ -│ http://smeshariki.ru/world/photofider_credit=0&view │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://novosibirsk.irr.ru%26bid │ 6 │ -│ http://irr.ru/index.php?showalbum/login-12.html%26custom │ 6 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 6 │ -│ http://solutions.diary.ru/realtitroenie_v_jurman.ru/albums/frame-owa.html?stric... │ 6 │ -│ http://mr7.ru/newsru.com/iframe_right%3D43 │ 6 │ -│ http://autodoc.ru/real-estate/apart │ 6 │ -│ http://smeshariki.ru/catalog │ 6 │ -│ http://edp2.adriver.ru/hocketshop.ru/moscow/detailanude │ 6 │ -│ http://povari.yandex.ru/greecondary/Products_id=&auto_vaz_2111 │ 6 │ -│ http://video.yandex.ru/catalog │ 6 │ -│ http://ereal-estate/rent │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://msk/platia-nashing/vantralitsa_transion │ 6 │ -│ http://3dnewsru.com/iframe_right.html?1=1&cid=51538 │ 6 │ -│ http://afisha.yandex.php/board,39.04839 │ 6 │ -│ http://novo/detail.aspx?group_cod_1s │ 6 │ -│ http://bdsmpeople.ru/saledParams=rhost%3D43%26bid%3D1 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://kalininmyclonus1 │ 6 │ -│ http://forum/topnews/22294&op_category │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://auto_hyundai_sarator │ 6 │ -│ http://love.ru/?p=17055335 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://real-estate=week/page=1&expand_search?film/298677435615.html │ 6 │ -│ http://afisha.yandex.ru/mymail.php │ 6 │ -│ http://video=0&with_exchangeType │ 6 │ -│ http://afisha.yandex.ru/index.ru/recipe │ 6 │ -│ http://koolinar.ru/port.ru/doc │ 6 │ -│ http://kinel-lab.com/rus/20130709_117485994,93304&op_seo_entry=1&gearbox=0&type... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ https://slovakia-600dd903c07022,101595,91194&op_seo_entry │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://diary.ru/exp?sid=3205 │ 6 │ -│ http://video.yandex.php?search?text=\xD0\x9A\xD0\xBE\xD0\xBD\xD1\x8C\xD1\x8F\xD... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://video.yandex.php?from]=&interapy-wkti/ &cd │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://love.ru/ru/irk/event/search/ │ 6 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D8393224 │ 6 │ -│ http://kinopoisk.ru/catalog │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://auto_volkswagen/vologdano/il_dlya-dnevka.ru │ 5 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 5 │ -│ http://barnaul/details/?cauth │ 5 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD0\xB5\xD1\x80&where=... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://metal-lodku-Obzor │ 5 │ -│ http://sp-mamrostova │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 5 │ -│ http://tp66.ru/exp?sid=3860217/rooms=2/men.aspx#location%3D0%26rnd │ 5 │ -│ http://zvukovo-gorodsk │ 5 │ -│ http://rmnt.ru/search/offilia_Sovetov_living_chamber/?78142 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://e96.ru/movies/614418821/artir.ua/search │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 5 │ -│ http:%2F%2Fwwww.bonprix.ru/?id=2013&where=all&filmId │ 5 │ -│ http://video.yandex.by/?state_id=&auth=1..630;IC,7711588 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_seo_entry=&op_category_... │ 5 │ -│ http://povari.yandex │ 5 │ -│ http://slovari.yandex.ru%26orderovskij-index.ru │ 5 │ -│ http://tks.ru/cat/publish-cherkalnaya-ttpodporyadushek │ 5 │ -│ http://video.yandex.ru/GameMain.aspx?Link │ 5 │ -│ http://slovarenok.com │ 5 │ -│ http://auto.ria.ua/auto_id=241269.html?1=1&cid=2127970 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://rustnye-sht-riemnikoi │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://slovari.yandex │ 5 │ -│ http://afisha.yandex.php?r=23436303135353.html?1 │ 5 │ -│ http://auto_volkswagen-Palities/horobki │ 5 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 40007&pt │ 5 │ -│ http://en.lyrsenses/zamba_zaborah_chamberk │ 5 │ -│ http://radio&planet.ru/work.ru/catalog │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 5 │ -│ http://love.ru/product_id=0&po_yers=0&po_yers=2&refererro/model=1346488078722&c... │ 5 │ -│ http://topnews.ru/GameMain.aspx?group_cod_1s=1983&pt=b&pd=9&pw=0 │ 5 │ -│ http://myloveplanet.ru/v14572&lb_id=13000001216629 │ 5 │ -│ http://sslow_135000008&position=search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 5 │ -│ http://real-estate=week&m=Dvigenie.html?option │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://pogoda.yandex.php?gidcar │ 5 │ -│ http://autodoc.ru/moscow │ 5 │ -│ http://bibidohertki-i-OOOO_REPORT/07_2013 │ 5 │ -│ http://avtomobile/motory.ru/comp.ru/view.php │ 5 │ -│ http://loveplanet.ru/mymail/rudi │ 5 │ -│ http://video.yandex.ru/firms.turizm │ 5 │ -│ http://video.yandex.ru%2FkategoriendflowerTo=&powerTo= │ 5 │ -│ http://auto.ria.ua/auto │ 5 │ -│ http://msk/platia-nashing/vannyie-product_id=1841&page2 │ 5 │ -│ http://auto.ria.ua │ 5 │ -│ http://nizhnieiewva88/photo/101246465376&cmd=show-to-buchaiev-pugache=51db32a68... │ 5 │ -│ http://direct.yandex.ru/index │ 5 │ -│ http://tks.ru/cat/publish-chemec.ru/search?filmId=CktclMBmUXI │ 5 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D158197%26width │ 5 │ -│ http://club.ru/cinema/movies/no-pos │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://kinopoisk.ru/spb.pulscen.ru/exp?sid=3159&op_category_id=&auth=0&checked=... │ 5 │ -│ http://holodilnik-rp-ploschaya-obuv/?ci=1280&with_video=0&choosOyg==&op_uid=111... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://wildberries.ru/real │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://slovariant_neu%3D1%26bid%3D1216/0001216629%26bt%3Dad │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://md.mirkovskaya-obl.irr.ru/jobinmoscow │ 5 │ -│ http://wildberrior/kia/ │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://povari.yandex.php?showalbum/login.pl?cl=all&film/497794,90458 │ 5 │ -│ http://jcmotorom-921205&bt=7 │ 5 │ -│ http://loveplanet.ru/\xD0\xB0\xD1\x80\xD0\xBE\xD0\xBC/curre-ap-i-showalbum/loui... │ 5 │ -│ http://en.lyrsense.com/obshchin-idieiala │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://smeshariki.ru/topic │ 5 │ -│ http://video.yandex.ru%26bt%3Dad.adriver.ru/recipe/view/10217/?from │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://moscow/detail/Torgovuyu-organske.ru/real-estate │ 5 │ -│ http:%2F%2Fwww.bonprix.ru │ 5 │ -│ http://gaylyU │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://alib.mist.html%26custom%3D%26c2%3D278888592138 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://loveplanet.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 5 │ -│ http://alpari.ru/gallery/pic845274 │ 5 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 4 │ -│ http://loveplanet │ 4 │ -│ http://whoyougle.ru/images/images/00000i/specifiers.ru/image=1&furniture │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.ua │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 4 │ -│ http://video.yandex.ru/price │ 4 │ -│ http://bdsmpeople.ru/Web/Pages=1/feedsmag.ru/~\xD0\xBA\xD0\xBD\xD0\xB8\xD0\xB3\... │ 4 │ -│ http://irr.ru/index.php?showalbum/login-10618968476372773 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/filmId │ 4 │ -│ http://sslovarenok.ru │ 4 │ -│ http://afisha.yandex.ru/zoom.php?f=5162613838.html_partments │ 4 │ -│ http://auto.ria.ua/auto_id=0 │ 4 │ -│ http://samarskii_krai/tuapse/detail/result.aspx │ 4 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 4 │ -│ http://loveche.html │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://wildberries.ru/catalog │ 4 │ -│ http://afisha.yandex.ru%252f7769%252fe │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://echoradar-s-Levoshcha │ 4 │ -│ http://e96.ru/news/39733/page2=&input_sponsor=&o=1015219.html_partments-sale&pa... │ 4 │ -│ http://direct.yandex.php?formsof(INFLECTION_ID=30861/14365-4b11&state/apartment... │ 4 │ -│ http://love.ru/?p=1#countpage/vacancies/events/738/0/3/women.aspx │ 4 │ -│ http://diary.ru/catalog=on&input_city[3 │ 4 │ -│ http://radio&planet.ru/marka=62&model=1178128455&pvno=2&evlg=VC,5 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://rasp.pl?cmd │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 4 │ -│ http://travel.ru/state/apartments-sale/rashinitit%2F537 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ https://smeshariki.ru/cinema/article10363136000001216629%26site_offilia_Sovets.... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://afisha.yandex.php?r=3&bs=&day │ 4 │ -│ http://video.yandex.php?view_type=2&driveresult.ru/replies │ 4 │ -│ http://auto.ria.ua/auto_id=63799.html_params │ 4 │ -│ http://ssl.hurranovskaya-ul-31-foto.ria │ 4 │ -│ http://kinopoisk.ru/saledParams%3Drhost%3Dad.adriver.ru/GameMain.aspx#location │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://smeshariki.ru/domchelkakh_location=search?text=\xD0\xBC\xD0\xB8vents │ 4 │ -│ http://vkirovoe-tourisma │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboy07/... │ 4 │ -│ http://smeshariki.ru/search/keukeru-soft │ 4 │ -│ http://guid=6&pw=2&pv=0&price_do=¤cy=1 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://card/windows)&bL=ru&cE │ 4 │ -│ http://kniga.ru/view=\xD0\xA1\xD1\x82\xD0\xB0\xD0\xB2\xD1\x80\xD0\xBE\xD0\xB2&w... │ 4 │ -│ http://afisha.yandex.ru/search │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://v102.ru/?s=Adaments-sale │ 4 │ -│ http://video.yandex.ru/GameMain.XYyZwYXRoPWEtbG9nb24vcG90Cw │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/real │ 4 │ -│ http://matched_car=373838928155755775482794,9453.116 Safari%2F537.36&he=10&s_ye... │ 4 │ -│ http://zarplata.ru/?p=12977-B26358/hasimages=1/page │ 4 │ -│ http://video.yandex.ru/GameMain.aspx#location-4.0.html │ 4 │ -│ https:%2F%2Fwwww.yandex.ru │ 4 │ -│ http://tks.ru/cat/publish-chernyjbelyj-9375966238&op_categoriya │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/social │ 4 │ -│ http://video.yandex.ru/realty/leaser_map=1/hasimay-2.html │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://auto.ria.ua/canel_pe_mascona Hilfigeratov/153228 │ 4 │ -│ http://nizhny novgorod │ 4 │ -│ http://video.yandex.ru%26target=search/ab_area=categorija80119 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://e-kuzbass.ru/a-shop │ 4 │ -│ http://guide.travel.ru/link │ 4 │ -│ http://victorhead.php?full&dom=780067167694.0; │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://kubikus.ru/search.php?r=4140211,1,7,7,7,7,0 │ 4 │ -│ http://afishi,Mudanted-belyj-974299099/guest-id=34089.html │ 4 │ -│ http://fitness/building │ 4 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/cart.php/cars/papago-d-plosch │ 4 │ -│ http://svpress_up.aspx#location[propfilter_pf[PODVAL]=&arrFilter][3463351841195... │ 4 │ -│ http:%2F%2Fwwww.bonprix.ru/real-estate/out-of-town/house │ 4 │ -│ https://produkty%2Fpulove.ru/gost/?page3 │ 4 │ -│ http://afisha.mail.aspx#locationalOffers │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/GameMain/dukhovyy │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadbor_sh... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http%3A//magnitogorod/page3/#over │ 4 │ -│ http://anketka.ru/bridget │ 4 │ -│ http://afisha.yandex.php?action/2741920 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://tks.ru/cat/public/gamemain.aspx#location │ 4 │ -│ http://love.ru/image=2&marka=84&model/mihailovo │ 4 │ -│ http://nepogoda.yandex.ru/search=1&target=search=0&can_be │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://kinopoisk.ru/searchAutoSearch=0&driver.ru/catalog/1/women.aspx#locationp... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://afisha.yandex.php?show=rlv&ru=1&expand_search │ 4 │ -│ http://love.ru/?p=1705 │ 4 │ -│ http://ssl.hurra.com │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://afisha.yandex.php?p=31&input │ 4 │ -│ http://loveche.ru/job/1162323&PAGEN_1=30&state │ 4 │ -│ http://video.yandex.ru/If yours.avtogsm.ru/animals/calculate │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://auto_volkswagen_cated_cartovskaya-obl.irr.ru/album/login │ 4 │ -│ http://rsdn.ru/details_103514,154;IC │ 4 │ -│ http://internet Explorer&aV=5.0 (Windows)&bL=en │ 4 │ -│ http://auto_s_product_name=\xD0\x9A\xD1\x80\xD1\x83\xD0\xB3\xD0\xBB\xD0\xBE\xD0... │ 4 │ -│ http://pogoda.yandex.ru/cooking_sm5_1148786993ab-417/photo/69363/26#formi.ru/co... │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboyzon... │ 4 │ -│ http://video.yandex.ru/topnews.ru/lanas-advert2713][to]=&int │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http%3A//edp1.adriverys/forum/view_type=city&custom=0&damages/0001216629%26bid%... │ 4 │ -│ http://inspelishchin-platjie-doma.ru/irkutsk.irr.ru │ 4 │ -│ http://auto_volkswagen_pass_1161967 │ 4 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD1\x82\xD0\xBE\xD1\x8... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://afisha.yandex.ru/content2.adriver │ 4 │ -│ http://en.lyrsenses/zamba_zabudka/photo-1/#page=0&sale/search?text=\xD0\x9C\xD0... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://auto_id=0&with_photo │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://b.kavanga.ru/?a=inneVolumeFrom │ 4 │ -│ http://smeshariki.ru/saint-petersburg-gorod/transfer/?id=7576149959760994861&op... │ 4 │ -│ http://arma/frl-4/transportnoy-kv-m-malchik.ru/show │ 4 │ -│ http://bdsmpeople.ru/film/64544.690078 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://tp66.ru/money.yandex.ru/albums_screenterval │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://irr.htm?from]=&int[1151;IC,112 │ 4 │ -│ http://afisha.yandex.ru&pvid=13733424 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_category_id=592b9e01c48ce9403%26bn%3D0%2... │ 4 │ -│ http://video.yandex.ru/GameMain │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://radiorecord.ru/login-m3w.html?1=1&cid │ 3 │ -│ http://video.yandex.ru%2Fplata.ru/ch │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ https://orenburg/?arrFiltersburg │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://afisha.yandex.php/topic104780204&op_uid=1954 │ 3 │ -│ http://afisha.yandex.php/tova.ru/uliya2076789599305953 │ 3 │ -│ http://afisha.yandex.ua/donetsk/urals │ 3 │ -│ http://afisha.yandex.ru/a-folders/misc │ 3 │ -│ http://afisha.yandex.ru/shop.ru/malta │ 3 │ -│ http://afisha.yandex.ru/project_price=&maxprice │ 3 │ -│ http://afisha.yandex.ru/kategoriya%2F5.0 (company │ 3 │ -│ http://afisha.yandex.ru/linkvac.php/board.php?topicseeng │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://romar/events/audio.ru/widget[]=vacancies/99.php?gidcar │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://irr.kz/realty/lease/3516093&pvno=2&evlg │ 3 │ -│ https://produkty%2Fplatjie-kuzbass.ru/newsru.com/iframe_right=0&auto_ria=0&meta... │ 3 │ -│ http://fuckfind=rent/view/2021/3 │ 3 │ -│ http://banantikov-nadushenie_orleona_server=sc.cheloveplant_11612/page │ 3 │ -│ http://radioscannerica/filmId=Ba_id=13733568414&city=\xD0\x9C\xD0\xBE\xD1\x81\x... │ 3 │ -│ http://kinopoisk.ru/service.ru/iframe │ 3 │ -│ http://samara.irr.ru%2Fproduct │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://video.yandex.ru&pvid=13735/?_h │ 3 │ -│ http://ssl.hurral=messages │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ https://slovakia-600dd903c06c999c226647639.html%3Fhtml │ 3 │ -│ https://slovakia-600dd903c07022,101595,9143531427800648_elit │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category_id=0&wi=16000&... │ 3 │ -│ http://sp-money.yandex.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://love.qip.ru │ 3 │ -│ http://auto_kia_30 │ 3 │ -│ http://kaluga/?ext=\xD0\xB3\xD0\xB5\xD1\x80\xD0\xBE\xD1\x8F\xD1\x82\xD0\xBD\xD1... │ 3 │ -│ http://sp-mamrostokonkursovet │ 3 │ -│ http://afisha.yandex.ru/forum.materinburg │ 3 │ -│ http://afisha.yandex.php?addriver.ru │ 3 │ -│ http://sp-money.yandex.ru/work rushki-sien-natalog/8570/page=0&expand │ 3 │ -│ http://love.ru/forum.cofe.ru/forum/view_type=city=790&Selectronics-technik │ 3 │ -│ http://astrobank.ru/image │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://forum/topnews/2229605699574.html?1=1 │ 3 │ -│ http://video.yandex.by/search/?target%3D43%26bid%3D2 │ 3 │ -│ http://direct&sortdirect.yandex.ru/imagesize%3D0%26ar │ 3 │ -│ http://wildberrior/bedroom]=&int[17][to]=&int[858 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://rmnt.ru/film/88677/russia/rio.ru/search?filmId=NNr6aJrm4s3M │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://bdsmpeople.ru/search │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://rmnt.ru/stars │ 3 │ -│ http://b.kavanga.ru │ 3 │ -│ http://video.yandex.ua/auto_id │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http:%2F%2Fwww.bonprix_ru}%2Fnizhniynovgorod/request-id │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://edp2.adriver.ru/catalog/181 │ 3 │ -│ http://video.yandex.ru/circle&state/out-of-town/houses │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://pogoda │ 3 │ -│ http://nigma.ru/product&op_category_name=\xD0\x91\xD0\xB8\xD0\xBA\xD0\xB8\xD0\x... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://svpress_w1t1042796786/6/?category │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://radiorecord.ru/catalog/idShare │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://afisha.yandex.ru/forum.donfiscategory │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://edp2.adriver.ru/jobinmoscow/detail │ 3 │ -│ http://kazan.irr.ru/location │ 3 │ -│ http://pogoda.yandex.kz/family │ 3 │ -│ http://msk/planet.ru/mymail.aspx#comme_me_saydinne │ 3 │ -│ https://produkty%2Fpulove.ru/voronezh-sien-zhienskaia-moda-zhienskaia-moda-zhie... │ 3 │ -│ http://video.yandex.php?topbloveche │ 3 │ -│ http://card/windows NT 6.1) AppleWebKit%2F5 │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com/iframe-owa.html?1=1&cid=577&o... │ 3 │ -│ http://tks.ru/cat/publish-chin-play.php?categoriya%2Fzhiensmed │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26url%3D//ad.adriver.ru/link/justic/h2.php/... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty%2Ftanki │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/92054446660.ht... │ 3 │ -│ http://svpress/showbiz/photo.htm │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://loveche.ru/volzhskiy │ 3 │ -│ http://ussuriysk.irr.ru/catalog/premiere/628962851d7fd0b6eb17b321d336f5bc7de189... │ 3 │ -│ http://afisha.yandex.ru/tatatit_chto.php?industry │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://magnitka_1_series.ru/?favorite_id=636233644&op_category_id=937514 │ 3 │ -│ http://smeshariki.ru/a-phony │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://ekategoriya%2F9B206 Safari │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://smeshariki.ru/catalog/286/women.aspx │ 3 │ -│ http://svpressa.ru/content/search │ 3 │ -│ http://video_dvd/game/iframe-owa.html │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountpage │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://msk/platia%2Fzhienskaia │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://brand=498&pvno │ 3 │ -│ http://tp66.ru/search/?page=1&fuelRateTo │ 3 │ -│ http://lk.wildberries │ 3 │ -│ http://e96.ru/real-estate │ 3 │ -│ http://my.kp.ru/albumfoto-1/pol-2 │ 3 │ -│ http://holodilnik.ru/catalog │ 3 │ -│ http://povarenok.ru/files/eliteh.ru/perm.irr.ru/board,75.2013-07-09 │ 3 │ -│ http://personal/atlants/7292&xdm_c │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://smeshariki.ru/?win=82&stat=141882,373;IC,2552f48 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://sp-mamrostovestory │ 3 │ -│ http://love.ru/?p=1#country=&op_seo_entry=-1&target │ 3 │ -│ http://amobil-nye-pliazhnaia │ 3 │ -│ http://smeshariki.ru/goodavec/photo/6936325.html?id=223978/page=102 │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://love.ru/recipes/indows │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberries.ru/item_no=2&evlg=VC,0;VL,205;IC,14;VL,757138/currency=RUR/h... │ 3 │ -│ http://kaluzha-na-NovletedAutoSearch=0 │ 3 │ -│ http://irr.ru/index.php?showalbum/login-kupaljinik-chere │ 3 │ -│ http://irr.ru/washek-s-printom │ 3 │ -│ http://msuzie │ 3 │ -│ http://kinopoisk.ru/search │ 3 │ -│ http://smeshariki.ru/real │ 3 │ -│ http://afisha.yandex.php?app=membered │ 3 │ -│ http://afisha.yandex.php?ELEMENT_ID │ 3 │ -│ http://afisha.yandex.ru/lesyach-hotels │ 3 │ -│ http://3dnews.ru/?p=12636464/5#f │ 3 │ -│ http://afisha.yandex.php?id=727285 │ 3 │ -│ http://afisha.yandex.ru%26bt%3D90%26nid%3D1216629 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic=79799398/?_h=search.htm... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://str_ob.html?1=1&choosO8gPJSs3M&where=all&filmId=mAyiC7y6M2mGV2GoA9hFoN3q... │ 3 │ -│ http://product_brand=RAINBOW&op_cated_content/search/keup/en-ru │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory_id=577&search/ab_district/date_id=2271][from... │ 3 │ -│ http://love.ru/ru/irk/event=little&cated_country=-192.html?1=1&cid │ 3 │ -│ http://love.ru/?p=17057 │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostavia.travel.ru/s... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberries.ru/basket&ch │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://state/room=94720-recept-Salat-iz-glasya1lesyat │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://kinopoisk.ru/sessulyanovka.ru/photosessid=3205&bt │ 3 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD0\xB2\... │ 3 │ -│ http://li.ru/filmId=XpzlPj8P8gE&where=all&text=\xD1\x81\xD0\xBA\xD0\xB0\xD1\x87... │ 3 │ -│ http://you_hashkaf.ua/search │ 3 │ -│ http://auto_repairs=0&confiscategoriya │ 3 │ -│ http://wildberries.ru/GameMain │ 3 │ -│ http://povaria/chak_naytimes.ru │ 3 │ -│ http://afisha.yandex.ru%2Fobuv-sapozhkivka=23&price │ 3 │ -│ http://afisha.yandex.ru/hotel-agen-Goluboj-9730 │ 3 │ -│ http://notebooking pressa │ 3 │ -│ http://smeshariki.ru/GameMain.aspx#location │ 3 │ -│ http://afisha.yandex.ru/comment/search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://soft.oszone.ru │ 3 │ -│ http://pogoda.yandex.ru/real-estate │ 3 │ -│ http://auto_repairs=0&po_yers=0&price.ru/\xD0\xB6\xD0\xB5\xD0\xBB\xD0\xB5\xD0\x... │ 3 │ -│ http://video_dvd/ratesTypeSearch?text=\xD1\x87\xD0\xB5\xD0\xBB\xD0\xBE\xD0\xB2\... │ 3 │ -│ http://auto_id=0&engineVolumeFrom │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://kommersantamina │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://video.yandex.ru/personal/offeebe34c7e12944&op_product │ 3 │ -│ https://slovarenok │ 3 │ -│ http://pogoda.yandex.ru/vlas-moskovskaya │ 3 │ -│ http://slovari.yandex.ru │ 3 │ -│ http://afishers/story │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://v102.ru/investate/apartment/?id=137336IseNhcbx3J85GkHSnzgnsPdZUU&where=a... │ 3 │ -│ http://guid=6&pw=2&pv=0&with_video.yandex.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/photo/narod.irr.ru/katering=1&pr=569&s_yers │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://audio_video.yandex.ru/mosday.html?item=4#photo446962 │ 3 │ -│ http://alpari.yandex.ru/saint-petersburg.irr.ru/cars/page3 │ 3 │ -│ http://video.yandex.ru/extra │ 3 │ -│ http://dom.net/provoe-pervouralnaya/ChildGluZ19oZXJlci1kYXRpb25zPU4mbj0zJmlkPTM... │ 3 │ -│ http://trashbox.ru/book │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com.ua/kiev/detail.ru/auth=1..640... │ 3 │ -│ http://love.ru/ru/irk/event=little&category_id=731-643736&mode=1 │ 3 │ -│ http://auto_ford Mix).mp3.ucoz.ru/v1430497.html%3Fhtml │ 3 │ -│ http://state_shariki │ 3 │ -│ http://omsk/evential/housession%3D240%26rleurl%3D//ad.adriver.ru/marshavskaya-r... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://smeshariki.ru/cgi-bin/click.cgi%3Fsid%3D0%26pz │ 3 │ -│ http://kinopoisk.ru/registernet Explorer&aV=5.0 (Windows NT 5.1; ru-ru&cE=true&... │ 3 │ -│ http://video.yandex.ru%2Fkategory_id=9759527418 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberring │ 3 │ -│ http://video │ 3 │ -│ http://radiorecord │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://afisha.yandex.php?partments │ 3 │ -│ http://en.lyrsenses/zamba_zabor_polnitsa-s.narod.irr.ru/bank/otzyvy/12031%2F%23... │ 3 │ -│ http://auto_id=0&color=0&confiscategory_id=3205&bt=7&bn=1&bc=3&ct=1&prr=http:%2... │ 3 │ -│ http://balcon_caddy Club relove │ 3 │ -│ http://wildberries.ru/cgi-bin/click.cgi%3Fsize │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://video.yandex.ru/real-estate/out-of-town/house.ru&pvid=1&distreet_legkovo... │ 3 │ -│ http://loveplanet.ru/Bezli-all │ 3 │ -│ http://bdsmpeople.ru/show/39932/Itemid,40.0.html_params%3DfsSaHR0cDovL2pzLnNtaT... │ 3 │ -│ http://kinopoisk.ru/spb.pulscen.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&film/67... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_product_brand=4200&lo=h... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/manga.ru/?rtext=\xD0\xBF\xD1\x80\xD0\xB8\xD0\xB1\xD0\xB... │ 3 │ -│ http://smeshariki.ru/furniture.html5 │ 3 │ -│ http://ssl.hurra.com/iframe-owa │ 3 │ -│ http://saint-peter │ 3 │ -│ http://whoyougle.com/iframe/iframe_right.ru/spb │ 3 │ -│ http://lazarevskoe │ 3 │ -│ http:%2F%2Fwww.bonprix.ru/imagesize │ 3 │ -│ http://video.yandex.ru/realty/search/main.aspx?sort=popular │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://cxem.net/324487194836848 │ 3 │ -│ http://klubnich/zrh/ │ 3 │ -│ http://wildberries.ru/comment/search │ 3 │ -│ http://saint-petersburg-gorod/goodal │ 3 │ -│ http://sunmar.ru/cart&ch=utf-8&sF=11,7,700&aN=Opera&aV=9.80 (Windows │ 3 │ -│ http://un1.adriver.ru/page=30138117749516%252f110916%252fmedicinema/movie_ross ... │ 3 │ -│ http://myloveplanet.ru/passenger/kitched_country_id=4312&input │ 3 │ -│ http://wildberries.ru/comme%2F2.12.388 Version/1552/page │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://bdsmpeople.ru/index.by/ru/page=0&confiscategory_id │ 3 │ -│ http://gotovim-doma.ru/personal/commersant.ru/image=19&pvno=2&engineVolum │ 3 │ -│ http://krasnodar.irr.ru/yaransferapid │ 3 │ -│ http://jobs-education │ 3 │ -│ http://irr.ru/index.php?showalbum/logizer8/num-1/refremost │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://afisha.yandex.ru/sell/resident │ 3 │ -│ http://afisha.yandex.php?p=176d43f96ef32d5bc1272 │ 3 │ -│ http://auto_id=0&color=0&confiscategoriya%2Fzhienskaya-advert25593 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://pogoda.yandex.ua/telefon_shtukaturkey │ 3 │ -│ http://forum/topics/ingradskazka-lookoformalities/poetry/events/?date&csrf-8200... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://pogoda.yandex.ru/real-estate/apartments/73151 │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://video=0&input_with_video.yandex.ru/page/10/women.aspx │ 3 │ -│ http://video_dvd/suppoll/dleead6718.php?SECTION │ 3 │ -│ http://radioscannerica/film/47018.html?1=1&cid=691390&pvno │ 3 │ -│ http://smeshariki.ru/news/2013/peshnye-udivlekanka.ru/l_03_00/bodreamfood.ua │ 3 │ -│ http://msk/platia-nashing/vanny.diary.ru/otdam_daily │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/used/KIA-Cee-d... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=&is... │ 3 │ -│ http://tyva-5/country=-1&washestvo/den_sidentialAmount │ 3 │ -│ http://rukodel=0&sort=newly&trafkey=27065/2/child.aspx#location │ 3 │ -│ http://kinopoisk.ru/real-estate=2013-07-2089241607/photo=0&with_exchangeType │ 3 │ -│ http://maps#ru_5_ru_1_ru_ru_ru_202_ru_1_ru_ru_ru_ru_2_ru_1743.html?oscsid=36303... │ 3 │ -│ http://irr.ru/index.php?showalbum/login-do-500-kv-m-Maya │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://mylove.ru/zoom.php?GID=2&IsOrder │ 3 │ -│ http://zapchastny_fashing_mashes/index.ru │ 3 │ -│ http://rsdn.ru/info_all=yes&razdumy-i-trikshop/search&_h_page/1719920 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://chehod/zvenia8312&input_who2=1&input_age1=35&aN=Netscape │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory │ 3 │ -│ http://arma/frl-4/travel.ru/moscow/details │ 3 │ -│ http://direct.yandex.ru%2F&sr │ 3 │ -│ http://kinopoisk.ru/cgi-bin/click.cgi%3Fsid%3D1216 │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/procoolonelopitered/ministrict/3d-probeg-340-... │ 3 │ -│ http://ssl.hurra.com/iframe/iframe-owa.html%26custom%3D%26custom │ 3 │ -│ http://myloveplanet.ru/bad_by_sidential/nizhnieiewva88 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://smeshariki.ru/index.ru/main.pl?cmd=show/47555 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://afisha.mail.ru/stars/page │ 3 │ -│ http://whoyougle.ru/basket&ch=utf-8&sF=11,7,7,7,700 │ 3 │ -│ http://votpusk.ru/ │ 3 │ -│ http://kuharka=48&modeloveplanet │ 3 │ -│ http://avtoto.ashx/1001087496197797217530729; Media Centernet_mastersburg │ 3 │ -│ http://zagranimals-planet.ru/user/31059&Module │ 3 │ -│ http://sendflower │ 3 │ -│ http://smeshariki.ru/chipinfo │ 3 │ -│ http://afisha.yandex.ua/index.ru/recipe │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://auto_operey-v-v-meha.ru/catalog/public │ 3 │ -│ http://bdsmpeople.ru/Web/price │ 3 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://irr.ru/6323%26bn%3D27888895,96772&op_page47 │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://forum/topnews/222968695,910112_1164074834-908745 │ 3 │ -│ http://myfashihtzu.html?1=1&cid=65625f313230303&po_yers=2013/07 │ 3 │ -│ http://video.yandex.ru%2F%2Fwww.bonprix │ 3 │ -│ http://video=0&is_hot=0&che_simeis │ 3 │ -│ http://myloveplants_list-obl.irr.ru%2Fobuv-zhienskaia-modiezhda │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://pogoda.turizm.ru/communit=0&vip=0&order_by=2 │ 3 │ -│ http://stories.ru/art/MACKLEMORE │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://irr.htm?from]=&int[852][to]=10&lastdiscussins/?keyworld │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://afisha.yandex.php?w=3160/transports │ 3 │ -│ http://kurort/SP1399&op │ 3 │ -│ http://afisha.yandex.php?link=114735200&brand=23368 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://wildberries.ru/page=0&vip │ 3 │ -│ http://nizhnieie-bielie-bieriends&fb_source-temno │ 3 │ -│ http://forum/topnews/2229362067528195&op_categoriya │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountry │ 3 │ -│ http://smeshariki.ru/Web/price │ 3 │ -│ http://ur.hh.ru/file/news/2013 \xD0\xB3\xD0\xBE\xD0\xB4\xD0\xB0 │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&text=\x... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http:%2F%2Fmuzhchine/ru-g-Chelya_v_tsentyabrskii │ 3 │ -│ http://radiorecord.ru │ 3 │ -│ http://ej.ru/muss/roll_to_audi/mode=replies │ 3 │ -│ http://guid=6&pw=2&pv=0&po_yers=0&with_video │ 3 │ -│ http://zarplata.ru/?p=1290&op_product_price=990348531&schoosOSRquM8gE&where=all... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 3 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&top │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://gorbus.aspx#localiformalitic │ 3 │ -│ http://my.alpari.ru/filmId=yJg89hqV8gE&where=all&film/530/?fromCityCodeForcentr... │ 3 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 │ 3 │ -│ http://pogoda.yandex.ru&xdm_e=http://name=\xD0\x91\xD0\xB0\xD0\xBB\xD0\xB5\xD1\... │ 2 │ -│ http://krasnyj-chastory │ 2 │ -│ http://board=11.ua.150.html%3Fhtml │ 2 │ -│ http://wildberries.ru/index.ua │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 2 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 2 │ -│ http://masterlingvo/#I 1 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://news/398261_enl.jpg-1 │ 2 │ -│ http://auto_id=3159&input_age2 │ 2 │ -│ http://poisk.ru/news/articles │ 2 │ -│ http://sslow_13500000%26rnd%3D278888 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD1\x81\... │ 2 │ -│ http://pogoda.yandex.ru/images/0001216629/0 │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 2 │ -│ http://moscow/detail/Torgovljatory/storii_efferanslyatting │ 2 │ -│ http://kinopoisk │ 2 │ -│ http://video=0&input_state │ 2 │ -│ http://forum.php?t=420 │ 2 │ -│ http://mysw.info/node/21544 │ 2 │ -│ http://smeshariki.ru/tashkinsk │ 2 │ -│ http://smeshariki.ru/product&op │ 2 │ -│ http://pogoda.yandex.ru/jobinmoscow │ 2 │ -│ http://afisha.mail.ru/cheva.ru/reportby │ 2 │ -│ https://diary/2013-07-05/101090/currency │ 2 │ -│ http://slovarenok.ru/chapochki │ 2 │ -│ http://pogoda.yandex.php?showalbum │ 2 │ -│ http://pogoda.yandex.ru/real │ 2 │ -│ http://kinopoisk.ru/saint │ 2 │ -│ http://3dnews.ru/msk/events │ 2 │ -│ http://afisha.yandex.php?id=1620_4_licanel │ 2 │ -│ http://yartb.html?city=55&TopicID=2&IsOrderedProduct │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://yaroslavl.irr │ 2 │ -│ http://video.yandex.ru/rent/info/messa.ru │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://ssl.hurra.com.ua/auto_repairs=0&page=10&category_id=detail │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://pogoda.yandex.ru/chat/ivan │ 2 │ -│ http://afisha.yandex.php?s=4d450&pid │ 2 │ -│ http://afisha.yandex.ru/?trafkey=54073799 │ 2 │ -│ http://afisha.yandex.php?city=418&view │ 2 │ -│ http://guid=6&pw=2&pv=0&price=690&s_yers=1916 │ 2 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=2furii10.html?city=0&hide │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://diary.ru/search?familliklink │ 2 │ -│ http://omsk.spb.ru/filmId=RUb3Hf2m0Cw&where=all&text │ 2 │ -│ http://kaluga/?ext=\xD1\x80\xD0\xB5\xD0\xB9 \xD0\xBD\xD0\xB0 \xD0\xB1\xD1\x80\x... │ 2 │ -│ http://wildberrin/foton │ 2 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 2 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 2 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 2 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.040 user 0.368000 sys 0.012000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -┌────────────────┬────────────────┬─────────────┬────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ TraficSourceID │ SearchEngineID │ AdvEngineID │ src │ dst │ pageviews │ -├────────────────┼────────────────┼─────────────┼────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33069 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 24703 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 15817 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 14233 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 │ 6549 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 │ 5257 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 5257 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 3547 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login │ 3538 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik │ 3371 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php │ 3333 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 3309 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 2923 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 2616 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 2441 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php │ 2418 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 2390 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari │ 2367 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 2349 │ -│ 1 │ 0 │ 0 │ https://google.com.ua/url?sa=t&rct │ http://komme%2F27.0.1453.116 │ 2228 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 1920 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://komme%2F27.0.1453.116 │ 1679 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ekburg.irr.ru%2Fpuloveplanet │ 1340 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://aliningrad │ 1270 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1254 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1213 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1193 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1186 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1142 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F27.0.1453.116 │ 1007 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 954 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://aliningrad │ 859 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login │ 821 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 791 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 777 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 757 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 722 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://video.yandex.php │ 702 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 691 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/obucheyelants │ 684 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 632 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 593 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 577 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://smeshariki.ru/obucheyelants │ 566 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.php │ 452 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 445 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBA\xD0\xBE\xD0\xBD\xD1\x82\xD1\x80\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 422 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 421 │ -│ 0 │ 0 │ 0 │ │ http://obninsk/detail │ 407 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 386 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug.ru │ http://irr.ru/index.php │ 374 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru/index │ 360 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 348 │ -│ 2 │ 0 │ 13 │ │ http://komme%2F27.0.1453.116 │ 347 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-2011/43597 │ 293 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 293 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6HS │ 282 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 268 │ -│ 3 │ 72 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 267 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 259 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 257 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari │ 256 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 │ 241 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login │ 237 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 │ 236 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/28435&lr=157 │ http://komme%2F27.0.1453.116 │ 233 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/ru/photo/6936313555&text=\xD1\x85\xD0\xBA \xD0\xBB\xD0\xBE\xD... │ http://komme%2F27.0.1453.116 │ 224 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php │ 223 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cat/dushkirillovyj │ http://komme%2F27.0.1453.116 │ 222 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=359&op_page2/... │ 218 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kriminally-bezhevsk │ 217 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0 │ 212 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 207 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 202 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 199 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://ekburg.irr.ru%2Fpuloveplanet │ 199 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 198 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 197 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 195 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/414526863.xlsx │ 191 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CDMQFjAJ │ http://irr.ru/index.php?showalbum/login-kiriskaya-obl.irr.ru/index.ru/GameMain.... │ 190 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=0&with_photo.ashx/101/4/?cat=6257271 │ 188 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=partments-sale/search?text=\xD1\x81\xD0... │ 187 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://afisha.yandex.ru/index │ 183 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://ekburg.irr.ru%2Fpuloveplanet │ 171 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 168 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 164 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 163 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://ekburg.irr.ru%2Fpuloveplanet │ 160 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 160 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/life.ru/cars/misc/travel.ru/?trafkey=058143&p... │ 155 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 │ 154 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=577&oki=1&oby=&op_s... │ 152 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://wildberrior/uphold │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6HS │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 149 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 146 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustic/meterburg │ 144 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 144 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 144 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E │ 141 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 140 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 136 │ -│ 1 │ 0 │ 0 │ http://kombardighantnie │ http://komme%2F27.0.1453.116 │ 136 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login │ 135 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kysjacevtika │ 132 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://komme%2F27.0.1453.116 Safari │ 124 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 121 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category/stroy/dachines... │ 114 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/?state │ http://komme%2F27.0.1453.116 │ 113 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/GameMain │ http://komme%2F27.0.1453.116 │ 113 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 111 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login │ 110 │ -│ 3 │ 85 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 110 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kardigan │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 107 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosoftwarenok.ru/projects/zhbi.po... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 105 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268143.html?1=1&cid=577&o... │ 104 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 104 │ -│ 0 │ 0 │ 0 │ │ http://tvidi.ru/photo=0&confiscategory_id=0&engineVolumeFrom=&fuelRateFrom=type... │ 103 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 102 │ -│ 3 │ 14 │ 0 │ │ http://komme%2F27.0.1453.116 │ 102 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 100 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 98 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://video.yandex.php │ 97 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 96 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.21150895 │ 96 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?sort=price │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 95 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login │ 95 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.12.388 │ 93 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 93 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 92 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberrior/uphold │ 92 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 91 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-tanks/search=1&ady=62&modeloveplanet.ru │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 90 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.145332.15&he=10&category_id=2740387993 │ 89 │ -│ 1 │ 0 │ 0 │ http://in-the-weightEnd=2351&numphoto=&isExclusiver.ru/alertljus │ http://komme%2F27.0.1453.116 │ 88 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fprodazha_Italja_unit=1&av=1&nm=1&lang=ru │ http://irr.ru/index.ru/show/414526863.xlsx │ 87 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 86 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb1.html?item_no=386703/?bundle=7172&msid=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 86 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 │ 85 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 84 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733582852/ │ 84 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupit/action │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 82 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 82 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?bodystyle │ http://komme%2F27.0.1453.116 │ 81 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F22.14&he=768486 │ 81 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusk.ru/ru/lookaginitial/Prodayu-Dach... │ 80 │ -│ 3 │ 4 │ 0 │ │ http://komme%2F27.0.1453.116 │ 80 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 79 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://aliningrad │ 78 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 78 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login │ 76 │ -│ 1 │ 0 │ 0 │ https://go.1ps.ru/show&showforum │ http://komme%2F27.0.1453.116 │ 76 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandex.ru │ http://komme%2F27.0.1453.116 │ 76 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26221/detail.ru/v1496366&... │ 75 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php │ 75 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 74 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%2F&sr=http://bonprix.ru... │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len80/page/product │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filtr/all/perm.pulscen... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 73 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 73 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F1.7.1364.172 │ 72 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://diary.ru/forum/intries │ 72 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/show/414526863_112 │ 72 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2618561&pp=1059&op_produc... │ 72 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://video.yandex.php │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 71 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 71 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari │ 71 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://aliningrad │ 71 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=product_id=42&Selection.chelov.ru/searc... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/yandsearch?te... │ http://irr.ru/index.php │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/server=sc.chel.ru/main... │ http://irr.ru/index.php │ 70 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification/?year=\xD0\xB1\... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2791954~43.87725656132&op... │ 69 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://komme%2F27.0.1453.116 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/produkty/bleacs/udilis... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 69 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&wi=1366&br... │ 68 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 67 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731&CgID=124jc&where=all... │ 66 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/Yozh/Goodda │ http://komme%2F27.0.1453.116 │ 66 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2725215195&ti=\xD0\x9F\xD... │ 66 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://ekburg.irr.ru%2Fpuloveplanet │ 65 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru/index │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 64 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://afisha.yandex.ru/index │ 64 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 63 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nmg.com%2F4.0 Safari%26clients-sale/search&e... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 63 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 62 │ -│ 2 │ 13 │ 13 │ │ http://komme%2F27.0.1453.116 │ 62 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login │ 62 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://ekburg.irr.ru%2Fpuloveplanet │ 61 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 61 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 61 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26092.html?s_text=\x5C\x5... │ 59 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/thenon-houses/public/g... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27755f32316.30; .NET CLR ... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742/details/?cauth=0&dam... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fautoad/kniga.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26004/?_h=galle/\xD0\xBF\... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert27930555&sob=1&p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2692.html%3Fhtml?period=3... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/world/photo31469:Album... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=leting "HalUXI&where=\xD0\x9F\xD0\xB5\xD1\x8... │ http://irr.ru/index.php?showalbum/lofiver.ru/articles/86121%26uid%3D139750%26ad... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27000,224648804-recept-so... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/koshka.com/ig/iframe-o... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory=cinema.perm.p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/realty/suntime-5/extre... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742357/detskii_gosts.xml... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742304][from]=&int[14670... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694336/photo/suzannason/... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Forum25/top... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268320995,968650f45491882... │ 57 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupit/action │ 57 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1375605&ga... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://video.yandex.php │ 56 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga/choice/... │ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_product_... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?showtopic,5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26017/quarius_Moscow/cavi... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2726125413975d77cf&search... │ 55 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 54 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266848223/review_type=pro... │ 53 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 53 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26989%26bt%3Dad.adriver.r... │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/album/login-1800002&pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 52 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0 │ 51 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2698172,93932353064614618... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605085268a9c4d61-9862.49... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19&numphoto/login=A-CL-MS-36575c72937][to]=&int[12822304 │ http://irr.ru/index.ru/widgetchrome%2F&ti=no&dom_v_bordovye-printime.ru │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/investate/comple/make/?page5/&docid=jlMNIrXw... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 50 │ -│ 0 │ 0 │ 0 │ │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 50 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F11.7.1364 │ 50 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 50 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26051.htBeg=6&NightRegist... │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xBE\xD1\x81\xD0\xB8\xD0\xB1... │ 49 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 49 │ -│ 5 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 49 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?group=days=3&text=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27410/photo-2.xhtml&serve... │ 48 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandsearch │ http://komme%2F27.0.1453.116 │ 48 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 48 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusty-i-vkont.at.ua/search=0&userId=0... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266128182&op_seo_entry/de... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?sort=&br... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/intrumen │ 46 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://afisha.yandex.ru/index │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumanegenre=33155?analog/kitchen_mini... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik │ 46 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login%2Flick.g.doubleclick.diary.ru/car/kw/3061/us... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://afisha.yandex.ru/index │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734.html_params%3Dfh_loc... │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/GameMain.p1Yo4A │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ua/searchads/jo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605906/frl-2/sportal.ru/... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fshow/lpp/cre.ru/pers... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.asp?search&ev... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logize.ru/msk/events_liver.ru/russert-plies.r... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26099486633033&countpage=... │ 44 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 43 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://sslow_135000008&position=search │ 43 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischet-solik/odezhda-plos... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert277553/38021/66936575776/... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271050&with_photo.kurortm... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 43 │ -│ 1 │ 0 │ 0 │ http://forums/liii-kuler S22oHgBJTngegotavgorod55.ru/filtr[2]=42 │ http://komme%2F27.0.1453.116 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-folders/?end=9a08488... │ http://irr.ru/index.php?showalbum/litamak.irr.ru/catalog/8570/travel.ru/cgi-bin... │ 41 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 41 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 41 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/0001216629/#top_by │ 41 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26040931&s_yers=0&with_vi... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://ekburg.irr.ru%2Fpuloveplanet │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.137508&s_yers=200... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://aliningrad │ 40 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logito8ergo_6470/page_type_id=20872/0/001:12:... │ 40 │ -│ 0 │ 0 │ 0 │ │ http://afisha.mail.ru/dmitrij │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/readar-nashi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27924563724&key=46960/med... │ 40 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://lib.ru/exp?sid=3205&bt=7&bn │ 40 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 39 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credit │ http://komme%2F27.0.1453.116 │ 38 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=8570/page=5&s_yers=2006 │ 38 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2679/detail/5482,935033/2... │ 38 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 38 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login │ 37 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dnevnoy-kv-Samara.irr.ru/searchAutos&marka=4... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://komme%2F27.0.1453.116 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 37 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=theating&page=... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?group │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/forum/topicID=269&stat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2784389/room=39695,966681... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/votkim-rukav-i-kova-ul-adve... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filmId=xFX1UbTNJjxe4yF... │ http://irr.ru/index.php?showalbum/login.asp?razdel7/test/matched=115909d9_dsc07... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wolf-club478561/OPEL | \xD0\xB0\xD0\xB2\xD1\... │ http://irr.ru/index.php?showalbum/login-marka=13&city&custom=0&damages/0001216 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27553&s_yers=0&po_yers/46... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert275226607660?design=6efxq... │ 36 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=show=&fgroup-tab-mara.irinables/yearT... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278171&size=52094&op_seo_... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27330528/photo/6223799%26... │ 36 │ -│ 5 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 36 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36&he │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://smeshariki.ru/obucheyelants │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=post_neu%3D22%26pz%3D0%26rleurl%3D%26xp... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 35 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/8569 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/index.ru/katego... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vip.ru/book.com/photo/7086/page=0&state=mont... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-cukickdrops_bak_yant4tlGtZgy3BQ&bvm=bv.49478... │ http://irr.ru/index.php?showalbum/login-501-98552&op_uid=1060948/6#f │ 35 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (Windows │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19&m_staraya-obl │ http://irr.ru/index.ru/show/414526863_112 │ 34 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.12.388 │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/promo=C-Tease/rent/dat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 34 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_uid=1... │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/list │ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 34 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/directadvert/kavanga_728x90.html │ http://komme%2F27.0.1453.116 │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 33 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshops.xml?typ │ http://komme%2F27.0.1453.116 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?cPath=default.ru/yandex.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33 │ -│ 1 │ 0 │ 0 │ http://auto.ria.ua/\xD0\x9A\xD0\xBB\xD0\xB8\xD0\xBC\xD0\xB0\xD1\x82\xD1\x8C\xD1... │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 33 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://video.yandex.php │ 33 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 33 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser │ 33 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265748ba3f755671_1162684&... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertif?sle=85000 \xD0\xB4\xD0... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/pub │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364 │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 32 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zxcvbnm1990&search/jobinmoscow.ru/yandex.by/... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements-obschaya │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fproduct/foto-4/login... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&op_category_id=1174]... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniturers_id=575386110 │ http://irr.ru/index.php?showalbum/login.exit/rem/php/board/10_kruizer_8267049&o... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiepochto_id=93231238][to... │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://afisha.yandex.ru/index │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ssl.hurra.com/iframe │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischuk_Kerasivye-klavik/?... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/jobs-education/m_acces... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26791e3a7c543c9bf9ef5c0-9... │ 31 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt │ http://komme%2F27.0.1453.116 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_api.php?cid=41389/rooms/perm.pulscen.ru/sear... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertificia/?letteo_985428161/... │ 30 │ -│ 1 │ 0 │ 0 │ http://domchelov.html&lr=55&text=\xD0\xB4\xD1\x80\xD1\x83\xD0\xB3\xD0\xB8\xD0\x... │ http://komme%2F27.0.1453.116 │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoryID=12452929587/... │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 30 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 30 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268630533&site_ot=&price_... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-k608i.trashbox.ru/cars/page5/&docid=577&lr=2... │ http://irr.ru/index.php?showalbum/logii_58247.php?forum.borovichy77/page5 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2716/~3/150160947694,9745... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/products/?category_id=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2696374/#page/tab=com_avt... │ 30 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/topic │ 30 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.14&he=768&wi=1920 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605446518%2F&sr=http://e... │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26979553039353936694&ch=u... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://smeshariki.ru/obucheyelants │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-carava/fotok-8-steklove.ru │ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26928142&op_products/view... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kupibatikov/?recrete/personal/klimat.ru/work... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/wedditeli_foto-2/__48/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x9A\xD0\xBE\xD1\x80\xD0\xBE\xD1\... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertifications[0][150]=60&inp... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_countpage/1 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/click.net/paged=696&lo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274229696.0 (Windows)&bL=... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-14132&with_photo=1&srtime-zone.net/catalog/t... │ http://irr.ru/index.php?showalbum/login-sierra%2Fdlia-zhienskaia-moda-zhiensmed │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 29 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704.html%3Fhtml5/v12/?fr... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2664952&xdm_p=1#country=-... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734/fotographic_2017666&... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategoriya/zhienskaya-... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9649.html%3... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/small_mercial-shoppicc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27350350394&ch=UTF-8&sF=1... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_seo_entr... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-baza/gadge... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27425.3120%26ntype_id=2&r... │ 28 │ -│ 1 │ 0 │ 0 │ https://directions[0][1541769377921968 │ http://komme%2F1.7.1364.172 │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert272291684/hasimage=0&view... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13733097&op_categ... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/mann japanel/?text=\xD1\x81\xD0\xBC\xD0... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269500469412216388/detail... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-my-sovies/bebybum.homestion bonprix.ru/... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikarbox=2&tz=9b81998&po_y... │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-176-kakaya-obl.irr.ru/search?lr=191&ei=N6rrg... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?lang=all&fi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 28 │ -│ 3 │ 95 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\xA4\... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/anime-2/#album/l... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-eve-nn.ru/cars/passenger/\xD0\x92\xD0\x90\xD... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274236&pv=15&prr=8&sid=35... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/41452626u1zIq0SGLXCI │ 28 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic/8939858-59929477/detailshop │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://ekburg.irr.ru/#lingvo │ 28 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F22.14&he=768486 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678/page=2&marka=84&mode... │ 27 │ -│ 3 │ 2 │ 0 │ │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/search?cl4url=http://yandex.ru/online... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27013.ya.ru/sportalznakom... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert24061.jpg.html%3Fhtml?1=1... │ 27 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031343-sh-898675/sort=pri... │ http://irr.ru/index.php?showalbum/login-5158&jenre/slyudya_metroman.ru/GameMain... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19195/offset=101&district=&city&with_photo=¤cy │ http://irr.ru/img/catalog/53485785/topic/8939850.php?page=3&marka=84&mm │ 27 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 27 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 27 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-oleginnesujeli-i-centum=etarget=search?clid=... │ http://irr.ru/index.php?showalbum/login-kupe.html_params%3Drhost%3Dad.adriver.r... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fhit_list.html?s... │ http://irr.ru/index.php?showalbum/login-irake-myservierk-supierts%2F&ti=\xD0\x9... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27928092_price=\xD0\xBC\x... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/mazda-dievochnye_avari... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-19.xhtml?city_to_one=All&film]/on/orders/810... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/1.19.mobile_photo=6001... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27630/?item_id=0&bodystyl... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ 1 │ 0 │ 0 │ http://bonprix_ru_11559&lr=12&usln │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lizova_ii_1112_1150&option[price_do=300... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E&wher... │ 26 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 26 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detall/seconomicsmovinki │ http://komme%2F27.0.1453.116 │ 26 │ -│ 1 │ 0 │ 0 │ http://yandex.ua/?target=search/tab=user │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://obninsk/detail │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login-jjjekrju-s-pring=1&price[price=\xD0\xBC... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/loginsk.irr.ru%2Fkategory_id=9204471-650/?_h=... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification.ru/messa.ru/rea... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lejandsearch?text=\xD1\x82\xD0\xB8\xD1\x85\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert21124631/?Search.php │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 26 │ -│ -1 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_category... │ 26 │ -│ 3 │ 197 │ 0 │ │ http://komme%2F27.0.1453.116 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?messa.ru/worlds/page2/7277932582&text=\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 26 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-kupe-20010120652838799.html?n=7148.html... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemey-volvom80Cw&where=a... │ 26 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=607&state=4... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.22 (KHTML, like Gecko) │ 25 │ -│ 1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 25 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/people.ru/cars/passets... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731/?city=0&pvno=2&evlg=... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vivarki/redki_sdelaet-law.ru/produkty/kartgo... │ http://irr.ru/index.php?showalbum/login=driff/en-ru/?p=2#messages/00001216629 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.html&ei=GcXrUYe8F2DAU-RbuRlLMczoJ--5uDCx8 │ http://irr.ru/index.php?showalbum/login-132/#images/0000.h95/\xD1\x82\xD1\x8E\x... │ 25 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert264314953); ru) AppleWebK... │ 25 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://obninsk/detail │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search?text=myaccountry │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguard-payments-sale/se... │ http://irr.ru/index.php?showalbum/login-88i8LaDyEkCVv6-DhRfEDcw==&action │ 25 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/yandex.ru/?f... │ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.ru/film/46351/frl-2/bage │ 25 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605838.html5/v12/?from=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-link=1147373-1&p=1&p=2&text=\xD0\xBC\xD0\xBE... │ http://irr.ru/index.php?showalbum/logisterlingvo/#! │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741559&t=5204/make=Chrys... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694589/detail/555-javata... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=shops.html?id=1&body_type=\xD0\xBF\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781472265&op_seo_entry=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626rleurl%3D//ad.adriver... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685/price_ot=&price=\xD0... │ 24 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://diary.ru/forum/intries │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26685857947301_Zoryatiya/... │ 24 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-117031&op_category&op_seo_entry=&op_uid │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27399992.php/board,13.0) ... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671291&fridgets/2012&pri... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguar.ru/page4/?emain.a... │ http://irr.ru/index.php?showalbum/login=ogabass.ru/filmId=9WOqzzitive&view │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-annative-expressageId=0oVXx21hUXI&where=all&... │ http://irr.ru/index.php?showalbum/login-rybnovlevojj_mamountry=-1&type%3D0 │ 24 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://video.yandex.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo/7... │ http://irr.ru/index.php?showalbum/login-kupit-plitka-s-korsiety/art/151/n4p/160... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/intrumen │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kupalnaya_elit=&id=3&clid=9403&lr=1913849 │ http://irr.ru/index.php?showalbum/login-gallebia_zhurnalCall D.R.E.mp3ex.net/ph... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lolacake/tatyaka.html?1=1&cid=577&oki=1 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinglish.rus-l7-p70505-1... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/yandsearch?lr=1... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-yvaken/topic=7702.jpg.html?1=1&cid=577&... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=1&bc=3&ct=1&pr │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-547552/1/?sma=RAINBOW&op_uid=fdd1b9d2721728&... │ http://irr.ru/index.php?showalbum/login-kapusta-advertika/search/room=1&damage=... │ 23 │ -│ 1 │ 0 │ 0 │ http://auto_volkswagency=1&text │ http://komme%2F27.0.1453.116 │ 23 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgarials/commend/searchvage-2.html&dt=13734... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-tuniki_510... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263090024189-528619c2077/... │ 23 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/ru/index.ru%26bid │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333572620201709/page/10... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news/radio&clid=198555... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26939.html?page/3/#count=... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/kategory_id=5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2740974%26pz%3D0%26ar_sli... │ 23 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 23 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/resultatya9176da22f1a521a5853.html&... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26600721376439322%26pz%3D... │ 23 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 23 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga_728x90.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://afisha.yandex.ru │ 22 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemennoke0070553995f27d6... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2697574697.html_param=0&u... │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-105vr/ │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=search.tutung/s... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/wheel.ceratornaya_solnechka │ http://komme%2F27.0.1453.116 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-10760b31b65633eZateq eb806e887d9f15ccf593280... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2091&local=1&TypeSearch?mail.ru/yandsearch&c... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26058&nnum=s4746835895&op... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-the-ili-tempera-minsk.irr.ru/produkty/bluzki... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodny │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2649.html?pg=menu_29.jpg&... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739693071/pomer... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/v1465][from]=&int[157281 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267830840994,95688781470%... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-business/sale/ru/search?mail.ru/film%2Fbarna... │ http://irr.ru/index.php?showalbum/login-zakon_type=1&fuelRatesTypeSearch │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 22 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27200,55.7654.html_parat-... │ 22 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26601629][from=&fuelRateT... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&l... │ http://irr.ru/introlux_page5/2/page/Renaul.irr.ru/start=1500-rublic/gamemain/Tu... │ 22 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://ekburg.irr.ru%2Fpuloveplanet │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liliate_id=24145602&with_phoney.yandex.ru;ya... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-natashkarola.irr.ru%2Fviewforum/MsgList.html... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26900/technics-technics-t... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&city[7]=23239330794,25826044_Black_lis... │ 21 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?price_till beli.php?cx=015216684_36 │ http://komme%2F27.0.1453.116 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691_ru_17832523.html_par... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694,978825315f373400/det... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1) AppleWebKit%2F537 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/world/photo/41/62b1dfa450/3/women-clother%26tms%3D%26ev_ltx%3D... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/basketshop.ru/yandsear... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/forum%2Fukhov/l... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27025&Select[4][]=32598.h... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265228.html?1=1&cid=577&o... │ 21 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://wildberrior/uphold │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/houses/passenge... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo44953.116.php?t=4... │ http://irr.ru/index.php?showalbum/login=ko&page=0&view.aspx?group_cod_1s=85&key... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/?from │ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-dance/index.ru/?favorite_off=1&encoding&hl=\... │ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logino-s-grigerator/page1=&input_age1 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login.html?1=1&input_who1=2&input_who2=1&inpu... │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cars/commeddesk.ru/google_s&12.5.746.59954.30 (KHTML, like/\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ramblery/pic/893985650697... │ http://smeshariki.ru/obucheyelants │ 21 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26737865/?page/12186/#/ec... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/logii-bin/click.cgi%3Fsid%3D158195,97987231-9... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/club.ru/anime-zone.ru/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dietsik.ru/razdel_id=111,7,700&w=728x90.html... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929&from%3D%26custom=0&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-brie... │ 21 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739353/details.... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2186/detail.aspx#location... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kupalnik │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/9-0--navam-zhien... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741465][from]=&input_wit... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification%2F4.0; XBLWP7; ... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_uid=1147... │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login=uzpp2XnEw&bvm=bv.49784469][from │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 20 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-Larki-i-korobeg-1124-95367/guestblowinp... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26219&wi=1280&lo=http://w... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/yandex.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estatic.diary.ru/... │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=58&Selectronics-technics/k... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?er=2&text=\xD0\xBA\xD1\x83\xD0\xBF\xD0\xB8\... │ http://irr.ru/index.php?showalbum/login=razer2.moikrug.ru/recipes/shop.spb │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b706234651&op_uid=1&b... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&price │ 20 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://video.yandex.php │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=NeIdeaCenternet Explorer&aV=5.0 │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertificirovka.ru/noteating-i... │ 20 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271161076&st=261&t=170977... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate=0&ReturnUr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313163307/?from]=&int[15... │ http://che.ru/produkty_zarubezhei-niepochekhly │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604370757034dea482207549... │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_api.php?id=7262882,9454472&op_product_n... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 20 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/cars │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-karbox=0&top=0&category/1961/#images/00... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26637&model=1705.html?1=1... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumilora481 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt=13733129/room │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-iulyanovskaya-obl/lyubvi?page=166733&eid=485... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27502/detail/?folders/#pa... │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://video.yandex.php │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626u1zIq0SGLXCI&where=al... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=240&u_cd=242035165&input_who2=1... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news_id_72387877055/nu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikulina.ru/real-estate/ou... │ 19 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?sort=popup/casualg... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/filmId=eRaB4pYAXCI │ http://komme%2F27.0.1453.116 │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBB\xD0\xB8\xD0\xBA\xD0\xBE\xD0\xBB\xD0\... │ http://irr.ru/index.php │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.ru/daily │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260540718534/?dateOn=1&bo... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login=perfect.yandex.ru%2Fkategoriya%2F537 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26672655462.html?city.ru/... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261921&pr=2441185112.html... │ 19 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostok.ru/cars │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/check=1&redir=1&lang=a... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671&Destige-all.html_par... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27404895,95923.html%26cus... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313437&pvno=2&evlg=VC,2;... │ http://irr.ru/index.php?showalbum/login-rizova.ru/look/timashirtsevm.cofe │ 19 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2638/?from=&raceTo=&power... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6HS │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27355f3132366336601&uuid=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166333/price[price_till]=&pri... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F_liveresum... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.php?f=11340.1j1j2.12...5998-103358071/8/page... │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/00007/10/1320006&p... │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto_repai... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://smeshariki.ru/obucheyelants │ 19 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.15751 Safari%2F5.0 (Windows │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2675406828ac956028692581/... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://obninsk/detail │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26968794618/irr.ru/real-e... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2629&lr=65&rstr=-213&text=back=False&withmos... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2702483,631;IC,444041%2F&... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=http://video.yandex.ru/real-estate │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?search?text=\xD1\x80\xD1\x83\xD1\x81\xD0\xB... │ http://irr.ru/index.php?showalbum/login-inadlinny-sched_car=0&city=0&price=18&p... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26669E116-1-1550046560013... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274026181891&rn=52&brand=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/dlia-zhiensk... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&checked=0&state/room=10002-g-v-program... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-165&jenre]/49047&lr=163&text=samsung ga-moda... │ http://irr.ru/index.php?showalbum/login-2-komn-v-na-92f5-6ccf-fef3-013f9f926a21... │ 18 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ru/index │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5 │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?elmt=popular&where=all&film/723/num │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupaljinik-slidi/places/?PAGEN_1=2 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/restate/out-of-town-resh=1&cnt=1373... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2012-recept-Ragu-iz-2-x │ 18 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=deltec&type=citymo=&version%2F4.0 (Linu... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27534.30 (KHTML, like Gec... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kochtinsk.pulscendinskiy-krasnodar.irr.ru │ http://irr.ru/index.php?showalbum/login-kuplyuskij/zapchastorii_state/out │ 18 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login.moikrug.ru/catalog/otchenko-ul-advert27... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zzzi.ru/a-shop and blademas/?id=1&marka=0&ma... │ http://irr.ru/index.php?showalbum/login-2003prev=/search/index.ru/meteor.html%3... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\... │ 18 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278453&with_exchange=0&ex... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=367132&pv=2... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27132069304;IC,1511e8d158... │ 18 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27071038_114736895,923401... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x9A\... │ 18 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/?target=search?... │ http://irr.ru/index.php?showalbum/login-bigutensiysk.irr.ru/regular&bid=3205&bt... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/8018&l=malkogo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-natura.ru/jobs-education=post_137336ID0... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kanza_tab_2499472&lr=1074&sort=popular │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27420780790931.html5/v12/... │ 18 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/obuv%2F │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-21102&text=\xD0\xB1\xD0\xB0\xD0\xB1\xD0\xBA\... │ http://irr.ru/index.php?showalbum/login.mospsy.ru/super-na-kostronics/tv-audio.... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685&numphoto=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertimeshariki.ru/cgi-bin/cli... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626p/ │ 18 │ -│ 1 │ 0 │ 0 │ https://diary.ru/magnitogorsk │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sien-zhienskaya-i-kvartir.html?id=1955451-01... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260908861%26point-peter=&... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=yators/?jumping.ru/series.ru/priceup&page=3&... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1453.116 Safari%2F8536 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeSearch?text=\xD0\xB2\xD0\xBA\xD1\x83\xD1... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga1788778.137329TVmbFNmbVNaR0YwW$UyNDg4fxS... │ http://irr.ru/index.php?showalbum/login-sienskaia-moda-zhienskikh_rabotjaga.ru/... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=36427929&stat... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinger/\xD0\x93\xD0\x90\... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929750%26ar_sliceid%3D2... │ 18 │ -│ 4 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691c352005&pvno=2&evlg=V... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru%2Ffilm/4858?pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/cars/communalux.ru/dos... │ http://irr.ru/index.php?showalbum/loshaya-obl.irr.ru/film/35562C?analytics/kto-... │ 17 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27422544/3/womens.ru/ufa.... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271527091/frl-2/bage-2.ht... │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#7878... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.103402.html%3Fhtml │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=artira-Lyubyat │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.aspx#location │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=yes&y1=2013-07-07.ya.ru/5_search?text=\... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.15.html?1=1 │ 17 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://aliningrad │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266149_51928/detail.ru/ka... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%26rleurl%3D%26CompPath%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2670308&po_yers=0&price_o... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2613168312.71 Safari%2F__... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_Bolers-device.aspx#commersanta_premiery... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=8&ch=UTF-8&ifr=1&av=1&nm=1&ved=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 17 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6HS │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-dlia-zhiensmed.ru/el... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginyazevskaya%2F5.0 (Windows NT 5.1) Prestorygoo... │ http://irr.ru/index.php?showalbum/login-kupilomatic.aspx#location=rr&days&city&... │ 17 │ -│ 1 │ 0 │ 0 │ http://bdsmpeople.ru/register2123 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logistereda.ru/photo/70861/6#f │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263161366&bid=2823&numpho... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowthreadreplies=24&ids=74&cu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27595,9305f3931339322%26n... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273363331251798796&op_seo... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313&ussp=mstp&usln=1&inp... │ http://irr.ru/index.php?showalbum/login-3767436/roomootofile/obzor.com/iframe │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26041.htm%3Fsid%3D0%26pz%... │ 17 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/stan/index.kz/yandex │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertistia.html?partment/searc... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-search%3Futm_source=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031323532363&win=70&mode=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 3 │ 22 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://diary.ru/forum/intries │ 17 │ -│ 1 │ 0 │ 0 │ http://bibieters_sig=5ea8QizwGSPy0yGcm14ATH74D4Bw&usg=AFQjCNHxMZlonova.ru/p1847... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_style... │ 17 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/messar.ru/professig │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261636/detail/odnoklassim... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lenfiki-zhienskaia-moda-muzhchin-sitafa ishq... │ http://irr.ru/index.php?showalbum/logical.ru/link=11485-9065383235348%2F&sr=htt... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-futbolki-k... │ 17 │ -│ 1 │ 0 │ 0 │ http://acase.php?input_who1=2&input_who2=1 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.10 │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733145625/details... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694&option%2F12.14&he=76... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-irina19910735%2F27.0.1453.116 Safari │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268814453957595,94406/det... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26618531&t=111246.html?p=... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=Like Gecko) Chrome%2Fpotnik=1&price_do=&curr... │ http://irr.ru/index.php?showalbum/login-the-poxudet_sponsor=&o=6000708498/?caut... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b76faact/less=1&ru=1&... │ 17 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-53584715,1327288/belgium/arch=1&themec.... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=bestwo.diary/details&id=9160/0/posti.ri... │ 17 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/category=cinem rasmusic │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 16 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search?clid=19554786.0.8.0.2.2003558_3497926... │ http://irr.ru/index.php?showalbum/logie-niz-doktops/electronics/4185570c72 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098707209463401090/deta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgas.rQRAX2Ua3IF1iHJcaz4ATv34DABw │ http://irr.ru/index.php?showalbum/login/?page=1024&wi=144438-88-3906740/currenc... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267044005320AA76)&vendor_... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27034195,9520160173/5/wom... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-herokee_tatus=1&rm=18606bAxFEcQT6smBB4W... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search/cuZXdzJTJGJmxyPTI4fQ,... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-plan... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&pt=b&pd=7&pw=2&page5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26396284046.xhtml%3Fhtml%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate/out-of-tow... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260612d7cf.4082813,74,756... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/page=91552&q=\... │ http://irr.ru/index.php │ 16 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login.aspx?sle=13&s_yers=0&page_type=0&door=0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?&clients-sale&siteurl%3D//ads/search?text... │ http://irr.ru/index.php?showalbum/login-kapusta-advert273786/foto-5/#photo=2817... │ 16 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/movinki... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnaja-prost-petersburg/detail/?fold... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-krasok.html_params%3Drhost%3Dad.adriver │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiensmed.ru/dl/\xD0\xB7\x... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti/redmonds %2F offset=403134333.2201560&c... │ http://irr.ru/index.php?showalbum/login-v40-velopiter.feriod=0&scroll_to_auto.r... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661232&streamfood.com/if... │ 16 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26765.html_params%3Drhost... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166303133334%2F&ei=horia+iudi... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-57548811426&text=\xD0\xBB\xD1\x83\xD1\x87\xD... │ http://irr.ru/index.php?showalbum/login-coolonellana-Molodilnik.ru/index.php?vi... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/70948/35.htm?id=... │ http://irr.ru/index.php?showalbum/logiy-luchaiev%2F&sr=http:%2F%2Fwww.bonprix c... │ 16 │ -│ 1 │ 0 │ 0 │ http://histore/profile;u= │ http://irr.ru/index.php?showalbum/login-kupe-20013&uuid=1373356164564_1sort/sta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2755597.html%3Fhtml?1=1&c... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=findjob.ru/planet.ru/index.ru/carinov.h... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/showtopic,803813.html&... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_catalog/1250/?itemsg/cd... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert260477/cities/visas/exp?s... │ 16 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://video.yandex.php │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27429&input_bdsm_position... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26887301/Zona/simferopol.... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2728024/detail/508/?instv... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-153-4b9c4.4331626.htm?size%3D0%26rleurl │ 16 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect │ http://komme%2F27.0.1453.116 │ 16 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://msuzie-shop/premiery-c-38208_2.html │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/posts&q=\xD0\xB2\xD0\xB5\xD0\xB4\xD1\x83\xD1... │ http://irr.ru/index.php?showalbum/login-mistore/#CATALOG_LIST-s-ovosibirsk/deta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-marka=23&model&desting?field_town/search/pri... │ http://irr.ru/index.php?showalbum/login-492ea9&show/417582,9626512874":10157628... │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/sch/price][min]=41&lr=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27364229586121%26url%3D%2... │ 16 │ -│ 1 │ 0 │ 0 │ http://volgografiyah_27_iyunya_50_let%2FgetId │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661251-6efa-d61f-fef3-01... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-maritkiN │ http://irr.ru/index.php?showalbum/login-ladimir.irr.ru/remiery-telefon.ru │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333%26bid%3D1%26rleurl%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new=127201148][fr... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/tv-audi/a7-spec=9894797179698712/p1/8... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=view/5/item3963... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti%2F%3Ff%3D100%&http://afisha.yandsearch&... │ http://irr.ru/index.php?showalbum/login-6030d.html_params%3Drhost%3D90%26height │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logiste's a di galaxy-s4-story1.aspx#location... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269899458/price/3815&pvno... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/7427510/detail/5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5.0 (Win... │ 16 │ -│ 1 │ 0 │ 0 │ http://yandsearch?lr=2&color │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new10152954vac │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2735030373434-152495.php?... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2753][from]=&pricedownloa... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-gorod/search?p=7&oprnd=9902.jpg&img_url=http... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27222.vk.me/u3166 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makeevka.ru/perm.irr.ru/slingvo/#1\xD0\... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26708662307][]=&selection... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313033393633310,932803][... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-irinakai/page_num_read2306e4574&ei=Ot_rUfjPG... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691.html?1=1&input_age2/... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2010000087378687/pic/8939375f303839303935373... │ http://irr.ru/index.php?showalbum/login-souse=2/path=39_1.html?1=1&cid=577&oki=... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/artira │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/?strict=5710... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2695/19545602cf94d77/repl... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumy.ua/search?text=\xD1\x81\xD0\xB5\... │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertists/736850/?item_no=737&... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikal.ru/search/room=1&dam... │ 16 │ -│ 1 │ 0 │ 0 │ http://forums/liiiervierk-suppohudeemvmesting/294465&op_categoriya │ http://komme%2F27.0.1453.116 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&model=1173... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login/?do=ready-mansion%3D0%26url%3D%26bn%3D0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=new&cad=rjt&fu=0&input_country_id=228&lr=961... │ http://irr.ru/index.php?showalbum/login-cam.shtml#objdesc=true&uA=Mozilla │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb=1&prepairs=0&city[1 │ http://irr.ru/index.php?showalbum/lounona-muzhchin-kupaljinik-chernyj-odnoe/san... │ 15 │ -│ 1 │ 0 │ 0 │ http://launcher-searchads/search │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781308;IC,238208836746/0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-album/login.2/second... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_produkty/kiev.org/forum... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginsk.eu/ru/Pagesize=13&m1=07 │ http://irr.ru/index.php?showalbum/login-3muda/truction/vacancies/750207190765.h... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26981583/page=1089592445/... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-4433140796851/detail/536180&all=False&i... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273389; U; Android 4.0.14... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945HPS?analog/r10418/sort=price_do=200&site_off=1 │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F1.7.1364.172 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273631253711/foto-6320166... │ 15 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshop/id_art_type=7&s_yers │ http://irr.ru/index.php?showalbum/login │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2724/?_h=search?text=\xD0... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-goluboj-podserial&dfs=13 │ http://irr.ru/index.php?showalbum/login-haus.html#news/228670,257&pci=3012/frl │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#67 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266356163f39ea6f8/7a8745&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26605566254/room=1.6&wher... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/washes/built-in_two_ch... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?Topic │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/state=1909644 │ http://afisha.yandex.ua/auto_id=0&with_photo.kurortmag.ru/razdnitsya-vishnury │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://irr.ru/index.ru/show/414526863_112 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/produkty%2Fplatj... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?transk.i... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginskikh-viana.irr.ru/GameMain.aspx?sle=1/house.... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-land_search?text=\xD0\xB0\xD0\xBC\xD0\xB1\xD... │ http://irr.ru/6323%26bn%3D27888895,963095425 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/index.ru/ange=0&s_yers=(3000&static.diary.ru... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?page37/?sta... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678&op_page=60000/curren... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list.html?typ=SMA&anbieter=aleks_evilkos.com │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?Id=9582 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=alini... │ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liloveplanet.ru/futurer533/women.aspx?group-... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2631920&lo=http://person_... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-tumbov.irr.ru/page1=&input_who1=2&id=29... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.1Uv1UfqCA8SonYC4BQ&usg=AFQjCNGB3pBUuKY1jJPP3... │ http://irr.ru/index.php?showalbum/login-tank-sale/search%3Fmode=&page_type │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2718599/photo=0&is_hot=0&... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.php?city[1]=700003 │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27515/418695&st=327-B110Q │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26761,55.654289600/detail... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login=vladimir/page_type=0&expand_search?text... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2699364224073532663835386... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-lential/secondary/details9.html_params%... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?dok=001&rightbox/week&From=0&user │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/frl-4/trailer/view/3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_appliazh-pliance/mista-bez-uchaiev/000202&cl... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=25127691%2F... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page9/#14... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2730675595,9292fa-d61f-fe... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienshchin-planet.ru/myma... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/gamemain.asp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27877437494,943082&stat_t... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27265484158197%26width%3D... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-676216b8af/4fd00fa61b3185631821/page_ty... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/ch/floore troubleclick... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1955451&lr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 15 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 3 │ 123 │ 0 │ │ http://komme%2F27.0.1453.116 │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Ford-antent... │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nashinger/users/search?textilead&353%26ev_pl... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginy_name/3196906481/currency=6465373200&price=\... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2776/?date=0&doorov.irr.r... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-102-821922237&arrFilter2013/08-iH4AT5zIGQDA │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_type=&freetao.diary.ru/sea... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/877/ru/buildin... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9584%26pz%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert276278-ploschaya_checked_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2693_763613.html_params%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274298799461981/?date=129... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert262997385f32313335781094&... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.10 │ 15 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credir=1 │ http://komme%2F27.0.1453.116 │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://ekburg.irr.ru%2Fpuloveplanet │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=on&input_onliner... │ http://irr.ru/index.php?showalbum/login.html%26custom%3D%26CompPath.2; WOW64; r... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/gosthelp.ru/auto.ria.... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=on&in... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2651734&pt=b&pd=7&pw=1&pr... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/litlip/saint-peter557932E-8C62-4917%26ad%3D64... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga664.1721/deti74.ru/animals-planet.ru │ http://irr.ru/index.php?showalbum/login=K-SR-B-13-9635095,9661/9#f │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search?text=\xD1\x84\x... │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 15 │ -└────────────────┴────────────────┴─────────────┴────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.132 user 1.200000 sys 0.020000 -D 07-01' AND (DATE '1970-01-01' + 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; -Run Time: real 0.014 user 0.184000 sys 0.000000 -D D (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND DontCountHits = 0 AND URLHash = 686716256552154761 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Run Time: real 0.019 user 0.248000 sys 0.000000 -D " = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) ORDER BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))); -Run Time: real 0.012 user 0.152000 sys 0.000000 -D -D -D '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(URL) > 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -┌──────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├──────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 102341 │ -│ http://komme%2F27.0.1453.116 │ 51218 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18315 │ -│ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16461 │ -│ http://irr.ru/index.php │ 12577 │ -│ http://irr.ru/index.php?showalbum/login │ 10880 │ -│ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 7627 │ -│ http://irr.ru/index.php?showalbum/login-kupalnik │ 4369 │ -│ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 4058 │ -│ http://komme%2F27.0.1453.116 Safari │ 3021 │ -└──────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.077 user 0.900000 sys 0.016000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(Title) > 0 GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ Title │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ \xD0\xA2\xD0\xB5\xD1\x81\xD1\x82 (\xD0\xA0\xD0\xBE\xD1\x81\xD1\x81\xD0\xB8\xD1\... │ 122407 │ -│ \xD0\xA8\xD0\xB0\xD1\x80\xD0\xB0\xD1\x80\xD0\xB0\xD0\xB9), \xD0\x92\xD1\x8B\xD0... │ 82935 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA - IRR.ru │ 80958 │ -│ \xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\xB8 New Era H (\xD0\x90\xD1\x81\xD1\x83\xD... │ 39098 │ -│ \xD0\xA2\xD0\xB5\xD0\xBF\xD0\xBB\xD0\xBE\xD1\x81\xD0\xBA\xD1\x83 \xD0\xBD\xD0\x... │ 23123 │ -│ Dave and Hotpoint sport \xE2\x80\x93 \xD1\x81\xD0\xB0\xD0\xBC\xD1\x8B\xD0\xB5 \... │ 14329 │ -│ AUTO.ria.ua \xE2\x84\xA2 - \xD0\x90\xD0\xBF\xD0\xBF\xD0\xB5\xD1\x80 │ 14053 │ -│ \xD0\x9F\xD1\x80\xD0\xB8\xD0\xBC\xD0\xBE\xD1\x80\xD1\x81\xD0\xBA (\xD0\xA0\xD0\... │ 13912 │ -│ OWAProfessign), \xD0\xBF\xD1\x80\xD0\xBE\xD0\xB4\xD0\xB0\xD1\x82\xD1\x8C │ 10919 │ -│ \xD0\xA2\xD1\x80\xD1\x83\xD1\x81\xD0\xB8 - \xD0\xA8\xD0\xBE\xD1\x83\xD0\xB1\xD0... │ 10157 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.051 user 0.688000 sys 0.000000 -D AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND IsLink != 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -┌────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ URL │ pageviews │ -├────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 7479 │ -│ http://aliningrad │ 4791 │ -│ http://ekburg.irr.ru%2Fpuloveplanet │ 3584 │ -│ http://smeshariki.ru/obucheyelants │ 3064 │ -│ http://video.yandex.php │ 2887 │ -│ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 1084 │ -│ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 891 │ -│ http://afisha.yandex.ru/index │ 855 │ -│ http://sslow_13507.html?aspx?naId=6HS │ 521 │ -│ http://wildberrior/uphold │ 484 │ -│ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 289 │ -│ http://obninsk/detail │ 241 │ -│ http://diary.ru/forum/intries │ 208 │ -│ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 185 │ -│ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 185 │ -│ http://kurort/SINA, ADRIAN │ 157 │ -│ http://afisha.yandex.ru │ 132 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 124 │ -│ http://ssl.hurra.com/iframe │ 123 │ -│ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 119 │ -│ http://stars-visa.html_params%3Drhost%3Dad.adriver.ru/catalog.php │ 105 │ -│ http://komme%2F27.0.1453.116 │ 83 │ -│ http://pogoda.yandex │ 80 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 79 │ -│ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 73 │ -│ http://video.yandex │ 71 │ -│ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 68 │ -│ http://wildberries.ru/daily │ 68 │ -│ http://smeshariki.ru/ru/index.ru%26bid │ 68 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 64 │ -│ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 64 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn │ 63 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 59 │ -│ http://sslow_135000008&position=search │ 58 │ -│ http:%2F%2Fwwwwww.bonprix.ru/topic │ 53 │ -│ http://sslow_13507.html/articles │ 52 │ -│ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 51 │ -│ http://rsdn.ru/rss.ya.ru/catalog │ 51 │ -│ http://ekburg.irr.ru/#lingvo │ 46 │ -│ http://pogoda.yandex.ru │ 45 │ -│ http://sslow_13500000%26rnd%3D2788881.html │ 44 │ -│ http://maps#ru_5_ru_22106.377648194,975924][to]=&int[14270pa106&op_uid=17759/6#... │ 42 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 37 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 37 │ -│ http://video.yandex.ru/film/46351/frl-2/bage │ 36 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 35 │ -│ http://direct.yandex │ 34 │ -│ http://sslow_13507.html?aspx?naId=3X_3bhLcs3M │ 33 │ -│ http://gotovim-doma │ 32 │ -│ http://che.ru/produkty_zarubezhei-niepochekhly │ 31 │ -│ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 30 │ -│ http://video.yandex.ru/Newsletter │ 29 │ -│ http://sslow_13507.html?aspx?naId=6D8IzMGys3M │ 29 │ -│ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ http://video.yandex.ru │ 28 │ -│ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 28 │ -│ http://notes=1/currency │ 27 │ -│ http://sslow_13507.html?aspx?naId=649&state/out-of-town=\xD0\xA5\xD0\xB0\xD0\xB... │ 27 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 26 │ -│ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 26 │ -│ http:%2F%2Fwwww.bonprix.ru/tambov │ 24 │ -│ http://kinopoisk.ru/shoppich.ru/search?clid │ 24 │ -│ http://wildberries │ 23 │ -│ http:%2F%2Fwwwwww.bonprix.ru/searchAutoSearch?text=\xD0\xB2\xD0\xB5\xD0\xBB\xD0... │ 22 │ -│ http://myloveplanet │ 20 │ -│ http://stars-visa-litraj.txt","lpu":"http://pogoda │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/secondary │ 20 │ -│ http://msuzie-shop/premiery-c-38208_2.html │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 20 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 20 │ -│ http://smeshariki │ 19 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 19 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ http://irr.ru/6323%26bn%3D27888895,96772,97436 │ 18 │ -│ http://kinopoisk.ru │ 18 │ -│ http://nepogoda.yandex.ru%2Fproducts/search?text=subscripts/busineshop │ 17 │ -│ http://wildberries.ru │ 17 │ -│ http://kinopoisk.ru/catalog/9902224 │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://direct.yandex.html │ 17 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 17 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 17 │ -│ http://irr.ru/6323%26bn%3D27888895,963095425 │ 16 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 16 │ -│ http://afisha.yandex.php?gidcar=36281664 │ 16 │ -│ http://auto.ria.ua/search │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://direct │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ http://irr.ru/index.php?showalbum/login-kupalnaya-obl │ 15 │ -│ http://kinopoisk.ru/odessya │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 15 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 14 │ -│ http://ekburg.irr.irr.ru/maker │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.yandex │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 14 │ -│ http://afisha.mail/rnd=0.9788 │ 14 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 13 │ -│ http://auto_many_to_auto.ria.ua/igrush43/ │ 13 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 13 │ -│ http://afisha.yandex.ru/cars │ 13 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 13 │ -│ http://love.ru/a-myprofi │ 13 │ -│ http: │ 13 │ -│ http:%2F%2Fwww.bonprix │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://real-estate/aparther/offiliates/corruption/russinsk │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 12 │ -│ http://on-online=on&accetti │ 12 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 12 │ -│ http://samara.irr.html5/v12/?from]= │ 12 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 12 │ -│ http://kinopoisk.ru/saledParams │ 12 │ -│ http://en.lyrsenses/zamba_zaborah_coldplay=1&gearbox │ 12 │ -│ http://zvukovo/hondar/2007&state/renlew/rigma.ru/scribed │ 12 │ -│ http://pogoda.yandex.php?SECTION │ 12 │ -│ http://msk/platia-nashing/vanny.diary.ru/moscow │ 12 │ -│ http://video.yandex.ru/GameMain.E6smreQhiu_hXR4&where=all&film │ 12 │ -│ http://news/6483731559676/Unlocknotebooks/m83/800_D_Black_list │ 11 │ -│ http://nizhnieiene/p17378705/currency=1&with_photo-ideapadeno │ 11 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 11 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty/photo-12/#imag... │ 11 │ -│ http://msk/platia-nashing/vanny.diary.ru/sale/liver │ 11 │ -│ http://video.yandex.ru/GameMain.aspx#location │ 11 │ -│ http://guid=6&pw=6&pv=13 │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 11 │ -│ http://wildberries.ru/rost.html?1 │ 11 │ -│ http://pogoda.yandex.ru/catalog/jokers │ 11 │ -│ http://smeshariki.ru/?win=82&stat=1&page/196264&pt │ 11 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 11 │ -│ http://en.lyrsenses/zamba_zabudtrimazok.html?page=12&prr=http://fap1.adrive_typ... │ 11 │ -│ http://bonprix.ru%26bid │ 11 │ -│ http://loveplanet.ru/GameMain │ 10 │ -│ http://bdsmpeople.ru │ 10 │ -│ http://video.yandex.ru&pvid │ 10 │ -│ http://liver.ru/cheboksicily/foto.aspx?sort=newly&trafkey │ 10 │ -│ http:%2F%2F%2Fwwww.bonprix │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://club.ru/spokoiteli/photo37775280000 │ 10 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 10 │ -│ http:%2F%2Fwwww.bonprix │ 10 │ -│ http://auto_id=240&n=13901038 │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 10 │ -│ http://smeshariki.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://slovariant_new3077940810/detail │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 10 │ -│ http://en.lyrsenses/zamba_zabor_id=1012_blank%26site │ 10 │ -│ http://sslow_13500000%26rnd%3D2788881.html?parts/passe │ 10 │ -│ http://pogoda.html%3Fhtml_params%3Drhost%3D43 │ 10 │ -│ http://irr.ru/index.php?showalbum/logout%26verstova.ru │ 10 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://pogoda.yandex.php │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film/6781203.html?id=242037047/detai... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://mysw.info/blog/sankt-peter%3D1216/00001216629 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 9 │ -│ http://e96.ru/albumfotok-15-fotki │ 9 │ -│ http://alpari.yandex.html?html_param=0&users/#page/Search/ab_dob%2Ffieiie-razvo... │ 9 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu │ 9 │ -│ http://bonprix.ru/catalog/8570/1006790 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http://b2b.testered/main/discuss/matched_country=-1&top=0&cityid=1024&wi=1366&o... │ 9 │ -│ http://svpressa.ru/topic=17082630 │ 9 │ -│ http://bonprix.ru │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 9 │ -│ http:%2F%2Fwwww.bonprix.ru/filmId=8j5j97LRs3M&where=all&sources │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 9 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/#win_13.html_p... │ 9 │ -│ http:%2F%2Fwwwwww.bonprix.ru/mymail/?folders/4744089758 │ 9 │ -│ http://nail=Yes&target=search │ 9 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_produkty%2F&sr=http://slovaria │ 9 │ -│ http://rlsnet.ru/vacancy/view_type_id=9677548268010367 │ 9 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/imagecache/wm/2013&where=all&filmId │ 8 │ -│ https://m.myloveplanet.ru/forum/abrika-kobelenie_nebestsenal │ 8 │ -│ http://zapchast.com/iframe-owa.html?1=1&cid=577&oki=1&op │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 8 │ -│ http://poisk.ru/price_ot=&price_ot=&price │ 8 │ -│ http://afisha.yandex.php?t=141880517 │ 8 │ -│ http://zarplata.ru/velika_all=\xD0\xBE\xD1\x82 │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http:%2F%2Fwwwwww.bonprix.ru/news/222974895&op │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://fore=3&marka=0&top=0 │ 8 │ -│ http://direct.yandex.ru/catalog │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96777&oki │ 8 │ -│ http://video.yandex.ru&xdm_p=1#item/search │ 8 │ -│ http://masterh4.adriver.yandex │ 8 │ -│ http://zarplata.ru/?p=12977-B26358/currency=RUR/page=1080&wi=1024&lo=http://rzh... │ 8 │ -│ http://en.lyrsenses/zamba_zabor/bedroomolsk │ 8 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ==&page_avtomodules.php?f=100&ref... │ 8 │ -│ http://mastered/main.aspx#location=1&bc=3&ct=1&pr=60322056107100919/page5/?_h=s... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://omsk.mlsn.ru │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://irr.ru/6323%26bn%3D27888895,96779/87 │ 8 │ -│ http://smeshariki.ru/GameMain │ 8 │ -│ http://che.ru&pvid=13733142835/100/topic,5240556895&ch=UTF-8&sF=11,7,7,0 │ 8 │ -│ http://afisha.yandex.php?gidcar=367108851%2Fr%2F1 │ 8 │ -│ http://love.ru/?p=17059 │ 8 │ -│ http://wildberries.xml?from]=&input │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 8 │ -│ http://irr.ru/index.php?showalbum/login-kupit-topy%2Fplatjie-gotovlexandex.html... │ 8 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 8 │ -│ http://auto.ria.ua/auto_id=24126629/0/index.ru/real-estate/out │ 8 │ -│ http://victor?page_type=city.stol-yar.ru/cars │ 8 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://video.yandex.ru%2F&sr=http://loveplants/65398f55 │ 7 │ -│ http://kinopoisk.ru/ch/feed/letniaiaprice/1609 │ 7 │ -│ http://real-estate/aparts/Aquarevski │ 7 │ -│ http://video.yandex.ru/a-album/login-vitiju/photo │ 7 │ -│ http://bonprix.ru/social/product_id │ 7 │ -│ http://moscow/detail/5552/0/2792834&m=111,7,7,5 │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 7 │ -│ http://irr.ru/Registered/main/topnewsru.com/page=6 │ 7 │ -│ http://irr.ru/index.php?showalbum/login-kapustics?sort=pogoda.yandex.ru%26bt%3D... │ 7 │ -│ http://bonprix.ru/catalog/8570/14139489 │ 7 │ -│ http://afisha.yandex.ru/\xD0\xB4\xD0\xBE\xD0\xBC\xD0\xB0/\xD0\x91\xD0\xA1\xD0\x... │ 7 │ -│ http://irr.htm?from]=&int[85][from]=&input_vsegodnyie │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://love.ru/?p=1#country=&op_seo │ 7 │ -│ http://bdsmpeople.ru/niktory/shtory/308/roomed.ru/p59473682740295 │ 7 │ -│ http:%2F%2Fwwwww.bonprix │ 7 │ -│ http://spb/event=big&marka=84&model=0&auto_id=0&s_yers=0&pv=10&can_be_check_PP │ 7 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 7 │ -│ http://myloveplantrackIt?tid │ 7 │ -│ http://pogoda.yandex.ru&pvid=1 │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 7 │ -│ http://irr.ru/index.php?showalbum/login-a-5-advert27114 │ 7 │ -│ http://bdsmpeople │ 7 │ -│ http://video=0&is_hot │ 7 │ -│ http://irr.ru/index.php?showalbum/login-zk34/pages/0001216629 │ 7 │ -│ http://svpressa.ru │ 7 │ -│ http://money.yandex │ 7 │ -│ http://gotovim-doma.ru │ 7 │ -│ http://afisha │ 7 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 7 │ -│ http://omsk/evential/house.ru/catalog/kitchedule=213-606361653965283 │ 6 │ -│ │ 6 │ -│ http://video.yandex.ru%2Fkategory_id │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://afisha.yandex.ru%2F%2Fmail/169 │ 6 │ -│ http://love.ru/?p=1#country=-1&sq_liver.ru/kyrgyzstan │ 6 │ -│ http://video.yandex.ru/ekt │ 6 │ -│ http://stars-varenok.ru/16745959680706/800_0.jpeg.html%3Fhtml5/v123593 │ 6 │ -│ http://bdsmpeople.ru/film/64544.690022.rar.html_params%3Drhost%3D_black_list=0&... │ 6 │ -│ http://afisha.yandex.ru/catalog=on │ 6 │ -│ http://bdsmpeople.ru/GameMain │ 6 │ -│ http://v102.ru/investate/apartments-sale │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://omsk/evential/housession%3D0 │ 6 │ -│ http://wildberries.ru/search │ 6 │ -│ http://video.yandex.ru/GameMain.aspx#location/page_type=category │ 6 │ -│ http://direct.yandex.ru/refererprofile%2F2.10 │ 6 │ -│ http://video.yandex.ru/a-topy │ 6 │ -│ http://afisha.yandex.ru%26bt%3D43%26anbietersburg │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://rukodel=0&sort=newly&trafkey=2750 │ 6 │ -│ http://moscow/details │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://smeshariki.ru/world/photofider_credit=0&view │ 6 │ -│ http://irr.ru/index.php?showalbum/login-12.html%26custom │ 6 │ -│ http://novosibirsk.irr.ru%26bid │ 6 │ -│ http://solutions.diary.ru/realtitroenie_v_jurman.ru/albums/frame-owa.html?stric... │ 6 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 6 │ -│ http://mr7.ru/newsru.com/iframe_right%3D43 │ 6 │ -│ http://autodoc.ru/real-estate/apart │ 6 │ -│ http://smeshariki.ru/catalog │ 6 │ -│ http://povari.yandex.ru/greecondary/Products_id=&auto_vaz_2111 │ 6 │ -│ http://edp2.adriver.ru/hocketshop.ru/moscow/detailanude │ 6 │ -│ http://video.yandex.ru/catalog │ 6 │ -│ http://ereal-estate/rent │ 6 │ -│ http://msk/platia-nashing/vantralitsa_transion │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://3dnewsru.com/iframe_right.html?1=1&cid=51538 │ 6 │ -│ http://afisha.yandex.php/board,39.04839 │ 6 │ -│ http://novo/detail.aspx?group_cod_1s │ 6 │ -│ http://auto_hyundai_sarator │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://kalininmyclonus1 │ 6 │ -│ http://bdsmpeople.ru/saledParams=rhost%3D43%26bid%3D1 │ 6 │ -│ http://forum/topnews/22294&op_category │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://love.ru/?p=17055335 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://real-estate=week/page=1&expand_search?film/298677435615.html │ 6 │ -│ http://afisha.yandex.ru/mymail.php │ 6 │ -│ http://koolinar.ru/port.ru/doc │ 6 │ -│ http://video=0&with_exchangeType │ 6 │ -│ http://afisha.yandex.ru/index.ru/recipe │ 6 │ -│ http://kinel-lab.com/rus/20130709_117485994,93304&op_seo_entry=1&gearbox=0&type... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ https://slovakia-600dd903c07022,101595,91194&op_seo_entry │ 6 │ -│ http://video.yandex.php?search?text=\xD0\x9A\xD0\xBE\xD0\xBD\xD1\x8C\xD1\x8F\xD... │ 6 │ -│ http://diary.ru/exp?sid=3205 │ 6 │ -│ http://video.yandex.php?from]=&interapy-wkti/ &cd │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 6 │ -│ http://love.ru/ru/irk/event/search/ │ 6 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D8393224 │ 6 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 6 │ -│ http://kinopoisk.ru/catalog │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://auto_volkswagen/vologdano/il_dlya-dnevka.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 5 │ -│ http://barnaul/details/?cauth │ 5 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD0\xB5\xD1\x80&where=... │ 5 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 5 │ -│ http://metal-lodku-Obzor │ 5 │ -│ http://sp-mamrostova │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 5 │ -│ http://zvukovo-gorodsk │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://tp66.ru/exp?sid=3860217/rooms=2/men.aspx#location%3D0%26rnd │ 5 │ -│ http://rmnt.ru/search/offilia_Sovetov_living_chamber/?78142 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://e96.ru/movies/614418821/artir.ua/search │ 5 │ -│ http://video.yandex.by/?state_id=&auth=1..630;IC,7711588 │ 5 │ -│ http:%2F%2Fwwww.bonprix.ru/?id=2013&where=all&filmId │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://povari.yandex │ 5 │ -│ http://slovari.yandex.ru%26orderovskij-index.ru │ 5 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_seo_entry=&op_category_... │ 5 │ -│ http://tks.ru/cat/publish-cherkalnaya-ttpodporyadushek │ 5 │ -│ http://slovarenok.com │ 5 │ -│ http://video.yandex.ru/GameMain.aspx?Link │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://auto.ria.ua/auto_id=241269.html?1=1&cid=2127970 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://auto_volkswagen-Palities/horobki │ 5 │ -│ http://slovari.yandex │ 5 │ -│ http://radio&planet.ru/work.ru/catalog │ 5 │ -│ http://afisha.yandex.php?r=23436303135353.html?1 │ 5 │ -│ http://en.lyrsenses/zamba_zaborah_chamberk │ 5 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 40007&pt │ 5 │ -│ http://rustnye-sht-riemnikoi │ 5 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 5 │ -│ http://sslow_135000008&position=search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 5 │ -│ http://pogoda.yandex.php?gidcar │ 5 │ -│ http://myloveplanet.ru/v14572&lb_id=13000001216629 │ 5 │ -│ http://topnews.ru/GameMain.aspx?group_cod_1s=1983&pt=b&pd=9&pw=0 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://bibidohertki-i-OOOO_REPORT/07_2013 │ 5 │ -│ http://real-estate=week&m=Dvigenie.html?option │ 5 │ -│ http://autodoc.ru/moscow │ 5 │ -│ http://avtomobile/motory.ru/comp.ru/view.php │ 5 │ -│ http://love.ru/product_id=0&po_yers=0&po_yers=2&refererro/model=1346488078722&c... │ 5 │ -│ http://auto.ria.ua/auto │ 5 │ -│ http://loveplanet.ru/mymail/rudi │ 5 │ -│ http://video.yandex.ru/firms.turizm │ 5 │ -│ http://video.yandex.ru%2FkategoriendflowerTo=&powerTo= │ 5 │ -│ http://auto.ria.ua │ 5 │ -│ http://msk/platia-nashing/vannyie-product_id=1841&page2 │ 5 │ -│ http://nizhnieiewva88/photo/101246465376&cmd=show-to-buchaiev-pugache=51db32a68... │ 5 │ -│ http://direct.yandex.ru/index │ 5 │ -│ http://tks.ru/cat/publish-chemec.ru/search?filmId=CktclMBmUXI │ 5 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid%3D158197%26width │ 5 │ -│ http://club.ru/cinema/movies/no-pos │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://kinopoisk.ru/spb.pulscen.ru/exp?sid=3159&op_category_id=&auth=0&checked=... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://holodilnik-rp-ploschaya-obuv/?ci=1280&with_video=0&choosOyg==&op_uid=111... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://wildberries.ru/real │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://slovariant_neu%3D1%26bid%3D1216/0001216629%26bt%3Dad │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://md.mirkovskaya-obl.irr.ru/jobinmoscow │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://wildberrior/kia/ │ 5 │ -│ http://loveplanet.ru/\xD0\xB0\xD1\x80\xD0\xBE\xD0\xBC/curre-ap-i-showalbum/loui... │ 5 │ -│ http://povari.yandex.php?showalbum/login.pl?cl=all&film/497794,90458 │ 5 │ -│ http://jcmotorom-921205&bt=7 │ 5 │ -│ http://en.lyrsense.com/obshchin-idieiala │ 5 │ -│ http://smeshariki.ru/topic │ 5 │ -│ http:%2F%2Fwww.bonprix.ru │ 5 │ -│ http://gaylyU │ 5 │ -│ http://moscow/detail/Torgovuyu-organske.ru/real-estate │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://video.yandex.ru%26bt%3Dad.adriver.ru/recipe/view/10217/?from │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&w... │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 5 │ -│ http://loveplanet.ru │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://alpari.ru/gallery/pic845274 │ 5 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 5 │ -│ http://alib.mist.html%26custom%3D%26c2%3D278888592138 │ 5 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 4 │ -│ http://loveplanet │ 4 │ -│ http://video.yandex.ua │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 4 │ -│ http://video.yandex.ru/price │ 4 │ -│ http://whoyougle.ru/images/images/00000i/specifiers.ru/image=1&furniture │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://sslovarenok.ru │ 4 │ -│ http://samarskii_krai/tuapse/detail/result.aspx │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/filmId │ 4 │ -│ http://auto.ria.ua/auto_id=0 │ 4 │ -│ http://loveche.html │ 4 │ -│ http://bdsmpeople.ru/Web/Pages=1/feedsmag.ru/~\xD0\xBA\xD0\xBD\xD0\xB8\xD0\xB3\... │ 4 │ -│ http://afisha.yandex.ru/zoom.php?f=5162613838.html_partments │ 4 │ -│ http://irr.ru/index.php?showalbum/login-10618968476372773 │ 4 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://wildberries.ru/catalog │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 4 │ -│ http://radio&planet.ru/marka=62&model=1178128455&pvno=2&evlg=VC,5 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://direct.yandex.php?formsof(INFLECTION_ID=30861/14365-4b11&state/apartment... │ 4 │ -│ http://afisha.yandex.ru%252f7769%252fe │ 4 │ -│ http://love.ru/?p=1#countpage/vacancies/events/738/0/3/women.aspx │ 4 │ -│ http://echoradar-s-Levoshcha │ 4 │ -│ http://e96.ru/news/39733/page2=&input_sponsor=&o=1015219.html_partments-sale&pa... │ 4 │ -│ http://diary.ru/catalog=on&input_city[3 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://rasp.pl?cmd │ 4 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://travel.ru/state/apartments-sale/rashinitit%2F537 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.php?view_type=2&driveresult.ru/replies │ 4 │ -│ https://smeshariki.ru/cinema/article10363136000001216629%26site_offilia_Sovets.... │ 4 │ -│ http://auto.ria.ua/auto_id=63799.html_params │ 4 │ -│ http://kinopoisk.ru/saledParams%3Drhost%3Dad.adriver.ru/GameMain.aspx#location │ 4 │ -│ http://ssl.hurranovskaya-ul-31-foto.ria │ 4 │ -│ http://afisha.yandex.php?r=3&bs=&day │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboy07/... │ 4 │ -│ http://smeshariki.ru/search/keukeru-soft │ 4 │ -│ http://vkirovoe-tourisma │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://smeshariki.ru/domchelkakh_location=search?text=\xD0\xBC\xD0\xB8vents │ 4 │ -│ http://card/windows)&bL=ru&cE │ 4 │ -│ http://kniga.ru/view=\xD0\xA1\xD1\x82\xD0\xB0\xD0\xB2\xD1\x80\xD0\xBE\xD0\xB2&w... │ 4 │ -│ http://guid=6&pw=2&pv=0&price_do=¤cy=1 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://video.yandex.ru/real │ 4 │ -│ http://afisha.yandex.ru/search │ 4 │ -│ http://v102.ru/?s=Adaments-sale │ 4 │ -│ http://video.yandex.ru/GameMain.XYyZwYXRoPWEtbG9nb24vcG90Cw │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://matched_car=373838928155755775482794,9453.116 Safari%2F537.36&he=10&s_ye... │ 4 │ -│ http://zarplata.ru/?p=12977-B26358/hasimages=1/page │ 4 │ -│ http://tks.ru/cat/publish-chernyjbelyj-9375966238&op_categoriya │ 4 │ -│ http://video.yandex.ru/GameMain.aspx#location-4.0.html │ 4 │ -│ https:%2F%2Fwwww.yandex.ru │ 4 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 4 │ -│ http://msk/platia-nashing/vanny.diary.ru/social │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://auto.ria.ua/canel_pe_mascona Hilfigeratov/153228 │ 4 │ -│ http://video.yandex.ru/realty/leaser_map=1/hasimay-2.html │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://nizhny novgorod │ 4 │ -│ http://video.yandex.ru%26target=search/ab_area=categorija80119 │ 4 │ -│ http://kubikus.ru/search.php?r=4140211,1,7,7,7,7,0 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://e-kuzbass.ru/a-shop │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://guide.travel.ru/link │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://victorhead.php?full&dom=780067167694.0; │ 4 │ -│ http://afishi,Mudanted-belyj-974299099/guest-id=34089.html │ 4 │ -│ http://fitness/building │ 4 │ -│ http://afisha.mail.aspx#locationalOffers │ 4 │ -│ http:%2F%2Fwwww.bonprix.ru/real-estate/out-of-town/house │ 4 │ -│ http:%2F%2Fbrjuki-lic-shop.ru/cart.php/cars/papago-d-plosch │ 4 │ -│ http://svpress_up.aspx#location[propfilter_pf[PODVAL]=&arrFilter][3463351841195... │ 4 │ -│ https://produkty%2Fpulove.ru/gost/?page3 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://anketka.ru/bridget │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.ru/GameMain/dukhovyy │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadbor_sh... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http%3A//magnitogorod/page3/#over │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://afisha.yandex.php?action/2741920 │ 4 │ -│ http://tks.ru/cat/public/gamemain.aspx#location │ 4 │ -│ http://love.ru/image=2&marka=84&model/mihailovo │ 4 │ -│ http://nepogoda.yandex.ru/search=1&target=search=0&can_be │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://kinopoisk.ru/searchAutoSearch=0&driver.ru/catalog/1/women.aspx#locationp... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://afisha.yandex.php?show=rlv&ru=1&expand_search │ 4 │ -│ http://love.ru/?p=1705 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://ssl.hurra.com │ 4 │ -│ http://internet Explorer&aV=5.0 (Windows)&bL=en │ 4 │ -│ http://auto_volkswagen_cated_cartovskaya-obl.irr.ru/album/login │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://pogoda.yandex.ru/cooking_sm5_1148786993ab-417/photo/69363/26#formi.ru/co... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.ru/If yours.avtogsm.ru/animals/calculate │ 4 │ -│ http://afisha.yandex.php?p=31&input │ 4 │ -│ http://auto_s_product_name=\xD0\x9A\xD1\x80\xD1\x83\xD0\xB3\xD0\xBB\xD0\xBE\xD0... │ 4 │ -│ http://loveche.ru/job/1162323&PAGEN_1=30&state │ 4 │ -│ http://rsdn.ru/details_103514,154;IC │ 4 │ -│ http://video.yandex.ru/topnews.ru/lanas-advert2713][to]=&int │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://video.yandex.ru/filmId=Xtvman98/num-1/refresh/russia/chapter/broadboyzon... │ 4 │ -│ http://auto_volkswagen_pass_1161967 │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http%3A//edp1.adriverys/forum/view_type=city&custom=0&damages/0001216629%26bid%... │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://inspelishchin-platjie-doma.ru/irkutsk.irr.ru │ 4 │ -│ http://video.yandex.ru/index.ru/\xD0\x9F\xD0\xBE\xD0\xB2\xD1\x82\xD0\xBE\xD1\x8... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://smeshariki.ru/saint-petersburg-gorod/transfer/?id=7576149959760994861&op... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 4 │ -│ http://en.lyrsenses/zamba_zabudka/photo-1/#page=0&sale/search?text=\xD0\x9C\xD0... │ 4 │ -│ http://afisha.yandex.ru/content2.adriver │ 4 │ -│ http://arma/frl-4/transportnoy-kv-m-malchik.ru/show │ 4 │ -│ http://b.kavanga.ru/?a=inneVolumeFrom │ 4 │ -│ http://auto_id=0&with_photo │ 4 │ -│ http://tp66.ru/money.yandex.ru/albums_screenterval │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 4 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&w... │ 4 │ -│ http://irr.htm?from]=&int[1151;IC,112 │ 4 │ -│ http://yoshka.diary.ru/exp?sid=3149&op_category_id=592b9e01c48ce9403%26bn%3D0%2... │ 4 │ -│ http://bdsmpeople.ru/film/64544.690078 │ 4 │ -│ http://afisha.yandex.ru&pvid=13733424 │ 4 │ -│ http://video.yandex.ru/GameMain │ 4 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 4 │ -│ http://kinopoisk.ru/service.ru/iframe │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ https://orenburg/?arrFiltersburg │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://radiorecord.ru/login-m3w.html?1=1&cid │ 3 │ -│ http://video.yandex.ru%2Fplata.ru/ch │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://banantikov-nadushenie_orleona_server=sc.cheloveplant_11612/page │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://romar/events/audio.ru/widget[]=vacancies/99.php?gidcar │ 3 │ -│ http://afisha.yandex.php/tova.ru/uliya2076789599305953 │ 3 │ -│ http://afisha.yandex.ru/shop.ru/malta │ 3 │ -│ http://afisha.yandex.ru/kategoriya%2F5.0 (company │ 3 │ -│ http://afisha.yandex.php/topic104780204&op_uid=1954 │ 3 │ -│ http://afisha.yandex.ua/donetsk/urals │ 3 │ -│ http://afisha.yandex.ru/a-folders/misc │ 3 │ -│ http://afisha.yandex.ru/project_price=&maxprice │ 3 │ -│ http://afisha.yandex.ru/linkvac.php/board.php?topicseeng │ 3 │ -│ http://radioscannerica/filmId=Ba_id=13733568414&city=\xD0\x9C\xD0\xBE\xD1\x81\x... │ 3 │ -│ http://irr.kz/realty/lease/3516093&pvno=2&evlg │ 3 │ -│ https://produkty%2Fplatjie-kuzbass.ru/newsru.com/iframe_right=0&auto_ria=0&meta... │ 3 │ -│ http://fuckfind=rent/view/2021/3 │ 3 │ -│ http://samara.irr.ru%2Fproduct │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://auto_kia_30 │ 3 │ -│ http://sp-mamrostokonkursovet │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://astrobank.ru/image │ 3 │ -│ http://love.qip.ru │ 3 │ -│ http://direct&sortdirect.yandex.ru/imagesize%3D0%26ar │ 3 │ -│ http://kaluga/?ext=\xD0\xB3\xD0\xB5\xD1\x80\xD0\xBE\xD1\x8F\xD1\x82\xD0\xBD\xD1... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ https://slovakia-600dd903c06c999c226647639.html%3Fhtml │ 3 │ -│ https://slovakia-600dd903c07022,101595,9143531427800648_elit │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category_id=0&wi=16000&... │ 3 │ -│ http://love.ru/forum.cofe.ru/forum/view_type=city=790&Selectronics-technik │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://sp-money.yandex.ru │ 3 │ -│ http://sp-money.yandex.ru/work rushki-sien-natalog/8570/page=0&expand │ 3 │ -│ http://video.yandex.ru&pvid=13735/?_h │ 3 │ -│ http://forum/topnews/2229605699574.html?1=1 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberrior/bedroom]=&int[17][to]=&int[858 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://afisha.yandex.ru/forum.materinburg │ 3 │ -│ http://afisha.yandex.php?addriver.ru │ 3 │ -│ http://video.yandex.by/search/?target%3D43%26bid%3D2 │ 3 │ -│ http://ssl.hurral=messages │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://bdsmpeople.ru/search │ 3 │ -│ http://video.yandex.ua/auto_id │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://rmnt.ru/stars │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://rmnt.ru/film/88677/russia/rio.ru/search?filmId=NNr6aJrm4s3M │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://video.yandex.ru/circle&state/out-of-town/houses │ 3 │ -│ http://irr.ru/imagecache/wm/2013&where=all&film.ru │ 3 │ -│ http://b.kavanga.ru │ 3 │ -│ http:%2F%2Fwww.bonprix_ru}%2Fnizhniynovgorod/request-id │ 3 │ -│ http://edp2.adriver.ru/catalog/181 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://pogoda │ 3 │ -│ http://pogoda.yandex.kz/family │ 3 │ -│ http://svpress_w1t1042796786/6/?category │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://edp2.adriver.ru/jobinmoscow/detail │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://radiorecord.ru/catalog/idShare │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com/iframe-owa.html?1=1&cid=577&o... │ 3 │ -│ http://msk/planet.ru/mymail.aspx#comme_me_saydinne │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://card/windows NT 6.1) AppleWebKit%2F5 │ 3 │ -│ http://afisha.yandex.ru/forum.donfiscategory │ 3 │ -│ http://tks.ru/cat/publish-chin-play.php?categoriya%2Fzhiensmed │ 3 │ -│ http://video.yandex.php?topbloveche │ 3 │ -│ http://kazan.irr.ru/location │ 3 │ -│ https://produkty%2Fpulove.ru/voronezh-sien-zhienskaia-moda-zhienskaia-moda-zhie... │ 3 │ -│ http://nigma.ru/product&op_category_name=\xD0\x91\xD0\xB8\xD0\xBA\xD0\xB8\xD0\x... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26url%3D//ad.adriver.ru/link/justic/h2.php/... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://loveche.ru/volzhskiy │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=1&s... │ 3 │ -│ http://smeshariki.ru/a-phony │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://magnitka_1_series.ru/?favorite_id=636233644&op_category_id=937514 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://ussuriysk.irr.ru/catalog/premiere/628962851d7fd0b6eb17b321d336f5bc7de189... │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountpage │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_produkty%2Ftanki │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/92054446660.ht... │ 3 │ -│ http://ekategoriya%2F9B206 Safari │ 3 │ -│ http://afisha.yandex.ru/tatatit_chto.php?industry │ 3 │ -│ http://smeshariki.ru/catalog/286/women.aspx │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://svpress/showbiz/photo.htm │ 3 │ -│ http://svpressa.ru/content/search │ 3 │ -│ http://video_dvd/game/iframe-owa.html │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://tp66.ru/search/?page=1&fuelRateTo │ 3 │ -│ http://e96.ru/real-estate │ 3 │ -│ http://lk.wildberries │ 3 │ -│ http://my.kp.ru/albumfoto-1/pol-2 │ 3 │ -│ http://holodilnik.ru/catalog │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://msk/platia%2Fzhienskaia │ 3 │ -│ http://personal/atlants/7292&xdm_c │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://povarenok.ru/files/eliteh.ru/perm.irr.ru/board,75.2013-07-09 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://brand=498&pvno │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://sp-mamrostovestory │ 3 │ -│ http://love.ru/?p=1#country=&op_seo_entry=-1&target │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://smeshariki.ru/?win=82&stat=141882,373;IC,2552f48 │ 3 │ -│ http://amobil-nye-pliazhnaia │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://irr.ru/washek-s-printom │ 3 │ -│ http://msuzie │ 3 │ -│ http://kinopoisk.ru/search │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://irr.ru/index.php?showalbum/login-kupaljinik-chere │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://smeshariki.ru/real │ 3 │ -│ http://wildberries.ru/item_no=2&evlg=VC,0;VL,205;IC,14;VL,757138/currency=RUR/h... │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostavia.travel.ru/s... │ 3 │ -│ http://3dnews.ru/?p=12636464/5#f │ 3 │ -│ http://smeshariki.ru/goodavec/photo/6936325.html?id=223978/page=102 │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic=79799398/?_h=search.htm... │ 3 │ -│ http://product_brand=RAINBOW&op_cated_content/search/keup/en-ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://love.ru/recipes/indows │ 3 │ -│ http://kaluzha-na-NovletedAutoSearch=0 │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory_id=577&search/ab_district/date_id=2271][from... │ 3 │ -│ http://afisha.yandex.php?app=membered │ 3 │ -│ http://afisha.yandex.php?ELEMENT_ID │ 3 │ -│ http://afisha.yandex.ru/lesyach-hotels │ 3 │ -│ http://afisha.yandex.ru%26bt%3D90%26nid%3D1216629 │ 3 │ -│ http://afisha.yandex.php?id=727285 │ 3 │ -│ http://love.ru/ru/irk/event=little&cated_country=-192.html?1=1&cid │ 3 │ -│ http://love.ru/?p=17057 │ 3 │ -│ http://str_ob.html?1=1&choosO8gPJSs3M&where=all&filmId=mAyiC7y6M2mGV2GoA9hFoN3q... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D1216629/0/&&puid1=m&puid2=23&pvno=2... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://you_hashkaf.ua/search │ 3 │ -│ http://auto_repairs=0&confiscategoriya │ 3 │ -│ http://wildberries.ru/GameMain │ 3 │ -│ http://povaria/chak_naytimes.ru │ 3 │ -│ http://pogoda.yandex.ru/real-estate │ 3 │ -│ http://afisha.yandex.ru/comment/search?text=\xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://kinopoisk.ru/sessulyanovka.ru/photosessid=3205&bt │ 3 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD0\xB2\... │ 3 │ -│ http://li.ru/filmId=XpzlPj8P8gE&where=all&text=\xD1\x81\xD0\xBA\xD0\xB0\xD1\x87... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://wildberries.ru/basket&ch │ 3 │ -│ http://soft.oszone.ru │ 3 │ -│ http://afisha.yandex.ru%2Fobuv-sapozhkivka=23&price │ 3 │ -│ http://afisha.yandex.ru/hotel-agen-Goluboj-9730 │ 3 │ -│ http://notebooking pressa │ 3 │ -│ http://auto_id=0&engineVolumeFrom │ 3 │ -│ http://smeshariki.ru/GameMain.aspx#location │ 3 │ -│ http://state/room=94720-recept-Salat-iz-glasya1lesyat │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://auto_repairs=0&po_yers=0&price.ru/\xD0\xB6\xD0\xB5\xD0\xBB\xD0\xB5\xD0\x... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://video_dvd/ratesTypeSearch?text=\xD1\x87\xD0\xB5\xD0\xBB\xD0\xBE\xD0\xB2\... │ 3 │ -│ http://omsk/evential/housession%3D0%26rnd%3D2%26bt%3D2%26nid%3D158197%26ad%3D21... │ 3 │ -│ http://slovari.yandex.ru │ 3 │ -│ http://kommersantamina │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://audio_video.yandex.ru/mosday.html?item=4#photo446962 │ 3 │ -│ http://afishers/story │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://video.yandex.ru/personal/offeebe34c7e12944&op_product │ 3 │ -│ http://v102.ru/investate/apartment/?id=137336IseNhcbx3J85GkHSnzgnsPdZUU&where=a... │ 3 │ -│ http://alpari.yandex.ru/saint-petersburg.irr.ru/cars/page3 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ https://slovarenok │ 3 │ -│ http://afisha.yandex.ru/real-esta.info/newsru.com.ua/kiev/detail.ru/auth=1..640... │ 3 │ -│ http://trashbox.ru/book │ 3 │ -│ http://video.yandex.ru/extra │ 3 │ -│ http://dom.net/provoe-pervouralnaya/ChildGluZ19oZXJlci1kYXRpb25zPU4mbj0zJmlkPTM... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/photo/narod.irr.ru/katering=1&pr=569&s_yers │ 3 │ -│ http://guid=6&pw=2&pv=0&with_video.yandex.ru │ 3 │ -│ http://love.ru/ru/irk/event=little&category_id=731-643736&mode=1 │ 3 │ -│ http://pogoda.yandex.ru/vlas-moskovskaya │ 3 │ -│ http://auto_ford Mix).mp3.ucoz.ru/v1430497.html%3Fhtml │ 3 │ -│ http://omsk/evential/housession%3D240%26rleurl%3D//ad.adriver.ru/marshavskaya-r... │ 3 │ -│ http://video │ 3 │ -│ http://radiorecord │ 3 │ -│ http://wildberring │ 3 │ -│ http://video.yandex.ru%2Fkategory_id=9759527418 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://smeshariki.ru/cgi-bin/click.cgi%3Fsid%3D0%26pz │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://bdsmpeople.ru/show/39932/Itemid,40.0.html_params%3DfsSaHR0cDovL2pzLnNtaT... │ 3 │ -│ http://state_shariki │ 3 │ -│ http://wildberries.ru/cgi-bin/click.cgi%3Fsize │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://balcon_caddy Club relove │ 3 │ -│ http://loveplanet.ru/Bezli-all │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://afisha.yandex.php?partments │ 3 │ -│ http://en.lyrsenses/zamba_zabor_polnitsa-s.narod.irr.ru/bank/otzyvy/12031%2F%23... │ 3 │ -│ http://video.yandex.ru/real-estate/out-of-town/house.ru&pvid=1&distreet_legkovo... │ 3 │ -│ http://auto_id=0&color=0&confiscategory_id=3205&bt=7&bn=1&bc=3&ct=1&prr=http:%2... │ 3 │ -│ http://kinopoisk.ru/registernet Explorer&aV=5.0 (Windows NT 5.1; ru-ru&cE=true&... │ 3 │ -│ http://kinopoisk.ru/spb.pulscen.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://saint-peter │ 3 │ -│ http://lazarevskoe │ 3 │ -│ http://whoyougle.com/iframe/iframe_right.ru/spb │ 3 │ -│ http:%2F%2Fwww.bonprix.ru/imagesize │ 3 │ -│ http://afisha.yandex.ru/manga.ru/?rtext=\xD0\xBF\xD1\x80\xD0\xB8\xD0\xB1\xD0\xB... │ 3 │ -│ http://smeshariki.ru/furniture.html5 │ 3 │ -│ http://cxem.net/324487194836848 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,947... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://video.yandex.ru/realty/search/main.aspx?sort=popular │ 3 │ -│ http://myloveplanet.ru/passenger/kitched_country_id=4312&input │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_product_brand=4200&lo=h... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://sunmar.ru/cart&ch=utf-8&sF=11,7,700&aN=Opera&aV=9.80 (Windows │ 3 │ -│ http://ssl.hurra.com/iframe-owa │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://saint-petersburg-gorod/goodal │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://un1.adriver.ru/page=30138117749516%252f110916%252fmedicinema/movie_ross ... │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&film/67... │ 3 │ -│ http://klubnich/zrh/ │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://wildberries.ru/comment/search │ 3 │ -│ http://wildberries.ru/comme%2F2.12.388 Version/1552/page │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://krasnodar.irr.ru/yaransferapid │ 3 │ -│ http://jobs-education │ 3 │ -│ http://gotovim-doma.ru/personal/commersant.ru/image=19&pvno=2&engineVolum │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 3 │ -│ http://irr.ru/index.php?showalbum/logizer8/num-1/refremost │ 3 │ -│ http://bdsmpeople.ru/index.by/ru/page=0&confiscategory_id │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://pogoda.yandex.ru/real-estate/apartments/73151 │ 3 │ -│ http://pogoda.yandex.ua/telefon_shtukaturkey │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://video=0&input_with_video.yandex.ru/page/10/women.aspx │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://afisha.yandex.ru/sell/resident │ 3 │ -│ http://afisha.yandex.php?p=176d43f96ef32d5bc1272 │ 3 │ -│ http://radioscannerica/film/47018.html?1=1&cid=691390&pvno │ 3 │ -│ http://auto_id=0&color=0&confiscategoriya%2Fzhienskaya-advert25593 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://forum/topics/ingradskazka-lookoformalities/poetry/events/?date&csrf-8200... │ 3 │ -│ http://video_dvd/suppoll/dleead6718.php?SECTION │ 3 │ -│ http://kinopoisk.ru/cgi-bin/click.cgi%3Fsid%3D1216 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=&is... │ 3 │ -│ http://rukodel=0&sort=newly&trafkey=27065/2/child.aspx#location │ 3 │ -│ http://zapchastny_fashing_mashes/index.ru │ 3 │ -│ http://msk/platia-nashing/vanny.diary.ru/otdam_daily │ 3 │ -│ http://smeshariki.ru/news/2013/peshnye-udivlekanka.ru/l_03_00/bodreamfood.ua │ 3 │ -│ http://myloveplanet.ru/bad_by_sidential/nizhnieiewva88 │ 3 │ -│ http://kinopoisk.ru/real-estate=2013-07-2089241607/photo=0&with_exchangeType │ 3 │ -│ http://video.yandex.ru/page=0&category&op_seo_entry=&op_category/used/KIA-Cee-d... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://direct.yandex.ru%2F&sr │ 3 │ -│ http://tyva-5/country=-1&washestvo/den_sidentialAmount │ 3 │ -│ http://mylove.ru/zoom.php?GID=2&IsOrder │ 3 │ -│ http://ssl.hurra.com/iframe/iframe-owa.html%26custom%3D%26custom │ 3 │ -│ http://en.lyrsenses/zamba_zabudka/procoolonelopitered/ministrict/3d-probeg-340-... │ 3 │ -│ http://arma/frl-4/travel.ru/moscow/details │ 3 │ -│ http://maps#ru_5_ru_1_ru_ru_ru_202_ru_1_ru_ru_ru_ru_2_ru_1743.html?oscsid=36303... │ 3 │ -│ http://rsdn.ru/info_all=yes&razdumy-i-trikshop/search&_h_page/1719920 │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://irr.ru/index.php?showalbum/login-do-500-kv-m-Maya │ 3 │ -│ http://chehod/zvenia8312&input_who2=1&input_age1=35&aN=Netscape │ 3 │ -│ http://pogoda.yandex.ru%2Fkategory │ 3 │ -│ http://kuharka=48&modeloveplanet │ 3 │ -│ http://sendflower │ 3 │ -│ http://avtoto.ashx/1001087496197797217530729; Media Centernet_mastersburg │ 3 │ -│ http://kaluga/?ext=\xD0\xB1\xD0\xB0\xD0\xB4\xD0\xBC\xD0\xB8\xD0\xBD\xD0\xBA\xD0... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://smeshariki.ru/index.ru/main.pl?cmd=show/47555 │ 3 │ -│ http://zagranimals-planet.ru/user/31059&Module │ 3 │ -│ http://afisha.mail.ru/stars/page │ 3 │ -│ http://myloveplants_list-obl.irr.ru%2Fobuv-zhienskaia-modiezhda │ 3 │ -│ http://smeshariki.ru/chipinfo │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://forum/topnews/222968695,910112_1164074834-908745 │ 3 │ -│ http://myfashihtzu.html?1=1&cid=65625f313230303&po_yers=2013/07 │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://bdsmpeople.ru/Web/price │ 3 │ -│ http://votpusk.ru/ │ 3 │ -│ http://afisha.yandex.ua/index.ru/recipe │ 3 │ -│ http://whoyougle.ru/basket&ch=utf-8&sF=11,7,7,7,700 │ 3 │ -│ http://video.yandex.ru%2F%2Fwww.bonprix │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://auto_operey-v-v-meha.ru/catalog/public │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://irr.ru/6323%26bn%3D27888895,96772&op_page47 │ 3 │ -│ http://video=0&is_hot=0&che_simeis │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D0%26ad%3D1216629/0/index.ru%26bn%3D0%26nid%3D... │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://wildberries.ru/page=0&vip │ 3 │ -│ http://kurort/SP1399&op │ 3 │ -│ http://irr.htm?from]=&int[852][to]=10&lastdiscussins/?keyworld │ 3 │ -│ http://wildberries.ru/filmId=4920/roomamountry │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 3 │ -│ http://nizhnieie-bielie-bieriends&fb_source-temno │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://forum/topnews/2229362067528195&op_categoriya │ 3 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 3 │ -│ http://afisha.yandex.php?w=3160/transports │ 3 │ -│ http://afisha.yandex.php?link=114735200&brand=23368 │ 3 │ -│ http://stories.ru/art/MACKLEMORE │ 3 │ -│ http://irr.ru/bank/otkrovnja-instvo.ru/search?text=\xD0\xBF\xD0\xB8\xD0\xBA\xD1... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 3 │ -│ http://pogoda.turizm.ru/communit=0&vip=0&order_by=2 │ 3 │ -│ http://maps#ru_5_ru_227_ru_3630&state/apartments-sale/secondary/pic/89395&op_pr... │ 3 │ -│ http://smeshariki.ru/Web/price │ 3 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 3 │ -│ http://radiorecord.ru │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 3 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 3 │ -│ http://ej.ru/muss/roll_to_audi/mode=replies │ 3 │ -│ http://zarplata.ru/?p=1290&op_product_price=990348531&schoosOSRquM8gE&where=all... │ 3 │ -│ http:%2F%2Fmuzhchine/ru-g-Chelya_v_tsentyabrskii │ 3 │ -│ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&top │ 3 │ -│ http://ur.hh.ru/file/news/2013 \xD0\xB3\xD0\xBE\xD0\xB4\xD0\xB0 │ 3 │ -│ http://my.alpari.ru/filmId=yJg89hqV8gE&where=all&film/530/?fromCityCodeForcentr... │ 3 │ -│ http://auto.ria.ua/auto_id=1&bc=3&ct=1&pr=9476648245557.html%26custom=1&damage=... │ 3 │ -│ http://bdsmpeople.ru/cgi-bin/click.cgi%3Fsid │ 3 │ -│ http://love.ru/?p=1#country=-1&sq_total=\xD0\xBE\xD1\x82 │ 3 │ -│ http://guid=6&pw=2&pv=0&po_yers=0&with_video │ 3 │ -│ http://tks.ru/filmId=rQRZO_mhUXI&where=all&filmId=z7pOMYOJ8gE&where=all&text=\x... │ 3 │ -│ http://gorbus.aspx#localiformalitic │ 3 │ -│ http://msk/events/7401438966/page_type=0&m_city.info/forum.rostov.irr.ru/msk/ev... │ 3 │ -│ http://omsk/evential/housession%3D90%26rnd%3D839322%26ntype=0&expand_search/obm... │ 2 │ -│ http://omsk/evential/housession%3D%26custom=0&damages/0000&with_photo/photo/708... │ 2 │ -│ http://kinopoisk │ 2 │ -│ http://video=0&input_state │ 2 │ -│ http://poisk.ru/news/articles │ 2 │ -│ http://forum.php?t=420 │ 2 │ -│ http://mysw.info/node/21544 │ 2 │ -│ http://smeshariki.ru/tashkinsk │ 2 │ -│ http://smeshariki.ru/product&op │ 2 │ -│ http://yaroslavl.irr │ 2 │ -│ http://board=11.ua.150.html%3Fhtml │ 2 │ -│ http://pogoda.yandex.ru/jobinmoscow │ 2 │ -│ http://afisha.mail.ru/cheva.ru/reportby │ 2 │ -│ http://3dnews.ru/msk/events │ 2 │ -│ http://slovarenok.ru/chapochki │ 2 │ -│ http://pogoda.yandex.ru/real │ 2 │ -│ https://diary/2013-07-05/101090/currency │ 2 │ -│ http://krasnyj-chastory │ 2 │ -│ http://kinopoisk.ru/saint │ 2 │ -│ http://pogoda.yandex.php?showalbum │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://rlsnet.ru/nogin.html5/v12 │ 2 │ -│ http://spb/event=big&marka=84&model=0&auto_id=0&s_yers=0&pv=10&category_name=\x... │ 2 │ -│ http://moscow/detail/Torgovljatory/storii_efferanslyatting │ 2 │ -│ http://sslow_13500000%26rnd%3D278888 │ 2 │ -│ http://kaluga/?ext=\xD1\x80\xD0\xB5\xD0\xB9 \xD0\xBD\xD0\xB0 \xD0\xB1\xD1\x80\x... │ 2 │ -│ http://auto.ria.ua/auto_auto.ria │ 2 │ -│ http://mysw.inform%26q%3Dversion.cheltyj │ 2 │ -│ http://diary.ru/search?familliklink │ 2 │ -│ http://ftp.auto.ria.ua/search?text=\xD0\xB8\xD0\xB3\xD1\x80\xD0\xB0 5 \xD1\x81\... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,940... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://ssl.hurra.com.ua/auto_repairs=0&page=10&category_id=detail │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://pogoda.yandex.ru/chat/ivan │ 2 │ -│ http://svpressages/00012166260001216629%26sliceid%3D0%26ad │ 2 │ -│ http://video.yandex.ru/rent/info/messa.ru │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?so... │ 2 │ -│ http://yartb.html?city=55&TopicID=2&IsOrderedProduct │ 2 │ -│ http://wildberries.ru/editem_no=100¤cy=1#country=&op_proizvodskaya-obuv-z... │ 2 │ -│ http://wildberries.ru/filmId=4920/room=1&lang=all&filmId=DnGbyVQVUXI&wheretomug... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D//a... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D0%26pz%3D0%26rleurl%3D%26... │ 2 │ -│ http://news/398261_enl.jpg-1 │ 2 │ -│ http://afisha.yandex.ru/?trafkey=54073799 │ 2 │ -│ http://afisha.yandex.php?id=1620_4_licanel │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://ur.hh.ru/page=10&can_be_checked_auto_region=1&rm=1&lang=all&film/18212.1... │ 2 │ -│ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 2 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 2 │ -│ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 2 │ -│ http://wildberries.aspx#location/group_cod_1s=53&butto_repairs=0&with_photo=0&i... │ 2 │ -│ http://wildberries.ru/index.ua │ 2 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.045 user 0.440000 sys 0.004000 -D 0-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -┌────────────────┬────────────────┬─────────────┬────────────────────────────────────────────────────────────────────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────────┐ -│ TraficSourceID │ SearchEngineID │ AdvEngineID │ src │ dst │ pageviews │ -├────────────────┼────────────────┼─────────────┼────────────────────────────────────────────────────────────────────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────────┤ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33069 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 24703 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 15817 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 14233 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 │ 6549 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 │ 5257 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 5257 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 3547 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login │ 3538 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik │ 3371 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php │ 3333 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 3309 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 2923 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 2616 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 2441 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php │ 2418 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 2390 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari │ 2367 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 2349 │ -│ 1 │ 0 │ 0 │ https://google.com.ua/url?sa=t&rct │ http://komme%2F27.0.1453.116 │ 2228 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 1920 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://komme%2F27.0.1453.116 │ 1679 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ekburg.irr.ru%2Fpuloveplanet │ 1340 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://aliningrad │ 1270 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1254 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1213 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1193 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1186 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 1142 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F27.0.1453.116 │ 1007 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 954 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://aliningrad │ 859 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login │ 821 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 791 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555419/page=show_photo/70946/detail/55212.15&he │ 777 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 757 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 722 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://video.yandex.php │ 702 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 691 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/obucheyelants │ 684 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 632 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 593 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 577 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://smeshariki.ru/obucheyelants │ 566 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.php │ 452 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 445 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBA\xD0\xBE\xD0\xBD\xD1\x82\xD1\x80\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 422 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 421 │ -│ 0 │ 0 │ 0 │ │ http://obninsk/detail │ 407 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 386 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug.ru │ http://irr.ru/index.php │ 374 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru/index │ 360 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 348 │ -│ 2 │ 0 │ 13 │ │ http://komme%2F27.0.1453.116 │ 347 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-2011/43597 │ 293 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 293 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6HS │ 282 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 268 │ -│ 3 │ 72 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 267 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 259 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 257 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari │ 256 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 │ 241 │ -│ 1 │ 0 │ 0 │ http://google.ru/realty │ http://irr.ru/index.php?showalbum/login │ 237 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 │ 236 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/28435&lr=157 │ http://komme%2F27.0.1453.116 │ 233 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/ru/photo/6936313555&text=\xD1\x85\xD0\xBA \xD0\xBB\xD0\xBE\xD... │ http://komme%2F27.0.1453.116 │ 224 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php │ 223 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cat/dushkirillovyj │ http://komme%2F27.0.1453.116 │ 222 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=359&op_page2/... │ 218 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kriminally-bezhevsk │ 217 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0 │ 212 │ -│ 3 │ 1 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 207 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 202 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 199 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://ekburg.irr.ru%2Fpuloveplanet │ 199 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 198 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 197 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 195 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/414526863.xlsx │ 191 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CDMQFjAJ │ http://irr.ru/index.php?showalbum/login-kiriskaya-obl.irr.ru/index.ru/GameMain.... │ 190 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=0&with_photo.ashx/101/4/?cat=6257271 │ 188 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=partments-sale/search?text=\xD1\x81\xD0... │ 187 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://afisha.yandex.ru/index │ 183 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://ekburg.irr.ru%2Fpuloveplanet │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 178 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://ekburg.irr.ru%2Fpuloveplanet │ 171 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 168 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 164 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 163 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://ekburg.irr.ru%2Fpuloveplanet │ 160 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 160 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/life.ru/cars/misc/travel.ru/?trafkey=058143&p... │ 155 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 │ 154 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=577&oki=1&oby=&op_s... │ 152 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://wildberrior/uphold │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6HS │ 150 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 149 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login │ 146 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustic/meterburg │ 144 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 144 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 144 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php │ 143 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E │ 141 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 140 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://kinopoisk.ru/news/2146555f3530316995264from]=&int[27][]=&selection/01a54... │ 136 │ -│ 1 │ 0 │ 0 │ http://kombardighantnie │ http://komme%2F27.0.1453.116 │ 136 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login │ 135 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kysjacevtika │ 132 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://komme%2F27.0.1453.116 Safari │ 124 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 121 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category/stroy/dachines... │ 114 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/?state │ http://komme%2F27.0.1453.116 │ 113 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/GameMain │ http://komme%2F27.0.1453.116 │ 113 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 111 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login │ 110 │ -│ 3 │ 85 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 110 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kardigan │ 109 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 107 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosoftwarenok.ru/projects/zhbi.po... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 105 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268143.html?1=1&cid=577&o... │ 104 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 104 │ -│ 0 │ 0 │ 0 │ │ http://tvidi.ru/photo=0&confiscategory_id=0&engineVolumeFrom=&fuelRateFrom=type... │ 103 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 102 │ -│ 3 │ 14 │ 0 │ │ http://komme%2F27.0.1453.116 │ 102 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 100 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 98 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://video.yandex.php │ 97 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId=GVlrcUaGUXI&wher... │ 96 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.21150895 │ 96 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?sort=price │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 95 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login │ 95 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.12.388 │ 93 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 93 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 92 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberrior/uphold │ 92 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 91 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-tanks/search=1&ady=62&modeloveplanet.ru │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 90 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.145332.15&he=10&category_id=2740387993 │ 89 │ -│ 1 │ 0 │ 0 │ http://in-the-weightEnd=2351&numphoto=&isExclusiver.ru/alertljus │ http://komme%2F27.0.1453.116 │ 88 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fprodazha_Italja_unit=1&av=1&nm=1&lang=ru │ http://irr.ru/index.ru/show/414526863.xlsx │ 87 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 86 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb1.html?item_no=386703/?bundle=7172&msid=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 86 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 │ 85 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 84 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733582852/ │ 84 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupit/action │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 83 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 82 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 82 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?bodystyle │ http://komme%2F27.0.1453.116 │ 81 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F22.14&he=768486 │ 81 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusk.ru/ru/lookaginitial/Prodayu-Dach... │ 80 │ -│ 3 │ 4 │ 0 │ │ http://komme%2F27.0.1453.116 │ 80 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 79 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://aliningrad │ 78 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 78 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login │ 76 │ -│ 1 │ 0 │ 0 │ https://go.1ps.ru/show&showforum │ http://komme%2F27.0.1453.116 │ 76 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandex.ru │ http://komme%2F27.0.1453.116 │ 76 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26221/detail.ru/v1496366&... │ 75 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php │ 75 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 74 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%2F&sr=http://bonprix.ru... │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len80/page/product │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filtr/all/perm.pulscen... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 74 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://irr.ru/introlux_page5/2/pageType=product_name=1&menu_7 │ 73 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 73 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F1.7.1364.172 │ 72 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://diary.ru/forum/intries │ 72 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/show/414526863_112 │ 72 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2618561&pp=1059&op_produc... │ 72 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://video.yandex.php │ 72 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 71 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 71 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari │ 71 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://aliningrad │ 71 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=product_id=42&Selection.chelov.ru/searc... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/yandsearch?te... │ http://irr.ru/index.php │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/server=sc.chel.ru/main... │ http://irr.ru/index.php │ 70 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification/?year=\xD0\xB1\... │ 70 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2791954~43.87725656132&op... │ 69 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://komme%2F27.0.1453.116 │ 69 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/produkty/bleacs/udilis... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 69 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&wi=1366&br... │ 68 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 67 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731&CgID=124jc&where=all... │ 66 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/Yozh/Goodda │ http://komme%2F27.0.1453.116 │ 66 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2725215195&ti=\xD0\x9F\xD... │ 66 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://ekburg.irr.ru%2Fpuloveplanet │ 65 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru/index │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 65 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 64 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://afisha.yandex.ru/index │ 64 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 63 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nmg.com%2F4.0 Safari%26clients-sale/search&e... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 63 │ -│ 2 │ 13 │ 13 │ │ http://komme%2F27.0.1453.116 │ 62 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 62 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://irr.ru/index.php?showalbum/login │ 62 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://ekburg.irr.ru%2Fpuloveplanet │ 61 │ -│ 1 │ 0 │ 0 │ https://gotovka/hotels.turizm │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 61 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia │ 61 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 61 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26092.html?s_text=\x5C\x5... │ 59 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/thenon-houses/public/g... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27755f32316.30; .NET CLR ... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742/details/?cauth=0&dam... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fautoad/kniga.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26004/?_h=galle/\xD0\xBF\... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert27930555&sob=1&p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2692.html%3Fhtml?period=3... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/world/photo31469:Album... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=leting "HalUXI&where=\xD0\x9F\xD0\xB5\xD1\x8... │ http://irr.ru/index.php?showalbum/lofiver.ru/articles/86121%26uid%3D139750%26ad... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27000,224648804-recept-so... │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/koshka.com/ig/iframe-o... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 58 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory=cinema.perm.p... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/realty/suntime-5/extre... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742357/detskii_gosts.xml... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2742304][from]=&int[14670... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694336/photo/suzannason/... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Forum25/top... │ 57 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268320995,968650f45491882... │ 57 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupit/action │ 57 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1375605&ga... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 57 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://video.yandex.php │ 56 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga/choice/... │ http://nizhnieie/novo/a78920&lo=http://sravni.ru/reposition/vacancies/eduard_32... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 56 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_product_... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?showtopic,5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26017/quarius_Moscow/cavi... │ 55 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2726125413975d77cf&search... │ 55 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 54 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266848223/review_type=pro... │ 53 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://obninsk/detail │ 53 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26989%26bt%3Dad.adriver.r... │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/metro=144&ved=0CEUQFjAB&url=http://bonp... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 52 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/album/login-1800002&pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 52 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0 │ 51 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2698172,93932353064614618... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605085268a9c4d61-9862.49... │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19&numphoto/login=A-CL-MS-36575c72937][to]=&int[12822304 │ http://irr.ru/index.ru/widgetchrome%2F&ti=no&dom_v_bordovye-printime.ru │ 51 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/investate/comple/make/?page5/&docid=jlMNIrXw... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 50 │ -│ 0 │ 0 │ 0 │ │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 50 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F11.7.1364 │ 50 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http:%2F%2Fwwwwww.bonprix.ru/GameMain.aspx │ 50 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://kurort/SINA, ADRIAN - Foreversant.ru/busineshevsk │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26051.htBeg=6&NightRegist... │ 50 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xBE\xD1\x81\xD0\xB8\xD0\xB1... │ 49 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 49 │ -│ 5 │ 0 │ 0 │ http://state=19&text=\xD0\xB4\xD0\xB0\xD1\x91\xD1\x88\xD1\x8C │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 49 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?group=days=3&text=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27410/photo-2.xhtml&serve... │ 48 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect.yandsearch │ http://komme%2F27.0.1453.116 │ 48 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http:%2F%2Fwwwwww.bonprix.ru/voskres.php?gr=1665773aad1900%26ntype │ 48 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusty-i-vkont.at.ua/search=0&userId=0... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266128182&op_seo_entry/de... │ 47 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?sort=&br... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/intrumen │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://afisha.yandex.ru/index │ 46 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupalnik │ 46 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumanegenre=33155?analog/kitchen_mini... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 46 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://irr.ru/index.php?showalbum/login │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login%2Flick.g.doubleclick.diary.ru/car/kw/3061/us... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734.html_params%3Dfh_loc... │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://afisha.yandex.ru/index │ 45 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/GameMain.p1Yo4A │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ua/searchads/jo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 45 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605906/frl-2/sportal.ru/... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fshow/lpp/cre.ru/pers... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostovskiy-kray.irr.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/login │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.asp?search&ev... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 44 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logize.ru/msk/events_liver.ru/russert-plies.r... │ 44 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26099486633033&countpage=... │ 44 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://sslow_135000008&position=search │ 43 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 43 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischet-solik/odezhda-plos... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert277553/38021/66936575776/... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271050&with_photo.kurortm... │ 43 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 43 │ -│ 1 │ 0 │ 0 │ http://forums/liii-kuler S22oHgBJTngegotavgorod55.ru/filtr[2]=42 │ http://komme%2F27.0.1453.116 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 42 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-folders/?end=9a08488... │ http://irr.ru/index.php?showalbum/litamak.irr.ru/catalog/8570/travel.ru/cgi-bin... │ 41 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 41 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 Safari │ 41 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/0001216629/#top_by │ 41 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26040931&s_yers=0&with_vi... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://ekburg.irr.ru%2Fpuloveplanet │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.137508&s_yers=200... │ 40 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://aliningrad │ 40 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logito8ergo_6470/page_type_id=20872/0/001:12:... │ 40 │ -│ 0 │ 0 │ 0 │ │ http://afisha.mail.ru/dmitrij │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/readar-nashi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 40 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27924563724&key=46960/med... │ 40 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://lib.ru/exp?sid=3205&bt=7&bn │ 40 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 39 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credit │ http://komme%2F27.0.1453.116 │ 38 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=8570/page=5&s_yers=2006 │ 38 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2679/detail/5482,935033/2... │ 38 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 38 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://irr.ru/index.php?showalbum/login │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dnevnoy-kv-Samara.irr.ru/searchAutos&marka=4... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 37 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://komme%2F27.0.1453.116 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=1 │ 37 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=theating&page=... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?group │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2784389/room=39695,966681... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/forum/topicID=269&stat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/votkim-rukav-i-kova-ul-adve... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wolf-club478561/OPEL | \xD0\xB0\xD0\xB2\xD1\... │ http://irr.ru/index.php?showalbum/login-marka=13&city&custom=0&damages/0001216 │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/filmId=xFX1UbTNJjxe4yF... │ http://irr.ru/index.php?showalbum/login.asp?razdel7/test/matched=115909d9_dsc07... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27553&s_yers=0&po_yers/46... │ 37 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert275226607660?design=6efxq... │ 36 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http:%2F%2Fbrjuki-lic-shop.ru/ch/metersburg/contertype%3D158197%26ad%3D1216629/... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=show=&fgroup-tab-mara.irinables/yearT... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278171&size=52094&op_seo_... │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27330528/photo/6223799%26... │ 36 │ -│ 5 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 36 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36&he │ 36 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://smeshariki.ru/obucheyelants │ 36 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=post_neu%3D22%26pz%3D0%26rleurl%3D%26xp... │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vip.ru/book.com/photo/7086/page=0&state=mont... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 35 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/catalog/8569 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/index.ru/katego... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-cukickdrops_bak_yant4tlGtZgy3BQ&bvm=bv.49478... │ http://irr.ru/index.php?showalbum/login-501-98552&op_uid=1060948/6#f │ 35 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (Windows │ 35 │ -│ -1 │ 0 │ 0 │ http://state=19&m_staraya-obl │ http://irr.ru/index.ru/show/414526863_112 │ 34 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.12.388 │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 34 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/promo=C-Tease/rent/dat... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 34 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_uid=1... │ 34 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/list │ http://lib.ru/exp?sid=3205&bt=7&bn=1&gearbox=0&type_id=0&last_auto_ria=0&type=0... │ 34 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/directadvert/kavanga_728x90.html │ http://komme%2F27.0.1453.116 │ 34 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshops.xml?typ │ http://komme%2F27.0.1453.116 │ 33 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/exp?sid=3205 │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?cPath=default.ru/yandex.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 33 │ -│ 1 │ 0 │ 0 │ http://auto.ria.ua/\xD0\x9A\xD0\xBB\xD0\xB8\xD0\xBC\xD0\xB0\xD1\x82\xD1\x8C\xD1... │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 33 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate/apartner │ http://video.yandex.php │ 33 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 33 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 33 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser │ 33 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga_728x90 │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265748ba3f755671_1162684&... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertif?sle=85000 \xD0\xB4\xD0... │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 32 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F27.0.1364 │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/pub │ 32 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zxcvbnm1990&search/jobinmoscow.ru/yandex.by/... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 32 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements-obschaya │ 32 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru%2Fproduct/foto-4/login... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&op_category_id=1174]... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniturers_id=575386110 │ http://irr.ru/index.php?showalbum/login.exit/rem/php/board/10_kruizer_8267049&o... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiepochto_id=93231238][to... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://afisha.yandex.ru/index │ 31 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://ssl.hurra.com/iframe │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertischuk_Kerasivye-klavik/?... │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/jobs-education/m_acces... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26791e3a7c543c9bf9ef5c0-9... │ 31 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 31 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt │ http://komme%2F27.0.1453.116 │ 31 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_api.php?cid=41389/rooms/perm.pulscen.ru/sear... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertificia/?letteo_985428161/... │ 30 │ -│ 1 │ 0 │ 0 │ http://domchelov.html&lr=55&text=\xD0\xB4\xD1\x80\xD1\x83\xD0\xB3\xD0\xB8\xD0\x... │ http://komme%2F27.0.1453.116 │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoryID=12452929587/... │ 30 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 30 │ -│ 1 │ 0 │ 0 │ http://video.yandsearch │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 30 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6DQgE4LmUXI&where=all&filmId │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert268630533&site_ot=&price_... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-k608i.trashbox.ru/cars/page5/&docid=577&lr=2... │ http://irr.ru/index.php?showalbum/logii_58247.php?forum.borovichy77/page5 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2716/~3/150160947694,9745... │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/products/?category_id=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2696374/#page/tab=com_avt... │ 30 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru/topic │ 30 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26979553039353936694&ch=u... │ 30 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.14&he=768&wi=1920 │ 30 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605446518%2F&sr=http://e... │ 30 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://smeshariki.ru/obucheyelants │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-carava/fotok-8-steklove.ru │ http://irr.ru/index.php?showalbum/login-yuoocor.ua/user │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26928142&op_products/view... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kupibatikov/?recrete/personal/klimat.ru/work... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 29 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x9A\xD0\xBE\xD1\x80\xD0\xBE\xD1\... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertifications[0][150]=60&inp... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/wedditeli_foto-2/__48/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_countpage/1 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/click.net/paged=696&lo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274229696.0 (Windows)&bL=... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-14132&with_photo=1&srtime-zone.net/catalog/t... │ http://irr.ru/index.php?showalbum/login-sierra%2Fdlia-zhienskaia-moda-zhiensmed │ 29 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2664952&xdm_p=1#country=-... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704.html%3Fhtml5/v12/?fr... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2734/fotographic_2017666&... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategoriya/zhienskaya-... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9649.html%3... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/small_mercial-shoppicc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27350350394&ch=UTF-8&sF=1... │ 29 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-baza/gadge... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_seo_entr... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27425.3120%26ntype_id=2&r... │ 28 │ -│ 1 │ 0 │ 0 │ https://directions[0][1541769377921968 │ http://komme%2F1.7.1364.172 │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13733097&op_categ... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert272291684/hasimage=0&view... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makuma.html?category_id=1017&lr=213&tex... │ http://irr.ru/index.php │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/mann japanel/?text=\xD1\x81\xD0\xBC\xD0... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269500469412216388/detail... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-my-sovies/bebybum.homestion bonprix.ru/... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikarbox=2&tz=9b81998&po_y... │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://video.yandex │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F8536.26 (KHTML │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-176-kakaya-obl.irr.ru/search?lr=191&ei=N6rrg... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?lang=all&fi... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\xA4\... │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/anime-2/#album/l... │ 28 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML, like Gecko │ 28 │ -│ 3 │ 95 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-eve-nn.ru/cars/passenger/\xD0\x92\xD0\x90\xD... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274236&pv=15&prr=8&sid=35... │ 28 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 28 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/poned │ http://irr.ru/img/catalog/53485785/topic/8939858-59929477/detailshop │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19&m_static.diary.ru%2Fpugache=51dba668ea2feb2Xw │ http://irr.ru/index.ru/show/41452626u1zIq0SGLXCI │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=Mitsundai/malitics/katersburg.irr.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 28 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/gold │ http://ekburg.irr.ru/#lingvo │ 28 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F22.14&he=768486 │ 28 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login.html?1=1&cid │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678/page=2&marka=84&mode... │ 27 │ -│ 3 │ 2 │ 0 │ │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/search?cl4url=http://yandex.ru/online... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27013.ya.ru/sportalznakom... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert24061.jpg.html%3Fhtml?1=1... │ 27 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031343-sh-898675/sort=pri... │ http://irr.ru/index.php?showalbum/login-5158&jenre/slyudya_metroman.ru/GameMain... │ 27 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19195/offset=101&district=&city&with_photo=¤cy │ http://irr.ru/img/catalog/53485785/topic/8939850.php?page=3&marka=84&mm │ 27 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://auto_map6%26pz%3D0%26geozone.net/201597547,8.0.146/imagecachel │ 27 │ -│ 1 │ 0 │ 0 │ http://forums/liiie/?target │ http://obninsk/detail │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fhit_list.html?s... │ http://irr.ru/index.php?showalbum/login-irake-myservierk-supierts%2F&ti=\xD0\x9... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-oleginnesujeli-i-centum=etarget=search?clid=... │ http://irr.ru/index.php?showalbum/login-kupe.html_params%3Drhost%3Dad.adriver.r... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27928092_price=\xD0\xBC\x... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/mazda-dievochnye_avari... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-kupalnik.10065%26bn%3D0%26ad%3D158197%2... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-19.xhtml?city_to_one=All&film]/on/orders/810... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/1.19.mobile_photo=6001... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27630/?item_id=0&bodystyl... │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId │ 27 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/poned#pa\xD0\xBE\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 27 │ -│ 1 │ 0 │ 0 │ http://bonprix_ru_11559&lr=12&usln │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lizova_ii_1112_1150&option[price_do=300... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search=0&choosO7a_rEk3E&wher... │ 26 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detall/seconomicsmovinki │ http://komme%2F27.0.1453.116 │ 26 │ -│ 1 │ 0 │ 0 │ http://yandex.ua/?target=search/tab=user │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 26 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 26 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://obninsk/detail │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 26 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/loginsk.irr.ru%2Fkategory_id=9204471-650/?_h=... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login-jjjekrju-s-pring=1&price[price=\xD0\xBC... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification.ru/messa.ru/rea... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lejandsearch?text=\xD1\x82\xD0\xB8\xD1\x85\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert21124631/?Search.php │ 26 │ -│ -1 │ 0 │ 0 │ http:%2F%2FwebcamMax │ http://afisha.yandex.ua/auto_id=1430][to]=&int[260][20][to] │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_category... │ 26 │ -│ 3 │ 197 │ 0 │ │ http://komme%2F27.0.1453.116 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?messa.ru/worlds/page2/7277932582&text=\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 26 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/openson XA2oYUXI │ http://irr.ru/index.php?showalbum/login-kupe-20010120652838799.html?n=7148.html... │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemey-volvom80Cw&where=a... │ 26 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://afisha.yandex.ru │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandsearch?text=\xD0\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 26 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_category │ 26 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=607&state=4... │ 26 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.22 (KHTML, like Gecko) │ 25 │ -│ 1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http:%2F%2Fwwwwww.bonprix.ru%2Fkategoriya │ 25 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978184 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=1&state │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/people.ru/cars/passets... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2731/?city=0&pvno=2&evlg=... │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-vivarki/redki_sdelaet-law.ru/produkty/kartgo... │ http://irr.ru/index.php?showalbum/login=driff/en-ru/?p=2#messages/00001216629 │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/offection │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.html&ei=GcXrUYe8F2DAU-RbuRlLMczoJ--5uDCx8 │ http://irr.ru/index.php?showalbum/login-132/#images/0000.h95/\xD1\x82\xD1\x8E\x... │ 25 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert264314953); ru) AppleWebK... │ 25 │ -│ 1 │ 0 │ 0 │ http://auto/auto.ria.ua/search/tab │ http://obninsk/detail │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search?text=myaccountry │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 25 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguard-payments-sale/se... │ http://irr.ru/index.php?showalbum/login-88i8LaDyEkCVv6-DhRfEDcw==&action │ 25 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/yandex.ru/?f... │ http://video.yandex.ru/page=0&category&op_seo_entry=&op_categoriya │ 25 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F537.36 (KHTML │ 25 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://video.yandex.ru/film/46351/frl-2/bage │ 25 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=7329 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2605838.html5/v12/?from=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694589/detail/555-javata... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741559&t=5204/make=Chrys... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-link=1147373-1&p=1&p=2&text=\xD0\xBC\xD0\xBE... │ http://irr.ru/index.php?showalbum/logisterlingvo/#! │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?cmd=shops.html?id=1&body_type=\xD0\xBF\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://irr.ru/index.php?showalbum/login-kupaljiteraturka │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781472265&op_seo_entry=&... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626rleurl%3D//ad.adriver... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685/price_ot=&price=\xD0... │ 24 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://diary.ru/forum/intries │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27399992.php/board,13.0) ... │ 24 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/login-117031&op_category&op_seo_entry=&op_uid │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671291&fridgets/2012&pri... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26685857947301_Zoryatiya/... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-annative-expressageId=0oVXx21hUXI&where=all&... │ http://irr.ru/index.php?showalbum/login-rybnovlevojj_mamountry=-1&type%3D0 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo/7... │ http://irr.ru/index.php?showalbum/login-kupit-plitka-s-korsiety/art/151/n4p/160... │ 24 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://video.yandex.php │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumyi-dachat/jaguar.ru/page4/?emain.a... │ http://irr.ru/index.php?showalbum/login=ogabass.ru/filmId=9WOqzzitive&view │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/intrumen │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kupalnaya_elit=&id=3&clid=9403&lr=1913849 │ http://irr.ru/index.php?showalbum/login-gallebia_zhurnalCall D.R.E.mp3ex.net/ph... │ 24 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-lolacake/tatyaka.html?1=1&cid=577&oki=1 │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinglish.rus-l7-p70505-1... │ 24 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/yandsearch?lr=1... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-yvaken/topic=7702.jpg.html?1=1&cid=577&... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=1&bc=3&ct=1&pr │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-547552/1/?sma=RAINBOW&op_uid=fdd1b9d2721728&... │ http://irr.ru/index.php?showalbum/login-kapusta-advertika/search/room=1&damage=... │ 23 │ -│ 1 │ 0 │ 0 │ http://auto_volkswagency=1&text │ http://komme%2F27.0.1453.116 │ 23 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgarials/commend/searchvage-2.html&dt=13734... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-tuniki_510... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263090024189-528619c2077/... │ 23 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://smeshariki.ru/ru/index.ru%26bid │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333572620201709/page/10... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news/radio&clid=198555... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26939.html?page/3/#count=... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2740974%26pz%3D0%26ar_sli... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/index.ru/kategory_id=5... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 23 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo │ 23 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://liver.ru/a/far_applunzsxi.cmle.ru/search?text │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/resultatya9176da22f1a521a5853.html&... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 23 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapustom%3D%26xpid%3DBBn-investate=toda... │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26600721376439322%26pz%3D... │ 23 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page │ 23 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://stalker-pub-20087898675494,960948/#page_type%3D260117152337&spn=1395,945... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/advert/kavanga_728x90.... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://afisha.yandex.ru │ 22 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=2512592 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisemennoke0070553995f27d6... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2697574697.html_param=0&u... │ 22 │ -│ 5 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-105vr/ │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=search.tutung/s... │ 22 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/wheel.ceratornaya_solnechka │ http://komme%2F27.0.1453.116 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2091&local=1&TypeSearch?mail.ru/yandsearch&c... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26058&nnum=s4746835895&op... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-10760b31b65633eZateq eb806e887d9f15ccf593280... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-the-ili-tempera-minsk.irr.ru/produkty/bluzki... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739693071/pomer... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodny │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2649.html?pg=menu_29.jpg&... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnik.ru/v1465][from]=&int[157281 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267830840994,95688781470%... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-business/sale/ru/search?mail.ru/film%2Fbarna... │ http://irr.ru/index.php?showalbum/login-zakon_type=1&fuelRatesTypeSearch │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 22 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27200,55.7654.html_parat-... │ 22 │ -│ 1 │ 0 │ 0 │ http://radioscannetcat=threadreplies │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&l... │ http://irr.ru/introlux_page5/2/page/Renaul.irr.ru/start=1500-rublic/gamemain/Tu... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26601629][from=&fuelRateT... │ 22 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://ekburg.irr.ru%2Fpuloveplanet │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-natashkarola.irr.ru%2Fviewforum/MsgList.html... │ http://irr.ru/index.php?showalbum/login-kapusta-advert26900/technics-technics-t... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liliate_id=24145602&with_phoney.yandex.ru;ya... │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 22 │ -│ -1 │ 0 │ 0 │ http://state=19&num=5&s_yers │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&city[7]=23239330794,25826044_Black_lis... │ 21 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?price_till beli.php?cx=015216684_36 │ http://komme%2F27.0.1453.116 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691_ru_17832523.html_par... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694,978825315f373400/det... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1) AppleWebKit%2F537 │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/world/photo/41/62b1dfa450/3/women-clother%26tms%3D%26ev_ltx%3D... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachka/saledParam │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomaha.diary.ru/forum%2Fukhov/l... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27025&Select[4][]=32598.h... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/basketshop.ru/yandsear... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert265228.html?1=1&cid=577&o... │ 21 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://wildberrior/uphold │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo44953.116.php?t=4... │ http://irr.ru/index.php?showalbum/login=ko&page=0&view.aspx?group_cod_1s=85&key... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/?from │ http://en.lyrsense.com.ua/?tag=type=category_id=1555768&wi=136225..87245-937559... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/houses/passenge... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-dance/index.ru/?favorite_off=1&encoding&hl=\... │ http://maps#ru_5_ru_22106.37766/men.aspx#location/topic.php?razdnuyu-komnatalog... │ 21 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F5.0 (Windo... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logino-s-grigerator/page1=&input_age1 │ 21 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 21 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/cars/commeddesk.ru/google_s&12.5.746.59954.30 (KHTML, like/\xD... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ramblery/pic/893985650697... │ http://smeshariki.ru/obucheyelants │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26737865/?page/12186/#/ec... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=199450984062 │ http://irr.ru/index.php?showalbum/logii-bin/click.cgi%3Fsid%3D158195,97987231-9... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/histofelny.i... │ http://irr.ru/index.php?showalbum/login.html?1=1&input_who1=2&input_who2=1&inpu... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/img/catalog/53485785/topic,806;IC,33;VL,1430/photo │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-brie... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-dietsik.ru/razdel_id=111,7,700&w=728x90.html... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929&from%3D%26custom=0&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/club.ru/anime-zone.ru/... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2186/detail.aspx#location... │ 21 │ -│ 1 │ 0 │ 0 │ https://google.com/fee=\xD0\xBC\xD0\xB5\xD0\xBD\xD1\x8C\xD1\x88\xD0\xB5 │ http://irr.ru/index.php?showalbum/login-kapusta-advert2686305895&op_seo_entry=&... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098-1.13739353/details.... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kupalnik │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification%2F4.0; XBLWP7; ... │ 21 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/9-0--navam-zhien... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2741465][from]=&input_wit... │ 21 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapustom=0&Itemid=577&oki=1&op_uid=1147... │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login=uzpp2XnEw&bvm=bv.49784469][from │ http://auto_s_product_id=25292.1406.798352/women.aspx?group_cod │ 20 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&sr=http://afisha │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-Larki-i-korobeg-1124-95367/guestblowinp... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26219&wi=1280&lo=http://w... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/yandex.ru/real... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estatic.diary.ru/... │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=58&Selectronics-technics/k... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?er=2&text=\xD0\xBA\xD1\x83\xD0\xBF\xD0\xB8\... │ http://irr.ru/index.php?showalbum/login=razer2.moikrug.ru/recipes/shop.spb │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b706234651&op_uid=1&b... │ 20 │ -│ 1 │ 0 │ 0 │ https://mysw.info=sw-131726275 │ http://komme%2F27.0.1453.116 Safari │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estate=yestered │ http://video.yandex.php │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&price │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=NeIdeaCenternet Explorer&aV=5.0 │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertificirovka.ru/noteating-i... │ 20 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 20 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313163307/?from]=&int[15... │ http://che.ru/produkty_zarubezhei-niepochekhly │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate=0&ReturnUr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271161076&st=261&t=170977... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604370757034dea482207549... │ 20 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_api.php?id=7262882,9454472&op_product_n... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert27419&z=9&l=map&id=2211-9... │ 20 │ -│ 1 │ 0 │ 0 │ http://video.yandex.ru/cars │ http://komme%2F27.0.1453.116 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26637&model=1705.html?1=1... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-karbox=0&top=0&category/1961/#images/00... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumilora481 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-iulyanovskaya-obl/lyubvi?page=166733&eid=485... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27502/detail/?folders/#pa... │ 20 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credircnt=13733129/room │ http://komme%2F27.0.1453.116 │ 20 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://video.yandex.php │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626u1zIq0SGLXCI&where=al... │ 20 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=240&u_cd=242035165&input_who2=1... │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien%26ar_slice-russic.ru... │ 19 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state/apartments/parts.ru/GameMain.aspx?sort=popup/casualg... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/news_id_72387877055/nu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikulina.ru/real-estate/ou... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=HOtbySdOiUw │ 19 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/filmId=eRaB4pYAXCI │ http://komme%2F27.0.1453.116 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login=perfect.yandex.ru%2Fkategoriya%2F537 │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.ru/daily │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text=\xD0\xBB\xD0\xB8\xD0\xBA\xD0\xBE\xD0\xBB\xD0\... │ http://irr.ru/index.php │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260540718534/?dateOn=1&bo... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261921&pr=2441185112.html... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26672655462.html?city.ru/... │ 19 │ -│ 1 │ 0 │ 0 │ http://sp-mamrostok.ru/cars │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27252/women.aspx?group_co... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2671&Destige-all.html_par... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/check=1&redir=1&lang=a... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27404895,95923.html%26cus... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006&po_yers=20078816 │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 19 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2638/?from=&raceTo=&power... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313437&pvno=2&evlg=VC,2;... │ http://irr.ru/index.php?showalbum/login-rizova.ru/look/timashirtsevm.cofe │ 19 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-estation/vacancing │ http://sslow_13507.html?aspx?naId=6HS │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=487930 │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27355f3132366336601&uuid=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.php?f=11340.1j1j2.12...5998-103358071/8/page... │ http://irr.ru/index.php?showalbum/login/?ReturnUrl=%23images/00007/10/1320006&p... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166333/price[price_till]=&pri... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2F_liveresum... │ 19 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=&auto_repai... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://smeshariki.ru/obucheyelants │ 19 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.15751 Safari%2F5.0 (Windows │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2675406828ac956028692581/... │ 19 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://obninsk/detail │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26968794618/irr.ru/real-e... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2629&lr=65&rstr=-213&text=back=False&withmos... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27423026517034&pvno=2&evl... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2702483,631;IC,444041%2F&... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=9.80 (W... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26669E116-1-1550046560013... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=http://video.yandex.ru/real-estate │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?search?text=\xD1\x80\xD1\x83\xD1\x81\xD0\xB... │ http://irr.ru/index.php?showalbum/login-inadlinny-sched_car=0&city=0&price=18&p... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274026181891&rn=52&brand=... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/referery_besplay/4972&bL=ru-ru&xdm_... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/dlia-zhiensk... │ 19 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login.html?1=1&cid=577&oki=1&op_product │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list=0&checked=0&state/room=10002-g-v-program... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-165&jenre]/49047&lr=163&text=samsung ga-moda... │ http://irr.ru/index.php?showalbum/login-2-komn-v-na-92f5-6ccf-fef3-013f9f926a21... │ 18 │ -│ 5 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://afisha.yandex.ru/index │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5 │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login/?elmt=popular&where=all&film/723/num │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/man/restate/out-of-town-resh=1&cnt=1373... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2012-recept-Ragu-iz-2-x │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupaljinik-slidi/places/?PAGEN_1=2 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main/erle.com.ua/yandex.php?app=my&name │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 18 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://wildberries.aspx#location/group_cod_1s=53&butto_638_1360/3/women.aspx?na... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=deltec&type=citymo=&version%2F4.0 (Linu... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27534.30 (KHTML, like Gec... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-kochtinsk.pulscendinskiy-krasnodar.irr.ru │ http://irr.ru/index.php?showalbum/login-kuplyuskij/zapchastorii_state/out │ 18 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login.moikrug.ru/catalog/otchenko-ul-advert27... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-zzzi.ru/a-shop and blademas/?id=1&marka=0&ma... │ http://irr.ru/index.php?showalbum/login-2003prev=/search/index.ru/meteor.html%3... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F&ti=\xD0\x91\xD1\x80\xD1\x8E\xD0\xBA\xD0\... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=367132&pv=2... │ 18 │ -│ 1 │ 0 │ 0 │ http://radiorecord.ru/real-estatic.diary │ http://irr.ru/index.php │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert278453&with_exchange=0&ex... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27132069304;IC,1511e8d158... │ 18 │ -│ -1 │ 0 │ 0 │ http://go.mail.ru/yandsearch?lr │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27071038_114736895,923401... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estat_type-5/exte... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x9A\... │ 18 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=978152&ch=U... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-natura.ru/jobs-education=post_137336ID0... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/8018&l=malkogo... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=kanza_tab_2499472&lr=1074&sort=popular │ http://irr.ru/index.php?showalbum/login-kupaljinik-Internet Explorer&aV=5.0 (Wi... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/search/?target=search?... │ http://irr.ru/index.php?showalbum/login-bigutensiysk.irr.ru/regular&bid=3205&bt... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27420780790931.html5/v12/... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya/obuv%2F │ 18 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-21102&text=\xD0\xB1\xD0\xB0\xD0\xB1\xD0\xBA\... │ http://irr.ru/index.php?showalbum/login.mospsy.ru/super-na-kostronics/tv-audio.... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2685&numphoto=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626p/ │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertimeshariki.ru/cgi-bin/cli... │ 18 │ -│ 1 │ 0 │ 0 │ https://diary.ru/magnitogorsk │ http://komme%2F27.0.1453.116 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sien-zhienskaya-i-kvartir.html?id=1955451-01... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260908861%26point-peter=&... │ 18 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1453.116 Safari%2F8536 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=yators/?jumping.ru/series.ru/priceup&page=3&... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C \... │ http://irr.ru/introlux_page5/2/pageTypeSearch?text=\xD0\xB2\xD0\xBA\xD1\x83\xD1... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisinger/\xD0\x93\xD0\x90\... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olga1788778.137329TVmbFNmbVNaR0YwW$UyNDg4fxS... │ http://irr.ru/index.php?showalbum/login-sienskaia-moda-zhienskikh_rabotjaga.ru/... │ 18 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_car=36427929&stat... │ 18 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26929750%26ar_sliceid%3D2... │ 18 │ -│ 4 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691c352005&pvno=2&evlg=V... │ 17 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru%2Ffilm/4858?pa... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert271527091/frl-2/bage-2.ht... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27422544/3/womens.ru/ufa.... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/cars/communalux.ru/dos... │ http://irr.ru/index.php?showalbum/loshaya-obl.irr.ru/film/35562C?analytics/kto-... │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#7878... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser%2F12.103402.html%3Fhtml │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=artira-Lyubyat │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x92\xD0\x90\xD0\x97 │ http://irr.ru/index.php?showalbum/login.aspx#location │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://myloveplanet.ru/index.ru/registrict=3219&st=10# │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=yes&y1=2013-07-07.ya.ru/5_search?text=\... │ 17 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://aliningrad │ 17 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1453.116 Safari%2F5.15.html?1=1 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266149_51928/detail.ru/ka... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-adverts%26rleurl%3D%26CompPath%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2670308&po_yers=0&price_o... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2613168312.71 Safari%2F__... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?Returniture=8&ch=UTF-8&ifr=1&av=1&nm=1&ved=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login_Bolers-device.aspx#commersanta_premiery... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 17 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http://sslow_13507.html?aspx?naId=6HS │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginyazevskaya%2F5.0 (Windows NT 5.1) Prestorygoo... │ http://irr.ru/index.php?showalbum/login-kupilomatic.aspx#location=rr&days&city&... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-dlia-zhiensmed.ru/el... │ 17 │ -│ 1 │ 0 │ 0 │ http://bdsmpeople.ru/register2123 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/logistereda.ru/photo/70861/6#f │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert263161366&bid=2823&numpho... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=ForeightEnd │ http://irr.ru/index.php │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowthreadreplies=24&ids=74&cu... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27595,9305f3931339322%26n... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26041.htm%3Fsid%3D0%26pz%... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313&ussp=mstp&usln=1&inp... │ http://irr.ru/index.php?showalbum/login-3767436/roomootofile/obzor.com/iframe │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273363331251798796&op_seo... │ 17 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/stan/index.kz/yandex │ http://komme%2F27.0.1453.116 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertistia.html?partment/searc... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-search%3Futm_source=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2046097563356531663031323532363&win=70&mode=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 3 │ 22 │ 0 │ │ http://komme%2F27.0.1453.116 │ 17 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://diary.ru/forum/intries │ 17 │ -│ 1 │ 0 │ 0 │ http://bibieters_sig=5ea8QizwGSPy0yGcm14ATH74D4Bw&usg=AFQjCNHxMZlonova.ru/p1847... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_len40/page3/#57366/?date_i... │ 17 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=&op_style... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-lenfiki-zhienskaia-moda-muzhchin-sitafa ishq... │ http://irr.ru/index.php?showalbum/logical.ru/link=11485-9065383235348%2F&sr=htt... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienskaia-moda-futbolki-k... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert261636/detail/odnoklassim... │ 17 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/messar.ru/professig │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 17 │ -│ 1 │ 0 │ 0 │ http://acase.php?input_who1=2&input_who2=1 │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 17 │ -│ 3 │ 3 │ 0 │ │ http://komme%2F2.10 │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2694&option%2F12.14&he=76... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26618531&t=111246.html?p=... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_uid=13733145625/details... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-irina19910735%2F27.0.1453.116 Safari │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert268814453957595,94406/det... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=Like Gecko) Chrome%2Fpotnik=1&price_do=&curr... │ http://irr.ru/index.php?showalbum/login-the-poxudet_sponsor=&o=6000708498/?caut... │ 17 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/login-53584715,1327288/belgium/arch=1&themec.... │ 17 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login=bestwo.diary/details&id=9160/0/posti.ri... │ 17 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2626b76faact/less=1&ru=1&... │ 17 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/category=cinem rasmusic │ http://komme%2F27.0.1453.116 │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-herokee_tatus=1&rm=18606bAxFEcQT6smBB4W... │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-zhien... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-olgas.rQRAX2Ua3IF1iHJcaz4ATv34DABw │ http://irr.ru/index.php?showalbum/login/?page=1024&wi=144438-88-3906740/currenc... │ 16 │ -│ 1 │ 0 │ 0 │ http://autodoc.ru/a-search/?target=search?clid=19554786.0.8.0.2.2003558_3497926... │ http://irr.ru/index.php?showalbum/logie-niz-doktops/electronics/4185570c72 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26098707209463401090/deta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert267044005320AA76)&vendor_... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27034195,9520160173/5/wom... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19&text=\xD1\x81\xD0\xBB\xD1\x83\xD1\x88\xD0\xB0\xD1\x82\xD1\x8C&s... │ http://irr.ru/introlux_page5/2/pageTypeId=0&expand_search/cuZXdzJTJGJmxyPTI4fQ,... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsien-zhienskaia-moda-plan... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26396284046.xhtml%3Fhtml%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2660&pt=b&pd=7&pw=2&page5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/real-estate/out-of-tow... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert260612d7cf.4082813,74,756... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=&esrc... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ 1 │ 0 │ 0 │ http://kipirog-s-krug │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/movinki... │ 16 │ -│ 5 │ 0 │ 0 │ http://state=199450984062 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-kupalnaja-prost-petersburg/detail/?fold... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.rambler.ru/page=91552&q=\... │ http://irr.ru/index.php │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.pl?&clients-sale&siteurl%3D//ads/search?text... │ http://irr.ru/index.php?showalbum/login-kapusta-advert273786/foto-5/#photo=2817... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-krasok.html_params%3Drhost%3Dad.adriver │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9C\xD0\x90\xD0\x97/page4/?_r... │ http://irr.ru/index.php?showalbum/login.aspx?sle=13&s_yers=0&page_type=0&door=0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertsiensmed.ru/dl/\xD0\xB7\x... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti/redmonds %2F offset=403134333.2201560&c... │ http://irr.ru/index.php?showalbum/login-v40-velopiter.feriod=0&scroll_to_auto.r... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661232&streamfood.com/if... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login-kupalnyj-bandom.ua/a-foldersSubCliensme... │ 16 │ -│ 1 │ 0 │ 0 │ http://google.ru/forum │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo/70948/35.htm?id=... │ http://irr.ru/index.php?showalbum/logiy-luchaiev%2F&sr=http:%2F%2Fwww.bonprix c... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-57548811426&text=\xD0\xBB\xD1\x83\xD1\x87\xD... │ http://irr.ru/index.php?showalbum/login-coolonellana-Molodilnik.ru/index.php?vi... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-204609756335653166303133334%2F&ei=horia+iudi... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert26765.html_params%3Drhost... │ 16 │ -│ 1 │ 0 │ 0 │ http://histore/profile;u= │ http://irr.ru/index.php?showalbum/login-kupe-20013&uuid=1373356164564_1sort/sta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2755597.html%3Fhtml?1=1&c... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login=findjob.ru/planet.ru/index.ru/carinov.h... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/showtopic,803813.html&... │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_catalog/1250/?itemsg/cd... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/manga │ http://irr.ru/index.php?showalbum/login │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert260477/cities/visas/exp?s... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirosomahachkakh-2/?type=0&choos&lr... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 │ 16 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/real-esta.ru/election │ http://video.yandex.php │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27429&input_bdsm_position... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2728024/detail/508/?instv... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26887301/Zona/simferopol.... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-153-4b9c4.4331626.htm?size%3D0%26rleurl │ 16 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credirect │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http://komme%2F27.0.1364.172 YaBrowser%2F10B350 Safari%2F537 │ 16 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://msuzie-shop/premiery-c-38208_2.html │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/posts&q=\xD0\xB2\xD0\xB5\xD0\xB4\xD1\x83\xD1... │ http://irr.ru/index.php?showalbum/login-mistore/#CATALOG_LIST-s-ovosibirsk/deta... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-marka=23&model&desting?field_town/search/pri... │ http://irr.ru/index.php?showalbum/login-492ea9&show/417582,9626512874":10157628... │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_country=-1 │ 16 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_name=\xD0\x91\... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/sch/price][min]=41&lr=... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27364229586121%26url%3D%2... │ 16 │ -│ 1 │ 0 │ 0 │ http://volgografiyah_27_iyunya_50_let%2FgetId │ http://komme%2F27.0.1453.116 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-sovies/tv-audi/a7-spec=9894797179698712/p1/8... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27333%26bid%3D1%26rleurl%... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new=127201148][fr... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-maritkiN │ http://irr.ru/index.php?showalbum/login-ladimir.irr.ru/remiery-telefon.ru │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2661251-6efa-d61f-fef3-01... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert269899458/price/3815&pvno... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/7427510/detail/5... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login-7205011,1,7,700&aN=Netscape&aV=5.0 (Win... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-wkti%2F%3Ff%3D100%&http://afisha.yandsearch&... │ http://irr.ru/index.php?showalbum/login-6030d.html_params%3Drhost%3D90%26height │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertification=view/5/item3963... │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logiste's a di galaxy-s4-story1.aspx#location... │ 16 │ -│ 1 │ 0 │ 0 │ http://yandsearch?lr=2&color │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2753][from]=&pricedownloa... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-gorod/search?p=7&oprnd=9902.jpg&img_url=http... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27222.vk.me/u3166 │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2735030373434-152495.php?... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_cated_new10152954vac │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2691.html?1=1&input_age2/... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-irinakai/page_num_read2306e4574&ei=Ot_rUfjPG... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/page5/#over/190008/0... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makeevka.ru/perm.irr.ru/slingvo/#1\xD0\... │ http://irr.ru/index.php?showalbum/login-kapustic/product_name │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26708662307][]=&selection... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-20460975633565316630313033393633310,932803][... │ http://irr.ru/index.php?showalbum/login-kapusta-advert27256.html_params │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2010000087378687/pic/8939375f303839303935373... │ http://irr.ru/index.php?showalbum/login-souse=2/path=39_1.html?1=1&cid=577&oki=... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumy.ua/search?text=\xD1\x81\xD0\xB5\... │ http://irr.ru/index.php?showalbum/login-kapustya88/?sob │ 16 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.ru/widgetchrome%2F201001556&op_seo_entry │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/?strict=5710... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category=cinema/artira │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertists/736850/?item_no=737&... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertikal.ru/search/room=1&dam... │ 16 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2695/19545602cf94d77/repl... │ 16 │ -│ 1 │ 0 │ 0 │ http://forums/liiiervierk-suppohudeemvmesting/294465&op_categoriya │ http://komme%2F27.0.1453.116 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_product_id=0&model=1173... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login=new&cad=rjt&fu=0&input_country_id=228&lr=961... │ http://irr.ru/index.php?showalbum/login-cam.shtml#objdesc=true&uA=Mozilla │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-verb=1&prepairs=0&city[1 │ http://irr.ru/index.php?showalbum/lounona-muzhchin-kupaljinik-chernyj-odnoe/san... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makum │ http://irr.ru/index.php?showalbum/login/?do=ready-mansion%3D0%26url%3D%26bn%3D0... │ 15 │ -│ 1 │ 0 │ 0 │ http://launcher-searchads/search │ http://komme%2F27.0.1453.116 Safari%2F5.0 (compatible; MSIE 9.0; │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_produkty/kiev.org/forum... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/a-album/login.2/second... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2781308;IC,238208836746/0... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-4433140796851/detail/536180&all=False&i... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginsk.eu/ru/Pagesize=13&m1=07 │ http://irr.ru/index.php?showalbum/login-3muda/truction/vacancies/750207190765.h... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26981583/page=1089592445/... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945HPS?analog/r10418/sort=price_do=200&site_off=1 │ http://irr.ru/index.ru/\xD0\x9D\xD0\xBE\xD0\xB2\xD0\xB3\xD0\xBE\xD1\x80\xD0\xBE... │ 15 │ -│ 1 │ 0 │ 0 │ http://wildberrifiers?year_detailshop/id_art_type=7&s_yers │ http://irr.ru/index.php?showalbum/login │ 15 │ -│ 3 │ 2 │ 0 │ │ http://komme%2F1.7.1364.172 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-goluboj-podserial&dfs=13 │ http://irr.ru/index.php?showalbum/login-haus.html#news/228670,257&pci=3012/frl │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273389; U; Android 4.0.14... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert273631253711/foto-6320166... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2724/?_h=search?text=\xD0... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category&op_page4/#67 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-ona.ru/name=yandex.ru/busineshematov/offers=... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price_... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26605566254/room=1.6&wher... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert266356163f39ea6f8/7a8745&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/washes/built-in_two_ch... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http:%2F%2Fwebbootate/aparts/state=1909644 │ http://afisha.yandex.ua/auto_id=0&with_photo.kurortmag.ru/razdnitsya-vishnury │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginskikh-viana.irr.ru/GameMain.aspx?sle=1/house.... │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/?act=full&Form.aspx?g... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/main.aspx?Topic │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/rative │ http://irr.ru/index.ru/show/414526863_112 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/list.html?typ=SMA&anbieter=aleks_evilkos.com │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertisements/produkty%2Fplatj... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-land_search?text=\xD0\xB0\xD0\xBC\xD0\xB1\xD... │ http://irr.ru/6323%26bn%3D27888895,963095425 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/index.ru/ange=0&s_yers=(3000&static.diary.ru... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/GameMain.aspx?transk.i... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.php?page37/?sta... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2678&op_page=60000/curren... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F27.0.1364.172 YaBrowser │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login.1Uv1UfqCA8SonYC4BQ&usg=AFQjCNGB3pBUuKY1jJPP3... │ http://irr.ru/index.php?showalbum/login-tank-sale/search%3Fmode=&page_type │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/yandex.php?Id=9582 │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-liloveplanet.ru/futurer533/women.aspx?group-... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2631920&lo=http://person_... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advert2718599/photo=0&is_hot=0&... │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/url?sa=t&rct=j&q=alini... │ http://radio&planet.ru/moscow.ru/\xD0\xB8\xD0\xBB\xD0\xBB\xD1\x8E\xD0\xB7\xD0\x... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-tumbov.irr.ru/page1=&input_who1=2&id=29... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert26761,55.654289600/detail... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.php?city[1]=700003 │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2699364224073532663835386... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27515/418695&st=327-B110Q │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login=vladimir/page_type=0&expand_search?text... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login/?dok=001&rightbox/week&From=0&user │ http://irr.ru/index.php?showalbum/login-kapusta-advert2604/frl-4/trailer/view/3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=450&with_photo=7... │ http://irr.ru/index.php?showalbum/login-lential/secondary/details9.html_params%... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x95\x... │ http://irr.ru/index.php?showalbum/login-676216b8af/4fd00fa61b3185631821/page_ty... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=0&page9/#14... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2730675595,9292fa-d61f-fe... │ 15 │ -│ -1 │ 0 │ 0 │ http://kinopoisk.ru/?state │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=25127691%2F... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27265484158197%26width%3D... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login_appliazh-pliance/mista-bez-uchaiev/000202&cl... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0&price │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/ch/floore troubleclick... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/yandex.ru/gamemain.asp... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/kategory_id=1955451&lr... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 0 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-kapusta-advertsienshchin-planet.ru/myma... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert27877437494,943082&stat_t... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiruiushching │ http://irr.ru/index.php?showalbum/login-kapusta-advert2704&prr=http:/ │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Ford-antent... │ 15 │ -│ 3 │ 123 │ 0 │ │ http://komme%2F27.0.1453.116 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_categoriya%2Fdlia-doma.... │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/url?sa │ http://sslow_13507.html?aspx?naId=6r7RoWvxU9qQ= │ 15 │ -│ 4 │ 0 │ 0 │ │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-nashinger/users/search?textilead&353%26ev_pl... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-102-821922237&arrFilter2013/08-iH4AT5zIGQDA │ http://irr.ru/index.php?showalbum/login-kupe-2/#page_type=&freetao.diary.ru/sea... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/loginy_name/3196906481/currency=6465373200&price=\... │ http://irr.ru/index.php?showalbum/login-kapusta-advert2776/?date=0&doorov.irr.r... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2693_763613.html_params%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert262997385f32313335781094&... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert274298799461981/?date=129... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert276278-ploschaya_checked_... │ 15 │ -│ 1 │ 0 │ 0 │ http://mysw.info/node/215455&text │ http://irr.ru/index.php?showalbum/logabass.ru/cation&op_category_id=9584%26pz%3... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/catalog/877/ru/buildin... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ 5 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/make=\xD0\x9A\xD0\x90\xD0\x9C\xD0\x90\x... │ http://ekburg.irr.ru%2Fpuloveplanet │ 15 │ -│ 1 │ 0 │ 0 │ http://smeshariki.ru/diary.ru/yandex.ru/credir=1 │ http://komme%2F27.0.1453.116 │ 15 │ -│ 5 │ 0 │ 0 │ http://kinopoisk.ru/yandex.ru/auto/69363 │ http:%2F%2Fwwwwww.bonprix.ru/myAccountry │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/photo=on&input_onliner... │ http://irr.ru/index.php?showalbum/login.html%26custom%3D%26CompPath.2; WOW64; r... │ 15 │ -│ 0 │ 0 │ 0 │ │ http://komme%2F2.10 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advertist/gosthelp.ru/auto.ria.... │ 15 │ -│ 1 │ 0 │ 0 │ http://yandex.ru/search?q=\xD0\xBB\xD0\xB0\xD0\xB2\xD0\xBF\xD0\xBB\xD0\xB0\xD0\... │ http://irr.ru/index.php?showalbum/login-kupaljinik-2008-g-v-stroika/photo=on&in... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumirostova.ru/doc.ru/belgorod.irr.ru... │ http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert269257798044.html_partmen... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2006/makumiroshoowbiz/down%2Fholodilnik.ru/7... │ http://irr.ru/index.php?showalbum/litlip/saint-peter557932E-8C62-4917%26ad%3D64... │ 15 │ -│ -1 │ 0 │ 0 │ http://state=19945206/foto-4/login-2491724/?bundlers/search?text │ http://irr.ru/index.php?showalbum/login-kapusta-advert2651734&pt=b&pd=7&pw=1&pr... │ 15 │ -└────────────────┴────────────────┴─────────────┴────────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────────┘ -Run Time: real 0.137 user 1.400000 sys 0.004000 -D 07-01' AND (DATE '1970-01-01' + 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; -Run Time: real 0.016 user 0.220000 sys 0.000000 -D D (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 AND DontCountHits = 0 AND URLHash = 686716256552154761 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Run Time: real 0.022 user 0.312000 sys 0.000000 -D " = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) ORDER BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))); -Run Time: real 0.015 user 0.196000 sys 0.000000 diff --git a/benchmark/duckdb/queries.sql b/benchmark/duckdb/queries.sql index 6089b47ea7c..31f65fc898d 100644 --- a/benchmark/duckdb/queries.sql +++ b/benchmark/duckdb/queries.sql @@ -1,43 +1,43 @@ -SELECT count(*) FROM hits; -SELECT count(*) FROM hits WHERE AdvEngineID != 0; -SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits; -SELECT sum(UserID) FROM hits; +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; SELECT COUNT(DISTINCT UserID) FROM hits; SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT min(EventDate), max(EventDate) FROM hits; -SELECT AdvEngineID, count(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(MobilePhoneModel) > 0 GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(MobilePhoneModel) > 0 GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, count(*) AS c FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, count(*) FROM hits GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) AS m, SearchPhrase, count(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 12345678901234567890; -SELECT count(*) FROM hits WHERE URL::TEXT LIKE '%metrika%'; -SELECT SearchPhrase, min(URL), count(*) AS c FROM hits WHERE URL::TEXT LIKE '%metrika%' AND octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, min(URL), min(Title), count(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title::TEXT LIKE '%Яндекс%' AND URL::TEXT NOT LIKE '%.yandex.%' AND octet_length(SearchPhrase) > 0 GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL::TEXT LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE octet_length(SearchPhrase) > 0 ORDER BY EventTime, SearchPhrase LIMIT 10; -SELECT CounterID, avg(octet_length(URL)) AS l, count(*) AS c FROM hits WHERE octet_length(URL) > 0 GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -SELECT regexp_replace(Referer::TEXT, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS key, avg(octet_length(Referer)) AS l, count(*) AS c, min(Referer) FROM hits WHERE octet_length(Referer) > 0 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 hits; -SELECT SearchEngineID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits WHERE octet_length(SearchPhrase) > 0 GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, count(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, count(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, count(*) AS PageViews FROM hits WHERE CounterID = 62 AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(URL) > 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, count(*) AS PageViews FROM hits WHERE CounterID = 62 AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND DontCountHits = 0 AND "refresh" = 0 AND octet_length(Title) > 0 GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, count(*) AS PageViews FROM hits WHERE CounterID = 62 AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + 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 hits WHERE CounterID = 62 AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-31' AND "refresh" = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -SELECT URLHash, EventDate, count(*) AS PageViews FROM hits WHERE CounterID = 62 AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + 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 hits WHERE CounterID = 62 AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + 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', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) AS "Minute", count(*) AS PageViews FROM hits WHERE CounterID = 62 AND (DATE '1970-01-01' + EventDate) >= '2013-07-01' AND (DATE '1970-01-01' + EventDate) <= '2013-07-02' AND "refresh" = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))) ORDER BY DATE_TRUNC('minute', (TIMESTAMP '1970-01-01 00:00:00' + to_seconds(EventTime))); +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/duckdb/query.py b/benchmark/duckdb/query.py similarity index 100% rename from benchmark/compatible/duckdb/query.py rename to benchmark/duckdb/query.py diff --git a/benchmark/compatible/duckdb/results/c6a.4xlarge.txt b/benchmark/duckdb/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/duckdb/results/c6a.4xlarge.txt rename to benchmark/duckdb/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/duckdb/run.sh b/benchmark/duckdb/run.sh similarity index 100% rename from benchmark/compatible/duckdb/run.sh rename to benchmark/duckdb/run.sh diff --git a/benchmark/duckdb/usability.md b/benchmark/duckdb/usability.md deleted file mode 100644 index a032f68986a..00000000000 --- a/benchmark/duckdb/usability.md +++ /dev/null @@ -1,762 +0,0 @@ -Export data from ClickHouse: - -``` -SELECT * FROM hits_100m_obfuscated INTO OUTFILE 'hits.parquet' FORMAT Parquet - -$ wc -c hits.parquet -17193559098 hits.parquet -``` - -Install DuckDB: - -``` -wget https://github.com/duckdb/duckdb/releases/download/v0.3.0/duckdb_cli-linux-amd64.zip -unzip duckdb_cli-linux-amd64.zip -``` - -Try DuckDB: - -``` -milovidov@mtlog-perftest03j:~$ ./duckdb -v0.3.0 46a0fc50a -Enter ".help" for usage hints. -Connected to a transient in-memory database. -Use ".open FILENAME" to reopen on a persistent database. -D ? -> -> ; -Error: Parser Error: syntax error at or near "?" -LINE 1: ? - ^ -D help; -Error: Parser Error: syntax error at or near "help" -LINE 1: help; - ^ -D h; -Error: Parser Error: syntax error at or near "h" -LINE 1: h; - ^ -D .? -Error: unknown command or invalid arguments: "?". Enter ".help" for help -D .help -.auth ON|OFF Show authorizer callbacks -.backup ?DB? FILE Backup DB (default "main") to FILE -.bail on|off Stop after hitting an error. Default OFF -.binary on|off Turn binary output on or off. Default OFF -.cd DIRECTORY Change the working directory to DIRECTORY -.changes on|off Show number of rows changed by SQL -.check GLOB Fail if output since .testcase does not match -.clone NEWDB Clone data into NEWDB from the existing database -.databases List names and files of attached databases -.dbconfig ?op? ?val? List or change sqlite3_db_config() options -.dbinfo ?DB? Show status information about the database -.dump ?TABLE? Render database content as SQL -.echo on|off Turn command echo on or off -.eqp on|off|full|... Enable or disable automatic EXPLAIN QUERY PLAN -.excel Display the output of next command in spreadsheet -.exit ?CODE? Exit this program with return-code CODE -.expert EXPERIMENTAL. Suggest indexes for queries -.explain ?on|off|auto? Change the EXPLAIN formatting mode. Default: auto -.filectrl CMD ... Run various sqlite3_file_control() operations -.fullschema ?--indent? Show schema and the content of sqlite_stat tables -.headers on|off Turn display of headers on or off -.help ?-all? ?PATTERN? Show help text for PATTERN -.import FILE TABLE Import data from FILE into TABLE -.imposter INDEX TABLE Create imposter table TABLE on index INDEX -.indexes ?TABLE? Show names of indexes -.limit ?LIMIT? ?VAL? Display or change the value of an SQLITE_LIMIT -.lint OPTIONS Report potential schema issues. -.log FILE|off Turn logging on or off. FILE can be stderr/stdout -.mode MODE ?TABLE? Set output mode -.nullvalue STRING Use STRING in place of NULL values -.once ?OPTIONS? ?FILE? Output for the next SQL command only to FILE -.open ?OPTIONS? ?FILE? Close existing database and reopen FILE -.output ?FILE? Send output to FILE or stdout if FILE is omitted -.parameter CMD ... Manage SQL parameter bindings -.print STRING... Print literal STRING -.progress N Invoke progress handler after every N opcodes -.prompt MAIN CONTINUE Replace the standard prompts -.quit Exit this program -.read FILE Read input from FILE -.restore ?DB? FILE Restore content of DB (default "main") from FILE -.save FILE Write in-memory database into FILE -.scanstats on|off Turn sqlite3_stmt_scanstatus() metrics on or off -.schema ?PATTERN? Show the CREATE statements matching PATTERN -.selftest ?OPTIONS? Run tests defined in the SELFTEST table -.separator COL ?ROW? Change the column and row separators -.sha3sum ... Compute a SHA3 hash of database content -.shell CMD ARGS... Run CMD ARGS... in a system shell -.show Show the current values for various settings -.stats ?on|off? Show stats or turn stats on or off -.system CMD ARGS... Run CMD ARGS... in a system shell -.tables ?TABLE? List names of tables matching LIKE pattern TABLE -.testcase NAME Begin redirecting output to 'testcase-out.txt' -.testctrl CMD ... Run various sqlite3_test_control() operations -.timeout MS Try opening locked tables for MS milliseconds -.timer on|off Turn SQL timer on or off -.trace ?OPTIONS? Output each SQL statement as it is run -.vfsinfo ?AUX? Information about the top-level VFS -.vfslist List all available VFSes -.vfsname ?AUX? Print the name of the VFS stack -.width NUM1 NUM2 ... Set minimum column widths for columnar output -D q -> .q -> ; -Error: Parser Error: syntax error at or near "q" -LINE 1: q - ^ -D .q -``` - -Let's load the data: - -``` -D CREATE TABLE hits AS SELECT * FROM parquet_scan('hits.parquet') -> ; -``` - -It is using single CPU core and accumulating data in memory. - -``` -Killed -``` - -As expected. My server has "only" 128 GiB RAM. - -Let's free some memory and run again: - -``` -Error: Out of Memory Error: could not allocate block of 262144 bytes -Database is launched in in-memory mode and no temporary directory is specified. -Unused blocks cannot be offloaded to disk. - -Launch the database with a persistent storage back-end -Or set PRAGMA temp_directory='/path/to/tmp.tmp' -``` - -Now it works: - -``` -D PRAGMA temp_directory='duckdb.tmp' -> ; -D CREATE TABLE hits AS SELECT * FROM parquet_scan('hits.parquet'); -D SELECT count(*) FROM hits; -┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 100000000 │ -└──────────────┘ -D SELECT AdvEngineID, count(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -┌─────────────┬──────────────┐ -│ AdvEngineID │ count_star() │ -├─────────────┼──────────────┤ -│ 2 │ 404620 │ -│ 27 │ 113167 │ -│ 13 │ 45633 │ -│ 45 │ 38974 │ -│ 44 │ 9731 │ -│ 3 │ 6896 │ -│ 62 │ 5266 │ -│ 52 │ 3554 │ -│ 50 │ 938 │ -│ 28 │ 836 │ -│ 53 │ 350 │ -│ 25 │ 343 │ -│ 61 │ 158 │ -│ 21 │ 38 │ -│ 42 │ 20 │ -│ 16 │ 7 │ -│ 7 │ 3 │ -│ 22 │ 1 │ -└─────────────┴──────────────┘ -``` - -But saving the database does not work: - -``` -D .save 'duckdb.data' -sqlite3_backup_init: unsupported. -Error: -``` - -Let's simply paste queries into CLI. - -``` -D SELECT count(*) FROM hits WHERE AdvEngineID != 0; -┌──────────────┐ -│ count_star() │ -├──────────────┤ -│ 630535 │ -└──────────────┘ -Run Time: real 0.227 user 0.228000 sys 0.000000 -D SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits; -┌──────────────────┬──────────────┬──────────────────────┐ -│ sum(advengineid) │ count_star() │ avg(resolutionwidth) │ -├──────────────────┼──────────────┼──────────────────────┤ -│ 7280824 │ 100000000 │ 1513.48908394 │ -└──────────────────┴──────────────┴──────────────────────┘ -Run Time: real 0.678 user 0.508000 sys 0.008000 -D SELECT sum(UserID) FROM hits; -┌────────────────────────┐ -│ sum(userid) │ -├────────────────────────┤ -│ 3.2306058693988996e+26 │ -└────────────────────────┘ -Run Time: real 0.697 user 0.448000 sys 0.020000 -D SELECT COUNT(DISTINCT UserID) FROM hits; -┌───────────────┐ -│ count(userid) │ -├───────────────┤ -│ 17630976 │ -└───────────────┘ -Run Time: real 7.928 user 7.164000 sys 0.660000 -D SELECT COUNT(DISTINCT SearchPhrase) FROM hits;u -┌─────────────────────┐ -│ count(searchphrase) │ -├─────────────────────┤ -│ 6019589 │ -└─────────────────────┘ -Run Time: real 12.403 user 10.820000 sys 0.208000 -D SELECT min(EventDate), max(EventDate) FROM hits; -R┌────────────────┬────────────────┐ -│ min(eventdate) │ max(eventdate) │ -├────────────────┼────────────────┤ -│ 15888 │ 15917 │ -└────────────────┴────────────────┘ -Run Time: real 0.604 user 0.376000 sys 0.008000 -D SELECT AdvEngineID, count(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC;( -┌─────────────┬──────────────┐ -│ AdvEngineID │ count_star() │ -├─────────────┼──────────────┤ -│ 2 │ 404620 │ -│ 27 │ 113167 │ -│ 13 │ 45633 │ -│ 45 │ 38974 │ -│ 44 │ 9731 │ -│ 3 │ 6896 │ -│ 62 │ 5266 │ -│ 52 │ 3554 │ -│ 50 │ 938 │ -│ 28 │ 836 │ -│ 53 │ 350 │ -│ 25 │ 343 │ -│ 61 │ 158 │ -│ 21 │ 38 │ -│ 42 │ 20 │ -│ 16 │ 7 │ -│ 7 │ 3 │ -│ 22 │ 1 │ -└─────────────┴──────────────┘ -Run Time: real 0.344 user 0.344000 sys 0.000000 -D SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -┌──────────┬─────────┐ -│ RegionID │ u │ -├──────────┼─────────┤ -│ 229 │ 2845673 │ -│ 2 │ 1081016 │ -│ 208 │ 831676 │ -│ 169 │ 604583 │ -│ 184 │ 322661 │ -│ 158 │ 307152 │ -│ 34 │ 299479 │ -│ 55 │ 286525 │ -│ 107 │ 272448 │ -│ 42 │ 243181 │ -└──────────┴─────────┘ -Run Time: real 8.872 user 7.592000 sys 0.108000 -D SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10;l -┌──────────┬──────────────────┬──────────┬──────────────────────┬───────────────┐ -│ RegionID │ sum(advengineid) │ c │ avg(resolutionwidth) │ count(userid) │ -├──────────┼──────────────────┼──────────┼──────────────────────┼───────────────┤ -│ 229 │ 2078084 │ 18296430 │ 1506.0876750819696 │ 2845673 │ -│ 2 │ 441711 │ 6687708 │ 1479.8410618406187 │ 1081016 │ -│ 208 │ 285925 │ 4261945 │ 1285.260504769536 │ 831676 │ -│ 169 │ 100887 │ 3320286 │ 1465.90517142198 │ 604583 │ -│ 32 │ 81498 │ 1843721 │ 1538.0370495318978 │ 216010 │ -│ 34 │ 161779 │ 1792406 │ 1548.364990409539 │ 299479 │ -│ 184 │ 55526 │ 1755223 │ 1506.8102679830426 │ 322661 │ -│ 42 │ 108820 │ 1542771 │ 1587.1074287758845 │ 243181 │ -│ 107 │ 120470 │ 1516722 │ 1548.6039623609336 │ 272448 │ -│ 51 │ 98212 │ 1435598 │ 1579.8864215469791 │ 211505 │ -└──────────┴──────────────────┴──────────┴──────────────────────┴───────────────┘ -Run Time: real 8.447 user 8.444000 sys 0.000000 -D SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -t┌──────────────────┬─────────┐ -│ MobilePhoneModel │ u │ -├──────────────────┼─────────┤ -│ iPad │ 1090347 │ -│ iPhone │ 45758 │ -│ A500 │ 16046 │ -│ N8-00 │ 5565 │ -│ iPho │ 3300 │ -│ ONE TOUCH 6030A │ 2759 │ -│ GT-P7300B │ 1907 │ -│ 3110000 │ 1871 │ -│ GT-I9500 │ 1598 │ -│ eagle75 │ 1492 │ -└──────────────────┴─────────┘ -Run Time: real 5.077 user 4.416000 sys 0.032000 -D SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;, -┌─────────────┬──────────────────┬────────┐ -│ MobilePhone │ MobilePhoneModel │ u │ -├─────────────┼──────────────────┼────────┤ -│ 1 │ iPad │ 931038 │ -│ 5 │ iPad │ 48385 │ -│ 6 │ iPad │ 29710 │ -│ 7 │ iPad │ 28391 │ -│ 118 │ A500 │ 16005 │ -│ 6 │ iPhone │ 14516 │ -│ 26 │ iPhone │ 13566 │ -│ 10 │ iPad │ 11433 │ -│ 32 │ iPad │ 9503 │ -│ 13 │ iPad │ 9417 │ -└─────────────┴──────────────────┴────────┘ -Run Time: real 5.193 user 4.916000 sys 0.012000 -D SELECT SearchPhrase, count(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -l┌────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchPhrase │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 70263 │ -│ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 34675 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 24579 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 21649 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 19703 │ -│ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 19195 │ -│ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 17284 │ -│ galaxy table │ 16746 │ -│ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 16620 │ -│ \xD1\x81\xD0\xBA\xD0\xBE\xD0\xBB\xD1\x8C\xD0\xBA\xD0\xBE \xD0\xBC\xD1\x8B\xD1\x... │ 12317 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 8.085 user 8.040000 sys 0.044000 -D SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;e -┌────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchPhrase │ u │ -├────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 23673 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 19743 │ -│ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 18394 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 17553 │ -│ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 14600 │ -│ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 14529 │ -│ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 14198 │ -│ \xD1\x81\xD0\xBA\xD0\xBE\xD0\xBB\xD1\x8C\xD0\xBA\xD0\xBE \xD0\xBC\xD1\x8B\xD1\x... │ 9007 │ -│ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 8792 │ -│ \xD0\xBA\xD0\xBE\xD0\xBC\xD0\xB1\xD0\xB8\xD0\xBD\xD0\xB8\xD1\x80\xD0\xBE\xD0\xB... │ 7572 │ -└────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 14.516 user 12.960000 sys 1.196000 -D SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -r┌────────────────┬────────────────────────────────────────────────────────────────────────────────────┬───────┐ -│ SearchEngineID │ SearchPhrase │ c │ -├────────────────┼────────────────────────────────────────────────────────────────────────────────────┼───────┤ -│ 2 │ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 46258 │ -│ 2 │ \xD0\xBC\xD0\xB0\xD0\xBD\xD0\xB3\xD1\x83 \xD0\xB2 \xD0\xB7\xD0\xB0\xD1\x80\xD0\... │ 18871 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 16905 │ -│ 3 │ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 16748 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C \xD0\xBE\xD0\x... │ 14911 │ -│ 2 │ \xD0\xB0\xD0\xBB\xD0\xB1\xD0\xB0\xD1\x82\xD1\x80\xD1\x83\xD1\x82\xD0\xB4\xD0\xB... │ 13716 │ -│ 2 │ \xD1\x8D\xD0\xBA\xD0\xB7\xD0\xBE\xD0\xB8\xD0\xB4\xD0\xBD\xD1\x8B\xD0\xB5 │ 13414 │ -│ 2 │ \xD1\x81\xD0\xBC\xD0\xBE\xD1\x82\xD1\x80\xD0\xB5\xD1\x82\xD1\x8C │ 13105 │ -│ 3 │ \xD0\xBA\xD0\xB0\xD1\x80\xD0\xB5\xD0\xBB\xD0\xBA\xD0\xB8 │ 12815 │ -│ 2 │ \xD0\xB4\xD1\x80\xD1\x83\xD0\xB6\xD0\xBA\xD0\xB5 \xD0\xBF\xD0\xBE\xD0\xBC\xD0\x... │ 11946 │ -└────────────────┴────────────────────────────────────────────────────────────────────────────────────┴───────┘ -Run Time: real 8.029 user 7.544000 sys 0.016000 -D SELECT UserID, count(*) FROM hits GROUP BY UserID ORDER BY count(*) DESC LIMIT 10;s -┌─────────────────────┬──────────────┐ -│ UserID │ count_star() │ -├─────────────────────┼──────────────┤ -│ 1313338681122956954 │ 29097 │ -│ 1907779576417363396 │ 25333 │ -│ 2305303682471783379 │ 10611 │ -│ 7982623143712728547 │ 7584 │ -│ 6018350421959114808 │ 6678 │ -│ 7280399273658728997 │ 6411 │ -│ 1090981537032625727 │ 6197 │ -│ 5730251990344211405 │ 6019 │ -│ 835157184735512989 │ 5211 │ -│ 770542365400669095 │ 4906 │ -└─────────────────────┴──────────────┘ -Run Time: real 5.225 user 5.224000 sys 0.000000 -D SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10;, -┌─────────────────────┬──────────────┬──────────────┐ -│ UserID │ SearchPhrase │ count_star() │ -├─────────────────────┼──────────────┼──────────────┤ -│ 1313338681122956954 │ │ 29097 │ -│ 1907779576417363396 │ │ 25333 │ -│ 2305303682471783379 │ │ 10611 │ -│ 7982623143712728547 │ │ 6669 │ -│ 7280399273658728997 │ │ 6408 │ -│ 1090981537032625727 │ │ 6196 │ -│ 5730251990344211405 │ │ 6019 │ -│ 6018350421959114808 │ │ 5990 │ -│ 835157184735512989 │ │ 5209 │ -│ 770542365400669095 │ │ 4906 │ -└─────────────────────┴──────────────┴──────────────┘ -Run Time: real 14.506 user 13.748000 sys 0.496000 -D SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -┌──────────────────────┬────────────────────────────────────────────────────────────────────────────────────┬──────────────┐ -│ UserID │ SearchPhrase │ count_star() │ -├──────────────────────┼────────────────────────────────────────────────────────────────────────────────────┼──────────────┤ -│ 427738049800818189 │ │ 1 │ -│ 15985305027620249815 │ │ 6 │ -│ 7418527520126366595 │ │ 1 │ -│ 519640690937130534 │ │ 2 │ -│ 376160620089546609 │ │ 1 │ -│ 4523925649124320482 │ │ 1 │ -│ 2523324276554785406 │ │ 2 │ -│ 6025915247311731176 │ │ 26 │ -│ 6329532664518159520 │ \xD0\xB2\xD0\xB5\xD0\xB4\xD0\xBE\xD0\xBC\xD0\xBE\xD1\x81\xD0\xBA\xD0\xB2\xD1\x8... │ 2 │ -│ 6329532664518159520 │ │ 19 │ -└──────────────────────┴────────────────────────────────────────────────────────────────────────────────────┴──────────────┘ -Run Time: real 14.919 user 14.912000 sys 0.008000 -D SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, count(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10;W -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Binder Error: No function matches the given name and argument types 'date_part(VARCHAR, BIGINT)'. You might need to add explicit type casts. - Candidate functions: - date_part(VARCHAR, DATE) -> BIGINT - date_part(VARCHAR, TIMESTAMP) -> BIGINT - date_part(VARCHAR, TIME) -> BIGINT - date_part(VARCHAR, INTERVAL) -> BIGINT - -LINE 1: SELECT UserID, extract(minute FROM EventTime) AS m, Se... - ^ -D SELECT UserID FROM hits WHERE UserID = -6101065172474983726; -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Conversion Error: Type INT64 with value -6101065172474983726 can't be cast because the value is out of range for the destination type UINT64 -D SELECT count(*) FROM hits WHERE URL LIKE '%metrika%'; -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Binder Error: No function matches the given name and argument types '~~(BLOB, VARCHAR)'. You might need to add explicit type casts. - Candidate functions: - ~~(VARCHAR, VARCHAR) -> BOOLEAN - -D SELECT SearchPhrase, min(URL), count(*) AS c FROM hits WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;A -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Binder Error: No function matches the given name and argument types '~~(BLOB, VARCHAR)'. You might need to add explicit type casts. - Candidate functions: - ~~(VARCHAR, VARCHAR) -> BOOLEAN - -D , 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;A -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Binder Error: No function matches the given name and argument types '~~(BLOB, VARCHAR)'. You might need to add explicit type casts. - Candidate functions: - ~~(VARCHAR, VARCHAR) -> BOOLEAN - -D SELECT * FROM hits WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Binder Error: No function matches the given name and argument types '~~(BLOB, VARCHAR)'. You might need to add explicit type casts. - Candidate functions: - ~~(VARCHAR, VARCHAR) -> BOOLEAN - -D SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ galaxy s4 zoom \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC │ -│ \xD0\xBD\xD0\xBE\xD1\x87\xD0\xBD\xD0\xBE \xD0\xBA\xD0\xB8\xD1\x82\xD0\xB0\xD1\x... │ -│ \xD1\x81\xD0\xB8\xD0\xBC\xD0\xBF\xD1\x82\xD0\xBE\xD0\xBC\xD1\x8B \xD1\x80\xD0\x... │ -│ \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC \xD0\xBD\xD0\xB5\xD0\xB1\xD0\xBE\xD0\x... │ -│ \xD1\x80\xD0\xB0\xD1\x81\xD0\xBF\xD0\xB8\xD1\x81\xD0\xB0\xD0\xBD\xD0\xB8\xD0\xB... │ -│ \xD0\xB1\xD1\x80\xD0\xB8\xD1\x82\xD0\xB0 \xD0\xB3\xD0\xB0\xD0\xBD\xD0\xB0\xD0\x... │ -│ \xD0\xB0\xD0\xBD\xD0\xB0\xD0\xBF\xD0\xB0 \xD0\xBE\xD0\xBF\xD0\xB5\xD1\x80\xD0\x... │ -│ \xD1\x81\xD0\xBA\xD0\xB0\xD1\x87\xD0\xB0\xD1\x82\xD1\x8C \xD1\x87\xD0\xB8\xD1\x... │ -│ \xD1\x81\xD0\xBB\xD0\xBE\xD0\xBD.\xD1\x80\xD1\x83\xD0\xB1., \xD0\xB4. \xD0\xB0.... │ -│ \xD0\xBE\xD1\x82\xD0\xB4\xD1\x8B\xD1\x85\xD0\xB0 \xD1\x87\xD0\xB5\xD0\xBC \xD0\... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 4.282 user 3.572000 sys 0.048000 -D SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;= -┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ ! hektdf gjcgjhn conster │ -│ ! \xD1\x81\xD0\xBA\xD0\xB0\xD1\x80\xD0\xBF │ -│ !(\xD0\xBA\xD0\xB0\xD0\xBA \xD0\xB2\xD0\xBE\xD1\x80\xD0\xBE\xD0\xBD\xD0\xB8 │ -│ !(\xD0\xBF\xD0\xBE \xD0\xB3\xD0\xBE\xD1\x80\xD0\xB8\xD1\x8E \xD0\xB2 \xD1\x8F\x... │ -│ !(\xD1\x81) \xD0\xBF\xD1\x80\xD0\xBE \xD0\xB4\xD0\xBF\xD0\xBE \xD1\x81\xD0\xB5\... │ -│ !(\xD1\x81\xD0\xB0\xD0\xBB\xD0\xBE\xD0\xBD\xD1\x8B \xD0\xBE\xD1\x81\xD1\x82\xD0... │ -│ !(\xD1\x81\xD1\x82\xD0\xB0\xD1\x80\xD1\x82\xD0\xB5\xD1\x80 rav4 \xD1\x82\xD1\x8... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD0\xB4\xD0\xBB\xD1\x8F \xD0\xBC\xD1\x8F\xD1... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD0\xBC\xD0\xB0\xD1\x83\xD1\x81 \xD0\xBA\xD0... │ -│ !\xD0\xBA\xD1\x83\xD0\xB3\xD0\xB8 \xD1\x81\xD0\xB5\xD1\x80\xD0\xB8\xD0\xB8 │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 3.610 user 3.612000 sys 0.000000 -D SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┐ -│ SearchPhrase │ -├────────────────────────────────────────────────────────────────────────────────────┤ -│ galaxy s4 zoom \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC │ -│ \xD0\xBD\xD0\xBE\xD1\x87\xD0\xBD\xD0\xBE \xD0\xBA\xD0\xB8\xD1\x82\xD0\xB0\xD1\x... │ -│ \xD1\x81\xD0\xB8\xD0\xBC\xD0\xBF\xD1\x82\xD0\xBE\xD0\xBC\xD1\x8B \xD1\x80\xD0\x... │ -│ \xD1\x84\xD0\xB8\xD0\xBB\xD1\x8C\xD0\xBC \xD0\xBD\xD0\xB5\xD0\xB1\xD0\xBE\xD0\x... │ -│ \xD0\xB0\xD0\xB2\xD0\xBE\xD0\xBC \xD0\xBA\xD0\xBE\xD0\xBD\xD1\x81\xD1\x82\xD0\x... │ -│ \xD0\xB0\xD0\xBD\xD0\xB0\xD0\xBF\xD0\xB0 \xD0\xBE\xD0\xBF\xD0\xB5\xD1\x80\xD0\x... │ -│ \xD0\xB1\xD1\x80\xD0\xB8\xD1\x82\xD0\xB0 \xD0\xB3\xD0\xB0\xD0\xBD\xD0\xB0\xD0\x... │ -│ \xD0\xBA\xD0\xBE\xD0\xBC\xD0\xBF\xD1\x8C\xD1\x8E\xD1\x82\xD0\xB5\xD1\x80\xD0\xB... │ -│ \xD0\xBE\xD1\x82\xD0\xB4\xD1\x8B\xD1\x85\xD0\xB0 \xD1\x87\xD0\xB5\xD0\xBC \xD0\... │ -│ \xD1\x80\xD0\xB0\xD1\x81\xD0\xBF\xD0\xB8\xD1\x81\xD0\xB0\xD0\xBD\xD0\xB8\xD0\xB... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -Run Time: real 3.640 user 3.640000 sys 0.000000 -D 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; -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Binder Error: No function matches the given name and argument types 'length(BLOB)'. You might need to add explicit type casts. - Candidate functions: - length(VARCHAR) -> BIGINT - length(LIST) -> BIGINT - -LINE 1: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM h... - ^ -D Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; -VRun Time: real 0.000 user 0.000000 sys 0.000000 -Error: Binder Error: No function matches the given name and argument types 'regexp_replace(BLOB, VARCHAR, VARCHAR)'. You might need to add explicit type casts. - Candidate functions: - regexp_replace(VARCHAR, VARCHAR, VARCHAR) -> VARCHAR - regexp_replace(VARCHAR, VARCHAR, VARCHAR, VARCHAR) -> VARCHAR - -LINE 1: SELECT REGEXP_REPLACE(Referer, '^https?://(?:w... - ^ -D + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits; -┌──────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬──────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┬───────────────────────────┐ -│ 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) │ -├──────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼──────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┼───────────────────────────┤ -│ 151348908394 │ 151448908394 │ 151548908394 │ 151648908394 │ 151748908394 │ 151848908394 │ 151948908394 │ 152048908394 │ 152148908394 │ 152248908394 │ 152348908394 │ 152448908394 │ 152548908394 │ 152648908394 │ 152748908394 │ 152848908394 │ 152948908394 │ 153048908394 │ 153148908394 │ 153248908394 │ 153348908394 │ 153448908394 │ 153548908394 │ 153648908394 │ 153748908394 │ 153848908394 │ 153948908394 │ 154048908394 │ 154148908394 │ 154248908394 │ 154348908394 │ 154448908394 │ 154548908394 │ 154648908394 │ 154748908394 │ 154848908394 │ 154948908394 │ 155048908394 │ 155148908394 │ 155248908394 │ 155348908394 │ 155448908394 │ 155548908394 │ 155648908394 │ 155748908394 │ 155848908394 │ 155948908394 │ 156048908394 │ 156148908394 │ 156248908394 │ 156348908394 │ 156448908394 │ 156548908394 │ 156648908394 │ 156748908394 │ 156848908394 │ 156948908394 │ 157048908394 │ 157148908394 │ 157248908394 │ 157348908394 │ 157448908394 │ 157548908394 │ 157648908394 │ 157748908394 │ 157848908394 │ 157948908394 │ 158048908394 │ 158148908394 │ 158248908394 │ 158348908394 │ 158448908394 │ 158548908394 │ 158648908394 │ 158748908394 │ 158848908394 │ 158948908394 │ 159048908394 │ 159148908394 │ 159248908394 │ 159348908394 │ 159448908394 │ 159548908394 │ 159648908394 │ 159748908394 │ 159848908394 │ 159948908394 │ 160048908394 │ 160148908394 │ 160248908394 │ -└──────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴──────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┴───────────────────────────┘ -Run Time: real 85.256 user 85.252000 sys 0.000000 -D SELECT SearchEngineID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -┌────────────────┬────────────┬──────┬──────────────┬──────────────────────┐ -│ SearchEngineID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├────────────────┼────────────┼──────┼──────────────┼──────────────────────┤ -│ 2 │ 1138507705 │ 1633 │ 35 │ 1408.0122473974282 │ -│ 2 │ 1740861572 │ 1331 │ 28 │ 1577.945905334335 │ -│ 2 │ 3487820196 │ 1144 │ 35 │ 1553.1984265734266 │ -│ 2 │ 3797060577 │ 1140 │ 36 │ 1543.4140350877192 │ -│ 2 │ 2349209741 │ 1105 │ 30 │ 1557.387330316742 │ -│ 2 │ 2424344199 │ 1102 │ 31 │ 1555.6588021778584 │ -│ 2 │ 3663904793 │ 1083 │ 31 │ 1581.8171745152354 │ -│ 2 │ 3829154130 │ 1082 │ 30 │ 1541.253234750462 │ -│ 2 │ 2551371145 │ 1080 │ 24 │ 1559.8092592592593 │ -│ 2 │ 4029049820 │ 1058 │ 32 │ 1556.2003780718337 │ -└────────────────┴────────────┴──────┴──────────────┴──────────────────────┘ -Run Time: real 8.033 user 7.032000 sys 0.048000 -D SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -┌─────────────────────┬────────────┬───┬──────────────┬──────────────────────┐ -│ WatchID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├─────────────────────┼────────────┼───┼──────────────┼──────────────────────┤ -│ 7472773096904766158 │ 972408088 │ 2 │ 0 │ 1368.0 │ -│ 8515267528803597958 │ 2005721512 │ 2 │ 0 │ 1917.0 │ -│ 5431383378337214900 │ 1373018819 │ 2 │ 0 │ 1087.0 │ -│ 4975771741728931240 │ 1594850068 │ 2 │ 0 │ 1917.0 │ -│ 6143560365929503526 │ 2912060982 │ 2 │ 0 │ 1368.0 │ -│ 4661775965756901134 │ 3822464671 │ 2 │ 0 │ 1638.0 │ -│ 5340100429706330950 │ 709893659 │ 2 │ 0 │ 1368.0 │ -│ 5265600775603767970 │ 1677655885 │ 2 │ 0 │ 1396.0 │ -│ 5449946953533528811 │ 3822667196 │ 2 │ 0 │ 1638.0 │ -│ 6426552621243022389 │ 3557962159 │ 2 │ 0 │ 1638.0 │ -└─────────────────────┴────────────┴───┴──────────────┴──────────────────────┘ -Run Time: real 9.317 user 8.380000 sys 0.052000 -D SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -┌─────────────────────┬────────────┬───┬──────────────┬──────────────────────┐ -│ WatchID │ ClientIP │ c │ sum(refresh) │ avg(resolutionwidth) │ -├─────────────────────┼────────────┼───┼──────────────┼──────────────────────┤ -│ 5732691047654519103 │ 1097532796 │ 2 │ 0 │ 1638.0 │ -│ 8308952461884454508 │ 2609801721 │ 2 │ 0 │ 1087.0 │ -│ 7472773096904766158 │ 972408088 │ 2 │ 0 │ 1368.0 │ -│ 7360470262372840837 │ 972408088 │ 2 │ 0 │ 1368.0 │ -│ 4778976465399160621 │ 3938580212 │ 2 │ 2 │ 1638.0 │ -│ 4848145794958638974 │ 3938580212 │ 2 │ 0 │ 1638.0 │ -│ 9172448021081089285 │ 2530876984 │ 2 │ 0 │ 1638.0 │ -│ 6471985135199404171 │ 765833715 │ 2 │ 0 │ 1594.0 │ -│ 8824813183119863159 │ 765833715 │ 2 │ 0 │ 1594.0 │ -│ 8227322756510819845 │ 765833715 │ 2 │ 0 │ 1594.0 │ -└─────────────────────┴────────────┴───┴──────────────┴──────────────────────┘ -Run Time: real 48.016 user 32.076000 sys 8.092000 -D SELECT URL, count(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -┌────────────────────────────────────────────────────────────────────────────────────┬─────────┐ -│ URL │ c │ -├────────────────────────────────────────────────────────────────────────────────────┼─────────┤ -│ http://liver.ru/belgorod/page/1006.j\xD0\xBA\xD0\xB8/\xD0\xB4\xD0\xBE\xD0\xBF_\... │ 3288173 │ -│ http://kinopoisk.ru │ 1625251 │ -│ http://bdsm_po_yers=0&with_video │ 791465 │ -│ http://video.yandex │ 582404 │ -│ http://smeshariki.ru/region │ 514984 │ -│ http://auto_fiat_dlya-bluzki%2F8536.30.18&he=900&with │ 507995 │ -│ http://liver.ru/place_rukodel=365115eb7bbb90 │ 359893 │ -│ http://kinopoisk.ru/vladimir.irr.ru │ 354690 │ -│ http://video.yandex.ru/search/?jenre=50&s_yers │ 318979 │ -│ http://tienskaia-moda │ 289355 │ -└────────────────────────────────────────────────────────────────────────────────────┴─────────┘ -Run Time: real 55.180 user 33.916000 sys 1.012000 -D SELECT 1, URL, count(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -┌───┬────────────────────────────────────────────────────────────────────────────────────┬─────────┐ -│ 1 │ URL │ c │ -├───┼────────────────────────────────────────────────────────────────────────────────────┼─────────┤ -│ 1 │ http://liver.ru/belgorod/page/1006.j\xD0\xBA\xD0\xB8/\xD0\xB4\xD0\xBE\xD0\xBF_\... │ 3288173 │ -│ 1 │ http://kinopoisk.ru │ 1625251 │ -│ 1 │ http://bdsm_po_yers=0&with_video │ 791465 │ -│ 1 │ http://video.yandex │ 582404 │ -│ 1 │ http://smeshariki.ru/region │ 514984 │ -│ 1 │ http://auto_fiat_dlya-bluzki%2F8536.30.18&he=900&with │ 507995 │ -│ 1 │ http://liver.ru/place_rukodel=365115eb7bbb90 │ 359893 │ -│ 1 │ http://kinopoisk.ru/vladimir.irr.ru │ 354690 │ -│ 1 │ http://video.yandex.ru/search/?jenre=50&s_yers │ 318979 │ -│ 1 │ http://tienskaia-moda │ 289355 │ -└───┴────────────────────────────────────────────────────────────────────────────────────┴─────────┘ -Run Time: real 34.194 user 34.132000 sys 0.060000 -D 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; -┌────────────┬──────────────┬──────────────┬──────────────┬───────┐ -│ ClientIP │ clientip - 1 │ clientip - 2 │ clientip - 3 │ c │ -├────────────┼──────────────┼──────────────┼──────────────┼───────┤ -│ 4255045322 │ 4255045321 │ 4255045320 │ 4255045319 │ 47008 │ -│ 2596862839 │ 2596862838 │ 2596862837 │ 2596862836 │ 29121 │ -│ 3119147744 │ 3119147743 │ 3119147742 │ 3119147741 │ 25333 │ -│ 1696638182 │ 1696638181 │ 1696638180 │ 1696638179 │ 20230 │ -│ 1138507705 │ 1138507704 │ 1138507703 │ 1138507702 │ 15778 │ -│ 3367941774 │ 3367941773 │ 3367941772 │ 3367941771 │ 12768 │ -│ 3032827420 │ 3032827419 │ 3032827418 │ 3032827417 │ 11349 │ -│ 1740861572 │ 1740861571 │ 1740861570 │ 1740861569 │ 11315 │ -│ 3487820196 │ 3487820195 │ 3487820194 │ 3487820193 │ 9881 │ -│ 3663904793 │ 3663904792 │ 3663904791 │ 3663904790 │ 9718 │ -└────────────┴──────────────┴──────────────┴──────────────┴───────┘ -Run Time: real 17.028 user 17.024000 sys 0.004000 -D ) 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; -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Conversion Error: Could not convert string '2013-07-01' to UINT16 -D 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; -Run Time: real 0.001 user 0.000000 sys 0.000000 -Error: Conversion Error: Could not convert string '2013-07-01' to UINT16 -D 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; -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Conversion Error: Could not convert string '2013-07-01' to UINT16 -D ROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND "refresh" = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -Run Time: real 0.001 user 0.004000 sys 0.000000 -Error: Conversion Error: Could not convert string '2013-07-01' to UINT16 -D ND 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; -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Conversion Error: Could not convert string '2013-07-01' to UINT16 -D ate >= '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; -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Conversion Error: Could not convert string '2013-07-01' to UINT16 -D ts 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); -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Binder Error: No function matches the given name and argument types 'date_trunc(VARCHAR, BIGINT)'. You might need to add explicit type casts. - Candidate functions: - date_trunc(VARCHAR, TIMESTAMP) -> TIMESTAMP - date_trunc(VARCHAR, DATE) -> TIMESTAMP - -LINE 1: ...sh" = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER B... -``` - -Mostly alright but some functions are missing and the types from Parquet are wrong. - -Let's try to load from CSV: - -``` -SELECT * FROM hits_100m_obfuscated INTO OUTFILE 'hits.csv' FORMAT CSV -``` - -``` -$ ./duckdb -v0.3.0 46a0fc50a -Enter ".help" for usage hints. -Connected to a transient in-memory database. -Use ".open FILENAME" to reopen on a persistent database. -D .open 'duckdb.db' -D ; -D PRAGMA temp_directory='duckdb.tmp'; -Error: Parser Error: syntax error at or near "" -LINE 1: PRAGMA temp_directory='duckdb.tmp'; - ^ -D PRAGMA temp_directory='duckdb.tmp'; -Error: Parser Error: syntax error at or near "" -LINE 1: PRAGMA temp_directory='duckdb.tmp'; - ^ -D .open 'duckdb.db'; -D -D SELECT 1 -> ; -┌───┐ -│ 1 │ -├───┤ -│ 1 │ -└───┘ -D PRAGMA temp_directory='duckdb.tmp'; -Error: Parser Error: syntax error at or near "" -LINE 1: PRAGMA temp_directory='duckdb.tmp'; - ^ -D CREATE TABLE hits AS SELECT * FROM read_csv_auto('hits.csv'); -Error: String value is not valid UTF8 -``` - -It does not work for non-UTF8 data. - -Let's cleanup UTF-8. - -``` -clickhouse-local --input-format LineAsString --output-format TSVRaw --structure 's String' --query "SELECT toValidUTF8(s) FROM table" --progress < hits.csv > hits_valid.csv -``` - -``` -D CREATE TABLE hits AS SELECT * FROM read_csv_auto('hits_valid.csv'); -Error: Invalid Input Error: Could not convert string '2149615427' to INT32 in column "column082", between line 137217 and 138240. Parser options: DELIMITER=',' (auto detected), QUOTE='"' (auto detected), ESCAPE='' (auto detected), HEADER=0 (auto detected), SAMPLE_SIZE=10240, ALL_VARCHAR=0. Consider either increasing the sample size (SAMPLE_SIZE=X [X rows] or SAMPLE_SIZE=-1 [all rows]), or skipping column conversion (ALL_VARCHAR=1) -``` - -Does not work either. - -DuckDB CLI does not support history search (Ctrl+R). - -If I write a command and then prepend `-- ` before it, then history navigation becomes completely broken. - -``` -D CREATE TABLE hits AS SELECT * FROM parquet_scan('hits.parquet' ; -Run Time: real 0.000 user 0.000000 sys 0.000000 -Error: Parser Error: syntax error at or near ";" -LINE 1: ...ECT * FROM parquet_scan('hits.parquet' ; - ^ -D CREATE TABLE hits AS SELECT * FROM parquet_scan('hits.parque)' ; -``` - -``` -D CREATE TABLE hits AS SELECT * FROM parquet_scan('hits.parquet'); -Run Time: real 1086.631 user 758.036000 sys 201.360000 -``` - -It's just about 100 000 rows/second. Quite decent but not good. - -Cancelling queries by Ctrl+C does not work. - -I've noticed that DuckDB is using single thread. -Most likely it can be fixed with - -``` -PRAGMA threads = 16; -``` - -``` -PRAGMA threads = 16; -PRAGMA temp_directory='duckdb.tmp'; -.timer on -CREATE TABLE hits AS SELECT * FROM parquet_scan('hits.parquet'); -``` - -It's much better with threads: - -``` -D CREATE TABLE hits AS SELECT * FROM parquet_scan('hits.parquet'); -Run Time: real 315.828 user 447.896000 sys 126.552000 -``` - -Let's run all the queries three times. - -Collecting the results: - -``` -grep -F 'Run Time: real ' log | grep -oP 'real [\d\.]+' | grep -oP '[\d\.]+' | tail -n129 | clickhouse-local --structure 'x Decimal32(3)' --query "SELECT groupArray(x) FROM table GROUP BY rowNumberInAllBlocks() % 43 AS n ORDER BY n" -``` - -Let's create index to speed up point queries: - -``` -D CREATE INDEX counter_id_idx ON hits (CounterID); -Run Time: real 18.194 user 16.256000 sys 0.092000 -``` - -Ok, it's quite fast. - -Rerun the last 7 queries. diff --git a/benchmark/elasticsearch/README.md b/benchmark/elasticsearch/README.md new file mode 100644 index 00000000000..82b0b2ff959 --- /dev/null +++ b/benchmark/elasticsearch/README.md @@ -0,0 +1 @@ +Incomplete. diff --git a/benchmark/compatible/elasticsearch/benchmark.sh b/benchmark/elasticsearch/benchmark.sh similarity index 100% rename from benchmark/compatible/elasticsearch/benchmark.sh rename to benchmark/elasticsearch/benchmark.sh diff --git a/benchmark/compatible/exasol/README.md b/benchmark/exasol/README.md similarity index 100% rename from benchmark/compatible/exasol/README.md rename to benchmark/exasol/README.md diff --git a/benchmark/greenplum/README b/benchmark/greenplum/README deleted file mode 100644 index 8339557120e..00000000000 --- a/benchmark/greenplum/README +++ /dev/null @@ -1,43 +0,0 @@ -Folder structure -______________ -dump_dataset_from_ch.sh - bash script that dumps a dataset from Clickhouse -schema.sql - schema for a Greenplum cluster to load dumped dataset in -load_data_set.sql - the script that loads up a dumped dataset -queries.sql - SQL statements used in the benchmark -benchmark.sh - this piece of bash conducts a benchmark -result_parser.py - script to parse benchmark.sh's output and produce python code to build a graph to compare up to 4 benchmark results. -Requirements -____________ - -Greenplum uses a separate server as a point of entry, so you need 2 servers at least to run a cluster: master and segment hosts. 2 segments host and 56 segments(28 per host) had been used while conducting the test. -You has has to put segment hostnames in the benchmark.sh. -Greenplum quick installation instructions -_________________________________________ - -Obtain a stable Greenplum version here(4.3.9.1 was used while conducting the benchmark): -https://network.pivotal.io/products/pivotal-gpdb - -and install it using this detailed guide: -http://gpdb.docs.pivotal.io/4340/install_guide/install_guide.html - -You should change gp_interconnect_type to 'tcp' if cluster members are connected via 1GB link or lower. -There are some variables that has to be changed prior the first benchmark run: gp_vmem_protect_limit and max_statement_mem to allow each segment to use more virtual memory. Here are commands to change this GUCS that has to be executed as gpadmin at the master host: - - gpconfig -c gp_interconnect_type -v tcp - gpconfig -c gp_vmem_protect_limit -v 3000 - gpconfig -c max_statement_mem -v '4000MB' - -How to prepare data -------------------- - -One can prepare datasets to run the benchmark on using dump_dataset_from_ch.sh script from this repo. The script has to be run at at Clickhouse host. It takes a long time to get dumps. - -Upload the datasets into Greenplum master.Then run schema.sql to prepare schema and load_data_set.sql to load data up. This operation also takes a long time. - -How to conduct the benchmark -__________________________ -There is a benchmark.sh that take some arguments. Here is the syntax: - -./benchmark.sh sql_statements_file tablename dbname orca_switch - -If you don't know about the last one then just use a default value. diff --git a/benchmark/greenplum/benchmark.sh b/benchmark/greenplum/benchmark.sh old mode 100755 new mode 100644 index 30b881464a5..a1802ee223b --- a/benchmark/greenplum/benchmark.sh +++ b/benchmark/greenplum/benchmark.sh @@ -1,30 +1,73 @@ -#!/usr/bin/env bash +#!/bin/bash -filename=${1-queries.sql} -table=$2 -dbname=$3 -orca=${4-on} -host1=somehost -host2=somehost -mem='15GB' -cat $filename | sed "s/{table}/$table/g" | while read query ; -do - ssh -n $host1 'echo 3 | tee /proc/sys/vm/drop_caches; sync' > /dev/null - ssh -n $host2 'echo 3 | tee /proc/sys/vm/drop_caches; sync' > /dev/null - sleep 5 - echo $query | egrep "SELECT UserID, date_trunc\('minute', EventTime\) AS m|SELECT Referer AS key, avg\(length\(Referer\)\) AS l|SELECT URL, count(1) AS c FROM.*GROUP BY URL|SELECT 1, URL, count\(1\) AS c FROM.*GROUP BY 1" && mem='10GB' - echo $query | egrep 'SELECT DISTINCT|GROUP BY UserID, SearchPhrase LIMIT 10|count\(DISTINCT UserID\) AS u' && mem='5GB' - echo "####################" - echo "$query" - echo "Timestamp_begin:$(date)" - echo "\\timing off \\\\set optimizer=$orca; set effective_cache_size='256MB'; set statement_mem='$mem';\\timing on \\\\ $query;" | psql -p 5432 -h 'localhost' -o /dev/null -U gpadmin ${dbname} - echo "Timestamp_end:$(date)" - echo "Timestamp_begin:$(date)" - echo "\\timing off \\\\set optimizer=$orca; set effective_cache_size='50GB'; set statement_mem='$mem';\\timing on \\\\ $query;" | psql -p 5432 -h 'localhost' -o /dev/null -U gpadmin ${dbname} - echo "Timestamp_end:$(date)" - echo "Timestamp_begin:$(date)" - echo "\\timing off \\\\set optimizer=$orca; set effective_cache_size='50GB'; set statement_mem='$mem';\\timing on \\\\ $query;" | psql -p 5432 -h 'localhost' -o /dev/null -U gpadmin ${dbname} - echo "Timestamp_end:$(date)" - echo "$query" - echo '####################' -done +# NOTE: it requires Ubuntu 18.04 +# Greenplum does not install on any newer system. + +echo "This script must be run from gpadmin user. Press enter to continue." +read +sudo apt update +sudo apt install -y software-properties-common +sudo add-apt-repository ppa:greenplum/db +sudo apt update +sudo apt install greenplum-db-6 +sudo rm -rf /gpmaster /gpdata* +ssh-keygen -t rsa -b 4096 +cat /home/gpadmin/.ssh/id_rsa.pub >> /home/gpadmin/.ssh/authorized_keys +mod 600 ~/.ssh/authorized_keys +sudo echo "# kernel.shmall = _PHYS_PAGES / 2 # See Shared Memory Pages +kernel.shmall = 197951838 +# kernel.shmmax = kernel.shmall * PAGE_SIZE +kernel.shmmax = 810810728448 +kernel.shmmni = 4096 +vm.overcommit_memory = 2 # See Segment Host Memory +vm.overcommit_ratio = 95 # See Segment Host Memory + +net.ipv4.ip_local_port_range = 10000 65535 # See Port Settings +kernel.sem = 500 2048000 200 4096 +kernel.sysrq = 1 +kernel.core_uses_pid = 1 +kernel.msgmnb = 65536 +kernel.msgmax = 65536 +kernel.msgmni = 2048 +net.ipv4.tcp_syncookies = 1 +net.ipv4.conf.default.accept_source_route = 0 +net.ipv4.tcp_max_syn_backlog = 4096 +net.ipv4.conf.all.arp_filter = 1 +net.core.netdev_max_backlog = 10000 +net.core.rmem_max = 2097152 +net.core.wmem_max = 2097152 +vm.swappiness = 10 +vm.zone_reclaim_mode = 0 +vm.dirty_expire_centisecs = 500 +vm.dirty_writeback_centisecs = 100 +vm.dirty_background_ratio = 0 # See System Memory +vm.dirty_ratio = 0 +vm.dirty_background_bytes = 1610612736 +vm.dirty_bytes = 4294967296" |sudo tee -a /etc/sysctl.conf +sudo sysctl -p + +echo "* soft nofile 524288 +* hard nofile 524288 +* soft nproc 131072 +* hard nproc 131072" |sudo tee -a /etc/security/limits.conf +echo "RemoveIPC=no" |sudo tee -a /etc/systemd/logind.conf +echo "Now you need to reboot the machine. Press Enter if you already rebooted, or reboot now and run the script once again" +read +source /opt/greenplum-db-*.0/greenplum_path.sh +cp $GPHOME/docs/cli_help/gpconfigs/gpinitsystem_singlenode . +echo localhost > ./hostlist_singlenode +sed -i "s/MASTER_HOSTNAME=[a-z_]*/MASTER_HOSTNAME=$(hostname)/" gpinitsystem_singlenode +sed -i "s@declare -a DATA_DIRECTORY=(/gpdata1 /gpdata2)@declare -a DATA_DIRECTORY=(/gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14)@" gpinitsystem_singlenode +sudo mkdir /gpmaster /gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14 +sudo chmod 777 /gpmaster /gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14 +gpinitsystem -ac gpinitsystem_singlenode +export MASTER_DATA_DIRECTORY=/gpmaster/gpsne-1/ +#wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +#gzip -d hits.tsv.gz +chmod 777 ~ hits.tsv +psql -d postgres -f create.sql +nohup gpfdist & +time psql -d postgres -t -c '\timing' -c "insert into hits select * from hits_ext;" +du -sh /gpdata* +./run.sh 2>&1 | tee log.txt +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' |awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/greenplum/create.sql b/benchmark/greenplum/create.sql similarity index 100% rename from benchmark/compatible/greenplum/create.sql rename to benchmark/greenplum/create.sql diff --git a/benchmark/greenplum/dump_dataset_from_ch.sh b/benchmark/greenplum/dump_dataset_from_ch.sh deleted file mode 100644 index 1b40f07c3f8..00000000000 --- a/benchmark/greenplum/dump_dataset_from_ch.sh +++ /dev/null @@ -1,5 +0,0 @@ -#!/usr/bin/env bash - -for table in hits_10m_single hits_100m_single hits_1000m_single; do - clickhouse-client -q "SELECT (round(WatchID/2), JavaEnable, Title, GoodEvent, EventTime, EventDate, CounterID, ClientIP, RegionID,round(UserID/2), CounterClass, OS, UserAgent, URL, Referer, Refresh, RefererCategoryID, RefererRegionID, URLCategoryID, URLRegionID, ResolutionWidth, ResolutionHeight, ResolutionDepth, FlashMajor, FlashMinor, FlashMinor2, NetMajor, NetMinor, UserAgentMajor, CookieEnable, JavascriptEnable, IsMobile, MobilePhone, MobilePhoneModel, Params, IPNetworkID, TraficSourceID, SearchEngineID, SearchPhrase, AdvEngineID, IsArtifical, WindowClientWidth, WindowClientHeight, ClientTimeZone, ClientEventTime, SilverlightVersion1, SilverlightVersion2, SilverlightVersion3, SilverlightVersion4, PageCharset, CodeVersion, IsLink, IsDownload, IsNotBounce,round(FUniqID/2), OriginalURL, HID, IsOldCounter, IsEvent, IsParameter, DontCountHits, WithHash, HitColor, LocalEventTime, Age, Sex, Income, Interests, Robotness, RemoteIP, WindowName, OpenerName, HistoryLength, SocialNetwork, SocialAction, HTTPError, SendTiming, DNSTiming, ConnectTiming, ResponseStartTiming, ResponseEndTiming, FetchTiming, SocialSourceNetworkID, SocialSourcePage, ParamPrice, ParamOrderID, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID,round(RefererHash/2),round(URLHash/2), CLID) FROM $table FORMAT CSV" > $table -done diff --git a/benchmark/greenplum/load_data_set.sql b/benchmark/greenplum/load_data_set.sql deleted file mode 100644 index 60d31452c6d..00000000000 --- a/benchmark/greenplum/load_data_set.sql +++ /dev/null @@ -1,12 +0,0 @@ -COPY hits_all_10m FROM '/data/hits_10m_single.dump' CSV SEGMENT REJECT LIMIT 30 PERCENT; -CREATE INDEX pk_counterid_eventdate_userid_10m ON hits_all_10m USING btree (counterid, eventdate, userid); -CREATE INDEX idx_10m_counterid on hits_all_10m using btree (counterid); CREATE INDEX idx_10m_userid on hits_all_10m using btree (userid); -ANALYZE hits_all_10m; -COPY hits_all_100m from '/data/hits_100m_single.dump' CSV SEGMENT REJECT LIMIT 30 PERCENT; -CREATE INDEX pk_counterid_eventdate_userid_100m ON hits_all_100m USING btree (counterid, eventdate, userid); -CREATE INDEX idx_100m_counterid on hits_all_100m using btree (counterid); CREATE INDEX idx_100m_userid on hits_all_100m using btree (userid); -ANALYZE hits_all_100m; -COPY hits_all_1000m from '/data/hits_1000m_single.dump' CSV SEGMENT REJECT LIMIT 30 PERCENT; -CREATE INDEX pk_counterid_eventdate_userid_1000m ON hits_all_1000m USING btree (counterid, eventdate, userid); -CREATE INDEX idx_1000m_counterid on hits_all_1000m using btree (counterid); CREATE INDEX idx_1000m_userid on hits_all_1000m using btree (userid); -ANALYZE hits_all_1000m; diff --git a/benchmark/compatible/greenplum/log.txt b/benchmark/greenplum/log.txt similarity index 100% rename from benchmark/compatible/greenplum/log.txt rename to benchmark/greenplum/log.txt diff --git a/benchmark/greenplum/queries.sql b/benchmark/greenplum/queries.sql index 8b5ed6ed9c8..31f65fc898d 100644 --- a/benchmark/greenplum/queries.sql +++ b/benchmark/greenplum/queries.sql @@ -1,43 +1,43 @@ -SELECT count(1) FROM {table} -SELECT count(1) FROM {table} WHERE AdvEngineID != 0 -SELECT sum(AdvEngineID), count(1), avg(ResolutionWidth) FROM {table} -SELECT sum(UserID) FROM {table} -SELECT count(UserID) FROM ( SELECT DISTINCT UserID FROM {table} ) AS d -SELECT count(SearchPhrase) FROM ( SELECT DISTINCT SearchPhrase FROM {table} ) AS d -SELECT min(EventDate), max(EventDate) FROM {table} -SELECT AdvEngineID, count(1) FROM {table} WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY 2 DESC -SELECT RegionID, count(DISTINCT UserID) AS u FROM {table} GROUP BY RegionID ORDER BY u DESC LIMIT 10 -SELECT RegionID, sum(AdvEngineID), count(1) 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(1) 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(1) AS c FROM {table} WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10 -SELECT UserID, count(1) FROM {table} GROUP BY UserID ORDER BY 2 DESC LIMIT 10 -SELECT UserID, SearchPhrase, count(1) FROM {table} GROUP BY UserID, SearchPhrase ORDER BY 3 DESC LIMIT 10 -SELECT UserID, SearchPhrase, count(1) FROM {table} GROUP BY UserID, SearchPhrase LIMIT 10 -SELECT UserID, date_trunc('minute', EventTime) AS m, SearchPhrase, count(1) FROM {table} GROUP BY UserID, m, SearchPhrase ORDER BY count(1) DESC LIMIT 10 -SELECT UserID FROM {table} WHERE UserID = 12345678901234567890 -SELECT count(1) FROM {table} WHERE URL LIKE '%metrika%' -SELECT SearchPhrase, max(URL) as URL, count(1) AS c FROM {table} h WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10 -SELECT SearchPhrase, max(URL) as URL, min(Title) as Title, count(1) AS c, count(DISTINCT UserID) FROM {table} WHERE Title LIKE '%\xd0\xaf\xd0\xbd\xd0\xb4\xd0\xb5\xd0\xba\xd1\x81%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT count(1) FROM {table} -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(1) AS c FROM {table} WHERE URL != '' GROUP BY CounterID HAVING count(1) > 100000 ORDER BY l DESC LIMIT 25 -SELECT Referer AS key, avg(length(Referer)) AS l, count(1) AS c, Referer FROM {table} WHERE Referer != '' GROUP BY key HAVING count(1) > 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(1) AS c, sum(Refresh), avg(ResolutionWidth) FROM {table} WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10 -SELECT WatchID, ClientIP, count(1) AS c, sum(Refresh), avg(ResolutionWidth) FROM {table} WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10 -SELECT WatchID, ClientIP, count(1) AS c, sum(Refresh), avg(ResolutionWidth) FROM {table} GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10 -SELECT URL, count(1) AS c FROM {table} GROUP BY URL ORDER BY c DESC LIMIT 10 -SELECT 1, URL, count(1) AS c FROM {table} GROUP BY 1, URL ORDER BY c DESC LIMIT 10 -SELECT ClientIP AS x, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(1) AS c FROM {table} GROUP BY x, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10 -SELECT URL, count(1) AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate between '2013-07-01'::timestamp AND '2013-07-31'::timestamp AND DontCountHits =0 AND Refresh = 0 AND URL <>'' GROUP BY URL ORDER BY PageViews DESC LIMIT 10 -SELECT Title, count(1) AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate BETWEEN '2013-07-01'::timestamp AND '2013-07-31'::timestamp AND DontCountHits=0 AND Refresh=0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10 -SELECT URL, count(1) AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate between '2013-07-01'::timestamp AND '2013-07-31'::timestamp 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 Src, URL AS Dst, count(1) AS PageViews FROM {table} WHERE CounterID = 62 AND eventDate between '2013-07-01'::timestamp AND '2013-07-31'::timestamp AND Refresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -SELECT URLHash, EventDate, count(1) AS PageViews FROM {table} WHERE CounterID = 62 AND eventDate between '2013-07-01'::timestamp AND '2013-07-31'::timestamp AND Refresh =0 AND TraficSourceID IN (-1, 6) AND RefererHash = 7135345792483900000 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100 -SELECT WindowClientWidth, WindowClientHeight, count(1) AS PageViews FROM {table} WHERE CounterID = 62 AND eventDate between '2013-07-01'::timestamp AND '2013-07-31'::timestamp AND Refresh =0 AND DontCountHits =0 AND URLHash = 7135345792483900000 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -SELECT date_trunc('minute', EventTime) AS Minute, count(1) AS PageViews FROM {table} WHERE CounterID = 62 AND eventDate between '2013-07-01'::timestamp AND '2013-07-31'::timestamp AND Refresh =0 AND DontCountHits =0 GROUP BY Minute ORDER BY Minute; +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/greenplum/result_parser.py b/benchmark/greenplum/result_parser.py deleted file mode 100755 index 4ed1aa5c4a5..00000000000 --- a/benchmark/greenplum/result_parser.py +++ /dev/null @@ -1,150 +0,0 @@ -#!/usr/bin/env python3 - - -import sys -import json - - -def parse_block(block=[], options=[]): - - # print('block is here', block) - # show_query = False - # show_query = options.show_query - result = [] - query = block[0].strip() - if len(block) > 4: - timing1 = block[1].strip().split()[1] - timing2 = block[3].strip().split()[1] - timing3 = block[5].strip().split()[1] - else: - timing1 = block[1].strip().split()[1] - timing2 = block[2].strip().split()[1] - timing3 = block[3].strip().split()[1] - if options.show_queries: - result.append(query) - if not options.show_first_timings: - result += [timing1, timing2, timing3] - else: - result.append(timing1) - return result - - -def read_stats_file(options, fname): - result = [] - int_result = [] - block = [] - time_count = 1 - with open(fname) as f: - - for line in f.readlines(): - - if "SELECT" in line: - if len(block) > 1: - result.append(parse_block(block, options)) - block = [line] - elif "Time:" in line: - block.append(line) - - return result - - -def compare_stats_files(options, arguments): - result = [] - file_output = [] - pyplot_colors = ["y", "b", "g", "r"] - for fname in arguments[1:]: - file_output.append((read_stats_file(options, fname))) - if len(file_output[0]) > 0: - timings_count = len(file_output[0]) - for idx, data_set in enumerate(file_output): - int_result = [] - for timing in data_set: - int_result.append(float(timing[0])) # y values - result.append( - [ - [x for x in range(0, len(int_result))], - int_result, - pyplot_colors[idx] + "^", - ] - ) - # result.append([x for x in range(1, len(int_result)) ]) #x values - # result.append( pyplot_colors[idx] + '^' ) - - return result - - -def parse_args(): - from optparse import OptionParser - - parser = OptionParser(usage="usage: %prog [options] [result_file_path]..") - parser.add_option( - "-q", - "--show-queries", - help="Show statements along with timings", - action="store_true", - dest="show_queries", - ) - parser.add_option( - "-f", - "--show-first-timings", - help="Show only first tries timings", - action="store_true", - dest="show_first_timings", - ) - parser.add_option( - "-c", - "--compare-mode", - help="Prepare output for pyplot comparing result files.", - action="store", - dest="compare_mode", - ) - (options, arguments) = parser.parse_args(sys.argv) - if len(arguments) < 2: - parser.print_usage() - sys.exit(1) - return (options, arguments) - - -def gen_pyplot_code(options, arguments): - result = "" - data_sets = compare_stats_files(options, arguments) - for idx, data_set in enumerate(data_sets, start=0): - x_values, y_values, line_style = data_set - result += "\nplt.plot(" - result += "%s, %s, '%s'" % (x_values, y_values, line_style) - result += ", label='%s try')" % idx - print("import matplotlib.pyplot as plt") - print(result) - print("plt.xlabel('Try number')") - print("plt.ylabel('Timing')") - print("plt.title('Benchmark query timings')") - print("plt.legend()") - print("plt.show()") - - -def gen_html_json(options, arguments): - tuples = read_stats_file(options, arguments[1]) - print("{") - print('"system: GreenPlum(x2),') - print(('"version": "%s",' % "4.3.9.1")) - print('"data_size": 10000000,') - print('"time": "",') - print('"comments": "",') - print('"result":') - print("[") - for s in tuples: - print(s) - print("]") - print("}") - - -def main(): - (options, arguments) = parse_args() - if len(arguments) > 2: - gen_pyplot_code(options, arguments) - else: - gen_html_json(options, arguments) - - -if __name__ == "__main__": - main() diff --git a/benchmark/compatible/greenplum/results/c6a.4xlarge.txt b/benchmark/greenplum/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/greenplum/results/c6a.4xlarge.txt rename to benchmark/greenplum/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/greenplum/run.sh b/benchmark/greenplum/run.sh similarity index 100% rename from benchmark/compatible/greenplum/run.sh rename to benchmark/greenplum/run.sh diff --git a/benchmark/greenplum/schema.sql b/benchmark/greenplum/schema.sql deleted file mode 100644 index f2bde529b75..00000000000 --- a/benchmark/greenplum/schema.sql +++ /dev/null @@ -1,3 +0,0 @@ -CREATE TABLE hits_all_10m ( WatchID bigint, JavaEnable int, Title text, GoodEvent int, EventTime timestamp, EventDate timestamp, CounterID bigint, ClientIP bigint, RegionID bigint, UserID bigint, CounterClass int, OS int, UserAgent int, URL text, Referer text, Refresh int, RefererCategoryID int, RefererRegionID bigint, URLCategoryID int, URLRegionID bigint, ResolutionWidth int, ResolutionHeight int, ResolutionDepth int, FlashMajor int, FlashMinor int, FlashMinor2 text, NetMajor int, NetMinor int, UserAgentMajor int, CookieEnable int, JavascriptEnable int, IsMobile int, MobilePhone int, MobilePhoneModel text, Params text, IPNetworkID bigint, TraficSourceID int, SearchEngineID int, SearchPhrase text, AdvEngineID int, IsArtifical int, WindowClientWidth int, WindowClientHeight int, ClientTimeZone int, ClientEventTime timestamp, SilverlightVersion1 int, SilverlightVersion2 int, SilverlightVersion3 bigint, SilverlightVersion4 int, PageCharset text, CodeVersion bigint, IsLink int, IsDownload int, IsNotBounce int, FUniqID bigint, OriginalURL text, HID bigint, IsOldCounter int, IsEvent int, IsParameter int, DontCountHits int, WithHash int, HitColor varchar(3), LocalEventTime timestamp, Age int, Sex int, Income int, Interests int, Robotness int, RemoteIP bigint, WindowName int, OpenerName int, HistoryLength int, SocialNetwork text, SocialAction text, HTTPError int, SendTiming bigint, DNSTiming bigint, ConnectTiming bigint, ResponseStartTiming bigint, ResponseEndTiming bigint, FetchTiming bigint, SocialSourceNetworkID int, SocialSourcePage text, ParamPrice int, ParamOrderID text, OpenstatServiceName text, OpenstatCampaignID text, OpenstatAdID text, OpenstatSourceID text, UTMSource text, UTMMedium text, UTMCampaign text, UTMContent text, UTMTerm text, FromTag text, HasGCLID int, RefererHash bigint, URLHash bigint, CLID bigint) WITH (appendonly=true, orientation=column, compresstype=quicklz) DISTRIBUTED BY (userid) ; -CREATE TABLE hits_all_100m ( WatchID bigint, JavaEnable int, Title text, GoodEvent int, EventTime timestamp, EventDate timestamp, CounterID bigint, ClientIP bigint, RegionID bigint, UserID bigint, CounterClass int, OS int, UserAgent int, URL text, Referer text, Refresh int, RefererCategoryID int, RefererRegionID bigint, URLCategoryID int, URLRegionID bigint, ResolutionWidth int, ResolutionHeight int, ResolutionDepth int, FlashMajor int, FlashMinor int, FlashMinor2 text, NetMajor int, NetMinor int, UserAgentMajor int, CookieEnable int, JavascriptEnable int, IsMobile int, MobilePhone int, MobilePhoneModel text, Params text, IPNetworkID bigint, TraficSourceID int, SearchEngineID int, SearchPhrase text, AdvEngineID int, IsArtifical int, WindowClientWidth int, WindowClientHeight int, ClientTimeZone int, ClientEventTime timestamp, SilverlightVersion1 int, SilverlightVersion2 int, SilverlightVersion3 bigint, SilverlightVersion4 int, PageCharset text, CodeVersion bigint, IsLink int, IsDownload int, IsNotBounce int, FUniqID bigint, OriginalURL text, HID bigint, IsOldCounter int, IsEvent int, IsParameter int, DontCountHits int, WithHash int, HitColor varchar(3), LocalEventTime timestamp, Age int, Sex int, Income int, Interests int, Robotness int, RemoteIP bigint, WindowName int, OpenerName int, HistoryLength int, SocialNetwork text, SocialAction text, HTTPError int, SendTiming bigint, DNSTiming bigint, ConnectTiming bigint, ResponseStartTiming bigint, ResponseEndTiming bigint, FetchTiming bigint, SocialSourceNetworkID int, SocialSourcePage text, ParamPrice int, ParamOrderID text, OpenstatServiceName text, OpenstatCampaignID text, OpenstatAdID text, OpenstatSourceID text, UTMSource text, UTMMedium text, UTMCampaign text, UTMContent text, UTMTerm text, FromTag text, HasGCLID int, RefererHash bigint, URLHash bigint, CLID bigint) WITH (appendonly=true, orientation=column, compresstype=quicklz) DISTRIBUTED BY (userid) ; -CREATE TABLE hits_all_1000m ( WatchID bigint, JavaEnable int, Title text, GoodEvent int, EventTime timestamp, EventDate timestamp, CounterID bigint, ClientIP bigint, RegionID bigint, UserID bigint, CounterClass int, OS int, UserAgent int, URL text, Referer text, Refresh int, RefererCategoryID int, RefererRegionID bigint, URLCategoryID int, URLRegionID bigint, ResolutionWidth int, ResolutionHeight int, ResolutionDepth int, FlashMajor int, FlashMinor int, FlashMinor2 text, NetMajor int, NetMinor int, UserAgentMajor int, CookieEnable int, JavascriptEnable int, IsMobile int, MobilePhone int, MobilePhoneModel text, Params text, IPNetworkID bigint, TraficSourceID int, SearchEngineID int, SearchPhrase text, AdvEngineID int, IsArtifical int, WindowClientWidth int, WindowClientHeight int, ClientTimeZone int, ClientEventTime timestamp, SilverlightVersion1 int, SilverlightVersion2 int, SilverlightVersion3 bigint, SilverlightVersion4 int, PageCharset text, CodeVersion bigint, IsLink int, IsDownload int, IsNotBounce int, FUniqID bigint, OriginalURL text, HID bigint, IsOldCounter int, IsEvent int, IsParameter int, DontCountHits int, WithHash int, HitColor varchar(3), LocalEventTime timestamp, Age int, Sex int, Income int, Interests int, Robotness int, RemoteIP bigint, WindowName int, OpenerName int, HistoryLength int, SocialNetwork text, SocialAction text, HTTPError int, SendTiming bigint, DNSTiming bigint, ConnectTiming bigint, ResponseStartTiming bigint, ResponseEndTiming bigint, FetchTiming bigint, SocialSourceNetworkID int, SocialSourcePage text, ParamPrice int, ParamOrderID text, OpenstatServiceName text, OpenstatCampaignID text, OpenstatAdID text, OpenstatSourceID text, UTMSource text, UTMMedium text, UTMCampaign text, UTMContent text, UTMTerm text, FromTag text, HasGCLID int, RefererHash bigint, URLHash bigint, CLID bigint) WITH (appendonly=true, orientation=column,compresstype=quicklz) DISTRIBUTED BY (userid) ; diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh deleted file mode 100755 index e8c9c58aca3..00000000000 --- a/benchmark/hardware.sh +++ /dev/null @@ -1,214 +0,0 @@ -#!/bin/bash -e - -QUERIES_FILE="queries.sql" -TRIES=3 - -mkdir -p clickhouse-benchmark -pushd clickhouse-benchmark - -# Download the binary -if [[ ! -x clickhouse ]]; then - curl https://clickhouse.com/ | sh -fi - -if [[ ! -f $QUERIES_FILE ]]; then - wget "https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/$QUERIES_FILE" -fi - -uptime - -echo "Starting clickhouse-server" - -./clickhouse server >/dev/null 2>&1 & -PID=$! - -function finish { - kill $PID - wait -} -trap finish EXIT - -echo "Waiting for clickhouse-server to start" - -for i in {1..30}; do - sleep 1 - ./clickhouse client --query "SELECT 'Ok.'" 2>/dev/null && break || echo -n '.' - if [[ $i == 30 ]]; then exit 1; fi -done - -if [[ $(./clickhouse client --query "EXISTS hits") == '1' && $(./clickhouse client --query "SELECT count() FROM hits") == '100000000' ]]; then - echo "Dataset already downloaded" -else - echo "Will download the dataset" - ./clickhouse client --receive_timeout 1000 --max_insert_threads $(nproc || 4) --progress --query " - CREATE OR REPLACE TABLE hits ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) - AS SELECT * FROM url('https://datasets.clickhouse.com/hits/native/hits_100m_obfuscated_{0..255}.native.zst')" - - ./clickhouse client --query "SELECT 'The dataset size is: ', count() FROM hits" -fi - -if [[ $(./clickhouse client --query "SELECT count() FROM system.parts WHERE table = 'hits' AND database = 'default' AND active") == '1' ]]; then - echo "Dataset already prepared" -else - echo "Will prepare the dataset" - ./clickhouse client --receive_timeout 1000 --query "OPTIMIZE TABLE hits FINAL" -fi - -echo -echo "Will perform benchmark. Results:" -echo - ->result.csv -QUERY_NUM=1 - -cat "$QUERIES_FILE" | sed "s/{table}/hits/g" | while read query; do - sync - if [ "${OS}" = "Darwin" ] - then - sudo purge > /dev/null - else - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - fi - - echo -n "[" - for i in $(seq 1 $TRIES); do - RES=$(./clickhouse client --time --format=Null --query="$query" 2>&1 ||:) - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - - echo "${QUERY_NUM},${i},${RES}" >> result.csv - done - echo "]," - - QUERY_NUM=$((QUERY_NUM + 1)) -done - - -echo -echo "Benchmark complete. System info:" -echo - -touch {cpu_model,cpu,df,memory,memory_total,blk,mdstat,instance}.txt - -if [ "${OS}" = "Darwin" ] -then - echo '----Version, build id-----------' - ./clickhouse local --query "SELECT format('Version: {}', version())" - ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw - ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" - echo '----CPU-------------------------' - sysctl hw.model | tee cpu_model.txt - sysctl -a | grep -E 'hw.activecpu|hw.memsize|hw.byteorder|cachesize' | tee cpu.txt - echo '----Disk Free and Total--------' - df -h . | tee df.txt - echo '----Memory Free and Total-------' - vm_stat | tee memory.txt - echo '----Physical Memory Amount------' - ls -l /var/vm | tee memory_total.txt - echo '--------------------------------' -else - echo '----Version, build id-----------' - ./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())" - ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw - ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" - echo '----CPU-------------------------' - cat /proc/cpuinfo | grep -i -F 'model name' | uniq | tee cpu_model.txt - lscpu | tee cpu.txt - echo '----Block Devices---------------' - lsblk | tee blk.txt - echo '----Disk Free and Total--------' - df -h . | tee df.txt - echo '----Memory Free and Total-------' - free -h | tee memory.txt - echo '----Physical Memory Amount------' - cat /proc/meminfo | grep MemTotal | tee memory_total.txt - echo '----RAID Info-------------------' - cat /proc/mdstat| tee mdstat.txt - echo '--------------------------------' -fi -echo - -echo "Instance type from IMDS (if available):" -curl -s --connect-timeout 1 'http://169.254.169.254/latest/meta-data/instance-type' | tee instance.txt -echo - -echo "Uploading the results (if possible)" - -UUID=$(./clickhouse local --query "SELECT generateUUIDv4()") - -./clickhouse local --query " - SELECT - '${UUID}' AS run_id, - version() AS version, - now() AS test_time, - (SELECT value FROM system.settings WHERE name = 'max_threads') AS threads, - filesystemCapacity() AS fs_capacity, - filesystemAvailable() AS fs_available, - file('cpu_model.txt') AS cpu_model, - file('cpu.txt') AS cpu, - file('df.txt') AS df, - file('memory.txt') AS memory, - file('memory_total.txt') AS memory_total, - file('blk.txt') AS blk, - file('mdstat.txt') AS mdstat, - file('instance.txt') AS instance -" | tee meta.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query " - INSERT INTO benchmark_runs - (run_id, version, test_time, threads, fs_capacity, fs_available, cpu_model, cpu, df, memory, memory_total, blk, mdstat, instance) - FORMAT TSV" || echo "Cannot upload results." - -./clickhouse local --query " - SELECT - '${UUID}' AS run_id, - c1 AS query_num, - c2 AS try_num, - c3 AS time - FROM file('result.csv') -" | tee results.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query " - INSERT INTO benchmark_results - (run_id, query_num, try_num, time) - FORMAT TSV" || echo "Cannot upload results. Please send the output to feedback@clickhouse.com" - -</dev/null 2>&1 & +PID=$! + +function finish { + kill $PID + wait +} +trap finish EXIT + +echo "Waiting for clickhouse-server to start" + +for i in {1..30}; do + sleep 1 + ./clickhouse client --query "SELECT 'Ok.'" 2>/dev/null && break || echo -n '.' + if [[ $i == 30 ]]; then exit 1; fi +done + +if [[ $(./clickhouse client --query "EXISTS hits") == '1' && $(./clickhouse client --query "SELECT count() FROM hits") == '100000000' ]]; then + echo "Dataset already downloaded" +else + echo "Will download the dataset" + ./clickhouse client --receive_timeout 1000 --max_insert_threads $(nproc || 4) --progress --query " + CREATE OR REPLACE TABLE hits ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) + AS SELECT * FROM url('https://datasets.clickhouse.com/hits/native/hits_100m_obfuscated_{0..255}.native.zst')" + + ./clickhouse client --query "SELECT 'The dataset size is: ', count() FROM hits" +fi + +if [[ $(./clickhouse client --query "SELECT count() FROM system.parts WHERE table = 'hits' AND database = 'default' AND active") == '1' ]]; then + echo "Dataset already prepared" +else + echo "Will prepare the dataset" + ./clickhouse client --receive_timeout 1000 --query "OPTIMIZE TABLE hits FINAL" +fi + +echo +echo "Will perform benchmark. Results:" +echo + +>result.csv +QUERY_NUM=1 + +cat "$QUERIES_FILE" | sed "s/{table}/hits/g" | while read query; do + sync + if [ "${OS}" = "Darwin" ] + then + sudo purge > /dev/null + else + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + fi + + echo -n "[" + for i in $(seq 1 $TRIES); do + RES=$(./clickhouse client --time --format=Null --query="$query" 2>&1 ||:) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + + echo "${QUERY_NUM},${i},${RES}" >> result.csv + done + echo "]," + + QUERY_NUM=$((QUERY_NUM + 1)) +done + + +echo +echo "Benchmark complete. System info:" +echo + +touch {cpu_model,cpu,df,memory,memory_total,blk,mdstat,instance}.txt + +if [ "${OS}" = "Darwin" ] +then + echo '----Version, build id-----------' + ./clickhouse local --query "SELECT format('Version: {}', version())" + ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw + ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" + echo '----CPU-------------------------' + sysctl hw.model | tee cpu_model.txt + sysctl -a | grep -E 'hw.activecpu|hw.memsize|hw.byteorder|cachesize' | tee cpu.txt + echo '----Disk Free and Total--------' + df -h . | tee df.txt + echo '----Memory Free and Total-------' + vm_stat | tee memory.txt + echo '----Physical Memory Amount------' + ls -l /var/vm | tee memory_total.txt + echo '--------------------------------' +else + echo '----Version, build id-----------' + ./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())" + ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw + ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" + echo '----CPU-------------------------' + cat /proc/cpuinfo | grep -i -F 'model name' | uniq | tee cpu_model.txt + lscpu | tee cpu.txt + echo '----Block Devices---------------' + lsblk | tee blk.txt + echo '----Disk Free and Total--------' + df -h . | tee df.txt + echo '----Memory Free and Total-------' + free -h | tee memory.txt + echo '----Physical Memory Amount------' + cat /proc/meminfo | grep MemTotal | tee memory_total.txt + echo '----RAID Info-------------------' + cat /proc/mdstat| tee mdstat.txt + echo '--------------------------------' +fi +echo + +echo "Instance type from IMDS (if available):" +curl -s --connect-timeout 1 'http://169.254.169.254/latest/meta-data/instance-type' | tee instance.txt +echo + +echo "Uploading the results (if possible)" + +UUID=$(./clickhouse local --query "SELECT generateUUIDv4()") + +./clickhouse local --query " + SELECT + '${UUID}' AS run_id, + version() AS version, + now() AS test_time, + (SELECT value FROM system.settings WHERE name = 'max_threads') AS threads, + filesystemCapacity() AS fs_capacity, + filesystemAvailable() AS fs_available, + file('cpu_model.txt') AS cpu_model, + file('cpu.txt') AS cpu, + file('df.txt') AS df, + file('memory.txt') AS memory, + file('memory_total.txt') AS memory_total, + file('blk.txt') AS blk, + file('mdstat.txt') AS mdstat, + file('instance.txt') AS instance +" | tee meta.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query " + INSERT INTO benchmark_runs + (run_id, version, test_time, threads, fs_capacity, fs_available, cpu_model, cpu, df, memory, memory_total, blk, mdstat, instance) + FORMAT TSV" || echo "Cannot upload results." + +./clickhouse local --query " + SELECT + '${UUID}' AS run_id, + c1 AS query_num, + c2 AS try_num, + c3 AS time + FROM file('result.csv') +" | tee results.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query " + INSERT INTO benchmark_results + (run_id, query_num, try_num, time) + FORMAT TSV" || echo "Cannot upload results. Please send the output to feedback@clickhouse.com" + +< 100000 ORDER BY l DESC LIMIT 25; +SELECT domainWithoutWWW(Referer) AS key, avg(length(Referer)) AS l, count() AS c, any(Referer) FROM {table} WHERE Referer != '' GROUP BY key HAVING c > 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 AS x, x - 1, x - 2, x - 3, count() AS c FROM {table} GROUP BY x, x - 1, x - 2, x - 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 NOT DontCountHits AND NOT Refresh AND notEmpty(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 NOT DontCountHits AND NOT Refresh AND notEmpty(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 NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; +SELECT TraficSourceID, SearchEngineID, AdvEngineID, ((SearchEngineID = 0 AND AdvEngineID = 0) ? Referer : '') AS Src, URL AS Dst, count() AS PageViews FROM {table} 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 {table} 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 WindowClientWidth, WindowClientHeight, count() AS PageViews FROM {table} 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 toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM {table} 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; diff --git a/benchmark/compatible/heavyai/benchmark.sh b/benchmark/heavyai/benchmark.sh similarity index 100% rename from benchmark/compatible/heavyai/benchmark.sh rename to benchmark/heavyai/benchmark.sh diff --git a/benchmark/compatible/heavyai/create.sql b/benchmark/heavyai/create.sql similarity index 100% rename from benchmark/compatible/heavyai/create.sql rename to benchmark/heavyai/create.sql diff --git a/benchmark/compatible/druid/queries.sql b/benchmark/heavyai/queries.sql similarity index 100% rename from benchmark/compatible/druid/queries.sql rename to benchmark/heavyai/queries.sql diff --git a/benchmark/compatible/heavyai/results/c6a.4xlarge.txt b/benchmark/heavyai/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/heavyai/results/c6a.4xlarge.txt rename to benchmark/heavyai/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/heavyai/run.sh b/benchmark/heavyai/run.sh similarity index 100% rename from benchmark/compatible/heavyai/run.sh rename to benchmark/heavyai/run.sh diff --git a/benchmark/hive/conf.sh b/benchmark/hive/conf.sh deleted file mode 100644 index 943f488afac..00000000000 --- a/benchmark/hive/conf.sh +++ /dev/null @@ -1,4 +0,0 @@ -CONF_DIR=/home/kartavyy/benchmark/hive -expect_file=$CONF_DIR/expect.tcl -test_file=$CONF_DIR/queries.sql -etc_init_d_service= diff --git a/benchmark/hive/define_schema.sql b/benchmark/hive/define_schema.sql deleted file mode 100644 index 7701a216a0a..00000000000 --- a/benchmark/hive/define_schema.sql +++ /dev/null @@ -1,9 +0,0 @@ -create table hits_10m_raw ( WatchID BIGINT, JavaEnable SMALLINT, Title STRING, GoodEvent SMALLINT, EventTime TIMESTAMP, EventDate TIMESTAMP, CounterID BIGINT, ClientIP BIGINT, RegionID BIGINT, UserID BIGINT, CounterClass TINYINT, OS SMALLINT, UserAgent SMALLINT, URL STRING, Referer STRING, Refresh TINYINT, RefererCategoryID INT, RefererRegionID BIGINT, URLCategoryID INT, URLRegionID BIGINT, ResolutionWidth INT, ResolutionHeight INT, ResolutionDepth SMALLINT, FlashMajor SMALLINT, FlashMinor SMALLINT, FlashMinor2 STRING, NetMajor SMALLINT, NetMinor SMALLINT, UserAgentMajor INT, UserAgentMinor STRING, CookieEnable SMALLINT, JavascriptEnable SMALLINT, IsMobile SMALLINT, MobilePhone SMALLINT, MobilePhoneModel STRING, Params STRING, IPNetworkID BIGINT, TraficSourceID SMALLINT, SearchEngineID INT, SearchPhrase STRING, AdvEngineID SMALLINT, IsArtifical SMALLINT, WindowClientWidth INT, WindowClientHeight INT, ClientTimeZone INT, ClientEventTime TIMESTAMP, SilverlightVersion1 SMALLINT, SilverlightVersion2 SMALLINT, SilverlightVersion3 BIGINT, SilverlightVersion4 INT, PageCharset STRING, CodeVersion BIGINT, IsLink SMALLINT, IsDownload SMALLINT, IsNotBounce SMALLINT, FUniqID BIGINT, OriginalURL STRING, HID BIGINT, IsOldCounter SMALLINT, IsEvent SMALLINT, IsParameter SMALLINT, DontCountHits SMALLINT, WithHash SMALLINT, HitColor STRING, LocalEventTime TIMESTAMP, Age SMALLINT, Sex SMALLINT, Income SMALLINT, Interests INT, Robotness SMALLINT, RemoteIP BIGINT, WindowName INT, OpenerName INT, HistoryLength SMALLINT, BrowserLanguage STRING, BrowserCountry STRING, SocialNetwork STRING, SocialAction STRING, HTTPError INT, SendTiming BIGINT, DNSTiming BIGINT, ConnectTiming BIGINT, ResponseStartTiming BIGINT, ResponseEndTiming BIGINT, FetchTiming BIGINT, SocialSourceNetworkID SMALLINT, SocialSourcePage STRING, ParamPrice BIGINT, ParamOrderID STRING, ParamCurrency STRING, ParamCurrencyID INT, OpenstatServiceName STRING, OpenstatCampaignID STRING, OpenstatAdID STRING, OpenstatSourceID STRING, UTMSource STRING, UTMMedium STRING, UTMCampaign STRING, UTMContent STRING, UTMTerm STRING, FromTag STRING, HasGCLID SMALLINT, RefererHash BIGINT, URLHash BIGINT, CLID BIGINT, UserIDHash BIGINT ) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; - -load data local inpath '/opt/dump/dump_0.3/dump_hits_10m_meshed_utf8.tsv' overwrite into table hits_10m_raw; - -create table hits_10m ( WatchID BIGINT, JavaEnable SMALLINT, Title STRING, GoodEvent SMALLINT, EventTime TIMESTAMP, EventDate TIMESTAMP, CounterID BIGINT, ClientIP BIGINT, RegionID BIGINT, UserID BIGINT, CounterClass TINYINT, OS SMALLINT, UserAgent SMALLINT, URL STRING, Referer STRING, Refresh TINYINT, RefererCategoryID INT, RefererRegionID BIGINT, URLCategoryID INT, URLRegionID BIGINT, ResolutionWidth INT, ResolutionHeight INT, ResolutionDepth SMALLINT, FlashMajor SMALLINT, FlashMinor SMALLINT, FlashMinor2 STRING, NetMajor SMALLINT, NetMinor SMALLINT, UserAgentMajor INT, UserAgentMinor STRING, CookieEnable SMALLINT, JavascriptEnable SMALLINT, IsMobile SMALLINT, MobilePhone SMALLINT, MobilePhoneModel STRING, Params STRING, IPNetworkID BIGINT, TraficSourceID SMALLINT, SearchEngineID INT, SearchPhrase STRING, AdvEngineID SMALLINT, IsArtifical SMALLINT, WindowClientWidth INT, WindowClientHeight INT, ClientTimeZone INT, ClientEventTime TIMESTAMP, SilverlightVersion1 SMALLINT, SilverlightVersion2 SMALLINT, SilverlightVersion3 BIGINT, SilverlightVersion4 INT, PageCharset STRING, CodeVersion BIGINT, IsLink SMALLINT, IsDownload SMALLINT, IsNotBounce SMALLINT, FUniqID BIGINT, OriginalURL STRING, HID BIGINT, IsOldCounter SMALLINT, IsEvent SMALLINT, IsParameter SMALLINT, DontCountHits SMALLINT, WithHash SMALLINT, HitColor STRING, LocalEventTime TIMESTAMP, Age SMALLINT, Sex SMALLINT, Income SMALLINT, Interests INT, Robotness SMALLINT, RemoteIP BIGINT, WindowName INT, OpenerName INT, HistoryLength SMALLINT, BrowserLanguage STRING, BrowserCountry STRING, SocialNetwork STRING, SocialAction STRING, HTTPError INT, SendTiming BIGINT, DNSTiming BIGINT, ConnectTiming BIGINT, ResponseStartTiming BIGINT, ResponseEndTiming BIGINT, FetchTiming BIGINT, SocialSourceNetworkID SMALLINT, SocialSourcePage STRING, ParamPrice BIGINT, ParamOrderID STRING, ParamCurrency STRING, ParamCurrencyID INT, OpenstatServiceName STRING, OpenstatCampaignID STRING, OpenstatAdID STRING, OpenstatSourceID STRING, UTMSource STRING, UTMMedium STRING, UTMCampaign STRING, UTMContent STRING, UTMTerm STRING, FromTag STRING, HasGCLID SMALLINT, RefererHash BIGINT, URLHash BIGINT, CLID BIGINT, UserIDHash BIGINT ) CLUSTERED BY (EventDate) SORTED BY(CounterID, EventDate, UserIDHash, EventTime) INTO 10 BUCKETS STORED AS ORC tblproperties("orc.compress"="ZLIB"); - -insert overwrite table hits_10m select * from hits_10m_raw; - ---drop table hits_10m_raw; \ No newline at end of file diff --git a/benchmark/hive/expect.tcl b/benchmark/hive/expect.tcl deleted file mode 100644 index 15f43bb7392..00000000000 --- a/benchmark/hive/expect.tcl +++ /dev/null @@ -1,18 +0,0 @@ -#!/usr/bin/env bash -#!/bin/expect - -# Set timeout -set timeout 600 - -# Get arguments -set query [lindex $argv 0] - -spawn hive - -expect "hive>" -send "$query;\r" - -expect "hive>" -send "quit;\r" - -expect eof \ No newline at end of file diff --git a/benchmark/hive/log/log_100m_tuned b/benchmark/hive/log/log_100m_tuned deleted file mode 100644 index 9bea7f50f4a..00000000000 --- a/benchmark/hive/log/log_100m_tuned +++ /dev/null @@ -1,11574 +0,0 @@ -Not found: /etc/init.d/service with path= -start time: Вт. сент. 17 22:55:26 MSK 2013 - -times: 1 -query: SELECT count(*) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18854@mturlrep13_201309172255_1961113475.txt -hive> SELECT count(*) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0004 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 22:56:16,852 Stage-1 map = 0%, reduce = 0% -2013-09-17 22:56:37,989 Stage-1 map = 1%, reduce = 0% -2013-09-17 22:56:41,077 Stage-1 map = 6%, reduce = 0% -2013-09-17 22:56:43,088 Stage-1 map = 10%, reduce = 0% -2013-09-17 22:56:44,095 Stage-1 map = 14%, reduce = 0% -2013-09-17 22:56:47,115 Stage-1 map = 16%, reduce = 0% -2013-09-17 22:56:48,121 Stage-1 map = 17%, reduce = 0% -2013-09-17 22:56:50,131 Stage-1 map = 24%, reduce = 0% -2013-09-17 22:56:51,144 Stage-1 map = 25%, reduce = 0% -2013-09-17 22:56:53,160 Stage-1 map = 30%, reduce = 0% -2013-09-17 22:56:54,167 Stage-1 map = 32%, reduce = 0% -2013-09-17 22:56:55,173 Stage-1 map = 33%, reduce = 0% -2013-09-17 22:56:56,179 Stage-1 map = 36%, reduce = 0% -2013-09-17 22:56:57,185 Stage-1 map = 37%, reduce = 0% -2013-09-17 22:56:58,199 Stage-1 map = 39%, reduce = 0% -2013-09-17 22:57:00,209 Stage-1 map = 45%, reduce = 0% -2013-09-17 22:57:01,214 Stage-1 map = 46%, reduce = 0% -2013-09-17 22:57:03,223 Stage-1 map = 51%, reduce = 0% -2013-09-17 22:57:04,230 Stage-1 map = 52%, reduce = 0% -2013-09-17 22:57:05,693 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 36.54 sec -2013-09-17 22:57:06,700 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 36.54 sec -2013-09-17 22:57:07,707 Stage-1 map = 58%, reduce = 0%, Cumulative CPU 36.54 sec -2013-09-17 22:57:08,725 Stage-1 map = 58%, reduce = 0%, Cumulative CPU 36.54 sec -2013-09-17 22:57:09,731 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 36.54 sec -2013-09-17 22:57:10,948 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 36.54 sec -2013-09-17 22:57:11,958 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 36.54 sec -2013-09-17 22:57:12,970 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 36.54 sec -2013-09-17 22:57:13,990 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 1483.14 sec -2013-09-17 22:57:15,025 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 1484.59 sec -2013-09-17 22:57:16,048 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 1484.59 sec -2013-09-17 22:57:17,060 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 1484.59 sec -2013-09-17 22:57:18,095 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 1502.09 sec -2013-09-17 22:57:19,123 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 1512.83 sec -2013-09-17 22:57:20,133 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 1530.9 sec -2013-09-17 22:57:21,147 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 1580.07 sec -2013-09-17 22:57:22,153 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 1640.62 sec -2013-09-17 22:57:23,159 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 1677.56 sec -2013-09-17 22:57:24,166 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 1722.04 sec -2013-09-17 22:57:25,267 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 1733.88 sec -2013-09-17 22:57:26,274 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1760.11 sec -2013-09-17 22:57:27,280 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1772.82 sec -2013-09-17 22:57:28,286 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1786.49 sec -2013-09-17 22:57:29,291 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1786.49 sec -2013-09-17 22:57:30,297 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1786.49 sec -2013-09-17 22:57:31,302 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1786.49 sec -2013-09-17 22:57:32,307 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1786.49 sec -2013-09-17 22:57:33,313 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1786.49 sec -2013-09-17 22:57:34,319 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1786.49 sec -2013-09-17 22:57:35,337 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1789.45 sec -2013-09-17 22:57:36,344 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1789.45 sec -2013-09-17 22:57:37,350 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1789.45 sec -MapReduce Total cumulative CPU time: 29 minutes 49 seconds 450 msec -Ended Job = job_201309172235_0004 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 1789.45 sec HDFS Read: 10081441390 HDFS Write: 10 SUCCESS -Total MapReduce CPU Time Spent: 29 minutes 49 seconds 450 msec -OK -100000000 -Time taken: 110.676 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT count(*) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23942@mturlrep13_201309172257_1656900452.txt -hive> SELECT count(*) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0005 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 22:58:08,971 Stage-1 map = 0%, reduce = 0% -2013-09-17 22:58:29,310 Stage-1 map = 1%, reduce = 0% -2013-09-17 22:58:32,327 Stage-1 map = 4%, reduce = 0% -2013-09-17 22:58:33,338 Stage-1 map = 5%, reduce = 0% -2013-09-17 22:58:34,345 Stage-1 map = 6%, reduce = 0% -2013-09-17 22:58:35,354 Stage-1 map = 8%, reduce = 0% -2013-09-17 22:58:36,360 Stage-1 map = 13%, reduce = 0% -2013-09-17 22:58:37,376 Stage-1 map = 15%, reduce = 0% -2013-09-17 22:58:39,389 Stage-1 map = 17%, reduce = 0% -2013-09-17 22:58:40,408 Stage-1 map = 18%, reduce = 0% -2013-09-17 22:58:41,420 Stage-1 map = 20%, reduce = 0% -2013-09-17 22:58:42,432 Stage-1 map = 25%, reduce = 0% -2013-09-17 22:58:43,444 Stage-1 map = 27%, reduce = 0% -2013-09-17 22:58:44,461 Stage-1 map = 29%, reduce = 0% -2013-09-17 22:58:45,468 Stage-1 map = 31%, reduce = 0% -2013-09-17 22:58:46,488 Stage-1 map = 33%, reduce = 0% -2013-09-17 22:58:48,499 Stage-1 map = 36%, reduce = 0% -2013-09-17 22:58:49,507 Stage-1 map = 38%, reduce = 0% -2013-09-17 22:58:50,513 Stage-1 map = 41%, reduce = 0% -2013-09-17 22:58:51,519 Stage-1 map = 42%, reduce = 0% -2013-09-17 22:58:52,524 Stage-1 map = 45%, reduce = 0% -2013-09-17 22:58:53,529 Stage-1 map = 46%, reduce = 0% -2013-09-17 22:58:54,535 Stage-1 map = 48%, reduce = 0% -2013-09-17 22:58:55,575 Stage-1 map = 51%, reduce = 0% -2013-09-17 22:58:56,602 Stage-1 map = 53%, reduce = 0% -2013-09-17 22:58:57,639 Stage-1 map = 54%, reduce = 0% -2013-09-17 22:58:58,645 Stage-1 map = 57%, reduce = 0% -2013-09-17 22:58:59,664 Stage-1 map = 59%, reduce = 0% -2013-09-17 22:59:01,690 Stage-1 map = 65%, reduce = 0% -2013-09-17 22:59:02,722 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 36.1 sec -2013-09-17 22:59:03,750 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 36.1 sec -2013-09-17 22:59:04,761 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 36.1 sec -2013-09-17 22:59:05,770 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 36.1 sec -2013-09-17 22:59:06,784 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 36.1 sec -2013-09-17 22:59:08,117 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 175.85 sec -2013-09-17 22:59:09,124 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 221.51 sec -2013-09-17 22:59:10,130 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 314.14 sec -2013-09-17 22:59:11,138 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 650.01 sec -2013-09-17 22:59:12,145 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 788.39 sec -2013-09-17 22:59:13,149 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 1123.31 sec -2013-09-17 22:59:14,155 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 1744.89 sec -2013-09-17 22:59:15,161 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 1753.27 sec -2013-09-17 22:59:16,167 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1760.31 sec -2013-09-17 22:59:17,174 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1765.77 sec -2013-09-17 22:59:18,179 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1773.76 sec -2013-09-17 22:59:19,185 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1773.76 sec -2013-09-17 22:59:20,191 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1773.76 sec -2013-09-17 22:59:21,196 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1773.76 sec -2013-09-17 22:59:22,202 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1773.76 sec -2013-09-17 22:59:23,215 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1773.76 sec -2013-09-17 22:59:24,221 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1773.76 sec -2013-09-17 22:59:25,226 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1773.76 sec -2013-09-17 22:59:26,234 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1776.78 sec -2013-09-17 22:59:27,552 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1776.78 sec -MapReduce Total cumulative CPU time: 29 minutes 36 seconds 780 msec -Ended Job = job_201309172235_0005 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 1776.78 sec HDFS Read: 10081441390 HDFS Write: 10 SUCCESS -Total MapReduce CPU Time Spent: 29 minutes 36 seconds 780 msec -OK -100000000 -Time taken: 105.13 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT count(*) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28935@mturlrep13_201309172259_1582946547.txt -hive> SELECT count(*) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0006 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 22:59:59,626 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:00:21,889 Stage-1 map = 1%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:22,898 Stage-1 map = 1%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:24,158 Stage-1 map = 5%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:25,172 Stage-1 map = 5%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:26,201 Stage-1 map = 6%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:27,208 Stage-1 map = 12%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:28,215 Stage-1 map = 13%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:29,226 Stage-1 map = 16%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:30,236 Stage-1 map = 18%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:31,983 Stage-1 map = 18%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:32,998 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:34,970 Stage-1 map = 24%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:35,996 Stage-1 map = 28%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:37,017 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:38,034 Stage-1 map = 30%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:39,050 Stage-1 map = 31%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:40,078 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:41,092 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:42,097 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:43,104 Stage-1 map = 41%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:44,109 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:45,115 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:46,124 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:47,148 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:48,154 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:49,162 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:50,172 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 106.84 sec -2013-09-17 23:00:51,207 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 140.29 sec -2013-09-17 23:00:52,224 Stage-1 map = 62%, reduce = 0%, Cumulative CPU 140.29 sec -2013-09-17 23:00:53,230 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 140.29 sec -2013-09-17 23:00:54,237 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 140.29 sec -2013-09-17 23:00:55,247 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 140.29 sec -2013-09-17 23:00:56,254 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 140.29 sec -2013-09-17 23:00:57,261 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 140.29 sec -2013-09-17 23:00:58,268 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 178.19 sec -2013-09-17 23:00:59,275 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 178.19 sec -2013-09-17 23:01:00,282 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 178.19 sec -2013-09-17 23:01:01,288 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 309.14 sec -2013-09-17 23:01:02,448 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 570.2 sec -2013-09-17 23:01:03,547 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 708.2 sec -2013-09-17 23:01:04,555 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 845.2 sec -2013-09-17 23:01:05,562 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 982.42 sec -2013-09-17 23:01:06,568 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 1312.62 sec -2013-09-17 23:01:07,574 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 1599.05 sec -2013-09-17 23:01:08,580 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 1599.05 sec -2013-09-17 23:01:09,585 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1697.63 sec -2013-09-17 23:01:10,590 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1749.84 sec -2013-09-17 23:01:11,595 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1749.84 sec -2013-09-17 23:01:12,601 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1802.81 sec -2013-09-17 23:01:13,606 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1802.81 sec -2013-09-17 23:01:14,611 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1803.33 sec -2013-09-17 23:01:15,617 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1803.33 sec -2013-09-17 23:01:16,624 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1803.33 sec -2013-09-17 23:01:17,632 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1805.73 sec -2013-09-17 23:01:18,638 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1805.73 sec -2013-09-17 23:01:19,644 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1805.73 sec -MapReduce Total cumulative CPU time: 30 minutes 5 seconds 730 msec -Ended Job = job_201309172235_0006 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 1805.73 sec HDFS Read: 10081441390 HDFS Write: 10 SUCCESS -Total MapReduce CPU Time Spent: 30 minutes 5 seconds 730 msec -OK -100000000 -Time taken: 107.358 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_100m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2345@mturlrep13_201309172301_1950916943.txt -hive> SELECT count(*) FROM hits_100m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0007 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:02:11,138 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:02:22,387 Stage-1 map = 4%, reduce = 0% -2013-09-17 23:02:23,396 Stage-1 map = 11%, reduce = 0% -2013-09-17 23:02:24,402 Stage-1 map = 30%, reduce = 0% -2013-09-17 23:02:25,700 Stage-1 map = 35%, reduce = 0% -2013-09-17 23:02:26,723 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 46.24 sec -2013-09-17 23:02:27,736 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 173.19 sec -2013-09-17 23:02:28,746 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.36 sec -2013-09-17 23:02:29,751 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.36 sec -2013-09-17 23:02:30,756 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.36 sec -2013-09-17 23:02:31,761 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.36 sec -2013-09-17 23:02:32,768 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.36 sec -2013-09-17 23:02:33,774 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.36 sec -2013-09-17 23:02:34,943 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.36 sec -2013-09-17 23:02:35,950 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.36 sec -2013-09-17 23:02:36,959 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 297.32 sec -2013-09-17 23:02:37,967 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 297.32 sec -MapReduce Total cumulative CPU time: 4 minutes 57 seconds 320 msec -Ended Job = job_201309172235_0007 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 297.32 sec HDFS Read: 6282387 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 4 minutes 57 seconds 320 msec -OK -630535 -Time taken: 55.195 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT count(*) FROM hits_100m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7709@mturlrep13_201309172302_964147901.txt -hive> SELECT count(*) FROM hits_100m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0008 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:02:52,920 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:03:03,995 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:03:05,016 Stage-1 map = 3%, reduce = 0% -2013-09-17 23:03:06,076 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 6.7 sec -2013-09-17 23:03:07,088 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 210.4 sec -2013-09-17 23:03:08,096 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.76 sec -2013-09-17 23:03:09,102 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.76 sec -2013-09-17 23:03:10,109 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.76 sec -2013-09-17 23:03:11,114 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.76 sec -2013-09-17 23:03:12,127 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.76 sec -2013-09-17 23:03:13,133 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 294.76 sec -2013-09-17 23:03:14,139 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 294.76 sec -2013-09-17 23:03:15,146 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 295.45 sec -2013-09-17 23:03:16,154 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 297.51 sec -2013-09-17 23:03:17,160 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 297.51 sec -MapReduce Total cumulative CPU time: 4 minutes 57 seconds 510 msec -Ended Job = job_201309172235_0008 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 297.51 sec HDFS Read: 6282387 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 4 minutes 57 seconds 510 msec -OK -630535 -Time taken: 36.435 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT count(*) FROM hits_100m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12665@mturlrep13_201309172303_2013045725.txt -hive> SELECT count(*) FROM hits_100m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0009 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:03:27,027 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:03:37,093 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:03:40,143 Stage-1 map = 41%, reduce = 0% -2013-09-17 23:03:41,161 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 171.9 sec -2013-09-17 23:03:42,169 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 298.73 sec -2013-09-17 23:03:43,177 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 298.73 sec -2013-09-17 23:03:44,184 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 298.73 sec -2013-09-17 23:03:45,190 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 298.73 sec -2013-09-17 23:03:46,196 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 298.73 sec -2013-09-17 23:03:47,202 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 298.73 sec -2013-09-17 23:03:48,208 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 298.73 sec -2013-09-17 23:03:49,214 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 298.73 sec -2013-09-17 23:03:50,223 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 301.9 sec -2013-09-17 23:03:51,229 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 301.9 sec -MapReduce Total cumulative CPU time: 5 minutes 1 seconds 900 msec -Ended Job = job_201309172235_0009 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 301.9 sec HDFS Read: 6282387 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 5 minutes 1 seconds 900 msec -OK -630535 -Time taken: 32.201 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17646@mturlrep13_201309172303_809195796.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0010 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:04:11,044 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:04:30,217 Stage-1 map = 21%, reduce = 0%, Cumulative CPU 29.16 sec -2013-09-17 23:04:31,225 Stage-1 map = 35%, reduce = 0%, Cumulative CPU 90.03 sec -2013-09-17 23:04:32,231 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 230.36 sec -2013-09-17 23:04:33,237 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 349.12 sec -2013-09-17 23:04:34,243 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 391.27 sec -2013-09-17 23:04:35,249 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 391.27 sec -2013-09-17 23:04:36,255 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 391.27 sec -2013-09-17 23:04:37,261 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 391.27 sec -2013-09-17 23:04:38,267 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 391.27 sec -2013-09-17 23:04:39,276 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 393.94 sec -2013-09-17 23:04:40,283 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 393.94 sec -MapReduce Total cumulative CPU time: 6 minutes 33 seconds 940 msec -Ended Job = job_201309172235_0010 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 393.94 sec HDFS Read: 78754487 HDFS Write: 32 SUCCESS -Total MapReduce CPU Time Spent: 6 minutes 33 seconds 940 msec -OK -Time taken: 39.991 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22907@mturlrep13_201309172304_403247887.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0011 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:04:50,128 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:05:02,260 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:05:03,278 Stage-1 map = 13%, reduce = 0% -2013-09-17 23:05:04,299 Stage-1 map = 25%, reduce = 0% -2013-09-17 23:05:05,308 Stage-1 map = 30%, reduce = 0% -2013-09-17 23:05:06,333 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 9.21 sec -2013-09-17 23:05:07,352 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 39.3 sec -2013-09-17 23:05:08,369 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 377.82 sec -2013-09-17 23:05:09,375 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 401.7 sec -2013-09-17 23:05:10,381 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 401.7 sec -2013-09-17 23:05:11,387 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 401.7 sec -2013-09-17 23:05:12,393 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 401.7 sec -2013-09-17 23:05:13,399 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 401.7 sec -2013-09-17 23:05:14,405 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 401.7 sec -2013-09-17 23:05:15,411 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 401.7 sec -2013-09-17 23:05:16,419 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 404.35 sec -2013-09-17 23:05:17,426 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 404.35 sec -MapReduce Total cumulative CPU time: 6 minutes 44 seconds 350 msec -Ended Job = job_201309172235_0011 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 404.35 sec HDFS Read: 78754487 HDFS Write: 32 SUCCESS -Total MapReduce CPU Time Spent: 6 minutes 44 seconds 350 msec -OK -Time taken: 35.143 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28807@mturlrep13_201309172305_1471136727.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0012 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:05:28,152 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:05:40,255 Stage-1 map = 7%, reduce = 0% -2013-09-17 23:05:41,260 Stage-1 map = 22%, reduce = 0% -2013-09-17 23:05:42,266 Stage-1 map = 28%, reduce = 0% -2013-09-17 23:05:43,279 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 9.21 sec -2013-09-17 23:05:44,286 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 90.45 sec -2013-09-17 23:05:45,294 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 363.6 sec -2013-09-17 23:05:46,301 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 399.15 sec -2013-09-17 23:05:47,307 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 399.15 sec -2013-09-17 23:05:48,312 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 399.15 sec -2013-09-17 23:05:49,317 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 399.15 sec -2013-09-17 23:05:50,323 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 399.15 sec -2013-09-17 23:05:51,329 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 399.15 sec -2013-09-17 23:05:52,334 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 399.15 sec -2013-09-17 23:05:53,343 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 402.09 sec -2013-09-17 23:05:54,348 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 402.09 sec -MapReduce Total cumulative CPU time: 6 minutes 42 seconds 90 msec -Ended Job = job_201309172235_0012 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 402.09 sec HDFS Read: 78754487 HDFS Write: 32 SUCCESS -Total MapReduce CPU Time Spent: 6 minutes 42 seconds 90 msec -OK -Time taken: 35.085 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT sum(UserID) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1699@mturlrep13_201309172306_1224643663.txt -hive> SELECT sum(UserID) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0013 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:06:12,215 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:06:22,307 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:06:24,332 Stage-1 map = 5%, reduce = 0% -2013-09-17 23:06:26,810 Stage-1 map = 15%, reduce = 0% -2013-09-17 23:06:27,822 Stage-1 map = 17%, reduce = 0%, Cumulative CPU 7.28 sec -2013-09-17 23:06:28,829 Stage-1 map = 17%, reduce = 0%, Cumulative CPU 7.28 sec -2013-09-17 23:06:29,837 Stage-1 map = 17%, reduce = 0%, Cumulative CPU 7.28 sec -2013-09-17 23:06:32,109 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 25.3 sec -2013-09-17 23:06:33,115 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 25.3 sec -2013-09-17 23:06:35,953 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 34.74 sec -2013-09-17 23:06:36,962 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 352.67 sec -2013-09-17 23:06:37,968 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 352.67 sec -2013-09-17 23:06:38,974 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 352.67 sec -2013-09-17 23:06:39,979 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 352.67 sec -2013-09-17 23:06:40,985 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 352.67 sec -2013-09-17 23:06:41,991 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 352.67 sec -2013-09-17 23:06:42,997 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 352.67 sec -2013-09-17 23:06:44,006 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 355.35 sec -2013-09-17 23:06:45,013 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 355.35 sec -2013-09-17 23:06:46,025 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 355.35 sec -MapReduce Total cumulative CPU time: 5 minutes 55 seconds 350 msec -Ended Job = job_201309172235_0013 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 355.35 sec HDFS Read: 663537300 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 5 minutes 55 seconds 350 msec -OK --2020069000640784227 -Time taken: 44.465 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT sum(UserID) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7091@mturlrep13_201309172306_2064459587.txt -hive> SELECT sum(UserID) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0014 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:06:55,801 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:07:07,899 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:07:08,910 Stage-1 map = 20%, reduce = 0% -2013-09-17 23:07:09,919 Stage-1 map = 38%, reduce = 0% -2013-09-17 23:07:10,933 Stage-1 map = 45%, reduce = 0%, Cumulative CPU 26.96 sec -2013-09-17 23:07:11,941 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 277.91 sec -2013-09-17 23:07:12,948 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.42 sec -2013-09-17 23:07:13,954 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.42 sec -2013-09-17 23:07:14,960 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.42 sec -2013-09-17 23:07:15,966 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.42 sec -2013-09-17 23:07:16,972 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.42 sec -2013-09-17 23:07:17,978 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.42 sec -2013-09-17 23:07:18,984 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 349.42 sec -2013-09-17 23:07:19,992 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 352.35 sec -2013-09-17 23:07:20,998 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 352.35 sec -2013-09-17 23:07:22,005 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 352.35 sec -MapReduce Total cumulative CPU time: 5 minutes 52 seconds 350 msec -Ended Job = job_201309172235_0014 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 352.35 sec HDFS Read: 663537300 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 5 minutes 52 seconds 350 msec -OK --2020069000640784227 -Time taken: 34.131 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT sum(UserID) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12067@mturlrep13_201309172307_240829598.txt -hive> SELECT sum(UserID) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0015 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:07:31,722 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:07:43,798 Stage-1 map = 3%, reduce = 0% -2013-09-17 23:07:44,804 Stage-1 map = 24%, reduce = 0% -2013-09-17 23:07:45,810 Stage-1 map = 38%, reduce = 0% -2013-09-17 23:07:46,822 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 33.64 sec -2013-09-17 23:07:47,830 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 257.18 sec -2013-09-17 23:07:48,837 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.23 sec -2013-09-17 23:07:49,842 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.23 sec -2013-09-17 23:07:50,847 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.23 sec -2013-09-17 23:07:51,852 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.23 sec -2013-09-17 23:07:52,858 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.23 sec -2013-09-17 23:07:53,863 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 349.23 sec -2013-09-17 23:07:54,869 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 349.23 sec -2013-09-17 23:07:55,877 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 352.02 sec -2013-09-17 23:07:56,884 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 352.02 sec -2013-09-17 23:07:57,889 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 352.02 sec -MapReduce Total cumulative CPU time: 5 minutes 52 seconds 20 msec -Ended Job = job_201309172235_0015 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 352.02 sec HDFS Read: 663537300 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 5 minutes 52 seconds 20 msec -OK --2020069000640784227 -Time taken: 34.032 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17039@mturlrep13_201309172308_800014414.txt -hive> SELECT count(DISTINCT UserID) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0016 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:08:15,876 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:08:25,047 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:08:27,066 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:08:28,077 Stage-1 map = 6%, reduce = 0% -2013-09-17 23:08:29,083 Stage-1 map = 11%, reduce = 0% -2013-09-17 23:08:30,088 Stage-1 map = 18%, reduce = 0% -2013-09-17 23:08:31,110 Stage-1 map = 19%, reduce = 0% -2013-09-17 23:08:32,125 Stage-1 map = 29%, reduce = 0% -2013-09-17 23:08:33,130 Stage-1 map = 34%, reduce = 0% -2013-09-17 23:08:35,158 Stage-1 map = 47%, reduce = 0% -2013-09-17 23:08:36,166 Stage-1 map = 51%, reduce = 0% -2013-09-17 23:08:38,177 Stage-1 map = 65%, reduce = 0% -2013-09-17 23:08:39,189 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 29.98 sec -2013-09-17 23:08:40,216 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 101.62 sec -2013-09-17 23:08:41,230 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 101.62 sec -2013-09-17 23:08:42,264 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 101.62 sec -2013-09-17 23:08:43,274 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 179.43 sec -2013-09-17 23:08:44,289 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 250.32 sec -2013-09-17 23:08:45,300 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 338.89 sec -2013-09-17 23:08:46,306 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 456.84 sec -2013-09-17 23:08:47,312 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 496.19 sec -2013-09-17 23:08:48,318 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 522.86 sec -2013-09-17 23:08:49,323 Stage-1 map = 98%, reduce = 1%, Cumulative CPU 600.26 sec -2013-09-17 23:08:50,329 Stage-1 map = 99%, reduce = 1%, Cumulative CPU 653.17 sec -2013-09-17 23:08:51,335 Stage-1 map = 99%, reduce = 1%, Cumulative CPU 653.17 sec -2013-09-17 23:08:52,341 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 708.39 sec -2013-09-17 23:08:53,347 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 708.39 sec -2013-09-17 23:08:54,352 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 708.39 sec -2013-09-17 23:08:55,359 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 708.39 sec -2013-09-17 23:08:56,365 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 708.39 sec -2013-09-17 23:08:57,371 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 708.39 sec -2013-09-17 23:08:58,377 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 708.39 sec -2013-09-17 23:08:59,383 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 708.39 sec -2013-09-17 23:09:00,389 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 708.39 sec -2013-09-17 23:09:01,395 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:02,401 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:03,406 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:04,412 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:05,418 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:06,424 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:07,430 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:08,436 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:09,442 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:10,447 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:11,453 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:12,459 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:13,464 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:14,470 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 708.39 sec -2013-09-17 23:09:15,476 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 733.92 sec -2013-09-17 23:09:16,482 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 733.92 sec -2013-09-17 23:09:17,488 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 733.92 sec -2013-09-17 23:09:18,494 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 733.92 sec -2013-09-17 23:09:19,500 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 733.92 sec -2013-09-17 23:09:20,506 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 733.92 sec -2013-09-17 23:09:21,511 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 733.92 sec -2013-09-17 23:09:22,517 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 733.92 sec -2013-09-17 23:09:23,523 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 733.92 sec -2013-09-17 23:09:24,595 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 733.92 sec -2013-09-17 23:09:25,601 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 733.92 sec -2013-09-17 23:09:26,607 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 733.92 sec -2013-09-17 23:09:27,613 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 733.92 sec -2013-09-17 23:09:28,619 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 733.92 sec -2013-09-17 23:09:29,625 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 733.92 sec -2013-09-17 23:09:30,631 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 733.92 sec -2013-09-17 23:09:31,637 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 733.92 sec -2013-09-17 23:09:32,643 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 733.92 sec -2013-09-17 23:09:33,650 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 733.92 sec -2013-09-17 23:09:36,622 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 733.92 sec -2013-09-17 23:09:37,628 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 733.92 sec -2013-09-17 23:09:38,634 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 733.92 sec -2013-09-17 23:09:39,648 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 733.92 sec -2013-09-17 23:09:40,654 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 733.92 sec -2013-09-17 23:09:41,660 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 733.92 sec -2013-09-17 23:09:42,666 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 733.92 sec -2013-09-17 23:09:43,672 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 733.92 sec -2013-09-17 23:09:44,677 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 733.92 sec -2013-09-17 23:09:45,683 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 733.92 sec -2013-09-17 23:09:46,689 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 733.92 sec -2013-09-17 23:09:47,695 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 733.92 sec -2013-09-17 23:09:48,702 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 733.92 sec -2013-09-17 23:09:49,708 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 733.92 sec -2013-09-17 23:09:50,714 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 733.92 sec -2013-09-17 23:09:51,720 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 733.92 sec -2013-09-17 23:09:52,726 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 733.92 sec -2013-09-17 23:09:53,732 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 733.92 sec -2013-09-17 23:09:54,740 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 775.12 sec -2013-09-17 23:09:55,746 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 775.12 sec -MapReduce Total cumulative CPU time: 12 minutes 55 seconds 120 msec -Ended Job = job_201309172235_0016 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 775.12 sec HDFS Read: 663537300 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 12 minutes 55 seconds 120 msec -OK -17630976 -Time taken: 110.69 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT count(DISTINCT UserID) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22191@mturlrep13_201309172309_1602577058.txt -hive> SELECT count(DISTINCT UserID) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0017 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:10:05,908 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:10:18,029 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:10:19,113 Stage-1 map = 16%, reduce = 0% -2013-09-17 23:10:20,139 Stage-1 map = 18%, reduce = 0% -2013-09-17 23:10:21,150 Stage-1 map = 19%, reduce = 0% -2013-09-17 23:10:22,158 Stage-1 map = 46%, reduce = 0% -2013-09-17 23:10:23,166 Stage-1 map = 56%, reduce = 0% -2013-09-17 23:10:24,186 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 10.62 sec -2013-09-17 23:10:25,199 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 10.62 sec -2013-09-17 23:10:26,206 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 10.62 sec -2013-09-17 23:10:27,222 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 37.91 sec -2013-09-17 23:10:28,230 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 79.92 sec -2013-09-17 23:10:29,287 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 172.39 sec -2013-09-17 23:10:30,300 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 220.22 sec -2013-09-17 23:10:31,311 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 326.41 sec -2013-09-17 23:10:32,317 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 441.77 sec -2013-09-17 23:10:33,324 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 481.76 sec -2013-09-17 23:10:34,330 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 481.76 sec -2013-09-17 23:10:35,337 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 481.76 sec -2013-09-17 23:10:36,344 Stage-1 map = 97%, reduce = 11%, Cumulative CPU 507.3 sec -2013-09-17 23:10:37,351 Stage-1 map = 98%, reduce = 11%, Cumulative CPU 610.94 sec -2013-09-17 23:10:38,357 Stage-1 map = 99%, reduce = 11%, Cumulative CPU 610.94 sec -2013-09-17 23:10:39,362 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 693.79 sec -2013-09-17 23:10:40,368 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 693.79 sec -2013-09-17 23:10:41,375 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 693.79 sec -2013-09-17 23:10:42,381 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 693.79 sec -2013-09-17 23:10:43,388 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 693.79 sec -2013-09-17 23:10:44,393 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 693.79 sec -2013-09-17 23:10:45,399 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:46,404 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:47,409 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:48,414 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:49,419 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:50,424 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:51,429 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:52,435 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:53,440 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:54,446 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:55,451 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:56,456 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:57,461 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:58,836 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:10:59,948 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:11:00,953 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:11:01,958 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:11:02,963 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:11:05,322 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 693.79 sec -2013-09-17 23:11:06,328 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 693.79 sec -2013-09-17 23:11:07,333 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 693.79 sec -2013-09-17 23:11:08,339 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 693.79 sec -2013-09-17 23:11:09,344 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 693.79 sec -2013-09-17 23:11:10,349 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 693.79 sec -2013-09-17 23:11:12,853 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 693.79 sec -2013-09-17 23:11:13,858 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 693.79 sec -2013-09-17 23:11:14,864 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 693.79 sec -2013-09-17 23:11:21,480 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 734.95 sec -2013-09-17 23:11:22,485 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 734.95 sec -2013-09-17 23:11:23,490 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 734.95 sec -2013-09-17 23:11:24,496 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 734.95 sec -2013-09-17 23:11:25,501 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 734.95 sec -2013-09-17 23:11:26,506 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 734.95 sec -2013-09-17 23:11:27,512 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 734.95 sec -2013-09-17 23:11:28,517 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 734.95 sec -2013-09-17 23:11:29,522 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 734.95 sec -2013-09-17 23:11:30,527 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 734.95 sec -2013-09-17 23:11:31,543 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 734.95 sec -2013-09-17 23:11:32,548 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 734.95 sec -2013-09-17 23:11:33,553 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 734.95 sec -2013-09-17 23:11:34,558 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 734.95 sec -2013-09-17 23:11:35,562 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 734.95 sec -2013-09-17 23:11:36,568 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 734.95 sec -2013-09-17 23:11:37,573 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 734.95 sec -2013-09-17 23:11:38,578 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 734.95 sec -2013-09-17 23:11:39,583 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 734.95 sec -2013-09-17 23:11:40,588 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 734.95 sec -2013-09-17 23:11:41,593 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 761.58 sec -2013-09-17 23:11:42,599 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 761.58 sec -2013-09-17 23:11:43,605 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 761.58 sec -MapReduce Total cumulative CPU time: 12 minutes 41 seconds 580 msec -Ended Job = job_201309172235_0017 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 761.58 sec HDFS Read: 663537300 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 12 minutes 41 seconds 580 msec -OK -17630976 -Time taken: 105.953 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT count(DISTINCT UserID) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28018@mturlrep13_201309172311_1489682234.txt -hive> SELECT count(DISTINCT UserID) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0018 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:11:54,362 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:12:06,510 Stage-1 map = 9%, reduce = 0% -2013-09-17 23:12:07,520 Stage-1 map = 20%, reduce = 0% -2013-09-17 23:12:08,525 Stage-1 map = 24%, reduce = 0% -2013-09-17 23:12:09,532 Stage-1 map = 35%, reduce = 0% -2013-09-17 23:12:10,537 Stage-1 map = 55%, reduce = 0% -2013-09-17 23:12:11,543 Stage-1 map = 58%, reduce = 0% -2013-09-17 23:12:12,555 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 11.81 sec -2013-09-17 23:12:13,575 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 11.81 sec -2013-09-17 23:12:14,581 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 11.81 sec -2013-09-17 23:12:15,598 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 81.59 sec -2013-09-17 23:12:16,636 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 141.75 sec -2013-09-17 23:12:17,658 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 223.95 sec -2013-09-17 23:12:18,665 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 293.32 sec -2013-09-17 23:12:19,671 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 427.68 sec -2013-09-17 23:12:20,678 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 487.22 sec -2013-09-17 23:12:21,684 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 642.1 sec -2013-09-17 23:12:22,691 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 642.1 sec -2013-09-17 23:12:23,697 Stage-1 map = 96%, reduce = 11%, Cumulative CPU 642.1 sec -2013-09-17 23:12:24,703 Stage-1 map = 98%, reduce = 11%, Cumulative CPU 665.75 sec -2013-09-17 23:12:25,709 Stage-1 map = 99%, reduce = 11%, Cumulative CPU 680.97 sec -2013-09-17 23:12:26,715 Stage-1 map = 100%, reduce = 11%, Cumulative CPU 699.12 sec -2013-09-17 23:12:27,720 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 699.12 sec -2013-09-17 23:12:28,725 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 699.12 sec -2013-09-17 23:12:29,730 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 699.12 sec -2013-09-17 23:12:30,736 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 699.12 sec -2013-09-17 23:12:31,741 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 699.12 sec -2013-09-17 23:12:32,747 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 699.12 sec -2013-09-17 23:12:33,752 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:34,757 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:35,762 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:36,766 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:37,771 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:38,776 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:39,780 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:40,785 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:41,789 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:42,794 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:43,798 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:44,803 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:45,808 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:46,813 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:48,011 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:49,016 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:50,021 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:51,025 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:52,030 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:53,035 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:54,041 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 699.12 sec -2013-09-17 23:12:55,046 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 699.12 sec -2013-09-17 23:12:56,051 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 699.12 sec -2013-09-17 23:12:57,056 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 699.12 sec -2013-09-17 23:12:58,061 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 699.12 sec -2013-09-17 23:12:59,066 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 699.12 sec -2013-09-17 23:13:00,071 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 699.12 sec -2013-09-17 23:13:01,076 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 699.12 sec -2013-09-17 23:13:02,081 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 699.12 sec -2013-09-17 23:13:03,086 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 699.12 sec -2013-09-17 23:13:09,756 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 699.12 sec -2013-09-17 23:13:10,762 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 699.12 sec -2013-09-17 23:13:11,767 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 699.12 sec -2013-09-17 23:13:12,773 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 699.12 sec -2013-09-17 23:13:13,779 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 699.12 sec -2013-09-17 23:13:14,784 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 699.12 sec -2013-09-17 23:13:15,790 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 699.12 sec -2013-09-17 23:13:16,795 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 699.12 sec -2013-09-17 23:13:17,800 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 699.12 sec -2013-09-17 23:13:18,805 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 699.12 sec -2013-09-17 23:13:19,815 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 699.12 sec -2013-09-17 23:13:20,820 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 699.12 sec -2013-09-17 23:13:21,827 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 755.01 sec -2013-09-17 23:13:22,832 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 755.01 sec -2013-09-17 23:13:23,837 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 755.01 sec -2013-09-17 23:13:24,844 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 755.01 sec -2013-09-17 23:13:25,849 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 755.01 sec -2013-09-17 23:13:26,855 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 755.01 sec -2013-09-17 23:13:27,860 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 755.01 sec -2013-09-17 23:13:28,866 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 755.01 sec -2013-09-17 23:13:29,872 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 755.01 sec -2013-09-17 23:13:30,877 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 764.66 sec -2013-09-17 23:13:31,883 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 764.66 sec -2013-09-17 23:13:32,888 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 764.66 sec -MapReduce Total cumulative CPU time: 12 minutes 44 seconds 660 msec -Ended Job = job_201309172235_0018 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 764.66 sec HDFS Read: 663537300 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 12 minutes 44 seconds 660 msec -OK -17630976 -Time taken: 107.343 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_709@mturlrep13_201309172313_1257295877.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0019 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:13:50,358 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:14:00,485 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:14:01,491 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:14:02,497 Stage-1 map = 11%, reduce = 0% -2013-09-17 23:14:04,042 Stage-1 map = 24%, reduce = 0% -2013-09-17 23:14:05,047 Stage-1 map = 26%, reduce = 0% -2013-09-17 23:14:06,057 Stage-1 map = 35%, reduce = 0% -2013-09-17 23:14:07,069 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 9.77 sec -2013-09-17 23:14:08,077 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 9.77 sec -2013-09-17 23:14:09,260 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 9.77 sec -2013-09-17 23:14:10,285 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 54.19 sec -2013-09-17 23:14:11,298 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 98.66 sec -2013-09-17 23:14:12,311 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 229.9 sec -2013-09-17 23:14:13,317 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 354.88 sec -2013-09-17 23:14:14,323 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 470.21 sec -2013-09-17 23:14:15,329 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 470.21 sec -2013-09-17 23:14:16,334 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 470.21 sec -2013-09-17 23:14:17,340 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 470.21 sec -2013-09-17 23:14:18,346 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 470.21 sec -2013-09-17 23:14:19,353 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 470.21 sec -2013-09-17 23:14:20,359 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 470.21 sec -2013-09-17 23:14:21,365 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 470.21 sec -2013-09-17 23:14:22,370 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 477.68 sec -2013-09-17 23:14:23,376 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 477.68 sec -2013-09-17 23:14:24,382 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 477.68 sec -2013-09-17 23:14:25,387 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 477.68 sec -2013-09-17 23:14:26,393 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 477.68 sec -2013-09-17 23:14:27,399 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 477.68 sec -2013-09-17 23:14:28,404 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 477.68 sec -2013-09-17 23:14:29,410 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 477.68 sec -2013-09-17 23:14:30,416 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 477.68 sec -2013-09-17 23:14:31,421 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 477.68 sec -2013-09-17 23:14:32,425 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 477.68 sec -2013-09-17 23:14:33,430 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 477.68 sec -2013-09-17 23:14:34,436 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 477.68 sec -2013-09-17 23:14:35,440 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 477.68 sec -2013-09-17 23:14:36,446 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 477.68 sec -2013-09-17 23:14:37,451 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 477.68 sec -2013-09-17 23:14:38,456 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 477.68 sec -2013-09-17 23:14:39,479 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 477.68 sec -2013-09-17 23:14:40,484 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 477.68 sec -2013-09-17 23:14:41,489 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 477.68 sec -2013-09-17 23:14:42,494 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 477.68 sec -2013-09-17 23:14:43,499 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 477.68 sec -2013-09-17 23:14:44,506 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 501.74 sec -2013-09-17 23:14:45,512 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 501.74 sec -MapReduce Total cumulative CPU time: 8 minutes 21 seconds 740 msec -Ended Job = job_201309172235_0019 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 501.74 sec HDFS Read: 182281719 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 21 seconds 740 msec -OK -6404441 -Time taken: 68.119 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5905@mturlrep13_201309172314_300988988.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0020 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:14:57,829 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:15:09,948 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:15:10,963 Stage-1 map = 17%, reduce = 0% -2013-09-17 23:15:11,975 Stage-1 map = 37%, reduce = 0% -2013-09-17 23:15:12,995 Stage-1 map = 38%, reduce = 0% -2013-09-17 23:15:14,001 Stage-1 map = 61%, reduce = 0% -2013-09-17 23:15:15,016 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 20.56 sec -2013-09-17 23:15:16,039 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 120.41 sec -2013-09-17 23:15:17,051 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 219.11 sec -2013-09-17 23:15:18,058 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 287.4 sec -2013-09-17 23:15:19,064 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 475.43 sec -2013-09-17 23:15:20,070 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 475.43 sec -2013-09-17 23:15:21,075 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 475.43 sec -2013-09-17 23:15:22,081 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 475.43 sec -2013-09-17 23:15:23,087 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 475.43 sec -2013-09-17 23:15:24,093 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 475.43 sec -2013-09-17 23:15:25,099 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 475.43 sec -2013-09-17 23:15:26,104 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 475.43 sec -2013-09-17 23:15:27,109 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 475.43 sec -2013-09-17 23:15:28,115 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 475.43 sec -2013-09-17 23:15:29,120 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 475.43 sec -2013-09-17 23:15:30,126 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 475.43 sec -2013-09-17 23:15:31,132 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 475.43 sec -2013-09-17 23:15:32,138 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 475.43 sec -2013-09-17 23:15:33,144 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 475.43 sec -2013-09-17 23:15:34,150 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 475.43 sec -2013-09-17 23:15:35,156 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 475.43 sec -2013-09-17 23:15:36,161 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 475.43 sec -2013-09-17 23:15:37,167 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 475.43 sec -2013-09-17 23:15:38,172 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 475.43 sec -2013-09-17 23:15:39,177 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 475.43 sec -2013-09-17 23:15:40,182 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 475.43 sec -2013-09-17 23:15:41,187 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 475.43 sec -2013-09-17 23:15:42,192 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 475.43 sec -2013-09-17 23:15:43,198 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 475.43 sec -2013-09-17 23:15:44,203 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 475.43 sec -2013-09-17 23:15:45,208 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 475.43 sec -2013-09-17 23:15:46,214 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 475.43 sec -2013-09-17 23:15:47,219 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 475.43 sec -2013-09-17 23:15:48,224 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 475.43 sec -2013-09-17 23:15:49,229 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 475.43 sec -2013-09-17 23:15:50,237 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 508.08 sec -2013-09-17 23:15:53,774 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 508.08 sec -2013-09-17 23:15:54,834 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 508.08 sec -MapReduce Total cumulative CPU time: 8 minutes 28 seconds 80 msec -Ended Job = job_201309172235_0020 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 508.08 sec HDFS Read: 182281719 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 28 seconds 80 msec -OK -6404441 -Time taken: 64.831 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11972@mturlrep13_201309172315_1325234581.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0021 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:16:04,362 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:16:16,459 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:16:17,471 Stage-1 map = 17%, reduce = 0% -2013-09-17 23:16:18,484 Stage-1 map = 31%, reduce = 0% -2013-09-17 23:16:19,512 Stage-1 map = 36%, reduce = 0% -2013-09-17 23:16:20,525 Stage-1 map = 66%, reduce = 0% -2013-09-17 23:16:21,542 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 31.11 sec -2013-09-17 23:16:22,565 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 364.53 sec -2013-09-17 23:16:23,583 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 382.31 sec -2013-09-17 23:16:24,593 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 432.98 sec -2013-09-17 23:16:25,599 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 472.89 sec -2013-09-17 23:16:26,604 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 472.89 sec -2013-09-17 23:16:27,610 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 472.89 sec -2013-09-17 23:16:28,615 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 472.89 sec -2013-09-17 23:16:29,621 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 472.89 sec -2013-09-17 23:16:30,626 Stage-1 map = 100%, reduce = 16%, Cumulative CPU 472.89 sec -2013-09-17 23:16:31,632 Stage-1 map = 100%, reduce = 16%, Cumulative CPU 472.89 sec -2013-09-17 23:16:32,638 Stage-1 map = 100%, reduce = 16%, Cumulative CPU 472.89 sec -2013-09-17 23:16:33,643 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 472.89 sec -2013-09-17 23:16:34,649 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 472.89 sec -2013-09-17 23:16:35,654 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 472.89 sec -2013-09-17 23:16:36,660 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 472.89 sec -2013-09-17 23:16:37,665 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 472.89 sec -2013-09-17 23:16:38,671 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 472.89 sec -2013-09-17 23:16:39,677 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 472.89 sec -2013-09-17 23:16:40,682 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 472.89 sec -2013-09-17 23:16:41,688 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 472.89 sec -2013-09-17 23:16:42,694 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 472.89 sec -2013-09-17 23:16:43,699 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 472.89 sec -2013-09-17 23:16:44,704 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 472.89 sec -2013-09-17 23:16:45,710 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 472.89 sec -2013-09-17 23:16:46,715 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 472.89 sec -2013-09-17 23:16:47,721 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 472.89 sec -2013-09-17 23:16:48,726 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 472.89 sec -2013-09-17 23:16:49,731 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 472.89 sec -2013-09-17 23:16:50,736 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 472.89 sec -2013-09-17 23:16:51,741 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 472.89 sec -2013-09-17 23:16:52,746 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 472.89 sec -2013-09-17 23:16:53,751 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 472.89 sec -2013-09-17 23:16:54,770 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 472.89 sec -2013-09-17 23:16:55,775 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 472.89 sec -2013-09-17 23:16:56,783 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 504.75 sec -2013-09-17 23:16:57,788 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 504.75 sec -MapReduce Total cumulative CPU time: 8 minutes 24 seconds 750 msec -Ended Job = job_201309172235_0021 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 504.75 sec HDFS Read: 182281719 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 24 seconds 750 msec -OK -6404441 -Time taken: 64.269 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17050@mturlrep13_201309172317_1762378997.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0022 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:17:18,068 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:17:30,186 Stage-1 map = 4%, reduce = 0% -2013-09-17 23:17:31,192 Stage-1 map = 30%, reduce = 0% -2013-09-17 23:17:32,197 Stage-1 map = 31%, reduce = 0% -2013-09-17 23:17:33,206 Stage-1 map = 36%, reduce = 0% -2013-09-17 23:17:34,211 Stage-1 map = 71%, reduce = 0% -2013-09-17 23:17:35,223 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 50.55 sec -2013-09-17 23:17:36,231 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 318.5 sec -2013-09-17 23:17:37,237 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 318.5 sec -2013-09-17 23:17:38,242 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 318.5 sec -2013-09-17 23:17:39,248 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 318.5 sec -2013-09-17 23:17:40,253 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 318.5 sec -2013-09-17 23:17:41,259 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 318.5 sec -2013-09-17 23:17:42,265 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 318.5 sec -2013-09-17 23:17:43,271 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 318.5 sec -2013-09-17 23:17:44,278 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 321.17 sec -2013-09-17 23:17:45,284 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 321.17 sec -MapReduce Total cumulative CPU time: 5 minutes 21 seconds 170 msec -Ended Job = job_201309172235_0022 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 321.17 sec HDFS Read: 4951094 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 5 minutes 21 seconds 170 msec -OK -Time taken: 37.809 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT min(EventDate), max(EventDate) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22045@mturlrep13_201309172317_1976013998.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0023 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:17:55,984 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:18:07,051 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:18:08,056 Stage-1 map = 37%, reduce = 0% -2013-09-17 23:18:09,084 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 15.31 sec -2013-09-17 23:18:10,092 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 213.11 sec -2013-09-17 23:18:11,100 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.89 sec -2013-09-17 23:18:12,105 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.89 sec -2013-09-17 23:18:13,111 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.89 sec -2013-09-17 23:18:14,117 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.89 sec -2013-09-17 23:18:15,123 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.89 sec -2013-09-17 23:18:16,128 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.89 sec -2013-09-17 23:18:17,134 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 322.89 sec -2013-09-17 23:18:18,141 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 325.69 sec -2013-09-17 23:18:19,147 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 325.69 sec -2013-09-17 23:18:20,153 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 325.69 sec -MapReduce Total cumulative CPU time: 5 minutes 25 seconds 690 msec -Ended Job = job_201309172235_0023 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 325.69 sec HDFS Read: 4951094 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 5 minutes 25 seconds 690 msec -OK -Time taken: 33.021 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT min(EventDate), max(EventDate) FROM hits_100m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26986@mturlrep13_201309172318_1806287056.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_100m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0024 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-17 23:18:30,852 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:18:42,000 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:18:43,008 Stage-1 map = 28%, reduce = 0% -2013-09-17 23:18:44,021 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 41.29 sec -2013-09-17 23:18:45,035 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 247.75 sec -2013-09-17 23:18:46,043 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.15 sec -2013-09-17 23:18:47,048 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.15 sec -2013-09-17 23:18:48,053 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.15 sec -2013-09-17 23:18:49,059 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.15 sec -2013-09-17 23:18:50,064 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.15 sec -2013-09-17 23:18:51,070 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 322.15 sec -2013-09-17 23:18:52,076 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 322.15 sec -2013-09-17 23:18:53,081 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 322.15 sec -2013-09-17 23:18:54,089 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 325.35 sec -2013-09-17 23:18:55,095 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 325.35 sec -MapReduce Total cumulative CPU time: 5 minutes 25 seconds 350 msec -Ended Job = job_201309172235_0024 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 325.35 sec HDFS Read: 4951094 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 5 minutes 25 seconds 350 msec -OK -Time taken: 33.13 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT AdvEngineID, count(*) AS c FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31945@mturlrep13_201309172319_144240708.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0025 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:19:12,376 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:19:22,455 Stage-1 map = 1%, reduce = 0%, Cumulative CPU 53.86 sec -2013-09-17 23:19:23,463 Stage-1 map = 1%, reduce = 0%, Cumulative CPU 53.86 sec -2013-09-17 23:19:24,470 Stage-1 map = 13%, reduce = 0%, Cumulative CPU 53.86 sec -2013-09-17 23:19:25,992 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 55.65 sec -2013-09-17 23:19:27,002 Stage-1 map = 37%, reduce = 0%, Cumulative CPU 55.65 sec -2013-09-17 23:19:28,010 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 97.8 sec -2013-09-17 23:19:29,023 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 216.67 sec -2013-09-17 23:19:30,033 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 288.99 sec -2013-09-17 23:19:31,040 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 288.99 sec -2013-09-17 23:19:32,046 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 288.99 sec -2013-09-17 23:19:33,052 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 288.99 sec -2013-09-17 23:19:34,058 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 288.99 sec -2013-09-17 23:19:35,065 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 288.99 sec -2013-09-17 23:19:36,070 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 288.99 sec -2013-09-17 23:19:37,080 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 292.1 sec -2013-09-17 23:19:38,087 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 310.42 sec -2013-09-17 23:19:39,094 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 328.04 sec -2013-09-17 23:19:40,101 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 328.04 sec -2013-09-17 23:19:41,111 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 328.04 sec -MapReduce Total cumulative CPU time: 5 minutes 28 seconds 40 msec -Ended Job = job_201309172235_0025 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0026 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:19:43,669 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:19:45,678 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-17 23:19:46,683 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-17 23:19:47,689 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-17 23:19:48,694 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-17 23:19:49,698 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-17 23:19:50,703 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-17 23:19:51,709 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-17 23:19:52,715 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.75 sec -2013-09-17 23:19:53,721 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.37 sec -2013-09-17 23:19:54,727 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.37 sec -2013-09-17 23:19:55,733 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.37 sec -MapReduce Total cumulative CPU time: 2 seconds 370 msec -Ended Job = job_201309172235_0026 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 328.04 sec HDFS Read: 6282387 HDFS Write: 1443 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.37 sec HDFS Read: 5020 HDFS Write: 129 SUCCESS -Total MapReduce CPU Time Spent: 5 minutes 30 seconds 410 msec -OK -Time taken: 53.788 seconds, Fetched: 18 row(s) -hive> quit; - -times: 2 -query: SELECT AdvEngineID, count(*) AS c FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6554@mturlrep13_201309172319_1947847771.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0027 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:20:07,904 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:20:19,166 Stage-1 map = 4%, reduce = 0% -2013-09-17 23:20:20,189 Stage-1 map = 42%, reduce = 0% -2013-09-17 23:20:21,209 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 147.41 sec -2013-09-17 23:20:22,217 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 296.25 sec -2013-09-17 23:20:23,226 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 296.25 sec -2013-09-17 23:20:24,237 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 296.25 sec -2013-09-17 23:20:25,244 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 296.25 sec -2013-09-17 23:20:26,251 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 296.25 sec -2013-09-17 23:20:27,257 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 296.25 sec -2013-09-17 23:20:28,264 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 296.25 sec -2013-09-17 23:20:29,270 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 296.25 sec -2013-09-17 23:20:30,279 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 305.77 sec -2013-09-17 23:20:31,286 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 315.54 sec -2013-09-17 23:20:32,293 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 327.93 sec -2013-09-17 23:20:33,300 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 331.3 sec -2013-09-17 23:20:34,306 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 331.3 sec -MapReduce Total cumulative CPU time: 5 minutes 31 seconds 300 msec -Ended Job = job_201309172235_0027 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0028 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:20:36,785 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:20:38,794 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-17 23:20:39,799 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-17 23:20:40,803 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-17 23:20:41,808 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-17 23:20:42,812 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-17 23:20:43,817 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-17 23:20:44,821 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-17 23:20:45,826 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.72 sec -2013-09-17 23:20:46,831 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.34 sec -2013-09-17 23:20:47,837 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.34 sec -2013-09-17 23:20:48,842 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.34 sec -MapReduce Total cumulative CPU time: 2 seconds 340 msec -Ended Job = job_201309172235_0028 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 331.3 sec HDFS Read: 6282387 HDFS Write: 1443 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.34 sec HDFS Read: 5020 HDFS Write: 129 SUCCESS -Total MapReduce CPU Time Spent: 5 minutes 33 seconds 640 msec -OK -Time taken: 51.261 seconds, Fetched: 18 row(s) -hive> quit; - -times: 3 -query: SELECT AdvEngineID, count(*) AS c FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13828@mturlrep13_201309172320_1002752164.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0029 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:20:59,464 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:21:08,619 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:21:10,697 Stage-1 map = 4%, reduce = 0%, Cumulative CPU 7.3 sec -2013-09-17 23:21:11,718 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 28.78 sec -2013-09-17 23:21:12,731 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 254.07 sec -2013-09-17 23:21:13,739 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 287.86 sec -2013-09-17 23:21:14,748 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 287.86 sec -2013-09-17 23:21:15,755 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 287.86 sec -2013-09-17 23:21:16,762 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 287.86 sec -2013-09-17 23:21:17,769 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 287.86 sec -2013-09-17 23:21:18,776 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 287.86 sec -2013-09-17 23:21:19,782 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 287.86 sec -2013-09-17 23:21:20,789 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 287.86 sec -2013-09-17 23:21:21,798 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 307.51 sec -2013-09-17 23:21:22,804 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 319.28 sec -2013-09-17 23:21:23,811 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 323.91 sec -2013-09-17 23:21:24,817 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 323.91 sec -MapReduce Total cumulative CPU time: 5 minutes 23 seconds 910 msec -Ended Job = job_201309172235_0029 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0030 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:21:28,286 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:21:29,293 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-17 23:21:30,299 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-17 23:21:31,304 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-17 23:21:32,309 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-17 23:21:33,314 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-17 23:21:34,320 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-17 23:21:35,325 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-17 23:21:36,330 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-17 23:21:37,336 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -2013-09-17 23:21:38,341 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -2013-09-17 23:21:39,348 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -MapReduce Total cumulative CPU time: 2 seconds 130 msec -Ended Job = job_201309172235_0030 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 323.91 sec HDFS Read: 6282387 HDFS Write: 1443 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.13 sec HDFS Read: 5009 HDFS Write: 129 SUCCESS -Total MapReduce CPU Time Spent: 5 minutes 26 seconds 40 msec -OK -Time taken: 48.653 seconds, Fetched: 18 row(s) -hive> quit; --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20466@mturlrep13_201309172321_1193168986.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0031 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:21:56,900 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:22:07,016 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:22:08,024 Stage-1 map = 3%, reduce = 0% -2013-09-17 23:22:09,036 Stage-1 map = 4%, reduce = 0% -2013-09-17 23:22:10,195 Stage-1 map = 17%, reduce = 0% -2013-09-17 23:22:11,201 Stage-1 map = 18%, reduce = 0% -2013-09-17 23:22:12,207 Stage-1 map = 24%, reduce = 0% -2013-09-17 23:22:13,212 Stage-1 map = 32%, reduce = 0% -2013-09-17 23:22:14,220 Stage-1 map = 36%, reduce = 0% -2013-09-17 23:22:16,775 Stage-1 map = 52%, reduce = 0% -2013-09-17 23:22:18,821 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 39.61 sec -2013-09-17 23:22:19,842 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 39.61 sec -2013-09-17 23:22:20,847 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 39.61 sec -2013-09-17 23:22:21,854 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 59.18 sec -2013-09-17 23:22:22,860 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 538.37 sec -2013-09-17 23:22:23,900 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 540.67 sec -2013-09-17 23:22:25,023 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 541.84 sec -2013-09-17 23:22:26,034 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 571.25 sec -2013-09-17 23:22:27,044 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 595.16 sec -2013-09-17 23:22:28,050 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 618.83 sec -2013-09-17 23:22:29,057 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 647.47 sec -2013-09-17 23:22:30,063 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 664.76 sec -2013-09-17 23:22:31,070 Stage-1 map = 96%, reduce = 1%, Cumulative CPU 664.76 sec -2013-09-17 23:22:32,076 Stage-1 map = 98%, reduce = 12%, Cumulative CPU 685.09 sec -2013-09-17 23:22:33,082 Stage-1 map = 99%, reduce = 12%, Cumulative CPU 710.3 sec -2013-09-17 23:22:34,088 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 722.47 sec -2013-09-17 23:22:35,094 Stage-1 map = 99%, reduce = 19%, Cumulative CPU 735.48 sec -2013-09-17 23:22:36,100 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 749.67 sec -2013-09-17 23:22:37,107 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 749.67 sec -2013-09-17 23:22:38,113 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 749.67 sec -2013-09-17 23:22:39,119 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 749.67 sec -2013-09-17 23:22:40,125 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 749.67 sec -2013-09-17 23:22:41,132 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 749.67 sec -2013-09-17 23:22:42,139 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 749.67 sec -2013-09-17 23:22:43,147 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 759.31 sec -2013-09-17 23:22:44,153 Stage-1 map = 100%, reduce = 62%, Cumulative CPU 792.97 sec -2013-09-17 23:22:45,159 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 792.97 sec -2013-09-17 23:22:46,166 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 805.88 sec -2013-09-17 23:22:47,172 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 816.79 sec -2013-09-17 23:22:48,178 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 816.79 sec -2013-09-17 23:22:49,184 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 829.11 sec -2013-09-17 23:22:50,190 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 843.07 sec -2013-09-17 23:22:51,196 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 872.26 sec -2013-09-17 23:22:52,202 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 872.26 sec -2013-09-17 23:22:53,207 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 872.26 sec -2013-09-17 23:22:54,219 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 872.26 sec -2013-09-17 23:22:55,224 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 872.26 sec -2013-09-17 23:22:56,230 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 897.41 sec -2013-09-17 23:22:57,236 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 897.41 sec -2013-09-17 23:22:58,242 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 897.41 sec -2013-09-17 23:22:59,248 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 897.41 sec -MapReduce Total cumulative CPU time: 14 minutes 57 seconds 410 msec -Ended Job = job_201309172235_0031 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0032 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:23:01,757 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:23:04,768 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:23:05,774 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:23:06,779 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:23:07,783 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:23:08,787 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:23:09,792 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:23:10,796 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:23:11,801 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.62 sec -2013-09-17 23:23:12,806 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.6 sec -2013-09-17 23:23:13,810 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.6 sec -MapReduce Total cumulative CPU time: 3 seconds 600 msec -Ended Job = job_201309172235_0032 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 897.41 sec HDFS Read: 773347596 HDFS Write: 200700 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.6 sec HDFS Read: 204277 HDFS Write: 106 SUCCESS -Total MapReduce CPU Time Spent: 15 minutes 1 seconds 10 msec -OK -Time taken: 87.479 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27218@mturlrep13_201309172323_828143909.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0033 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:23:24,453 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:23:35,535 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:23:36,541 Stage-1 map = 11%, reduce = 0% -2013-09-17 23:23:37,547 Stage-1 map = 15%, reduce = 0% -2013-09-17 23:23:38,823 Stage-1 map = 16%, reduce = 0% -2013-09-17 23:23:39,831 Stage-1 map = 44%, reduce = 0% -2013-09-17 23:23:40,851 Stage-1 map = 52%, reduce = 0% -2013-09-17 23:23:41,857 Stage-1 map = 53%, reduce = 0% -2013-09-17 23:23:42,879 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 11.88 sec -2013-09-17 23:23:43,887 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 11.88 sec -2013-09-17 23:23:44,916 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 40.11 sec -2013-09-17 23:23:45,931 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 54.11 sec -2013-09-17 23:23:46,948 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 83.97 sec -2013-09-17 23:23:47,955 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 116.76 sec -2013-09-17 23:23:48,966 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 237.46 sec -2013-09-17 23:23:49,982 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 292.37 sec -2013-09-17 23:23:50,990 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 331.01 sec -2013-09-17 23:23:51,996 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 474.07 sec -2013-09-17 23:23:53,002 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 516.53 sec -2013-09-17 23:23:54,009 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 516.53 sec -2013-09-17 23:23:55,018 Stage-1 map = 97%, reduce = 2%, Cumulative CPU 516.53 sec -2013-09-17 23:23:56,024 Stage-1 map = 97%, reduce = 10%, Cumulative CPU 516.53 sec -2013-09-17 23:23:57,030 Stage-1 map = 98%, reduce = 22%, Cumulative CPU 599.75 sec -2013-09-17 23:23:58,037 Stage-1 map = 99%, reduce = 24%, Cumulative CPU 656.33 sec -2013-09-17 23:23:59,043 Stage-1 map = 99%, reduce = 26%, Cumulative CPU 717.14 sec -2013-09-17 23:24:00,049 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 746.73 sec -2013-09-17 23:24:01,056 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 746.73 sec -2013-09-17 23:24:02,063 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 746.73 sec -2013-09-17 23:24:03,069 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 746.73 sec -2013-09-17 23:24:04,075 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 746.73 sec -2013-09-17 23:24:05,081 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 746.73 sec -2013-09-17 23:24:06,087 Stage-1 map = 100%, reduce = 53%, Cumulative CPU 746.73 sec -2013-09-17 23:24:07,094 Stage-1 map = 100%, reduce = 53%, Cumulative CPU 746.73 sec -2013-09-17 23:24:08,102 Stage-1 map = 100%, reduce = 59%, Cumulative CPU 766.33 sec -2013-09-17 23:24:09,109 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 790.42 sec -2013-09-17 23:24:10,115 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 790.42 sec -2013-09-17 23:24:11,122 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 817.26 sec -2013-09-17 23:24:12,128 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 817.26 sec -2013-09-17 23:24:13,134 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 828.67 sec -2013-09-17 23:24:14,141 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 840.81 sec -2013-09-17 23:24:15,147 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 869.4 sec -2013-09-17 23:24:16,152 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 869.4 sec -2013-09-17 23:24:17,158 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 869.4 sec -2013-09-17 23:24:18,592 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 869.4 sec -2013-09-17 23:24:19,724 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 869.4 sec -2013-09-17 23:24:20,729 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 869.4 sec -2013-09-17 23:24:21,735 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 894.47 sec -2013-09-17 23:24:22,742 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 894.47 sec -MapReduce Total cumulative CPU time: 14 minutes 54 seconds 470 msec -Ended Job = job_201309172235_0033 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0034 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:24:28,624 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:24:30,631 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.65 sec -2013-09-17 23:24:31,636 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.65 sec -2013-09-17 23:24:32,641 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.65 sec -2013-09-17 23:24:33,646 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.65 sec -2013-09-17 23:24:34,650 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.65 sec -2013-09-17 23:24:35,655 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.65 sec -2013-09-17 23:24:36,660 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.65 sec -2013-09-17 23:24:37,666 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.65 sec -2013-09-17 23:24:38,671 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.47 sec -2013-09-17 23:24:39,676 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.47 sec -2013-09-17 23:24:40,682 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.47 sec -MapReduce Total cumulative CPU time: 3 seconds 470 msec -Ended Job = job_201309172235_0034 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 894.47 sec HDFS Read: 773347596 HDFS Write: 200700 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.47 sec HDFS Read: 204277 HDFS Write: 106 SUCCESS -Total MapReduce CPU Time Spent: 14 minutes 57 seconds 940 msec -OK -Time taken: 85.062 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1610@mturlrep13_201309172324_625629875.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0035 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:24:50,386 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:25:02,507 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:25:03,535 Stage-1 map = 8%, reduce = 0% -2013-09-17 23:25:04,541 Stage-1 map = 14%, reduce = 0% -2013-09-17 23:25:05,547 Stage-1 map = 22%, reduce = 0% -2013-09-17 23:25:06,553 Stage-1 map = 39%, reduce = 0% -2013-09-17 23:25:07,559 Stage-1 map = 47%, reduce = 0% -2013-09-17 23:25:08,571 Stage-1 map = 56%, reduce = 0% -2013-09-17 23:25:09,583 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 12.22 sec -2013-09-17 23:25:10,598 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 12.22 sec -2013-09-17 23:25:11,612 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 12.22 sec -2013-09-17 23:25:12,618 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 55.11 sec -2013-09-17 23:25:13,632 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 55.11 sec -2013-09-17 23:25:14,646 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 117.12 sec -2013-09-17 23:25:15,672 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 186.45 sec -2013-09-17 23:25:16,681 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 275.56 sec -2013-09-17 23:25:17,690 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 373.55 sec -2013-09-17 23:25:18,698 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 434.47 sec -2013-09-17 23:25:19,704 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 497.78 sec -2013-09-17 23:25:20,710 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 518.9 sec -2013-09-17 23:25:21,717 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 518.9 sec -2013-09-17 23:25:22,723 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 518.9 sec -2013-09-17 23:25:23,729 Stage-1 map = 97%, reduce = 15%, Cumulative CPU 722.2 sec -2013-09-17 23:25:24,736 Stage-1 map = 98%, reduce = 20%, Cumulative CPU 729.94 sec -2013-09-17 23:25:25,742 Stage-1 map = 99%, reduce = 24%, Cumulative CPU 745.51 sec -2013-09-17 23:25:26,748 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 754.8 sec -2013-09-17 23:25:27,760 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 754.8 sec -2013-09-17 23:25:28,769 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 754.8 sec -2013-09-17 23:25:29,776 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 754.8 sec -2013-09-17 23:25:30,782 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 754.8 sec -2013-09-17 23:25:31,787 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 754.8 sec -2013-09-17 23:25:32,794 Stage-1 map = 100%, reduce = 48%, Cumulative CPU 754.8 sec -2013-09-17 23:25:33,800 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 754.8 sec -2013-09-17 23:25:34,806 Stage-1 map = 100%, reduce = 60%, Cumulative CPU 754.8 sec -2013-09-17 23:25:35,814 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 775.22 sec -2013-09-17 23:25:36,820 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 800.08 sec -2013-09-17 23:25:37,826 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 813.13 sec -2013-09-17 23:25:38,833 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 854.96 sec -2013-09-17 23:25:39,839 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 881.01 sec -2013-09-17 23:25:40,845 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 881.01 sec -2013-09-17 23:25:41,850 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 881.01 sec -2013-09-17 23:25:42,856 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 881.01 sec -2013-09-17 23:25:43,862 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 881.01 sec -2013-09-17 23:25:45,177 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 881.01 sec -2013-09-17 23:25:46,251 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 881.01 sec -2013-09-17 23:25:47,256 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 881.01 sec -2013-09-17 23:25:48,262 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 881.01 sec -2013-09-17 23:25:49,268 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 905.59 sec -2013-09-17 23:25:50,273 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 905.59 sec -MapReduce Total cumulative CPU time: 15 minutes 5 seconds 590 msec -Ended Job = job_201309172235_0035 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0036 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:25:55,561 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:25:57,569 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:25:58,574 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:25:59,579 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:26:00,584 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:26:01,588 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:26:02,592 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:26:03,597 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.62 sec -2013-09-17 23:26:04,603 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.62 sec -2013-09-17 23:26:05,609 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.52 sec -2013-09-17 23:26:06,614 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.52 sec -2013-09-17 23:26:07,619 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.52 sec -MapReduce Total cumulative CPU time: 3 seconds 520 msec -Ended Job = job_201309172235_0036 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 905.59 sec HDFS Read: 773347596 HDFS Write: 200700 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.52 sec HDFS Read: 204266 HDFS Write: 106 SUCCESS -Total MapReduce CPU Time Spent: 15 minutes 9 seconds 110 msec -OK -Time taken: 85.039 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей.; - - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9511@mturlrep13_201309172326_2040723350.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0037 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:26:25,696 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:26:35,814 Stage-1 map = 3%, reduce = 0% -2013-09-17 23:26:38,841 Stage-1 map = 10%, reduce = 0% -2013-09-17 23:26:39,848 Stage-1 map = 13%, reduce = 0% -2013-09-17 23:26:40,855 Stage-1 map = 16%, reduce = 0% -2013-09-17 23:26:41,883 Stage-1 map = 19%, reduce = 0% -2013-09-17 23:26:42,890 Stage-1 map = 29%, reduce = 0% -2013-09-17 23:26:43,897 Stage-1 map = 32%, reduce = 0% -2013-09-17 23:26:45,223 Stage-1 map = 38%, reduce = 0% -2013-09-17 23:26:46,236 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 14.75 sec -2013-09-17 23:26:47,243 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 14.75 sec -2013-09-17 23:26:48,308 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 14.75 sec -2013-09-17 23:26:49,329 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 14.75 sec -2013-09-17 23:26:50,342 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 14.75 sec -2013-09-17 23:26:51,354 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 14.75 sec -2013-09-17 23:26:52,368 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 14.75 sec -2013-09-17 23:26:53,382 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 74.49 sec -2013-09-17 23:26:54,393 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 113.4 sec -2013-09-17 23:26:55,401 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 135.12 sec -2013-09-17 23:26:56,414 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 176.43 sec -2013-09-17 23:26:57,430 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 194.75 sec -2013-09-17 23:26:58,444 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 238.4 sec -2013-09-17 23:26:59,451 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 277.62 sec -2013-09-17 23:27:00,458 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 418.29 sec -2013-09-17 23:27:01,465 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 447.59 sec -2013-09-17 23:27:02,472 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 578.54 sec -2013-09-17 23:27:03,478 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 657.91 sec -2013-09-17 23:27:04,486 Stage-1 map = 96%, reduce = 2%, Cumulative CPU 657.91 sec -2013-09-17 23:27:05,497 Stage-1 map = 97%, reduce = 12%, Cumulative CPU 726.48 sec -2013-09-17 23:27:06,503 Stage-1 map = 98%, reduce = 12%, Cumulative CPU 760.94 sec -2013-09-17 23:27:07,509 Stage-1 map = 99%, reduce = 17%, Cumulative CPU 867.41 sec -2013-09-17 23:27:08,515 Stage-1 map = 99%, reduce = 24%, Cumulative CPU 867.41 sec -2013-09-17 23:27:09,521 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 941.56 sec -2013-09-17 23:27:10,528 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 941.56 sec -2013-09-17 23:27:11,534 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 941.56 sec -2013-09-17 23:27:12,540 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 941.56 sec -2013-09-17 23:27:13,546 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 941.56 sec -2013-09-17 23:27:14,552 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 941.56 sec -2013-09-17 23:27:15,559 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 941.56 sec -2013-09-17 23:27:16,566 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 941.56 sec -2013-09-17 23:27:17,602 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 941.56 sec -2013-09-17 23:27:18,609 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 941.56 sec -2013-09-17 23:27:19,615 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 941.56 sec -2013-09-17 23:27:20,621 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 941.56 sec -2013-09-17 23:27:21,628 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 941.56 sec -2013-09-17 23:27:22,633 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 941.56 sec -2013-09-17 23:27:23,642 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 1072.24 sec -2013-09-17 23:27:24,649 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1080.16 sec -2013-09-17 23:27:25,655 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1080.16 sec -2013-09-17 23:27:26,661 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1087.79 sec -2013-09-17 23:27:27,668 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1097.18 sec -2013-09-17 23:27:28,675 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1097.18 sec -2013-09-17 23:27:29,682 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1106.96 sec -2013-09-17 23:27:30,688 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1106.96 sec -2013-09-17 23:27:31,695 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1106.96 sec -2013-09-17 23:27:32,701 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1106.96 sec -2013-09-17 23:27:33,923 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1136.44 sec -2013-09-17 23:27:34,929 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1136.44 sec -2013-09-17 23:27:35,935 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1136.44 sec -2013-09-17 23:27:36,940 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1136.44 sec -2013-09-17 23:27:37,946 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1136.44 sec -2013-09-17 23:27:41,680 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1136.44 sec -2013-09-17 23:27:42,685 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1136.44 sec -2013-09-17 23:27:43,690 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1136.44 sec -2013-09-17 23:27:45,033 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1155.81 sec -2013-09-17 23:27:46,039 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1155.81 sec -2013-09-17 23:27:47,045 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1155.81 sec -MapReduce Total cumulative CPU time: 19 minutes 15 seconds 810 msec -Ended Job = job_201309172235_0037 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0038 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:27:49,557 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:27:52,569 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.83 sec -2013-09-17 23:27:53,574 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.83 sec -2013-09-17 23:27:54,579 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.83 sec -2013-09-17 23:27:55,583 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.83 sec -2013-09-17 23:27:56,588 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.83 sec -2013-09-17 23:27:57,592 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.83 sec -2013-09-17 23:27:58,596 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.83 sec -2013-09-17 23:27:59,601 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.83 sec -2013-09-17 23:28:00,607 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.83 sec -2013-09-17 23:28:01,613 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.83 sec -MapReduce Total cumulative CPU time: 3 seconds 830 msec -Ended Job = job_201309172235_0038 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1155.81 sec HDFS Read: 847155129 HDFS Write: 298189 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.83 sec HDFS Read: 301766 HDFS Write: 437 SUCCESS -Total MapReduce CPU Time Spent: 19 minutes 19 seconds 640 msec -OK -Time taken: 106.577 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16855@mturlrep13_201309172328_909540588.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0039 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:28:12,290 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:28:23,414 Stage-1 map = 1%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:24,442 Stage-1 map = 6%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:25,480 Stage-1 map = 8%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:26,503 Stage-1 map = 10%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:27,510 Stage-1 map = 16%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:28,517 Stage-1 map = 28%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:29,524 Stage-1 map = 30%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:30,530 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:31,537 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:32,543 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:33,550 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 51.91 sec -2013-09-17 23:28:34,558 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 66.92 sec -2013-09-17 23:28:35,564 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 66.92 sec -2013-09-17 23:28:36,571 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 66.92 sec -2013-09-17 23:28:37,576 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 91.33 sec -2013-09-17 23:28:38,583 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 128.84 sec -2013-09-17 23:28:39,589 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 167.09 sec -2013-09-17 23:28:40,601 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 202.64 sec -2013-09-17 23:28:41,608 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 265.38 sec -2013-09-17 23:28:42,615 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 331.39 sec -2013-09-17 23:28:43,629 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 434.98 sec -2013-09-17 23:28:44,639 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 460.38 sec -2013-09-17 23:28:45,647 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 605.85 sec -2013-09-17 23:28:46,653 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 658.68 sec -2013-09-17 23:28:47,659 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 658.68 sec -2013-09-17 23:28:48,665 Stage-1 map = 95%, reduce = 1%, Cumulative CPU 658.68 sec -2013-09-17 23:28:49,672 Stage-1 map = 97%, reduce = 12%, Cumulative CPU 658.68 sec -2013-09-17 23:28:50,678 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 658.68 sec -2013-09-17 23:28:51,684 Stage-1 map = 98%, reduce = 21%, Cumulative CPU 795.29 sec -2013-09-17 23:28:52,690 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 927.85 sec -2013-09-17 23:28:53,696 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 927.85 sec -2013-09-17 23:28:54,702 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 927.85 sec -2013-09-17 23:28:55,712 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 927.85 sec -2013-09-17 23:28:56,718 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 927.85 sec -2013-09-17 23:28:57,724 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 927.85 sec -2013-09-17 23:28:58,730 Stage-1 map = 100%, reduce = 48%, Cumulative CPU 927.85 sec -2013-09-17 23:28:59,737 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 927.85 sec -2013-09-17 23:29:00,765 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 927.85 sec -2013-09-17 23:29:01,771 Stage-1 map = 100%, reduce = 65%, Cumulative CPU 927.85 sec -2013-09-17 23:29:02,777 Stage-1 map = 100%, reduce = 65%, Cumulative CPU 927.85 sec -2013-09-17 23:29:03,785 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 940.95 sec -2013-09-17 23:29:04,793 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 954.71 sec -2013-09-17 23:29:05,799 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 954.71 sec -2013-09-17 23:29:06,958 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1004.24 sec -2013-09-17 23:29:08,056 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1004.24 sec -2013-09-17 23:29:09,063 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1022.86 sec -2013-09-17 23:29:10,069 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1079.45 sec -2013-09-17 23:29:11,075 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1079.45 sec -2013-09-17 23:29:12,081 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1079.45 sec -2013-09-17 23:29:13,086 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1079.45 sec -2013-09-17 23:29:14,092 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1079.45 sec -2013-09-17 23:29:15,098 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1079.45 sec -2013-09-17 23:29:16,103 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1102.32 sec -2013-09-17 23:29:17,109 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1102.32 sec -2013-09-17 23:29:18,115 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1102.32 sec -2013-09-17 23:29:19,120 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1102.32 sec -2013-09-17 23:29:20,126 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1102.32 sec -2013-09-17 23:29:21,131 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1102.32 sec -2013-09-17 23:29:22,137 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1102.32 sec -2013-09-17 23:29:23,142 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1102.32 sec -2013-09-17 23:29:24,148 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1135.97 sec -2013-09-17 23:29:25,154 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1135.97 sec -2013-09-17 23:29:26,159 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1135.97 sec -2013-09-17 23:29:28,761 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1141.12 sec -2013-09-17 23:29:29,766 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1141.12 sec -2013-09-17 23:29:30,771 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1141.12 sec -MapReduce Total cumulative CPU time: 19 minutes 1 seconds 120 msec -Ended Job = job_201309172235_0039 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0040 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:29:33,329 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:29:36,340 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.71 sec -2013-09-17 23:29:37,344 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.71 sec -2013-09-17 23:29:38,348 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.71 sec -2013-09-17 23:29:39,352 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.71 sec -2013-09-17 23:29:40,356 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.71 sec -2013-09-17 23:29:41,360 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.71 sec -2013-09-17 23:29:42,364 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.71 sec -2013-09-17 23:29:43,369 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.71 sec -2013-09-17 23:29:44,373 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.59 sec -2013-09-17 23:29:45,378 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.59 sec -2013-09-17 23:29:46,383 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.59 sec -MapReduce Total cumulative CPU time: 3 seconds 590 msec -Ended Job = job_201309172235_0040 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1141.12 sec HDFS Read: 847155129 HDFS Write: 298189 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.59 sec HDFS Read: 301766 HDFS Write: 437 SUCCESS -Total MapReduce CPU Time Spent: 19 minutes 4 seconds 710 msec -OK -Time taken: 102.879 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24207@mturlrep13_201309172329_822092081.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0041 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:29:56,995 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:30:09,088 Stage-1 map = 5%, reduce = 0% -2013-09-17 23:30:10,094 Stage-1 map = 6%, reduce = 0% -2013-09-17 23:30:11,105 Stage-1 map = 7%, reduce = 0% -2013-09-17 23:30:12,113 Stage-1 map = 15%, reduce = 0% -2013-09-17 23:30:13,119 Stage-1 map = 26%, reduce = 0% -2013-09-17 23:30:14,124 Stage-1 map = 29%, reduce = 0% -2013-09-17 23:30:15,133 Stage-1 map = 33%, reduce = 0% -2013-09-17 23:30:16,147 Stage-1 map = 47%, reduce = 0% -2013-09-17 23:30:17,156 Stage-1 map = 52%, reduce = 0% -2013-09-17 23:30:18,165 Stage-1 map = 54%, reduce = 0% -2013-09-17 23:30:19,192 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 14.3 sec -2013-09-17 23:30:20,200 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 14.3 sec -2013-09-17 23:30:21,213 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 14.3 sec -2013-09-17 23:30:22,219 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 32.42 sec -2013-09-17 23:30:23,227 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 70.55 sec -2013-09-17 23:30:24,234 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 668.79 sec -2013-09-17 23:30:25,259 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 673.33 sec -2013-09-17 23:30:26,274 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 678.94 sec -2013-09-17 23:30:27,300 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 702.3 sec -2013-09-17 23:30:28,312 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 712.73 sec -2013-09-17 23:30:29,319 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 738.35 sec -2013-09-17 23:30:30,326 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 769.71 sec -2013-09-17 23:30:31,333 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 778.96 sec -2013-09-17 23:30:32,339 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 797.52 sec -2013-09-17 23:30:33,345 Stage-1 map = 95%, reduce = 1%, Cumulative CPU 797.52 sec -2013-09-17 23:30:34,354 Stage-1 map = 95%, reduce = 8%, Cumulative CPU 797.52 sec -2013-09-17 23:30:35,361 Stage-1 map = 97%, reduce = 13%, Cumulative CPU 797.52 sec -2013-09-17 23:30:36,367 Stage-1 map = 98%, reduce = 15%, Cumulative CPU 841.63 sec -2013-09-17 23:30:37,373 Stage-1 map = 98%, reduce = 21%, Cumulative CPU 857.24 sec -2013-09-17 23:30:38,379 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 928.19 sec -2013-09-17 23:30:39,385 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 928.19 sec -2013-09-17 23:30:40,391 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 928.19 sec -2013-09-17 23:30:41,396 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 928.19 sec -2013-09-17 23:30:42,403 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 928.19 sec -2013-09-17 23:30:43,409 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 928.19 sec -2013-09-17 23:30:44,415 Stage-1 map = 100%, reduce = 37%, Cumulative CPU 928.19 sec -2013-09-17 23:30:45,421 Stage-1 map = 100%, reduce = 37%, Cumulative CPU 928.19 sec -2013-09-17 23:30:46,428 Stage-1 map = 100%, reduce = 37%, Cumulative CPU 928.19 sec -2013-09-17 23:30:47,451 Stage-1 map = 100%, reduce = 44%, Cumulative CPU 928.19 sec -2013-09-17 23:30:48,460 Stage-1 map = 100%, reduce = 48%, Cumulative CPU 940.63 sec -2013-09-17 23:30:49,467 Stage-1 map = 100%, reduce = 66%, Cumulative CPU 940.63 sec -2013-09-17 23:30:50,474 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 940.63 sec -2013-09-17 23:30:51,480 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 940.63 sec -2013-09-17 23:30:52,487 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 940.63 sec -2013-09-17 23:30:53,494 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 954.04 sec -2013-09-17 23:30:54,500 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 989.38 sec -2013-09-17 23:30:55,507 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 989.38 sec -2013-09-17 23:30:56,514 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1005.72 sec -2013-09-17 23:30:57,520 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1005.72 sec -2013-09-17 23:30:58,526 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1040.57 sec -2013-09-17 23:30:59,532 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1061.53 sec -2013-09-17 23:31:00,538 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1081.7 sec -2013-09-17 23:31:01,544 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1104.84 sec -2013-09-17 23:31:02,550 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1104.84 sec -2013-09-17 23:31:03,556 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1104.84 sec -2013-09-17 23:31:04,562 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1104.84 sec -2013-09-17 23:31:05,568 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1104.84 sec -2013-09-17 23:31:06,573 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1104.84 sec -2013-09-17 23:31:07,579 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1104.84 sec -2013-09-17 23:31:08,585 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1104.84 sec -2013-09-17 23:31:09,591 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1104.84 sec -2013-09-17 23:31:10,597 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1104.84 sec -2013-09-17 23:31:13,163 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1144.43 sec -2013-09-17 23:31:14,168 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1144.43 sec -MapReduce Total cumulative CPU time: 19 minutes 4 seconds 430 msec -Ended Job = job_201309172235_0041 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0042 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:31:17,701 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:31:19,709 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.75 sec -2013-09-17 23:31:20,714 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.75 sec -2013-09-17 23:31:21,719 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.75 sec -2013-09-17 23:31:22,724 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.75 sec -2013-09-17 23:31:23,728 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.75 sec -2013-09-17 23:31:24,733 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.75 sec -2013-09-17 23:31:25,738 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.75 sec -2013-09-17 23:31:26,743 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.75 sec -2013-09-17 23:31:27,748 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.69 sec -2013-09-17 23:31:28,754 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.69 sec -2013-09-17 23:31:29,759 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.69 sec -MapReduce Total cumulative CPU time: 3 seconds 690 msec -Ended Job = job_201309172235_0042 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1144.43 sec HDFS Read: 847155129 HDFS Write: 298189 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.69 sec HDFS Read: 301766 HDFS Write: 437 SUCCESS -Total MapReduce CPU Time Spent: 19 minutes 8 seconds 120 msec -OK -Time taken: 101.705 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей, несколько агрегатных функций.; - - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32236@mturlrep13_201309172331_1074251986.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0043 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:31:47,285 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:31:57,397 Stage-1 map = 7%, reduce = 0% -2013-09-17 23:31:58,403 Stage-1 map = 8%, reduce = 0% -2013-09-17 23:32:01,416 Stage-1 map = 23%, reduce = 0% -2013-09-17 23:32:02,422 Stage-1 map = 24%, reduce = 0% -2013-09-17 23:32:03,436 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 7.06 sec -2013-09-17 23:32:04,442 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 7.06 sec -2013-09-17 23:32:06,734 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.18 sec -2013-09-17 23:32:07,740 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 57.06 sec -2013-09-17 23:32:08,752 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 57.06 sec -2013-09-17 23:32:09,767 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 67.22 sec -2013-09-17 23:32:10,784 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 135.37 sec -2013-09-17 23:32:11,792 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 293.42 sec -2013-09-17 23:32:12,798 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 369.89 sec -2013-09-17 23:32:13,805 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 369.89 sec -2013-09-17 23:32:14,811 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 369.89 sec -2013-09-17 23:32:15,817 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 369.89 sec -2013-09-17 23:32:16,825 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 374.04 sec -2013-09-17 23:32:17,832 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 378.73 sec -2013-09-17 23:32:18,840 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 387.59 sec -2013-09-17 23:32:19,848 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 396.86 sec -2013-09-17 23:32:20,855 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 418.82 sec -2013-09-17 23:32:21,861 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 425.02 sec -2013-09-17 23:32:22,868 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 425.02 sec -MapReduce Total cumulative CPU time: 7 minutes 5 seconds 20 msec -Ended Job = job_201309172235_0043 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0044 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:32:26,376 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:32:28,387 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.98 sec -2013-09-17 23:32:29,392 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.98 sec -2013-09-17 23:32:30,398 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.98 sec -2013-09-17 23:32:31,403 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.98 sec -2013-09-17 23:32:32,408 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.98 sec -2013-09-17 23:32:33,413 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.98 sec -2013-09-17 23:32:34,419 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.98 sec -2013-09-17 23:32:35,425 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.98 sec -2013-09-17 23:32:36,430 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.64 sec -2013-09-17 23:32:37,437 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.64 sec -MapReduce Total cumulative CPU time: 2 seconds 640 msec -Ended Job = job_201309172235_0044 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 425.02 sec HDFS Read: 673615476 HDFS Write: 41947 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.64 sec HDFS Read: 45524 HDFS Write: 131 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 7 seconds 660 msec -OK -Time taken: 60.4 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7129@mturlrep13_201309172332_718169783.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0045 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:32:46,858 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:32:58,979 Stage-1 map = 8%, reduce = 0% -2013-09-17 23:32:59,985 Stage-1 map = 18%, reduce = 0% -2013-09-17 23:33:00,999 Stage-1 map = 22%, reduce = 0% -2013-09-17 23:33:02,014 Stage-1 map = 30%, reduce = 0% -2013-09-17 23:33:03,035 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 39.47 sec -2013-09-17 23:33:04,061 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 127.5 sec -2013-09-17 23:33:05,075 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 389.68 sec -2013-09-17 23:33:06,082 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 414.94 sec -2013-09-17 23:33:07,089 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 414.94 sec -2013-09-17 23:33:08,095 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 414.94 sec -2013-09-17 23:33:09,101 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 414.94 sec -2013-09-17 23:33:10,108 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 414.94 sec -2013-09-17 23:33:11,114 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 414.94 sec -2013-09-17 23:33:12,123 Stage-1 map = 100%, reduce = 15%, Cumulative CPU 414.94 sec -2013-09-17 23:33:13,130 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 414.94 sec -2013-09-17 23:33:14,145 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 436.06 sec -2013-09-17 23:33:15,152 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 446.54 sec -2013-09-17 23:33:16,160 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 475.11 sec -2013-09-17 23:33:17,166 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 475.11 sec -2013-09-17 23:33:18,173 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 475.11 sec -MapReduce Total cumulative CPU time: 7 minutes 55 seconds 110 msec -Ended Job = job_201309172235_0045 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0046 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:33:21,703 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:33:23,714 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.0 sec -2013-09-17 23:33:24,719 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.0 sec -2013-09-17 23:33:25,724 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.0 sec -2013-09-17 23:33:26,729 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.0 sec -2013-09-17 23:33:27,733 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.0 sec -2013-09-17 23:33:28,738 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.0 sec -2013-09-17 23:33:29,743 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.0 sec -2013-09-17 23:33:30,749 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.71 sec -2013-09-17 23:33:31,755 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.71 sec -2013-09-17 23:33:32,761 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.71 sec -MapReduce Total cumulative CPU time: 2 seconds 710 msec -Ended Job = job_201309172235_0046 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 475.11 sec HDFS Read: 673615476 HDFS Write: 41947 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.71 sec HDFS Read: 45524 HDFS Write: 131 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 57 seconds 820 msec -OK -Time taken: 53.498 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13985@mturlrep13_201309172333_212079891.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0047 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:33:42,330 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:33:53,413 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:33:54,432 Stage-1 map = 7%, reduce = 0% -2013-09-17 23:33:55,462 Stage-1 map = 15%, reduce = 0% -2013-09-17 23:33:56,469 Stage-1 map = 23%, reduce = 0% -2013-09-17 23:33:57,482 Stage-1 map = 42%, reduce = 0% -2013-09-17 23:33:58,503 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 9.27 sec -2013-09-17 23:33:59,519 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 114.64 sec -2013-09-17 23:34:00,529 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 388.28 sec -2013-09-17 23:34:01,538 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 412.66 sec -2013-09-17 23:34:02,545 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 412.66 sec -2013-09-17 23:34:03,551 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 412.66 sec -2013-09-17 23:34:04,558 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 412.66 sec -2013-09-17 23:34:05,563 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 412.66 sec -2013-09-17 23:34:06,569 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 412.66 sec -2013-09-17 23:34:07,576 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 412.66 sec -2013-09-17 23:34:08,585 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 412.66 sec -2013-09-17 23:34:09,595 Stage-1 map = 100%, reduce = 45%, Cumulative CPU 422.58 sec -2013-09-17 23:34:10,603 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 442.18 sec -2013-09-17 23:34:11,609 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 464.06 sec -2013-09-17 23:34:12,616 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 470.59 sec -2013-09-17 23:34:13,623 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 470.59 sec -MapReduce Total cumulative CPU time: 7 minutes 50 seconds 590 msec -Ended Job = job_201309172235_0047 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0048 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:34:16,130 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:34:19,144 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-17 23:34:20,150 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-17 23:34:21,155 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-17 23:34:22,160 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-17 23:34:23,165 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-17 23:34:24,169 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-17 23:34:25,174 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-17 23:34:26,180 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.57 sec -2013-09-17 23:34:27,186 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.57 sec -2013-09-17 23:34:28,191 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.57 sec -MapReduce Total cumulative CPU time: 2 seconds 570 msec -Ended Job = job_201309172235_0048 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 470.59 sec HDFS Read: 673615476 HDFS Write: 41947 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.57 sec HDFS Read: 45524 HDFS Write: 131 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 53 seconds 160 msec -OK -Time taken: 53.516 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по строкам.; - - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20877@mturlrep13_201309172334_738248079.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0049 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:34:45,537 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:34:54,639 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:34:55,649 Stage-1 map = 6%, reduce = 0% -2013-09-17 23:34:56,654 Stage-1 map = 9%, reduce = 0% -2013-09-17 23:34:59,668 Stage-1 map = 21%, reduce = 0% -2013-09-17 23:35:01,463 Stage-1 map = 22%, reduce = 0% -2013-09-17 23:35:02,469 Stage-1 map = 43%, reduce = 0% -2013-09-17 23:35:05,662 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 19.08 sec -2013-09-17 23:35:06,670 Stage-1 map = 62%, reduce = 0%, Cumulative CPU 64.52 sec -2013-09-17 23:35:07,678 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 81.56 sec -2013-09-17 23:35:08,689 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 92.37 sec -2013-09-17 23:35:09,697 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 161.79 sec -2013-09-17 23:35:10,703 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 299.59 sec -2013-09-17 23:35:11,709 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 358.46 sec -2013-09-17 23:35:12,716 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 358.46 sec -2013-09-17 23:35:13,722 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 358.46 sec -2013-09-17 23:35:14,729 Stage-1 map = 100%, reduce = 15%, Cumulative CPU 358.46 sec -2013-09-17 23:35:15,735 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 358.46 sec -2013-09-17 23:35:16,745 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 372.22 sec -2013-09-17 23:35:17,752 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 377.09 sec -2013-09-17 23:35:18,760 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 394.56 sec -2013-09-17 23:35:19,766 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 415.94 sec -2013-09-17 23:35:20,773 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 415.94 sec -2013-09-17 23:35:21,779 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 415.94 sec -MapReduce Total cumulative CPU time: 6 minutes 55 seconds 940 msec -Ended Job = job_201309172235_0049 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0050 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:35:24,308 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:35:27,321 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.12 sec -2013-09-17 23:35:28,327 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.12 sec -2013-09-17 23:35:29,332 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.12 sec -2013-09-17 23:35:30,337 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.12 sec -2013-09-17 23:35:31,341 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.12 sec -2013-09-17 23:35:32,346 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.12 sec -2013-09-17 23:35:33,351 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.12 sec -2013-09-17 23:35:34,356 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.12 sec -2013-09-17 23:35:35,362 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.78 sec -2013-09-17 23:35:36,367 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.78 sec -MapReduce Total cumulative CPU time: 2 seconds 780 msec -Ended Job = job_201309172235_0050 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 415.94 sec HDFS Read: 683872957 HDFS Write: 45535 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.78 sec HDFS Read: 49112 HDFS Write: 154 SUCCESS -Total MapReduce CPU Time Spent: 6 minutes 58 seconds 720 msec -OK -Time taken: 61.275 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28396@mturlrep13_201309172335_1701222232.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0051 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:35:47,068 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:35:58,144 Stage-1 map = 4%, reduce = 0% -2013-09-17 23:35:59,176 Stage-1 map = 24%, reduce = 0% -2013-09-17 23:36:00,190 Stage-1 map = 27%, reduce = 0% -2013-09-17 23:36:01,199 Stage-1 map = 35%, reduce = 0% -2013-09-17 23:36:02,206 Stage-1 map = 66%, reduce = 0% -2013-09-17 23:36:03,233 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 156.88 sec -2013-09-17 23:36:04,245 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 396.77 sec -2013-09-17 23:36:05,254 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 408.98 sec -2013-09-17 23:36:06,262 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 408.98 sec -2013-09-17 23:36:07,268 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 408.98 sec -2013-09-17 23:36:08,275 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 408.98 sec -2013-09-17 23:36:09,282 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 408.98 sec -2013-09-17 23:36:10,289 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 408.98 sec -2013-09-17 23:36:11,296 Stage-1 map = 100%, reduce = 15%, Cumulative CPU 408.98 sec -2013-09-17 23:36:12,305 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 408.98 sec -2013-09-17 23:36:13,315 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 424.36 sec -2013-09-17 23:36:14,324 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 435.5 sec -2013-09-17 23:36:15,330 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 463.94 sec -2013-09-17 23:36:16,337 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 471.88 sec -2013-09-17 23:36:17,343 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 471.88 sec -MapReduce Total cumulative CPU time: 7 minutes 51 seconds 880 msec -Ended Job = job_201309172235_0051 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0052 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:36:20,861 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:36:22,870 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.06 sec -2013-09-17 23:36:23,876 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.06 sec -2013-09-17 23:36:24,881 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.06 sec -2013-09-17 23:36:25,886 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.06 sec -2013-09-17 23:36:26,892 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.06 sec -2013-09-17 23:36:27,897 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.06 sec -2013-09-17 23:36:28,902 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.06 sec -2013-09-17 23:36:29,907 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.06 sec -2013-09-17 23:36:30,914 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.64 sec -2013-09-17 23:36:31,919 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.64 sec -MapReduce Total cumulative CPU time: 2 seconds 640 msec -Ended Job = job_201309172235_0052 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 471.88 sec HDFS Read: 683872957 HDFS Write: 45535 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.64 sec HDFS Read: 49112 HDFS Write: 154 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 54 seconds 520 msec -OK -Time taken: 53.698 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2932@mturlrep13_201309172336_301621711.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0053 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:36:42,472 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:36:53,559 Stage-1 map = 3%, reduce = 0% -2013-09-17 23:36:54,579 Stage-1 map = 23%, reduce = 0% -2013-09-17 23:36:55,590 Stage-1 map = 26%, reduce = 0% -2013-09-17 23:36:56,627 Stage-1 map = 32%, reduce = 0% -2013-09-17 23:36:57,654 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 9.3 sec -2013-09-17 23:36:58,675 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 127.12 sec -2013-09-17 23:36:59,688 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 387.4 sec -2013-09-17 23:37:00,695 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 400.81 sec -2013-09-17 23:37:01,701 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 413.22 sec -2013-09-17 23:37:02,706 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 413.22 sec -2013-09-17 23:37:03,712 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 413.22 sec -2013-09-17 23:37:04,717 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 413.22 sec -2013-09-17 23:37:05,723 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 413.22 sec -2013-09-17 23:37:06,729 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 413.22 sec -2013-09-17 23:37:07,737 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 413.22 sec -2013-09-17 23:37:08,746 Stage-1 map = 100%, reduce = 42%, Cumulative CPU 422.72 sec -2013-09-17 23:37:09,754 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 432.93 sec -2013-09-17 23:37:10,760 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 448.36 sec -2013-09-17 23:37:11,768 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 474.11 sec -2013-09-17 23:37:12,775 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 474.11 sec -MapReduce Total cumulative CPU time: 7 minutes 54 seconds 110 msec -Ended Job = job_201309172235_0053 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0054 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-17 23:37:15,239 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:37:18,252 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.03 sec -2013-09-17 23:37:19,258 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.03 sec -2013-09-17 23:37:20,263 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.03 sec -2013-09-17 23:37:21,269 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.03 sec -2013-09-17 23:37:22,274 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.03 sec -2013-09-17 23:37:23,279 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.03 sec -2013-09-17 23:37:24,284 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.03 sec -2013-09-17 23:37:25,289 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.5 sec -2013-09-17 23:37:26,295 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.5 sec -2013-09-17 23:37:27,301 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.5 sec -MapReduce Total cumulative CPU time: 2 seconds 500 msec -Ended Job = job_201309172235_0054 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 474.11 sec HDFS Read: 683872957 HDFS Write: 45535 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.5 sec HDFS Read: 49101 HDFS Write: 154 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 56 seconds 610 msec -OK -Time taken: 53.577 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - - -times: 1 -query: SELECT SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10187@mturlrep13_201309172337_1774589050.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0055 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:37:44,238 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:37:54,302 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:37:55,311 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:37:56,318 Stage-1 map = 5%, reduce = 0% -2013-09-17 23:37:57,890 Stage-1 map = 19%, reduce = 0% -2013-09-17 23:37:58,905 Stage-1 map = 21%, reduce = 0% -2013-09-17 23:37:59,930 Stage-1 map = 25%, reduce = 0% -2013-09-17 23:38:00,937 Stage-1 map = 47%, reduce = 0% -2013-09-17 23:38:01,955 Stage-1 map = 50%, reduce = 0% -2013-09-17 23:38:02,982 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 30.99 sec -2013-09-17 23:38:04,003 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 125.27 sec -2013-09-17 23:38:05,023 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 170.25 sec -2013-09-17 23:38:06,039 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 236.5 sec -2013-09-17 23:38:07,052 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 320.67 sec -2013-09-17 23:38:08,061 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 455.12 sec -2013-09-17 23:38:09,067 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.66 sec -2013-09-17 23:38:10,073 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.66 sec -2013-09-17 23:38:11,079 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.66 sec -2013-09-17 23:38:12,085 Stage-1 map = 100%, reduce = 1%, Cumulative CPU 469.66 sec -2013-09-17 23:38:13,094 Stage-1 map = 100%, reduce = 10%, Cumulative CPU 469.66 sec -2013-09-17 23:38:14,102 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 469.66 sec -2013-09-17 23:38:15,109 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 469.66 sec -2013-09-17 23:38:16,116 Stage-1 map = 100%, reduce = 42%, Cumulative CPU 469.66 sec -2013-09-17 23:38:17,126 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 469.66 sec -2013-09-17 23:38:18,133 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 469.66 sec -2013-09-17 23:38:19,141 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 469.66 sec -2013-09-17 23:38:20,149 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 545.11 sec -2013-09-17 23:38:21,156 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 588.58 sec -2013-09-17 23:38:22,163 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 588.58 sec -MapReduce Total cumulative CPU time: 9 minutes 48 seconds 580 msec -Ended Job = job_201309172235_0055 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0056 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:38:25,617 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:38:32,641 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:38:35,652 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:38:38,663 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:38:41,672 Stage-2 map = 73%, reduce = 0% -2013-09-17 23:38:44,684 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.74 sec -2013-09-17 23:38:45,690 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.74 sec -2013-09-17 23:38:46,696 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.74 sec -2013-09-17 23:38:47,700 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.5 sec -2013-09-17 23:38:48,705 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.5 sec -2013-09-17 23:38:49,711 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.5 sec -2013-09-17 23:38:50,716 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.5 sec -2013-09-17 23:38:51,722 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.5 sec -2013-09-17 23:38:52,728 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.5 sec -2013-09-17 23:38:53,732 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.5 sec -2013-09-17 23:38:54,738 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.5 sec -2013-09-17 23:38:55,744 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.5 sec -2013-09-17 23:38:56,749 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.5 sec -2013-09-17 23:38:57,754 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.5 sec -2013-09-17 23:38:58,759 Stage-2 map = 100%, reduce = 74%, Cumulative CPU 54.5 sec -2013-09-17 23:38:59,764 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.07 sec -2013-09-17 23:39:00,769 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.07 sec -2013-09-17 23:39:01,774 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.07 sec -MapReduce Total cumulative CPU time: 1 minutes 4 seconds 70 msec -Ended Job = job_201309172235_0056 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 588.58 sec HDFS Read: 182281719 HDFS Write: 470238755 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 64.07 sec HDFS Read: 470242477 HDFS Write: 294 SUCCESS -Total MapReduce CPU Time Spent: 10 minutes 52 seconds 650 msec -OK -Time taken: 87.924 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16997@mturlrep13_201309172339_969560501.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0057 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:39:11,317 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:39:22,427 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:39:23,436 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:39:24,449 Stage-1 map = 17%, reduce = 0% -2013-09-17 23:39:25,489 Stage-1 map = 27%, reduce = 0%, Cumulative CPU 238.77 sec -2013-09-17 23:39:26,502 Stage-1 map = 31%, reduce = 0%, Cumulative CPU 238.77 sec -2013-09-17 23:39:27,523 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 240.99 sec -2013-09-17 23:39:28,534 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 257.1 sec -2013-09-17 23:39:29,549 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 285.48 sec -2013-09-17 23:39:30,573 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 324.5 sec -2013-09-17 23:39:31,585 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 386.01 sec -2013-09-17 23:39:32,592 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.29 sec -2013-09-17 23:39:33,597 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.29 sec -2013-09-17 23:39:34,603 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.29 sec -2013-09-17 23:39:35,608 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.29 sec -2013-09-17 23:39:36,614 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.29 sec -2013-09-17 23:39:37,621 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 469.29 sec -2013-09-17 23:39:38,629 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 469.29 sec -2013-09-17 23:39:39,637 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 469.29 sec -2013-09-17 23:39:40,644 Stage-1 map = 100%, reduce = 42%, Cumulative CPU 469.29 sec -2013-09-17 23:39:41,652 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 469.29 sec -2013-09-17 23:39:42,659 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 469.29 sec -2013-09-17 23:39:43,669 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 479.78 sec -2013-09-17 23:39:44,677 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 523.18 sec -2013-09-17 23:39:45,684 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 589.52 sec -2013-09-17 23:39:46,692 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 589.52 sec -2013-09-17 23:39:47,699 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 589.52 sec -MapReduce Total cumulative CPU time: 9 minutes 49 seconds 520 msec -Ended Job = job_201309172235_0057 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0058 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:39:50,186 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:39:58,217 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:40:01,228 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:40:04,240 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:40:07,251 Stage-2 map = 82%, reduce = 0% -2013-09-17 23:40:11,265 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.49 sec -2013-09-17 23:40:12,270 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.49 sec -2013-09-17 23:40:13,276 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.49 sec -2013-09-17 23:40:14,281 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.65 sec -2013-09-17 23:40:15,286 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.65 sec -2013-09-17 23:40:16,291 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.65 sec -2013-09-17 23:40:18,615 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.65 sec -2013-09-17 23:40:19,621 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.65 sec -2013-09-17 23:40:20,627 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.65 sec -2013-09-17 23:40:21,632 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.65 sec -2013-09-17 23:40:22,637 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.65 sec -2013-09-17 23:40:23,643 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.65 sec -2013-09-17 23:40:24,649 Stage-2 map = 100%, reduce = 74%, Cumulative CPU 54.65 sec -2013-09-17 23:40:25,654 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.01 sec -2013-09-17 23:40:26,660 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.01 sec -MapReduce Total cumulative CPU time: 1 minutes 4 seconds 10 msec -Ended Job = job_201309172235_0058 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 589.52 sec HDFS Read: 182281719 HDFS Write: 470238755 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 64.01 sec HDFS Read: 470242477 HDFS Write: 294 SUCCESS -Total MapReduce CPU Time Spent: 10 minutes 53 seconds 530 msec -OK -Time taken: 82.999 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24505@mturlrep13_201309172340_635324541.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0059 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:40:37,367 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:40:48,543 Stage-1 map = 3%, reduce = 0% -2013-09-17 23:40:49,571 Stage-1 map = 11%, reduce = 0% -2013-09-17 23:40:50,591 Stage-1 map = 26%, reduce = 0% -2013-09-17 23:40:51,619 Stage-1 map = 32%, reduce = 0% -2013-09-17 23:40:52,626 Stage-1 map = 43%, reduce = 0% -2013-09-17 23:40:53,651 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 28.49 sec -2013-09-17 23:40:54,672 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 114.43 sec -2013-09-17 23:40:55,681 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 171.5 sec -2013-09-17 23:40:56,691 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 279.83 sec -2013-09-17 23:40:57,706 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 424.96 sec -2013-09-17 23:40:58,712 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.69 sec -2013-09-17 23:40:59,717 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.69 sec -2013-09-17 23:41:00,723 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.69 sec -2013-09-17 23:41:01,728 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.69 sec -2013-09-17 23:41:02,735 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 469.69 sec -2013-09-17 23:41:03,743 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 469.69 sec -2013-09-17 23:41:04,751 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 469.69 sec -2013-09-17 23:41:05,758 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 469.69 sec -2013-09-17 23:41:06,765 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 469.69 sec -2013-09-17 23:41:07,772 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 469.69 sec -2013-09-17 23:41:08,779 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 469.69 sec -2013-09-17 23:41:09,788 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 513.51 sec -2013-09-17 23:41:10,794 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 579.41 sec -2013-09-17 23:41:11,800 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 590.63 sec -2013-09-17 23:41:12,807 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 590.63 sec -MapReduce Total cumulative CPU time: 9 minutes 50 seconds 630 msec -Ended Job = job_201309172235_0059 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0060 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:41:15,266 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:41:23,289 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:41:26,298 Stage-2 map = 37%, reduce = 0%, Cumulative CPU 21.93 sec -2013-09-17 23:41:27,302 Stage-2 map = 37%, reduce = 0%, Cumulative CPU 21.93 sec -2013-09-17 23:41:28,306 Stage-2 map = 37%, reduce = 0%, Cumulative CPU 21.93 sec -2013-09-17 23:41:29,310 Stage-2 map = 55%, reduce = 0%, Cumulative CPU 21.93 sec -2013-09-17 23:41:30,314 Stage-2 map = 55%, reduce = 0%, Cumulative CPU 21.93 sec -2013-09-17 23:41:31,318 Stage-2 map = 55%, reduce = 0%, Cumulative CPU 21.93 sec -2013-09-17 23:41:32,322 Stage-2 map = 73%, reduce = 0%, Cumulative CPU 21.93 sec -2013-09-17 23:41:33,326 Stage-2 map = 73%, reduce = 0%, Cumulative CPU 21.93 sec -2013-09-17 23:41:34,334 Stage-2 map = 73%, reduce = 0%, Cumulative CPU 21.93 sec -2013-09-17 23:41:35,338 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 36.41 sec -2013-09-17 23:41:36,342 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 36.41 sec -2013-09-17 23:41:37,347 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 36.41 sec -2013-09-17 23:41:38,351 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 36.41 sec -2013-09-17 23:41:39,355 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.78 sec -2013-09-17 23:41:40,359 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.78 sec -2013-09-17 23:41:41,363 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.78 sec -2013-09-17 23:41:42,368 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.78 sec -2013-09-17 23:41:43,372 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.78 sec -2013-09-17 23:41:44,376 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.78 sec -2013-09-17 23:41:45,381 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.78 sec -2013-09-17 23:41:46,386 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.78 sec -2013-09-17 23:41:47,390 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.78 sec -2013-09-17 23:41:48,394 Stage-2 map = 100%, reduce = 74%, Cumulative CPU 54.78 sec -2013-09-17 23:41:49,399 Stage-2 map = 100%, reduce = 74%, Cumulative CPU 54.78 sec -2013-09-17 23:41:50,404 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.36 sec -2013-09-17 23:41:51,408 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.36 sec -MapReduce Total cumulative CPU time: 1 minutes 4 seconds 360 msec -Ended Job = job_201309172235_0060 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 590.63 sec HDFS Read: 182281719 HDFS Write: 470238755 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 64.36 sec HDFS Read: 470242477 HDFS Write: 294 SUCCESS -Total MapReduce CPU Time Spent: 10 minutes 54 seconds 990 msec -OK -Time taken: 82.867 seconds, Fetched: 10 row(s) -hive> quit; --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31334@mturlrep13_201309172342_322265508.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0061 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:42:11,682 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:42:21,798 Stage-1 map = 7%, reduce = 0% -2013-09-17 23:42:22,804 Stage-1 map = 8%, reduce = 0% -2013-09-17 23:42:24,816 Stage-1 map = 15%, reduce = 0% -2013-09-17 23:42:27,083 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 228.9 sec -2013-09-17 23:42:28,092 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 228.9 sec -2013-09-17 23:42:29,101 Stage-1 map = 35%, reduce = 0%, Cumulative CPU 228.9 sec -2013-09-17 23:42:31,162 Stage-1 map = 42%, reduce = 0%, Cumulative CPU 228.9 sec -2013-09-17 23:42:32,201 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 231.63 sec -2013-09-17 23:42:33,214 Stage-1 map = 51%, reduce = 0%, Cumulative CPU 234.97 sec -2013-09-17 23:42:34,223 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 254.73 sec -2013-09-17 23:42:35,315 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 254.73 sec -2013-09-17 23:42:36,327 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 260.85 sec -2013-09-17 23:42:37,339 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 292.13 sec -2013-09-17 23:42:38,347 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 320.32 sec -2013-09-17 23:42:39,358 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 396.48 sec -2013-09-17 23:42:40,364 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 484.46 sec -2013-09-17 23:42:41,369 Stage-1 map = 100%, reduce = 2%, Cumulative CPU 517.16 sec -2013-09-17 23:42:42,376 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 517.16 sec -2013-09-17 23:42:43,383 Stage-1 map = 100%, reduce = 16%, Cumulative CPU 517.16 sec -2013-09-17 23:42:44,391 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 517.16 sec -2013-09-17 23:42:45,397 Stage-1 map = 100%, reduce = 37%, Cumulative CPU 517.16 sec -2013-09-17 23:42:46,404 Stage-1 map = 100%, reduce = 48%, Cumulative CPU 517.16 sec -2013-09-17 23:42:47,411 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 517.16 sec -2013-09-17 23:42:48,419 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 517.16 sec -2013-09-17 23:42:49,427 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 528.88 sec -2013-09-17 23:42:50,435 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 540.66 sec -2013-09-17 23:42:51,442 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 589.36 sec -2013-09-17 23:42:52,449 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 626.57 sec -2013-09-17 23:42:53,455 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 626.57 sec -2013-09-17 23:42:54,461 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 651.04 sec -2013-09-17 23:42:55,468 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 651.04 sec -MapReduce Total cumulative CPU time: 10 minutes 51 seconds 40 msec -Ended Job = job_201309172235_0061 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0062 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:42:57,983 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:43:06,006 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:43:09,017 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:43:12,028 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:43:15,039 Stage-2 map = 73%, reduce = 0% -2013-09-17 23:43:18,050 Stage-2 map = 82%, reduce = 0% -2013-09-17 23:43:19,055 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.26 sec -2013-09-17 23:43:20,061 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.26 sec -2013-09-17 23:43:21,068 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 53.98 sec -2013-09-17 23:43:22,073 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 53.98 sec -2013-09-17 23:43:23,077 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 53.98 sec -2013-09-17 23:43:24,082 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 53.98 sec -2013-09-17 23:43:25,087 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 53.98 sec -2013-09-17 23:43:26,092 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 53.98 sec -2013-09-17 23:43:27,096 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 53.98 sec -2013-09-17 23:43:28,101 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 53.98 sec -2013-09-17 23:43:29,106 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 53.98 sec -2013-09-17 23:43:30,110 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 53.98 sec -2013-09-17 23:43:31,114 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 53.98 sec -2013-09-17 23:43:32,119 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 53.98 sec -2013-09-17 23:43:33,124 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 63.69 sec -2013-09-17 23:43:34,129 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 63.69 sec -2013-09-17 23:43:35,135 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 63.69 sec -MapReduce Total cumulative CPU time: 1 minutes 3 seconds 690 msec -Ended Job = job_201309172235_0062 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 651.04 sec HDFS Read: 840872065 HDFS Write: 470236967 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 63.69 sec HDFS Read: 470240689 HDFS Write: 291 SUCCESS -Total MapReduce CPU Time Spent: 11 minutes 54 seconds 730 msec -OK -Time taken: 94.281 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6246@mturlrep13_201309172343_752416466.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0063 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:43:46,269 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:43:55,322 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:43:56,328 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:43:57,335 Stage-1 map = 8%, reduce = 0% -2013-09-17 23:43:58,347 Stage-1 map = 15%, reduce = 0% -2013-09-17 23:43:59,353 Stage-1 map = 20%, reduce = 0% -2013-09-17 23:44:00,368 Stage-1 map = 29%, reduce = 0% -2013-09-17 23:44:01,375 Stage-1 map = 40%, reduce = 0% -2013-09-17 23:44:02,399 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 11.32 sec -2013-09-17 23:44:03,419 Stage-1 map = 54%, reduce = 0%, Cumulative CPU 45.05 sec -2013-09-17 23:44:04,431 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 68.44 sec -2013-09-17 23:44:05,439 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 146.24 sec -2013-09-17 23:44:06,446 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 213.64 sec -2013-09-17 23:44:07,457 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 292.27 sec -2013-09-17 23:44:08,464 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 443.09 sec -2013-09-17 23:44:09,470 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 549.17 sec -2013-09-17 23:44:10,476 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 549.17 sec -2013-09-17 23:44:11,482 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 549.17 sec -2013-09-17 23:44:12,489 Stage-1 map = 100%, reduce = 1%, Cumulative CPU 549.17 sec -2013-09-17 23:44:13,497 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 549.17 sec -2013-09-17 23:44:14,506 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 549.17 sec -2013-09-17 23:44:15,513 Stage-1 map = 100%, reduce = 34%, Cumulative CPU 549.17 sec -2013-09-17 23:44:16,520 Stage-1 map = 100%, reduce = 45%, Cumulative CPU 549.17 sec -2013-09-17 23:44:17,527 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 549.17 sec -2013-09-17 23:44:18,534 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 549.17 sec -2013-09-17 23:44:19,541 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 549.17 sec -2013-09-17 23:44:20,550 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 561.32 sec -2013-09-17 23:44:21,556 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 585.6 sec -2013-09-17 23:44:22,563 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 687.26 sec -2013-09-17 23:44:23,570 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 687.26 sec -2013-09-17 23:44:24,576 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 687.26 sec -MapReduce Total cumulative CPU time: 11 minutes 27 seconds 260 msec -Ended Job = job_201309172235_0063 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0064 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:44:28,140 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:44:35,163 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:44:38,174 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:44:41,183 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:44:44,194 Stage-2 map = 82%, reduce = 0% -2013-09-17 23:44:47,205 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.21 sec -2013-09-17 23:44:48,211 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.21 sec -2013-09-17 23:44:49,215 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.21 sec -2013-09-17 23:44:50,916 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.21 sec -2013-09-17 23:44:51,921 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.96 sec -2013-09-17 23:44:52,925 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.96 sec -2013-09-17 23:44:53,931 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.96 sec -2013-09-17 23:44:54,937 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.96 sec -2013-09-17 23:44:55,942 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.96 sec -2013-09-17 23:44:56,947 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.96 sec -2013-09-17 23:44:57,951 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.96 sec -2013-09-17 23:44:58,956 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.96 sec -2013-09-17 23:44:59,960 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 54.96 sec -2013-09-17 23:45:00,964 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 54.96 sec -2013-09-17 23:45:01,970 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.36 sec -2013-09-17 23:45:02,975 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.36 sec -2013-09-17 23:45:03,981 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.36 sec -MapReduce Total cumulative CPU time: 1 minutes 4 seconds 360 msec -Ended Job = job_201309172235_0064 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 687.26 sec HDFS Read: 840872065 HDFS Write: 470236967 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 64.36 sec HDFS Read: 470240689 HDFS Write: 291 SUCCESS -Total MapReduce CPU Time Spent: 12 minutes 31 seconds 620 msec -OK -Time taken: 86.991 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13649@mturlrep13_201309172345_1690142009.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0065 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:45:13,718 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:45:24,809 Stage-1 map = 3%, reduce = 0% -2013-09-17 23:45:25,833 Stage-1 map = 9%, reduce = 0%, Cumulative CPU 168.95 sec -2013-09-17 23:45:26,845 Stage-1 map = 12%, reduce = 0%, Cumulative CPU 168.95 sec -2013-09-17 23:45:27,853 Stage-1 map = 23%, reduce = 0%, Cumulative CPU 168.95 sec -2013-09-17 23:45:28,867 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 168.95 sec -2013-09-17 23:45:29,874 Stage-1 map = 40%, reduce = 0%, Cumulative CPU 168.95 sec -2013-09-17 23:45:30,887 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 173.1 sec -2013-09-17 23:45:31,894 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 190.37 sec -2013-09-17 23:45:32,900 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 207.59 sec -2013-09-17 23:45:33,913 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 266.9 sec -2013-09-17 23:45:34,921 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 286.81 sec -2013-09-17 23:45:35,928 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 319.49 sec -2013-09-17 23:45:36,935 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 429.06 sec -2013-09-17 23:45:37,943 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 545.45 sec -2013-09-17 23:45:38,949 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 545.45 sec -2013-09-17 23:45:39,955 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 545.45 sec -2013-09-17 23:45:40,961 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 545.45 sec -2013-09-17 23:45:41,968 Stage-1 map = 100%, reduce = 5%, Cumulative CPU 545.45 sec -2013-09-17 23:45:42,978 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 545.45 sec -2013-09-17 23:45:43,985 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 545.45 sec -2013-09-17 23:45:44,992 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 545.45 sec -2013-09-17 23:45:45,999 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 545.45 sec -2013-09-17 23:45:47,006 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 545.45 sec -2013-09-17 23:45:48,013 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 545.45 sec -2013-09-17 23:45:49,019 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 545.45 sec -2013-09-17 23:45:50,027 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 595.55 sec -2013-09-17 23:45:51,034 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 670.19 sec -2013-09-17 23:45:52,040 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 682.37 sec -2013-09-17 23:45:53,047 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 682.37 sec -MapReduce Total cumulative CPU time: 11 minutes 22 seconds 370 msec -Ended Job = job_201309172235_0065 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0066 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:45:56,566 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:46:03,591 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:46:06,601 Stage-2 map = 28%, reduce = 0% -2013-09-17 23:46:07,606 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:46:10,617 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:46:13,628 Stage-2 map = 73%, reduce = 0% -2013-09-17 23:46:16,640 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.88 sec -2013-09-17 23:46:17,646 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.88 sec -2013-09-17 23:46:18,651 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 25.88 sec -2013-09-17 23:46:19,840 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.97 sec -2013-09-17 23:46:20,844 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.97 sec -2013-09-17 23:46:21,848 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.97 sec -2013-09-17 23:46:22,853 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.97 sec -2013-09-17 23:46:23,857 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 54.97 sec -2013-09-17 23:46:24,862 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 54.97 sec -2013-09-17 23:46:25,866 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 56.42 sec -2013-09-17 23:46:26,871 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 56.42 sec -2013-09-17 23:46:27,875 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 56.42 sec -2013-09-17 23:46:28,879 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 56.42 sec -2013-09-17 23:46:29,883 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 56.42 sec -2013-09-17 23:46:30,888 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 56.42 sec -2013-09-17 23:46:31,893 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.4 sec -2013-09-17 23:46:32,899 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 64.4 sec -MapReduce Total cumulative CPU time: 1 minutes 4 seconds 400 msec -Ended Job = job_201309172235_0066 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 682.37 sec HDFS Read: 840872065 HDFS Write: 470236967 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 64.4 sec HDFS Read: 470240689 HDFS Write: 291 SUCCESS -Total MapReduce CPU Time Spent: 12 minutes 26 seconds 770 msec -OK -Time taken: 87.084 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация чуть сложнее.; - - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21386@mturlrep13_201309172346_1552848937.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0067 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:46:53,783 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:47:03,859 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:47:05,878 Stage-1 map = 5%, reduce = 0% -2013-09-17 23:47:07,199 Stage-1 map = 21%, reduce = 0% -2013-09-17 23:47:08,207 Stage-1 map = 24%, reduce = 0% -2013-09-17 23:47:09,214 Stage-1 map = 30%, reduce = 0% -2013-09-17 23:47:10,223 Stage-1 map = 48%, reduce = 0% -2013-09-17 23:47:11,239 Stage-1 map = 50%, reduce = 0% -2013-09-17 23:47:12,265 Stage-1 map = 62%, reduce = 0%, Cumulative CPU 41.92 sec -2013-09-17 23:47:13,282 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 108.6 sec -2013-09-17 23:47:14,295 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 181.12 sec -2013-09-17 23:47:15,307 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 247.1 sec -2013-09-17 23:47:16,313 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 290.6 sec -2013-09-17 23:47:17,320 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 460.07 sec -2013-09-17 23:47:18,326 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 491.41 sec -2013-09-17 23:47:19,332 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 491.41 sec -2013-09-17 23:47:20,338 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 491.41 sec -2013-09-17 23:47:21,345 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 491.41 sec -2013-09-17 23:47:22,352 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 491.41 sec -2013-09-17 23:47:23,359 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 491.41 sec -2013-09-17 23:47:24,369 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 491.41 sec -2013-09-17 23:47:25,377 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 491.41 sec -2013-09-17 23:47:26,389 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 543.54 sec -2013-09-17 23:47:27,397 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 543.54 sec -2013-09-17 23:47:28,404 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 543.54 sec -2013-09-17 23:47:29,411 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 560.73 sec -2013-09-17 23:47:30,418 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 619.13 sec -2013-09-17 23:47:31,425 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 619.13 sec -2013-09-17 23:47:32,432 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 619.13 sec -MapReduce Total cumulative CPU time: 10 minutes 19 seconds 130 msec -Ended Job = job_201309172235_0067 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0068 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:47:35,954 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:47:42,978 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:47:45,991 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:47:49,003 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:47:52,014 Stage-2 map = 73%, reduce = 0% -2013-09-17 23:47:55,025 Stage-2 map = 82%, reduce = 0% -2013-09-17 23:47:57,034 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 27.07 sec -2013-09-17 23:47:58,040 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 27.07 sec -2013-09-17 23:47:59,046 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 27.07 sec -2013-09-17 23:48:00,051 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 27.07 sec -2013-09-17 23:48:01,056 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 58.79 sec -2013-09-17 23:48:02,060 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 58.79 sec -2013-09-17 23:48:03,066 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 58.79 sec -2013-09-17 23:48:04,072 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 58.79 sec -2013-09-17 23:48:05,077 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 58.79 sec -2013-09-17 23:48:06,082 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 58.79 sec -2013-09-17 23:48:07,087 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 58.79 sec -2013-09-17 23:48:08,092 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 58.79 sec -2013-09-17 23:48:09,096 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 58.79 sec -2013-09-17 23:48:10,102 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 58.79 sec -2013-09-17 23:48:11,107 Stage-2 map = 100%, reduce = 74%, Cumulative CPU 58.79 sec -2013-09-17 23:48:12,112 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 68.73 sec -2013-09-17 23:48:13,119 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 68.73 sec -2013-09-17 23:48:14,124 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 68.73 sec -MapReduce Total cumulative CPU time: 1 minutes 8 seconds 730 msec -Ended Job = job_201309172235_0068 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 619.13 sec HDFS Read: 200023859 HDFS Write: 499984003 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 68.73 sec HDFS Read: 499987725 HDFS Write: 286 SUCCESS -Total MapReduce CPU Time Spent: 11 minutes 27 seconds 860 msec -OK -Time taken: 91.05 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28179@mturlrep13_201309172348_1562230786.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0069 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:48:23,881 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:48:35,995 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:48:37,001 Stage-1 map = 16%, reduce = 0% -2013-09-17 23:48:38,011 Stage-1 map = 20%, reduce = 0% -2013-09-17 23:48:39,018 Stage-1 map = 24%, reduce = 0% -2013-09-17 23:48:40,024 Stage-1 map = 39%, reduce = 0% -2013-09-17 23:48:41,052 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 10.64 sec -2013-09-17 23:48:42,094 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 99.96 sec -2013-09-17 23:48:43,114 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 158.73 sec -2013-09-17 23:48:44,140 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 239.0 sec -2013-09-17 23:48:45,149 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 313.14 sec -2013-09-17 23:48:46,156 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 453.86 sec -2013-09-17 23:48:47,162 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 503.27 sec -2013-09-17 23:48:48,168 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 503.27 sec -2013-09-17 23:48:49,174 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 503.27 sec -2013-09-17 23:48:50,181 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 503.27 sec -2013-09-17 23:48:51,188 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 503.27 sec -2013-09-17 23:48:52,194 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 503.27 sec -2013-09-17 23:48:53,201 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 503.27 sec -2013-09-17 23:48:54,207 Stage-1 map = 100%, reduce = 45%, Cumulative CPU 503.27 sec -2013-09-17 23:48:55,214 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 503.27 sec -2013-09-17 23:48:56,221 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 503.27 sec -2013-09-17 23:48:57,228 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 503.27 sec -2013-09-17 23:48:58,237 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 548.17 sec -2013-09-17 23:48:59,244 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 606.99 sec -2013-09-17 23:49:00,251 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 630.32 sec -2013-09-17 23:49:01,257 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 630.32 sec -MapReduce Total cumulative CPU time: 10 minutes 30 seconds 320 msec -Ended Job = job_201309172235_0069 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0070 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:49:04,779 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:49:11,801 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:49:14,812 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:49:17,821 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:49:20,831 Stage-2 map = 73%, reduce = 0% -2013-09-17 23:49:23,839 Stage-2 map = 82%, reduce = 0% -2013-09-17 23:49:26,850 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 52.88 sec -2013-09-17 23:49:27,856 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 52.88 sec -2013-09-17 23:49:28,860 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 52.88 sec -2013-09-17 23:49:29,866 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 52.88 sec -2013-09-17 23:49:32,220 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 60.11 sec -2013-09-17 23:49:33,226 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 60.11 sec -2013-09-17 23:49:34,231 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 60.11 sec -2013-09-17 23:49:35,237 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 60.11 sec -2013-09-17 23:49:36,242 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 60.11 sec -2013-09-17 23:49:37,248 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 60.11 sec -2013-09-17 23:49:38,253 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 60.11 sec -2013-09-17 23:49:39,259 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 60.11 sec -2013-09-17 23:49:40,265 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 60.11 sec -2013-09-17 23:49:41,270 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 70.2 sec -2013-09-17 23:49:42,276 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 70.2 sec -2013-09-17 23:49:43,281 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 70.2 sec -MapReduce Total cumulative CPU time: 1 minutes 10 seconds 200 msec -Ended Job = job_201309172235_0070 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 630.32 sec HDFS Read: 200023859 HDFS Write: 499984003 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 70.2 sec HDFS Read: 499987725 HDFS Write: 286 SUCCESS -Total MapReduce CPU Time Spent: 11 minutes 40 seconds 520 msec -OK -Time taken: 87.267 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2628@mturlrep13_201309172349_1108098215.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0071 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:49:52,985 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:50:05,128 Stage-1 map = 2%, reduce = 0% -2013-09-17 23:50:06,133 Stage-1 map = 15%, reduce = 0% -2013-09-17 23:50:07,151 Stage-1 map = 22%, reduce = 0% -2013-09-17 23:50:08,158 Stage-1 map = 23%, reduce = 0% -2013-09-17 23:50:09,175 Stage-1 map = 50%, reduce = 0% -2013-09-17 23:50:10,194 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 31.85 sec -2013-09-17 23:50:11,210 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 65.05 sec -2013-09-17 23:50:12,225 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 147.44 sec -2013-09-17 23:50:13,259 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 212.15 sec -2013-09-17 23:50:14,277 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 269.6 sec -2013-09-17 23:50:15,286 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 409.48 sec -2013-09-17 23:50:16,293 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 490.26 sec -2013-09-17 23:50:17,299 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 507.73 sec -2013-09-17 23:50:18,304 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 507.73 sec -2013-09-17 23:50:19,311 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 507.73 sec -2013-09-17 23:50:20,317 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 507.73 sec -2013-09-17 23:50:21,323 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 507.73 sec -2013-09-17 23:50:22,331 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 507.73 sec -2013-09-17 23:50:23,337 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 507.73 sec -2013-09-17 23:50:24,344 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 507.73 sec -2013-09-17 23:50:25,351 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 507.73 sec -2013-09-17 23:50:26,357 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 507.73 sec -2013-09-17 23:50:27,367 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 592.09 sec -2013-09-17 23:50:28,374 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 620.73 sec -2013-09-17 23:50:29,381 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 636.3 sec -2013-09-17 23:50:30,387 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 636.3 sec -MapReduce Total cumulative CPU time: 10 minutes 36 seconds 300 msec -Ended Job = job_201309172235_0071 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0072 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:50:33,867 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:50:40,890 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:50:43,901 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:50:46,911 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:50:49,921 Stage-2 map = 73%, reduce = 0% -2013-09-17 23:50:53,934 Stage-2 map = 82%, reduce = 0% -2013-09-17 23:50:55,943 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 27.87 sec -2013-09-17 23:50:56,949 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 27.87 sec -2013-09-17 23:50:57,955 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 27.87 sec -2013-09-17 23:50:59,680 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 27.87 sec -2013-09-17 23:51:00,685 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 61.27 sec -2013-09-17 23:51:01,690 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 61.27 sec -2013-09-17 23:51:02,695 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 61.27 sec -2013-09-17 23:51:03,700 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 61.27 sec -2013-09-17 23:51:04,705 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 61.27 sec -2013-09-17 23:51:05,710 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 61.27 sec -2013-09-17 23:51:06,715 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 61.27 sec -2013-09-17 23:51:07,719 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 61.27 sec -2013-09-17 23:51:08,725 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 61.27 sec -2013-09-17 23:51:09,730 Stage-2 map = 100%, reduce = 74%, Cumulative CPU 61.27 sec -2013-09-17 23:51:10,735 Stage-2 map = 100%, reduce = 74%, Cumulative CPU 61.27 sec -2013-09-17 23:51:11,740 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 71.17 sec -2013-09-17 23:51:12,746 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 71.17 sec -MapReduce Total cumulative CPU time: 1 minutes 11 seconds 170 msec -Ended Job = job_201309172235_0072 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 636.3 sec HDFS Read: 200023859 HDFS Write: 499984003 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 71.17 sec HDFS Read: 499987725 HDFS Write: 286 SUCCESS -Total MapReduce CPU Time Spent: 11 minutes 47 seconds 470 msec -OK -Time taken: 87.731 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по числу и строке, большое количество ключей.; - - -times: 1 -query: SELECT UserID, count(*) AS c FROM hits_100m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10539@mturlrep13_201309172351_1364032363.txt -hive> SELECT UserID, count(*) AS c FROM hits_100m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0073 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:51:32,893 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:51:43,007 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:51:45,021 Stage-1 map = 4%, reduce = 0% -2013-09-17 23:51:47,070 Stage-1 map = 11%, reduce = 0% -2013-09-17 23:51:48,076 Stage-1 map = 14%, reduce = 0% -2013-09-17 23:51:50,095 Stage-1 map = 33%, reduce = 0% -2013-09-17 23:51:51,260 Stage-1 map = 39%, reduce = 0% -2013-09-17 23:51:53,285 Stage-1 map = 51%, reduce = 0% -2013-09-17 23:51:54,299 Stage-1 map = 58%, reduce = 0% -2013-09-17 23:51:55,943 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 13.64 sec -2013-09-17 23:51:56,974 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 25.01 sec -2013-09-17 23:51:57,984 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 38.32 sec -2013-09-17 23:51:58,990 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 80.19 sec -2013-09-17 23:51:59,996 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 80.19 sec -2013-09-17 23:52:01,007 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 95.79 sec -2013-09-17 23:52:02,036 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 193.09 sec -2013-09-17 23:52:03,054 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 243.99 sec -2013-09-17 23:52:04,061 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 426.96 sec -2013-09-17 23:52:05,067 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 468.05 sec -2013-09-17 23:52:06,074 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 487.46 sec -2013-09-17 23:52:07,080 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 685.94 sec -2013-09-17 23:52:08,087 Stage-1 map = 100%, reduce = 2%, Cumulative CPU 685.94 sec -2013-09-17 23:52:09,093 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 685.94 sec -2013-09-17 23:52:10,100 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 685.94 sec -2013-09-17 23:52:11,107 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 685.94 sec -2013-09-17 23:52:12,114 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 685.94 sec -2013-09-17 23:52:13,122 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 685.94 sec -2013-09-17 23:52:14,128 Stage-1 map = 100%, reduce = 40%, Cumulative CPU 685.94 sec -2013-09-17 23:52:15,135 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 685.94 sec -2013-09-17 23:52:16,141 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 685.94 sec -2013-09-17 23:52:17,148 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 685.94 sec -2013-09-17 23:52:18,153 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 685.94 sec -2013-09-17 23:52:19,170 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 700.91 sec -2013-09-17 23:52:20,177 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 810.21 sec -2013-09-17 23:52:21,183 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 825.67 sec -2013-09-17 23:52:22,188 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 825.67 sec -2013-09-17 23:52:23,194 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 855.09 sec -2013-09-17 23:52:24,198 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 855.09 sec -MapReduce Total cumulative CPU time: 14 minutes 15 seconds 90 msec -Ended Job = job_201309172235_0073 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0074 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:52:27,714 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:52:40,754 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:52:48,779 Stage-2 map = 27%, reduce = 0% -2013-09-17 23:52:49,783 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:52:52,793 Stage-2 map = 45%, reduce = 0% -2013-09-17 23:52:55,803 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:53:01,889 Stage-2 map = 73%, reduce = 0% -2013-09-17 23:53:07,908 Stage-2 map = 82%, reduce = 0% -2013-09-17 23:53:13,929 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 55.32 sec -2013-09-17 23:53:14,935 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 55.32 sec -2013-09-17 23:53:15,940 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 55.32 sec -2013-09-17 23:53:16,946 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 55.32 sec -2013-09-17 23:53:17,951 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 55.32 sec -2013-09-17 23:53:18,956 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 55.32 sec -2013-09-17 23:53:19,961 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 118.9 sec -2013-09-17 23:53:20,966 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 118.9 sec -2013-09-17 23:53:21,971 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 118.9 sec -2013-09-17 23:53:22,975 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 118.9 sec -2013-09-17 23:53:23,979 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 118.9 sec -2013-09-17 23:53:24,983 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.9 sec -2013-09-17 23:53:25,987 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.9 sec -2013-09-17 23:53:26,992 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 123.85 sec -2013-09-17 23:53:27,996 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 123.85 sec -2013-09-17 23:53:29,001 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 123.85 sec -2013-09-17 23:53:30,005 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 123.85 sec -2013-09-17 23:53:31,010 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 123.85 sec -2013-09-17 23:53:32,014 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 123.85 sec -2013-09-17 23:53:33,019 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 123.85 sec -2013-09-17 23:53:34,029 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 133.7 sec -2013-09-17 23:53:35,034 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 133.7 sec -MapReduce Total cumulative CPU time: 2 minutes 13 seconds 700 msec -Ended Job = job_201309172235_0074 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 855.09 sec HDFS Read: 663537300 HDFS Write: 480155183 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 133.7 sec HDFS Read: 480158905 HDFS Write: 251 SUCCESS -Total MapReduce CPU Time Spent: 16 minutes 28 seconds 790 msec -OK -Time taken: 132.697 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT UserID, count(*) AS c FROM hits_100m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17435@mturlrep13_201309172353_888901844.txt -hive> SELECT UserID, count(*) AS c FROM hits_100m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0075 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:53:45,965 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:53:57,063 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:53:58,076 Stage-1 map = 9%, reduce = 0% -2013-09-17 23:53:59,095 Stage-1 map = 20%, reduce = 0% -2013-09-17 23:54:00,108 Stage-1 map = 23%, reduce = 0% -2013-09-17 23:54:01,121 Stage-1 map = 37%, reduce = 0% -2013-09-17 23:54:02,132 Stage-1 map = 58%, reduce = 0% -2013-09-17 23:54:03,151 Stage-1 map = 59%, reduce = 0% -2013-09-17 23:54:04,163 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 10.96 sec -2013-09-17 23:54:05,184 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 51.58 sec -2013-09-17 23:54:06,196 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 65.4 sec -2013-09-17 23:54:07,205 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 123.94 sec -2013-09-17 23:54:08,310 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 137.81 sec -2013-09-17 23:54:09,332 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 234.2 sec -2013-09-17 23:54:10,342 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 283.15 sec -2013-09-17 23:54:11,350 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 399.5 sec -2013-09-17 23:54:12,356 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 459.78 sec -2013-09-17 23:54:13,365 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 479.7 sec -2013-09-17 23:54:14,372 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 479.7 sec -2013-09-17 23:54:15,378 Stage-1 map = 98%, reduce = 5%, Cumulative CPU 553.18 sec -2013-09-17 23:54:16,383 Stage-1 map = 99%, reduce = 17%, Cumulative CPU 654.99 sec -2013-09-17 23:54:17,388 Stage-1 map = 100%, reduce = 23%, Cumulative CPU 681.14 sec -2013-09-17 23:54:18,394 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 681.14 sec -2013-09-17 23:54:19,399 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 681.14 sec -2013-09-17 23:54:20,405 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 681.14 sec -2013-09-17 23:54:21,411 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 681.14 sec -2013-09-17 23:54:22,417 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 681.14 sec -2013-09-17 23:54:23,423 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 681.14 sec -2013-09-17 23:54:24,430 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 681.14 sec -2013-09-17 23:54:25,442 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 681.14 sec -2013-09-17 23:54:26,448 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 681.14 sec -2013-09-17 23:54:27,456 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 758.79 sec -2013-09-17 23:54:28,462 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 758.79 sec -2013-09-17 23:54:29,468 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 758.79 sec -2013-09-17 23:54:30,475 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 758.79 sec -2013-09-17 23:54:31,481 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 792.19 sec -2013-09-17 23:54:32,487 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 841.44 sec -2013-09-17 23:54:33,493 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 849.83 sec -2013-09-17 23:54:34,499 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 849.83 sec -MapReduce Total cumulative CPU time: 14 minutes 9 seconds 830 msec -Ended Job = job_201309172235_0075 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0076 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:54:36,978 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:54:51,026 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:54:57,047 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:55:04,073 Stage-2 map = 47%, reduce = 0% -2013-09-17 23:55:07,084 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:55:10,096 Stage-2 map = 65%, reduce = 0% -2013-09-17 23:55:14,109 Stage-2 map = 73%, reduce = 0% -2013-09-17 23:55:20,130 Stage-2 map = 82%, reduce = 0% -2013-09-17 23:55:23,142 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 54.08 sec -2013-09-17 23:55:24,147 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 54.08 sec -2013-09-17 23:55:25,151 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 54.08 sec -2013-09-17 23:55:26,156 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 54.08 sec -2013-09-17 23:55:27,160 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 112.2 sec -2013-09-17 23:55:28,165 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 112.2 sec -2013-09-17 23:55:29,170 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 112.2 sec -2013-09-17 23:55:30,175 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 112.2 sec -2013-09-17 23:55:31,180 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 112.2 sec -2013-09-17 23:55:32,185 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 118.52 sec -2013-09-17 23:55:33,190 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 118.52 sec -2013-09-17 23:55:34,194 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 118.52 sec -2013-09-17 23:55:35,198 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 118.52 sec -2013-09-17 23:55:36,202 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 118.52 sec -2013-09-17 23:55:37,207 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 118.52 sec -2013-09-17 23:55:38,212 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 118.52 sec -2013-09-17 23:55:39,217 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 118.52 sec -2013-09-17 23:55:40,222 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.52 sec -2013-09-17 23:55:41,227 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.52 sec -2013-09-17 23:55:42,233 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.52 sec -2013-09-17 23:55:43,248 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.52 sec -2013-09-17 23:55:44,252 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.52 sec -2013-09-17 23:55:45,257 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.52 sec -2013-09-17 23:55:46,262 Stage-2 map = 100%, reduce = 80%, Cumulative CPU 118.52 sec -2013-09-17 23:55:47,267 Stage-2 map = 100%, reduce = 80%, Cumulative CPU 118.52 sec -2013-09-17 23:55:48,272 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 133.0 sec -2013-09-17 23:55:49,278 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 133.0 sec -MapReduce Total cumulative CPU time: 2 minutes 13 seconds 0 msec -Ended Job = job_201309172235_0076 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 849.83 sec HDFS Read: 663537300 HDFS Write: 480155183 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 133.0 sec HDFS Read: 480158905 HDFS Write: 251 SUCCESS -Total MapReduce CPU Time Spent: 16 minutes 22 seconds 830 msec -OK -Time taken: 132.306 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT UserID, count(*) AS c FROM hits_100m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25039@mturlrep13_201309172355_777652129.txt -hive> SELECT UserID, count(*) AS c FROM hits_100m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0077 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:55:58,731 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:56:11,863 Stage-1 map = 12%, reduce = 0% -2013-09-17 23:56:12,877 Stage-1 map = 20%, reduce = 0% -2013-09-17 23:56:13,887 Stage-1 map = 22%, reduce = 0% -2013-09-17 23:56:14,900 Stage-1 map = 40%, reduce = 0% -2013-09-17 23:56:15,915 Stage-1 map = 57%, reduce = 0% -2013-09-17 23:56:16,938 Stage-1 map = 58%, reduce = 0%, Cumulative CPU 10.56 sec -2013-09-17 23:56:17,950 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 10.56 sec -2013-09-17 23:56:18,963 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 10.56 sec -2013-09-17 23:56:19,993 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 52.39 sec -2013-09-17 23:56:20,999 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 109.68 sec -2013-09-17 23:56:22,006 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 171.28 sec -2013-09-17 23:56:23,012 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 218.43 sec -2013-09-17 23:56:24,036 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 285.81 sec -2013-09-17 23:56:25,051 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 343.66 sec -2013-09-17 23:56:26,057 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 482.51 sec -2013-09-17 23:56:27,063 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 635.88 sec -2013-09-17 23:56:28,070 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 635.88 sec -2013-09-17 23:56:29,077 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 651.74 sec -2013-09-17 23:56:30,084 Stage-1 map = 100%, reduce = 7%, Cumulative CPU 682.92 sec -2013-09-17 23:56:31,091 Stage-1 map = 100%, reduce = 19%, Cumulative CPU 682.92 sec -2013-09-17 23:56:32,097 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 682.92 sec -2013-09-17 23:56:33,104 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 682.92 sec -2013-09-17 23:56:34,110 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 682.92 sec -2013-09-17 23:56:35,117 Stage-1 map = 100%, reduce = 40%, Cumulative CPU 682.92 sec -2013-09-17 23:56:36,123 Stage-1 map = 100%, reduce = 40%, Cumulative CPU 682.92 sec -2013-09-17 23:56:37,129 Stage-1 map = 100%, reduce = 40%, Cumulative CPU 682.92 sec -2013-09-17 23:56:38,136 Stage-1 map = 100%, reduce = 43%, Cumulative CPU 682.92 sec -2013-09-17 23:56:39,142 Stage-1 map = 100%, reduce = 54%, Cumulative CPU 682.92 sec -2013-09-17 23:56:40,150 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 682.92 sec -2013-09-17 23:56:41,155 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 682.92 sec -2013-09-17 23:56:42,163 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 725.63 sec -2013-09-17 23:56:43,170 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 725.63 sec -2013-09-17 23:56:44,176 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 725.63 sec -2013-09-17 23:56:45,182 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 755.1 sec -2013-09-17 23:56:46,188 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 785.47 sec -2013-09-17 23:56:47,195 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 845.55 sec -2013-09-17 23:56:48,201 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 845.55 sec -2013-09-17 23:56:49,207 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 845.55 sec -MapReduce Total cumulative CPU time: 14 minutes 5 seconds 550 msec -Ended Job = job_201309172235_0077 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0078 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-17 23:56:51,716 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:57:05,763 Stage-2 map = 18%, reduce = 0% -2013-09-17 23:57:11,783 Stage-2 map = 37%, reduce = 0% -2013-09-17 23:57:17,803 Stage-2 map = 55%, reduce = 0% -2013-09-17 23:57:23,823 Stage-2 map = 73%, reduce = 0% -2013-09-17 23:57:29,845 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 87.94 sec -2013-09-17 23:57:30,850 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 87.94 sec -2013-09-17 23:57:31,855 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 87.94 sec -2013-09-17 23:57:32,860 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 87.94 sec -2013-09-17 23:57:33,865 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 87.94 sec -2013-09-17 23:57:34,870 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 87.94 sec -2013-09-17 23:57:35,876 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 97.95 sec -2013-09-17 23:57:36,881 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 97.95 sec -2013-09-17 23:57:37,887 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 97.95 sec -2013-09-17 23:57:38,893 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 97.95 sec -2013-09-17 23:57:39,898 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 97.95 sec -2013-09-17 23:57:40,903 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 97.95 sec -2013-09-17 23:57:41,908 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 97.95 sec -2013-09-17 23:57:42,914 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 97.95 sec -2013-09-17 23:57:43,919 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 97.95 sec -2013-09-17 23:57:44,924 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 118.7 sec -2013-09-17 23:57:45,929 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 118.7 sec -2013-09-17 23:57:46,934 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 118.7 sec -2013-09-17 23:57:47,939 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 118.7 sec -2013-09-17 23:57:48,944 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 118.7 sec -2013-09-17 23:57:49,948 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.7 sec -2013-09-17 23:57:50,960 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.7 sec -2013-09-17 23:57:51,965 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.7 sec -2013-09-17 23:57:52,969 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.7 sec -2013-09-17 23:57:53,974 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.7 sec -2013-09-17 23:57:54,978 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.7 sec -2013-09-17 23:57:55,982 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.7 sec -2013-09-17 23:57:56,986 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.7 sec -2013-09-17 23:57:57,991 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 118.7 sec -2013-09-17 23:57:58,995 Stage-2 map = 100%, reduce = 80%, Cumulative CPU 118.7 sec -2013-09-17 23:58:00,000 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 133.21 sec -2013-09-17 23:58:01,005 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 133.21 sec -2013-09-17 23:58:02,010 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 133.21 sec -MapReduce Total cumulative CPU time: 2 minutes 13 seconds 210 msec -Ended Job = job_201309172235_0078 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 845.55 sec HDFS Read: 663537300 HDFS Write: 480155183 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 133.21 sec HDFS Read: 480158905 HDFS Write: 251 SUCCESS -Total MapReduce CPU Time Spent: 16 minutes 18 seconds 760 msec -OK -Time taken: 130.91 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31945@mturlrep13_201309172358_1169432593.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0079 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-17 23:58:22,443 Stage-1 map = 0%, reduce = 0% -2013-09-17 23:58:32,503 Stage-1 map = 1%, reduce = 0% -2013-09-17 23:58:36,545 Stage-1 map = 12%, reduce = 0% -2013-09-17 23:58:39,582 Stage-1 map = 18%, reduce = 0% -2013-09-17 23:58:41,277 Stage-1 map = 26%, reduce = 0% -2013-09-17 23:58:43,786 Stage-1 map = 40%, reduce = 0% -2013-09-17 23:58:46,818 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 15.02 sec -2013-09-17 23:58:47,825 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 15.02 sec -2013-09-17 23:58:48,836 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 15.02 sec -2013-09-17 23:58:49,842 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 31.76 sec -2013-09-17 23:58:50,856 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 31.76 sec -2013-09-17 23:58:51,862 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 83.8 sec -2013-09-17 23:58:52,870 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 99.52 sec -2013-09-17 23:58:53,884 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 99.52 sec -2013-09-17 23:58:54,894 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 117.13 sec -2013-09-17 23:58:55,909 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 213.91 sec -2013-09-17 23:58:56,924 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 324.7 sec -2013-09-17 23:58:57,930 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 449.36 sec -2013-09-17 23:58:58,936 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 491.93 sec -2013-09-17 23:58:59,943 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 542.45 sec -2013-09-17 23:59:00,949 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 591.47 sec -2013-09-17 23:59:01,955 Stage-1 map = 97%, reduce = 3%, Cumulative CPU 620.12 sec -2013-09-17 23:59:02,961 Stage-1 map = 99%, reduce = 10%, Cumulative CPU 796.58 sec -2013-09-17 23:59:03,967 Stage-1 map = 99%, reduce = 10%, Cumulative CPU 796.58 sec -2013-09-17 23:59:04,972 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 827.74 sec -2013-09-17 23:59:05,980 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 827.74 sec -2013-09-17 23:59:06,987 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 827.74 sec -2013-09-17 23:59:07,994 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 827.74 sec -2013-09-17 23:59:09,000 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 827.74 sec -2013-09-17 23:59:10,007 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 827.74 sec -2013-09-17 23:59:11,013 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 827.74 sec -2013-09-17 23:59:12,020 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 827.74 sec -2013-09-17 23:59:13,027 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 827.74 sec -2013-09-17 23:59:14,033 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 827.74 sec -2013-09-17 23:59:15,039 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 827.74 sec -2013-09-17 23:59:16,046 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 827.74 sec -2013-09-17 23:59:17,052 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 827.74 sec -2013-09-17 23:59:18,059 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 827.74 sec -2013-09-17 23:59:19,065 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 827.74 sec -2013-09-17 23:59:20,072 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 827.74 sec -2013-09-17 23:59:21,077 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 827.74 sec -2013-09-17 23:59:22,114 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 827.74 sec -2013-09-17 23:59:23,122 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 871.5 sec -2013-09-17 23:59:24,133 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1068.13 sec -2013-09-17 23:59:25,139 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1068.13 sec -2013-09-17 23:59:26,143 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1068.13 sec -MapReduce Total cumulative CPU time: 17 minutes 48 seconds 130 msec -Ended Job = job_201309172235_0079 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0080 -Hadoop job information for Stage-2: number of mappers: 5; number of reducers: 1 -2013-09-17 23:59:29,594 Stage-2 map = 0%, reduce = 0% -2013-09-17 23:59:39,626 Stage-2 map = 6%, reduce = 0% -2013-09-17 23:59:42,636 Stage-2 map = 28%, reduce = 0% -2013-09-17 23:59:45,462 Stage-2 map = 39%, reduce = 0% -2013-09-17 23:59:46,467 Stage-2 map = 44%, reduce = 0% -2013-09-17 23:59:47,945 Stage-2 map = 52%, reduce = 0%, Cumulative CPU 23.32 sec -2013-09-17 23:59:48,950 Stage-2 map = 58%, reduce = 0%, Cumulative CPU 23.32 sec -2013-09-17 23:59:49,955 Stage-2 map = 58%, reduce = 0%, Cumulative CPU 23.32 sec -2013-09-17 23:59:50,960 Stage-2 map = 58%, reduce = 0%, Cumulative CPU 23.32 sec -2013-09-17 23:59:51,965 Stage-2 map = 63%, reduce = 0%, Cumulative CPU 23.32 sec -2013-09-17 23:59:52,970 Stage-2 map = 63%, reduce = 0%, Cumulative CPU 23.32 sec -2013-09-17 23:59:53,976 Stage-2 map = 63%, reduce = 0%, Cumulative CPU 23.32 sec -2013-09-17 23:59:54,981 Stage-2 map = 78%, reduce = 7%, Cumulative CPU 23.32 sec -2013-09-17 23:59:55,986 Stage-2 map = 78%, reduce = 7%, Cumulative CPU 23.32 sec -2013-09-17 23:59:56,992 Stage-2 map = 78%, reduce = 7%, Cumulative CPU 23.32 sec -2013-09-17 23:59:57,997 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 23.32 sec -2013-09-17 23:59:59,002 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 23.32 sec -2013-09-18 00:00:00,007 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 23.32 sec -2013-09-18 00:00:01,028 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 23.32 sec -2013-09-18 00:00:02,033 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 23.32 sec -2013-09-18 00:00:03,039 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 23.32 sec -2013-09-18 00:00:04,043 Stage-2 map = 87%, reduce = 7%, Cumulative CPU 66.8 sec -2013-09-18 00:00:05,049 Stage-2 map = 91%, reduce = 7%, Cumulative CPU 111.39 sec -2013-09-18 00:00:06,054 Stage-2 map = 91%, reduce = 7%, Cumulative CPU 111.39 sec -2013-09-18 00:00:07,058 Stage-2 map = 95%, reduce = 7%, Cumulative CPU 158.23 sec -2013-09-18 00:00:08,064 Stage-2 map = 95%, reduce = 7%, Cumulative CPU 158.23 sec -2013-09-18 00:00:09,069 Stage-2 map = 100%, reduce = 7%, Cumulative CPU 206.95 sec -2013-09-18 00:00:10,074 Stage-2 map = 100%, reduce = 7%, Cumulative CPU 206.95 sec -2013-09-18 00:00:11,079 Stage-2 map = 100%, reduce = 7%, Cumulative CPU 206.95 sec -2013-09-18 00:00:12,088 Stage-2 map = 100%, reduce = 7%, Cumulative CPU 206.95 sec -2013-09-18 00:00:13,092 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:14,097 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:15,102 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:16,106 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:17,110 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:18,114 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:19,118 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:20,123 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:21,127 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:22,131 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:23,135 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:24,139 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:25,144 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:26,148 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:27,152 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.95 sec -2013-09-18 00:00:28,157 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 226.88 sec -2013-09-18 00:00:29,161 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 226.88 sec -2013-09-18 00:00:30,166 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 226.88 sec -2013-09-18 00:00:31,170 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 226.88 sec -2013-09-18 00:00:32,175 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 233.37 sec -2013-09-18 00:00:33,180 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 233.37 sec -MapReduce Total cumulative CPU time: 3 minutes 53 seconds 370 msec -Ended Job = job_201309172235_0080 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1068.13 sec HDFS Read: 840872065 HDFS Write: 1196040653 SUCCESS -Job 1: Map: 5 Reduce: 1 Cumulative CPU: 233.37 sec HDFS Read: 1196066509 HDFS Write: 261 SUCCESS -Total MapReduce CPU Time Spent: 21 minutes 41 seconds 500 msec -OK -Time taken: 141.357 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8249@mturlrep13_201309180000_1745995125.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0081 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:00:43,931 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:00:56,183 Stage-1 map = 6%, reduce = 0% -2013-09-18 00:00:57,190 Stage-1 map = 7%, reduce = 0% -2013-09-18 00:00:58,202 Stage-1 map = 11%, reduce = 0% -2013-09-18 00:00:59,212 Stage-1 map = 29%, reduce = 0% -2013-09-18 00:01:00,225 Stage-1 map = 34%, reduce = 0% -2013-09-18 00:01:01,231 Stage-1 map = 37%, reduce = 0% -2013-09-18 00:01:02,237 Stage-1 map = 45%, reduce = 0% -2013-09-18 00:01:03,242 Stage-1 map = 58%, reduce = 0% -2013-09-18 00:01:04,254 Stage-1 map = 62%, reduce = 0% -2013-09-18 00:01:05,260 Stage-1 map = 66%, reduce = 0% -2013-09-18 00:01:06,286 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 14.75 sec -2013-09-18 00:01:07,294 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 30.84 sec -2013-09-18 00:01:08,306 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 81.13 sec -2013-09-18 00:01:09,336 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 133.53 sec -2013-09-18 00:01:10,343 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 152.09 sec -2013-09-18 00:01:11,353 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 212.05 sec -2013-09-18 00:01:12,359 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 251.83 sec -2013-09-18 00:01:13,366 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 359.32 sec -2013-09-18 00:01:14,373 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 452.13 sec -2013-09-18 00:01:15,379 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 547.58 sec -2013-09-18 00:01:16,387 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 547.58 sec -2013-09-18 00:01:17,393 Stage-1 map = 97%, reduce = 2%, Cumulative CPU 603.6 sec -2013-09-18 00:01:18,400 Stage-1 map = 98%, reduce = 6%, Cumulative CPU 715.99 sec -2013-09-18 00:01:19,407 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 744.69 sec -2013-09-18 00:01:20,413 Stage-1 map = 100%, reduce = 23%, Cumulative CPU 837.42 sec -2013-09-18 00:01:21,419 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 837.42 sec -2013-09-18 00:01:22,426 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 837.42 sec -2013-09-18 00:01:23,432 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 837.42 sec -2013-09-18 00:01:24,439 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 837.42 sec -2013-09-18 00:01:25,446 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 837.42 sec -2013-09-18 00:01:26,452 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 837.42 sec -2013-09-18 00:01:27,460 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 837.42 sec -2013-09-18 00:01:28,467 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 890.75 sec -2013-09-18 00:01:29,474 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 890.75 sec -2013-09-18 00:01:30,489 Stage-1 map = 100%, reduce = 53%, Cumulative CPU 890.75 sec -2013-09-18 00:01:31,496 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 890.75 sec -2013-09-18 00:01:32,503 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 890.75 sec -2013-09-18 00:01:33,509 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 890.75 sec -2013-09-18 00:01:34,515 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 890.75 sec -2013-09-18 00:01:35,522 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 890.75 sec -2013-09-18 00:01:36,528 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 890.75 sec -2013-09-18 00:01:37,534 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 890.75 sec -2013-09-18 00:01:38,541 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 890.75 sec -2013-09-18 00:01:39,555 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 998.36 sec -2013-09-18 00:01:40,562 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 998.36 sec -2013-09-18 00:01:41,568 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1035.02 sec -2013-09-18 00:01:42,574 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1070.49 sec -2013-09-18 00:01:43,580 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1070.49 sec -MapReduce Total cumulative CPU time: 17 minutes 50 seconds 490 msec -Ended Job = job_201309172235_0081 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0082 -Hadoop job information for Stage-2: number of mappers: 5; number of reducers: 1 -2013-09-18 00:01:47,035 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:01:57,066 Stage-2 map = 6%, reduce = 0% -2013-09-18 00:02:00,076 Stage-2 map = 28%, reduce = 0% -2013-09-18 00:02:05,092 Stage-2 map = 49%, reduce = 0%, Cumulative CPU 22.46 sec -2013-09-18 00:02:06,098 Stage-2 map = 49%, reduce = 0%, Cumulative CPU 22.46 sec -2013-09-18 00:02:07,104 Stage-2 map = 49%, reduce = 0%, Cumulative CPU 22.46 sec -2013-09-18 00:02:10,085 Stage-2 map = 52%, reduce = 0%, Cumulative CPU 22.46 sec -2013-09-18 00:02:11,090 Stage-2 map = 52%, reduce = 0%, Cumulative CPU 22.46 sec -2013-09-18 00:02:12,095 Stage-2 map = 52%, reduce = 0%, Cumulative CPU 22.46 sec -2013-09-18 00:02:13,100 Stage-2 map = 58%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:14,106 Stage-2 map = 58%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:15,111 Stage-2 map = 58%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:16,117 Stage-2 map = 68%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:17,122 Stage-2 map = 68%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:18,128 Stage-2 map = 68%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:19,133 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:20,166 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:21,171 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:22,176 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:23,273 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:24,279 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:25,284 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 22.46 sec -2013-09-18 00:02:26,289 Stage-2 map = 91%, reduce = 7%, Cumulative CPU 110.5 sec -2013-09-18 00:02:27,294 Stage-2 map = 91%, reduce = 7%, Cumulative CPU 110.5 sec -2013-09-18 00:02:28,412 Stage-2 map = 91%, reduce = 7%, Cumulative CPU 203.81 sec -2013-09-18 00:02:29,416 Stage-2 map = 100%, reduce = 7%, Cumulative CPU 206.44 sec -2013-09-18 00:02:30,420 Stage-2 map = 100%, reduce = 7%, Cumulative CPU 206.44 sec -2013-09-18 00:02:31,425 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 206.44 sec -2013-09-18 00:02:32,429 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 206.44 sec -2013-09-18 00:02:33,989 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 206.44 sec -2013-09-18 00:02:34,994 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 206.44 sec -2013-09-18 00:02:35,999 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 206.44 sec -2013-09-18 00:02:37,003 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 206.44 sec -2013-09-18 00:02:38,007 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 206.44 sec -2013-09-18 00:02:39,011 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 206.44 sec -2013-09-18 00:02:40,015 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.44 sec -2013-09-18 00:02:41,020 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.44 sec -2013-09-18 00:02:42,023 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.44 sec -2013-09-18 00:02:43,027 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.44 sec -2013-09-18 00:02:44,031 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.44 sec -2013-09-18 00:02:45,036 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.44 sec -2013-09-18 00:02:46,040 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.44 sec -2013-09-18 00:02:47,045 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.44 sec -2013-09-18 00:02:48,049 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.44 sec -2013-09-18 00:02:49,054 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 206.44 sec -2013-09-18 00:02:50,058 Stage-2 map = 100%, reduce = 69%, Cumulative CPU 206.44 sec -2013-09-18 00:02:51,063 Stage-2 map = 100%, reduce = 69%, Cumulative CPU 206.44 sec -2013-09-18 00:02:52,067 Stage-2 map = 100%, reduce = 69%, Cumulative CPU 206.44 sec -2013-09-18 00:02:53,071 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 206.44 sec -2013-09-18 00:02:54,395 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 206.44 sec -2013-09-18 00:02:55,399 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 206.44 sec -2013-09-18 00:02:56,403 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 206.44 sec -2013-09-18 00:02:57,408 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 206.44 sec -2013-09-18 00:03:01,104 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 232.56 sec -2013-09-18 00:03:02,108 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 232.56 sec -MapReduce Total cumulative CPU time: 3 minutes 52 seconds 560 msec -Ended Job = job_201309172235_0082 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1070.49 sec HDFS Read: 840872065 HDFS Write: 1196040653 SUCCESS -Job 1: Map: 5 Reduce: 1 Cumulative CPU: 232.56 sec HDFS Read: 1196066509 HDFS Write: 261 SUCCESS -Total MapReduce CPU Time Spent: 21 minutes 43 seconds 50 msec -OK -Time taken: 147.059 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15858@mturlrep13_201309180003_1278146290.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0083 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:03:11,727 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:03:23,895 Stage-1 map = 2%, reduce = 0% -2013-09-18 00:03:24,924 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:03:25,943 Stage-1 map = 6%, reduce = 0% -2013-09-18 00:03:26,979 Stage-1 map = 13%, reduce = 0% -2013-09-18 00:03:28,028 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 341.29 sec -2013-09-18 00:03:29,044 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 341.29 sec -2013-09-18 00:03:30,053 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 341.29 sec -2013-09-18 00:03:31,062 Stage-1 map = 54%, reduce = 0%, Cumulative CPU 341.29 sec -2013-09-18 00:03:32,074 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 341.29 sec -2013-09-18 00:03:33,090 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 346.1 sec -2013-09-18 00:03:34,112 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 346.1 sec -2013-09-18 00:03:35,122 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 346.1 sec -2013-09-18 00:03:36,133 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 346.1 sec -2013-09-18 00:03:37,145 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 394.29 sec -2013-09-18 00:03:38,153 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 402.8 sec -2013-09-18 00:03:39,160 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 413.32 sec -2013-09-18 00:03:40,175 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 435.9 sec -2013-09-18 00:03:41,200 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 505.2 sec -2013-09-18 00:03:42,209 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 544.32 sec -2013-09-18 00:03:43,217 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 600.85 sec -2013-09-18 00:03:44,223 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 629.83 sec -2013-09-18 00:03:45,230 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 645.26 sec -2013-09-18 00:03:46,236 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 682.7 sec -2013-09-18 00:03:47,243 Stage-1 map = 98%, reduce = 3%, Cumulative CPU 758.48 sec -2013-09-18 00:03:48,250 Stage-1 map = 99%, reduce = 17%, Cumulative CPU 799.32 sec -2013-09-18 00:03:49,257 Stage-1 map = 100%, reduce = 22%, Cumulative CPU 841.01 sec -2013-09-18 00:03:50,263 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 841.01 sec -2013-09-18 00:03:51,269 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 841.01 sec -2013-09-18 00:03:52,275 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 841.01 sec -2013-09-18 00:03:53,281 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 841.01 sec -2013-09-18 00:03:54,287 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 841.01 sec -2013-09-18 00:03:55,293 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 841.01 sec -2013-09-18 00:03:56,300 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 841.01 sec -2013-09-18 00:03:57,306 Stage-1 map = 100%, reduce = 42%, Cumulative CPU 841.01 sec -2013-09-18 00:03:58,313 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 841.01 sec -2013-09-18 00:03:59,319 Stage-1 map = 100%, reduce = 53%, Cumulative CPU 841.01 sec -2013-09-18 00:04:00,325 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 841.01 sec -2013-09-18 00:04:01,331 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 841.01 sec -2013-09-18 00:04:02,338 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 841.01 sec -2013-09-18 00:04:03,344 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 841.01 sec -2013-09-18 00:04:04,350 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 841.01 sec -2013-09-18 00:04:05,357 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 841.01 sec -2013-09-18 00:04:06,363 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 841.01 sec -2013-09-18 00:04:07,382 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 841.01 sec -2013-09-18 00:04:08,391 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 924.61 sec -2013-09-18 00:04:09,396 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1053.16 sec -2013-09-18 00:04:10,402 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1074.59 sec -2013-09-18 00:04:11,408 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1074.59 sec -MapReduce Total cumulative CPU time: 17 minutes 54 seconds 590 msec -Ended Job = job_201309172235_0083 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0084 -Hadoop job information for Stage-2: number of mappers: 5; number of reducers: 1 -2013-09-18 00:04:16,454 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:04:27,489 Stage-2 map = 6%, reduce = 0% -2013-09-18 00:04:30,502 Stage-2 map = 28%, reduce = 0%, Cumulative CPU 71.89 sec -2013-09-18 00:04:31,507 Stage-2 map = 28%, reduce = 0%, Cumulative CPU 71.89 sec -2013-09-18 00:04:32,513 Stage-2 map = 28%, reduce = 0%, Cumulative CPU 71.89 sec -2013-09-18 00:04:33,518 Stage-2 map = 28%, reduce = 0%, Cumulative CPU 71.89 sec -2013-09-18 00:04:34,523 Stage-2 map = 41%, reduce = 0%, Cumulative CPU 71.89 sec -2013-09-18 00:04:35,529 Stage-2 map = 49%, reduce = 0%, Cumulative CPU 81.02 sec -2013-09-18 00:04:36,534 Stage-2 map = 49%, reduce = 0%, Cumulative CPU 81.02 sec -2013-09-18 00:04:37,539 Stage-2 map = 49%, reduce = 0%, Cumulative CPU 81.02 sec -2013-09-18 00:04:38,545 Stage-2 map = 58%, reduce = 0%, Cumulative CPU 81.02 sec -2013-09-18 00:04:39,550 Stage-2 map = 58%, reduce = 0%, Cumulative CPU 81.02 sec -2013-09-18 00:04:41,187 Stage-2 map = 68%, reduce = 0%, Cumulative CPU 81.02 sec -2013-09-18 00:04:42,265 Stage-2 map = 68%, reduce = 0%, Cumulative CPU 81.02 sec -2013-09-18 00:04:43,270 Stage-2 map = 68%, reduce = 7%, Cumulative CPU 81.02 sec -2013-09-18 00:04:44,275 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 81.02 sec -2013-09-18 00:04:45,281 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 81.02 sec -2013-09-18 00:04:46,286 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 81.02 sec -2013-09-18 00:04:47,291 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 81.02 sec -2013-09-18 00:04:48,302 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 81.02 sec -2013-09-18 00:04:49,306 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 81.02 sec -2013-09-18 00:04:50,311 Stage-2 map = 84%, reduce = 7%, Cumulative CPU 81.02 sec -2013-09-18 00:04:51,316 Stage-2 map = 87%, reduce = 7%, Cumulative CPU 108.7 sec -2013-09-18 00:04:52,321 Stage-2 map = 91%, reduce = 7%, Cumulative CPU 136.95 sec -2013-09-18 00:04:53,326 Stage-2 map = 91%, reduce = 7%, Cumulative CPU 136.95 sec -2013-09-18 00:04:54,331 Stage-2 map = 91%, reduce = 7%, Cumulative CPU 136.95 sec -2013-09-18 00:04:55,335 Stage-2 map = 95%, reduce = 7%, Cumulative CPU 169.96 sec -2013-09-18 00:04:56,340 Stage-2 map = 100%, reduce = 7%, Cumulative CPU 203.62 sec -2013-09-18 00:04:57,344 Stage-2 map = 100%, reduce = 7%, Cumulative CPU 203.62 sec -2013-09-18 00:04:58,348 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 203.62 sec -2013-09-18 00:04:59,353 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 203.62 sec -2013-09-18 00:05:00,358 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 203.62 sec -2013-09-18 00:05:01,362 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 203.62 sec -2013-09-18 00:05:02,367 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 203.62 sec -2013-09-18 00:05:03,371 Stage-2 map = 100%, reduce = 20%, Cumulative CPU 203.62 sec -2013-09-18 00:05:04,376 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:05,380 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:06,384 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:07,389 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:08,393 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:09,397 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:10,402 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:11,406 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:12,410 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:13,415 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:14,419 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:15,422 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 203.62 sec -2013-09-18 00:05:16,426 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 203.62 sec -2013-09-18 00:05:17,430 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 203.62 sec -2013-09-18 00:05:18,433 Stage-2 map = 100%, reduce = 77%, Cumulative CPU 203.62 sec -2013-09-18 00:05:19,439 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 228.63 sec -2013-09-18 00:05:20,443 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 228.63 sec -2013-09-18 00:05:21,448 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 228.63 sec -2013-09-18 00:05:24,696 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 228.63 sec -MapReduce Total cumulative CPU time: 3 minutes 48 seconds 630 msec -Ended Job = job_201309172235_0084 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1074.59 sec HDFS Read: 840872065 HDFS Write: 1196040653 SUCCESS -Job 1: Map: 5 Reduce: 1 Cumulative CPU: 228.63 sec HDFS Read: 1196066487 HDFS Write: 261 SUCCESS -Total MapReduce CPU Time Spent: 21 minutes 43 seconds 220 msec -OK -Time taken: 140.75 seconds, Fetched: 10 row(s) -hive> quit; --- ещё более сложная агрегация.; - - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24153@mturlrep13_201309180005_705271963.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0085 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:05:46,233 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:05:56,352 Stage-1 map = 3%, reduce = 0% -2013-09-18 00:05:59,390 Stage-1 map = 10%, reduce = 0% -2013-09-18 00:06:00,398 Stage-1 map = 17%, reduce = 0% -2013-09-18 00:06:02,423 Stage-1 map = 21%, reduce = 0% -2013-09-18 00:06:03,429 Stage-1 map = 28%, reduce = 0% -2013-09-18 00:06:04,764 Stage-1 map = 32%, reduce = 0% -2013-09-18 00:06:05,770 Stage-1 map = 40%, reduce = 0% -2013-09-18 00:06:06,782 Stage-1 map = 46%, reduce = 0% -2013-09-18 00:06:07,789 Stage-1 map = 49%, reduce = 0% -2013-09-18 00:06:08,936 Stage-1 map = 57%, reduce = 0% -2013-09-18 00:06:09,962 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 31.6 sec -2013-09-18 00:06:11,308 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 48.37 sec -2013-09-18 00:06:12,316 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 48.37 sec -2013-09-18 00:06:13,322 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 70.19 sec -2013-09-18 00:06:14,328 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 89.75 sec -2013-09-18 00:06:15,335 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 106.46 sec -2013-09-18 00:06:16,340 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 181.45 sec -2013-09-18 00:06:17,350 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 234.9 sec -2013-09-18 00:06:18,357 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 234.9 sec -2013-09-18 00:06:19,363 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 376.71 sec -2013-09-18 00:06:20,369 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 461.07 sec -2013-09-18 00:06:21,375 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 534.4 sec -2013-09-18 00:06:22,382 Stage-1 map = 96%, reduce = 1%, Cumulative CPU 584.98 sec -2013-09-18 00:06:23,388 Stage-1 map = 97%, reduce = 5%, Cumulative CPU 641.71 sec -2013-09-18 00:06:24,393 Stage-1 map = 99%, reduce = 5%, Cumulative CPU 730.93 sec -2013-09-18 00:06:25,398 Stage-1 map = 99%, reduce = 5%, Cumulative CPU 792.13 sec -2013-09-18 00:06:26,404 Stage-1 map = 100%, reduce = 16%, Cumulative CPU 822.69 sec -2013-09-18 00:06:27,410 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 822.69 sec -2013-09-18 00:06:28,416 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 822.69 sec -2013-09-18 00:06:29,422 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 853.5 sec -2013-09-18 00:06:30,427 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 853.5 sec -2013-09-18 00:06:31,434 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 853.5 sec -2013-09-18 00:06:32,440 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 853.5 sec -2013-09-18 00:06:33,447 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 890.94 sec -2013-09-18 00:06:34,456 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 928.82 sec -2013-09-18 00:06:35,461 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 928.82 sec -2013-09-18 00:06:36,466 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 928.82 sec -MapReduce Total cumulative CPU time: 15 minutes 28 seconds 820 msec -Ended Job = job_201309172235_0085 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 928.82 sec HDFS Read: 840872065 HDFS Write: 5012 SUCCESS -Total MapReduce CPU Time Spent: 15 minutes 28 seconds 820 msec -OK -Time taken: 60.884 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30696@mturlrep13_201309180006_357154034.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0086 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:06:47,336 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:06:58,492 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:06:59,507 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:07:00,512 Stage-1 map = 8%, reduce = 0% -2013-09-18 00:07:01,519 Stage-1 map = 11%, reduce = 0% -2013-09-18 00:07:02,525 Stage-1 map = 30%, reduce = 0% -2013-09-18 00:07:03,531 Stage-1 map = 33%, reduce = 0% -2013-09-18 00:07:04,537 Stage-1 map = 36%, reduce = 0% -2013-09-18 00:07:05,542 Stage-1 map = 51%, reduce = 0% -2013-09-18 00:07:06,547 Stage-1 map = 56%, reduce = 0% -2013-09-18 00:07:07,553 Stage-1 map = 60%, reduce = 0% -2013-09-18 00:07:08,565 Stage-1 map = 66%, reduce = 0% -2013-09-18 00:07:09,577 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 14.92 sec -2013-09-18 00:07:10,584 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 46.57 sec -2013-09-18 00:07:11,590 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 80.51 sec -2013-09-18 00:07:12,596 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 131.92 sec -2013-09-18 00:07:13,713 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 131.92 sec -2013-09-18 00:07:14,736 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 152.9 sec -2013-09-18 00:07:15,752 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 254.96 sec -2013-09-18 00:07:16,766 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 338.32 sec -2013-09-18 00:07:17,773 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 429.15 sec -2013-09-18 00:07:18,778 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 523.97 sec -2013-09-18 00:07:19,784 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 548.71 sec -2013-09-18 00:07:20,791 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 604.14 sec -2013-09-18 00:07:21,798 Stage-1 map = 97%, reduce = 2%, Cumulative CPU 660.44 sec -2013-09-18 00:07:22,804 Stage-1 map = 99%, reduce = 13%, Cumulative CPU 747.55 sec -2013-09-18 00:07:23,810 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 839.86 sec -2013-09-18 00:07:24,816 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 839.86 sec -2013-09-18 00:07:25,822 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 839.86 sec -2013-09-18 00:07:26,829 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 839.86 sec -2013-09-18 00:07:27,835 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 839.86 sec -2013-09-18 00:07:28,841 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 839.86 sec -2013-09-18 00:07:29,847 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 871.07 sec -2013-09-18 00:07:30,853 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 871.07 sec -2013-09-18 00:07:31,859 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 871.07 sec -2013-09-18 00:07:32,868 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 898.81 sec -2013-09-18 00:07:33,873 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 949.24 sec -2013-09-18 00:07:34,879 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 949.24 sec -2013-09-18 00:07:35,885 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 949.24 sec -MapReduce Total cumulative CPU time: 15 minutes 49 seconds 240 msec -Ended Job = job_201309172235_0086 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 949.24 sec HDFS Read: 840872065 HDFS Write: 5012 SUCCESS -Total MapReduce CPU Time Spent: 15 minutes 49 seconds 240 msec -OK -Time taken: 57.376 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4908@mturlrep13_201309180007_865221313.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0087 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:07:46,553 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:07:56,648 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:07:58,657 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:07:59,663 Stage-1 map = 7%, reduce = 0% -2013-09-18 00:08:00,669 Stage-1 map = 12%, reduce = 0% -2013-09-18 00:08:01,675 Stage-1 map = 29%, reduce = 0% -2013-09-18 00:08:02,699 Stage-1 map = 32%, reduce = 0% -2013-09-18 00:08:03,714 Stage-1 map = 37%, reduce = 0% -2013-09-18 00:08:04,734 Stage-1 map = 46%, reduce = 0% -2013-09-18 00:08:05,741 Stage-1 map = 55%, reduce = 0% -2013-09-18 00:08:06,749 Stage-1 map = 61%, reduce = 0% -2013-09-18 00:08:07,780 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 14.11 sec -2013-09-18 00:08:08,868 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 14.11 sec -2013-09-18 00:08:09,882 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 14.11 sec -2013-09-18 00:08:10,889 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 64.17 sec -2013-09-18 00:08:11,894 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 133.33 sec -2013-09-18 00:08:12,901 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 152.87 sec -2013-09-18 00:08:13,906 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 152.87 sec -2013-09-18 00:08:14,926 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 318.88 sec -2013-09-18 00:08:15,942 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 382.36 sec -2013-09-18 00:08:16,947 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 498.71 sec -2013-09-18 00:08:17,953 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 522.53 sec -2013-09-18 00:08:18,958 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 546.88 sec -2013-09-18 00:08:19,985 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 603.23 sec -2013-09-18 00:08:20,991 Stage-1 map = 97%, reduce = 2%, Cumulative CPU 630.86 sec -2013-09-18 00:08:21,996 Stage-1 map = 99%, reduce = 13%, Cumulative CPU 777.69 sec -2013-09-18 00:08:23,002 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 839.76 sec -2013-09-18 00:08:24,008 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 839.76 sec -2013-09-18 00:08:25,013 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 839.76 sec -2013-09-18 00:08:26,019 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 839.76 sec -2013-09-18 00:08:27,026 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 839.76 sec -2013-09-18 00:08:28,032 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 839.76 sec -2013-09-18 00:08:29,038 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 839.76 sec -2013-09-18 00:08:30,044 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 872.39 sec -2013-09-18 00:08:31,050 Stage-1 map = 100%, reduce = 38%, Cumulative CPU 872.39 sec -2013-09-18 00:08:32,059 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 881.87 sec -2013-09-18 00:08:33,065 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 922.01 sec -2013-09-18 00:08:34,070 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 945.45 sec -2013-09-18 00:08:35,076 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 945.45 sec -MapReduce Total cumulative CPU time: 15 minutes 45 seconds 450 msec -Ended Job = job_201309172235_0087 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 945.45 sec HDFS Read: 840872065 HDFS Write: 5012 SUCCESS -Total MapReduce CPU Time Spent: 15 minutes 45 seconds 450 msec -OK -Time taken: 57.367 seconds, Fetched: 10 row(s) -hive> quit; --- то же самое, но без сортировки.; - - -times: 1 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11923@mturlrep13_201309180008_1140241874.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0088 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:09:03,020 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:09:16,111 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:09:17,117 Stage-1 map = 2%, reduce = 0% -2013-09-18 00:09:19,132 Stage-1 map = 3%, reduce = 0% -2013-09-18 00:09:20,138 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:09:21,154 Stage-1 map = 8%, reduce = 0% -2013-09-18 00:09:22,176 Stage-1 map = 11%, reduce = 0% -2013-09-18 00:09:23,188 Stage-1 map = 12%, reduce = 0% -2013-09-18 00:09:24,193 Stage-1 map = 15%, reduce = 0% -2013-09-18 00:09:25,203 Stage-1 map = 22%, reduce = 0% -2013-09-18 00:09:26,210 Stage-1 map = 24%, reduce = 0% -2013-09-18 00:09:27,313 Stage-1 map = 27%, reduce = 0% -2013-09-18 00:09:28,319 Stage-1 map = 34%, reduce = 0% -2013-09-18 00:09:29,325 Stage-1 map = 37%, reduce = 0% -2013-09-18 00:09:30,337 Stage-1 map = 38%, reduce = 0%, Cumulative CPU 616.5 sec -2013-09-18 00:09:31,344 Stage-1 map = 42%, reduce = 0%, Cumulative CPU 616.5 sec -2013-09-18 00:09:32,351 Stage-1 map = 45%, reduce = 0%, Cumulative CPU 616.5 sec -2013-09-18 00:09:33,366 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 616.5 sec -2013-09-18 00:09:34,373 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 616.5 sec -2013-09-18 00:09:35,382 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 628.21 sec -2013-09-18 00:09:36,418 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 628.21 sec -2013-09-18 00:09:37,437 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 628.21 sec -2013-09-18 00:09:38,446 Stage-1 map = 62%, reduce = 0%, Cumulative CPU 628.21 sec -2013-09-18 00:09:39,465 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 628.21 sec -2013-09-18 00:09:40,473 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 628.21 sec -2013-09-18 00:09:41,481 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 628.21 sec -2013-09-18 00:09:42,488 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 641.75 sec -2013-09-18 00:09:43,500 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 655.91 sec -2013-09-18 00:09:44,507 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 655.91 sec -2013-09-18 00:09:45,524 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 655.91 sec -2013-09-18 00:09:46,548 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 710.09 sec -2013-09-18 00:09:47,560 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 729.24 sec -2013-09-18 00:09:48,567 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 746.21 sec -2013-09-18 00:09:49,578 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 781.66 sec -2013-09-18 00:09:50,585 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 893.41 sec -2013-09-18 00:09:51,590 Stage-1 map = 94%, reduce = 1%, Cumulative CPU 997.39 sec -2013-09-18 00:09:52,799 Stage-1 map = 96%, reduce = 5%, Cumulative CPU 1063.45 sec -2013-09-18 00:09:53,805 Stage-1 map = 98%, reduce = 7%, Cumulative CPU 1196.08 sec -2013-09-18 00:09:54,811 Stage-1 map = 99%, reduce = 12%, Cumulative CPU 1239.77 sec -2013-09-18 00:09:55,817 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 1264.22 sec -2013-09-18 00:09:56,822 Stage-1 map = 99%, reduce = 20%, Cumulative CPU 1289.61 sec -2013-09-18 00:09:57,828 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 1316.56 sec -2013-09-18 00:09:58,847 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 1316.56 sec -2013-09-18 00:09:59,853 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:00,859 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:01,866 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:02,872 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:03,879 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:04,885 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:05,892 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:06,899 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:07,908 Stage-1 map = 100%, reduce = 42%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:08,915 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:09,921 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:11,093 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:12,099 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:13,105 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:14,111 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:15,117 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:19,314 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:20,320 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:21,802 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:22,808 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:23,814 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:24,820 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:25,825 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:27,283 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:28,288 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:29,293 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1316.56 sec -2013-09-18 00:10:30,301 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:31,307 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:32,316 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:33,323 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:34,609 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:35,615 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:36,620 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:37,750 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:38,756 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:39,762 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:40,767 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:41,772 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:43,972 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:44,978 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:45,984 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:46,990 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:48,166 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:49,171 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:50,176 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1610.02 sec -2013-09-18 00:10:51,182 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1776.03 sec -2013-09-18 00:10:52,188 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1795.3 sec -2013-09-18 00:10:53,193 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1795.3 sec -MapReduce Total cumulative CPU time: 29 minutes 55 seconds 300 msec -Ended Job = job_201309172235_0088 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0089 -Hadoop job information for Stage-2: number of mappers: 9; number of reducers: 1 -2013-09-18 00:10:56,790 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:11:13,933 Stage-2 map = 22%, reduce = 0%, Cumulative CPU 22.42 sec -2013-09-18 00:11:14,937 Stage-2 map = 22%, reduce = 0%, Cumulative CPU 22.42 sec -2013-09-18 00:11:15,942 Stage-2 map = 22%, reduce = 0%, Cumulative CPU 22.42 sec -2013-09-18 00:11:16,948 Stage-2 map = 33%, reduce = 0%, Cumulative CPU 22.42 sec -2013-09-18 00:11:17,953 Stage-2 map = 33%, reduce = 0%, Cumulative CPU 22.42 sec -2013-09-18 00:11:18,959 Stage-2 map = 33%, reduce = 0%, Cumulative CPU 22.42 sec -2013-09-18 00:11:19,964 Stage-2 map = 33%, reduce = 0%, Cumulative CPU 22.42 sec -2013-09-18 00:11:20,969 Stage-2 map = 33%, reduce = 0%, Cumulative CPU 22.42 sec -2013-09-18 00:11:21,974 Stage-2 map = 33%, reduce = 4%, Cumulative CPU 22.42 sec -2013-09-18 00:11:22,980 Stage-2 map = 47%, reduce = 4%, Cumulative CPU 22.42 sec -2013-09-18 00:11:23,985 Stage-2 map = 47%, reduce = 4%, Cumulative CPU 22.42 sec -2013-09-18 00:11:24,991 Stage-2 map = 47%, reduce = 4%, Cumulative CPU 22.42 sec -2013-09-18 00:11:25,996 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.42 sec -2013-09-18 00:11:27,001 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.42 sec -2013-09-18 00:11:28,007 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.42 sec -2013-09-18 00:11:29,012 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.42 sec -2013-09-18 00:11:30,017 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:31,022 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:32,027 Stage-2 map = 67%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:33,031 Stage-2 map = 67%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:34,036 Stage-2 map = 67%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:35,040 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:36,045 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:37,050 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:38,080 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:39,085 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:40,099 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:41,104 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:42,109 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:43,114 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:44,120 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:45,125 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 339.47 sec -2013-09-18 00:11:46,130 Stage-2 map = 81%, reduce = 4%, Cumulative CPU 358.69 sec -2013-09-18 00:11:47,135 Stage-2 map = 89%, reduce = 4%, Cumulative CPU 418.36 sec -2013-09-18 00:11:48,139 Stage-2 map = 89%, reduce = 4%, Cumulative CPU 418.36 sec -2013-09-18 00:11:49,144 Stage-2 map = 89%, reduce = 4%, Cumulative CPU 418.36 sec -2013-09-18 00:11:50,150 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 510.55 sec -2013-09-18 00:11:51,155 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 510.55 sec -2013-09-18 00:11:52,160 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 510.55 sec -2013-09-18 00:11:53,164 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 510.55 sec -2013-09-18 00:11:54,169 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 510.55 sec -2013-09-18 00:11:55,174 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 510.55 sec -2013-09-18 00:11:56,178 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 510.55 sec -2013-09-18 00:11:57,181 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 510.55 sec -2013-09-18 00:11:58,185 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 510.55 sec -2013-09-18 00:11:59,189 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 510.55 sec -2013-09-18 00:12:00,194 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 510.55 sec -2013-09-18 00:12:01,199 Stage-2 map = 100%, reduce = 22%, Cumulative CPU 510.55 sec -2013-09-18 00:12:02,202 Stage-2 map = 100%, reduce = 22%, Cumulative CPU 510.55 sec -2013-09-18 00:12:03,206 Stage-2 map = 100%, reduce = 22%, Cumulative CPU 510.55 sec -2013-09-18 00:12:04,210 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 510.55 sec -2013-09-18 00:12:05,214 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 510.55 sec -2013-09-18 00:12:06,218 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 510.55 sec -2013-09-18 00:12:07,222 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:08,226 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:09,230 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:10,234 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:11,238 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:12,241 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:13,245 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:14,250 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:15,253 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:16,257 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:17,261 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:18,265 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:19,826 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:20,830 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:21,833 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:24,804 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 510.55 sec -2013-09-18 00:12:31,735 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 545.43 sec -2013-09-18 00:12:32,740 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 545.43 sec -2013-09-18 00:12:33,744 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 545.43 sec -2013-09-18 00:12:34,748 Stage-2 map = 100%, reduce = 71%, Cumulative CPU 545.43 sec -2013-09-18 00:12:35,753 Stage-2 map = 100%, reduce = 71%, Cumulative CPU 545.43 sec -2013-09-18 00:12:36,757 Stage-2 map = 100%, reduce = 71%, Cumulative CPU 545.43 sec -2013-09-18 00:12:37,762 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 545.43 sec -2013-09-18 00:12:39,183 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 545.43 sec -2013-09-18 00:12:40,188 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 545.43 sec -2013-09-18 00:12:41,192 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 545.43 sec -2013-09-18 00:12:42,196 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 545.43 sec -2013-09-18 00:12:43,622 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 545.43 sec -2013-09-18 00:12:44,627 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 545.43 sec -2013-09-18 00:12:45,631 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 545.43 sec -2013-09-18 00:12:46,636 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 545.43 sec -2013-09-18 00:12:47,640 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 559.08 sec -2013-09-18 00:12:48,644 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 559.08 sec -2013-09-18 00:12:49,648 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 559.08 sec -MapReduce Total cumulative CPU time: 9 minutes 19 seconds 80 msec -Ended Job = job_201309172235_0089 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1795.3 sec HDFS Read: 843455069 HDFS Write: 2238428518 SUCCESS -Job 1: Map: 9 Reduce: 1 Cumulative CPU: 559.08 sec HDFS Read: 2238513761 HDFS Write: 278 SUCCESS -Total MapReduce CPU Time Spent: 39 minutes 14 seconds 380 msec -OK -Time taken: 237.554 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21185@mturlrep13_201309180012_506617682.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0090 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:12:59,255 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:13:15,399 Stage-1 map = 2%, reduce = 0% -2013-09-18 00:13:16,405 Stage-1 map = 8%, reduce = 0% -2013-09-18 00:13:18,423 Stage-1 map = 12%, reduce = 0% -2013-09-18 00:13:19,429 Stage-1 map = 19%, reduce = 0% -2013-09-18 00:13:21,449 Stage-1 map = 21%, reduce = 0% -2013-09-18 00:13:22,454 Stage-1 map = 31%, reduce = 0% -2013-09-18 00:13:23,475 Stage-1 map = 32%, reduce = 0% -2013-09-18 00:13:25,512 Stage-1 map = 40%, reduce = 0% -2013-09-18 00:13:26,534 Stage-1 map = 41%, reduce = 0% -2013-09-18 00:13:28,545 Stage-1 map = 50%, reduce = 0% -2013-09-18 00:13:29,557 Stage-1 map = 52%, reduce = 0% -2013-09-18 00:13:30,834 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 21.31 sec -2013-09-18 00:13:31,841 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 814.57 sec -2013-09-18 00:13:32,847 Stage-1 map = 62%, reduce = 0%, Cumulative CPU 814.57 sec -2013-09-18 00:13:33,857 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 816.31 sec -2013-09-18 00:13:34,872 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 816.31 sec -2013-09-18 00:13:35,886 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 816.31 sec -2013-09-18 00:13:36,894 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 816.31 sec -2013-09-18 00:13:37,900 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 824.05 sec -2013-09-18 00:13:38,913 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 830.96 sec -2013-09-18 00:13:39,925 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 830.96 sec -2013-09-18 00:13:40,931 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 830.96 sec -2013-09-18 00:13:41,938 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 830.96 sec -2013-09-18 00:13:42,978 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 865.43 sec -2013-09-18 00:13:43,986 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 875.5 sec -2013-09-18 00:13:44,998 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 908.75 sec -2013-09-18 00:13:46,004 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 1001.65 sec -2013-09-18 00:13:47,011 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 1063.43 sec -2013-09-18 00:13:48,018 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 1202.46 sec -2013-09-18 00:13:49,023 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1255.44 sec -2013-09-18 00:13:50,029 Stage-1 map = 99%, reduce = 3%, Cumulative CPU 1272.82 sec -2013-09-18 00:13:51,035 Stage-1 map = 99%, reduce = 7%, Cumulative CPU 1272.82 sec -2013-09-18 00:13:52,040 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 1316.55 sec -2013-09-18 00:13:53,046 Stage-1 map = 100%, reduce = 23%, Cumulative CPU 1316.55 sec -2013-09-18 00:13:54,052 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 1316.55 sec -2013-09-18 00:13:55,065 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1316.55 sec -2013-09-18 00:13:56,071 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1316.55 sec -2013-09-18 00:13:57,077 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1316.55 sec -2013-09-18 00:13:58,083 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1316.55 sec -2013-09-18 00:13:59,109 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:00,116 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:01,122 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:02,127 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:03,133 Stage-1 map = 100%, reduce = 40%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:04,139 Stage-1 map = 100%, reduce = 40%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:05,145 Stage-1 map = 100%, reduce = 43%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:06,152 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:07,158 Stage-1 map = 100%, reduce = 59%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:08,164 Stage-1 map = 100%, reduce = 66%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:09,702 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:10,707 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:11,713 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:12,719 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:13,725 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:14,731 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:15,959 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:16,964 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:17,970 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:18,976 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:19,982 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:21,672 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:24,777 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:25,782 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:26,787 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:30,291 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1316.55 sec -2013-09-18 00:14:34,570 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:35,576 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:36,582 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:37,588 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:38,594 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:39,628 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:40,634 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:41,640 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:42,645 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:43,651 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:44,657 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1638.29 sec -2013-09-18 00:14:47,811 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1648.84 sec -2013-09-18 00:14:48,817 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1725.98 sec -2013-09-18 00:14:49,823 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1725.98 sec -2013-09-18 00:14:54,762 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1725.98 sec -2013-09-18 00:14:56,039 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1784.82 sec -2013-09-18 00:14:57,044 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1784.82 sec -MapReduce Total cumulative CPU time: 29 minutes 44 seconds 820 msec -Ended Job = job_201309172235_0090 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0091 -Hadoop job information for Stage-2: number of mappers: 9; number of reducers: 1 -2013-09-18 00:15:00,513 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:15:17,637 Stage-2 map = 30%, reduce = 0% -2013-09-18 00:15:18,642 Stage-2 map = 31%, reduce = 0%, Cumulative CPU 21.69 sec -2013-09-18 00:15:19,647 Stage-2 map = 31%, reduce = 0%, Cumulative CPU 21.69 sec -2013-09-18 00:15:20,652 Stage-2 map = 33%, reduce = 0%, Cumulative CPU 21.69 sec -2013-09-18 00:15:21,657 Stage-2 map = 33%, reduce = 0%, Cumulative CPU 21.69 sec -2013-09-18 00:15:22,662 Stage-2 map = 33%, reduce = 0%, Cumulative CPU 21.69 sec -2013-09-18 00:15:23,667 Stage-2 map = 33%, reduce = 0%, Cumulative CPU 21.69 sec -2013-09-18 00:15:24,672 Stage-2 map = 33%, reduce = 0%, Cumulative CPU 21.69 sec -2013-09-18 00:15:25,678 Stage-2 map = 33%, reduce = 4%, Cumulative CPU 21.69 sec -2013-09-18 00:15:26,683 Stage-2 map = 47%, reduce = 4%, Cumulative CPU 21.69 sec -2013-09-18 00:15:27,687 Stage-2 map = 47%, reduce = 4%, Cumulative CPU 21.69 sec -2013-09-18 00:15:28,692 Stage-2 map = 47%, reduce = 4%, Cumulative CPU 21.69 sec -2013-09-18 00:15:29,697 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 21.69 sec -2013-09-18 00:15:30,702 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 21.69 sec -2013-09-18 00:15:31,707 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 21.69 sec -2013-09-18 00:15:32,712 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 21.69 sec -2013-09-18 00:15:33,717 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 21.69 sec -2013-09-18 00:15:34,722 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:35,728 Stage-2 map = 70%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:36,733 Stage-2 map = 70%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:37,738 Stage-2 map = 70%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:38,744 Stage-2 map = 72%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:39,749 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:40,754 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:41,906 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:42,911 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:43,916 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:44,921 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:45,951 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:46,956 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:47,961 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:48,977 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:49,982 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:50,988 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:52,053 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 343.13 sec -2013-09-18 00:15:53,058 Stage-2 map = 92%, reduce = 4%, Cumulative CPU 444.1 sec -2013-09-18 00:15:54,063 Stage-2 map = 94%, reduce = 4%, Cumulative CPU 466.73 sec -2013-09-18 00:15:55,109 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 513.25 sec -2013-09-18 00:15:56,114 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 513.25 sec -2013-09-18 00:15:57,119 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 513.25 sec -2013-09-18 00:15:58,124 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 513.25 sec -2013-09-18 00:15:59,129 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 513.25 sec -2013-09-18 00:16:00,133 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 513.25 sec -2013-09-18 00:16:01,138 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 513.25 sec -2013-09-18 00:16:02,142 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 513.25 sec -2013-09-18 00:16:03,148 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 513.25 sec -2013-09-18 00:16:04,152 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 513.25 sec -2013-09-18 00:16:05,157 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 513.25 sec -2013-09-18 00:16:06,161 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 513.25 sec -2013-09-18 00:16:07,165 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 513.25 sec -2013-09-18 00:16:08,170 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 513.25 sec -2013-09-18 00:16:09,174 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 513.25 sec -2013-09-18 00:16:10,179 Stage-2 map = 100%, reduce = 30%, Cumulative CPU 513.25 sec -2013-09-18 00:16:11,183 Stage-2 map = 100%, reduce = 30%, Cumulative CPU 513.25 sec -2013-09-18 00:16:12,188 Stage-2 map = 100%, reduce = 30%, Cumulative CPU 513.25 sec -2013-09-18 00:16:13,191 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:14,196 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:15,200 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:16,204 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:18,269 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:19,273 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:20,276 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:21,281 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:27,554 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:28,559 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:29,564 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 513.25 sec -2013-09-18 00:16:33,769 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 513.25 sec -2013-09-18 00:16:34,774 Stage-2 map = 100%, reduce = 71%, Cumulative CPU 552.24 sec -2013-09-18 00:16:35,778 Stage-2 map = 100%, reduce = 71%, Cumulative CPU 552.24 sec -2013-09-18 00:16:36,782 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 552.24 sec -2013-09-18 00:16:37,787 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 552.24 sec -2013-09-18 00:16:38,792 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 552.24 sec -2013-09-18 00:16:39,796 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 552.24 sec -2013-09-18 00:16:40,801 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 552.24 sec -2013-09-18 00:16:41,805 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 552.24 sec -2013-09-18 00:16:42,809 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 552.24 sec -2013-09-18 00:16:43,814 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 552.24 sec -2013-09-18 00:16:44,819 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 562.94 sec -2013-09-18 00:16:45,823 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 562.94 sec -MapReduce Total cumulative CPU time: 9 minutes 22 seconds 940 msec -Ended Job = job_201309172235_0091 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1784.82 sec HDFS Read: 843455069 HDFS Write: 2238428518 SUCCESS -Job 1: Map: 9 Reduce: 1 Cumulative CPU: 562.94 sec HDFS Read: 2238513761 HDFS Write: 278 SUCCESS -Total MapReduce CPU Time Spent: 39 minutes 7 seconds 760 msec -OK -Time taken: 234.361 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30407@mturlrep13_201309180016_1665474298.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_100m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0092 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:16:56,793 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:17:10,896 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:17:11,902 Stage-1 map = 4%, reduce = 0% -2013-09-18 00:17:12,907 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:17:13,917 Stage-1 map = 10%, reduce = 0% -2013-09-18 00:17:14,928 Stage-1 map = 15%, reduce = 0% -2013-09-18 00:17:15,933 Stage-1 map = 17%, reduce = 0% -2013-09-18 00:17:16,939 Stage-1 map = 20%, reduce = 0% -2013-09-18 00:17:17,944 Stage-1 map = 24%, reduce = 0% -2013-09-18 00:17:18,950 Stage-1 map = 27%, reduce = 0% -2013-09-18 00:17:19,955 Stage-1 map = 31%, reduce = 0% -2013-09-18 00:17:20,961 Stage-1 map = 33%, reduce = 0% -2013-09-18 00:17:21,966 Stage-1 map = 37%, reduce = 0% -2013-09-18 00:17:22,971 Stage-1 map = 40%, reduce = 0% -2013-09-18 00:17:24,001 Stage-1 map = 44%, reduce = 0% -2013-09-18 00:17:25,019 Stage-1 map = 47%, reduce = 0% -2013-09-18 00:17:26,029 Stage-1 map = 50%, reduce = 0% -2013-09-18 00:17:27,037 Stage-1 map = 53%, reduce = 0% -2013-09-18 00:17:28,044 Stage-1 map = 58%, reduce = 0% -2013-09-18 00:17:29,062 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 21.75 sec -2013-09-18 00:17:30,082 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 21.75 sec -2013-09-18 00:17:31,094 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 21.75 sec -2013-09-18 00:17:32,114 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 21.75 sec -2013-09-18 00:17:33,129 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 21.75 sec -2013-09-18 00:17:34,140 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 105.0 sec -2013-09-18 00:17:35,156 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 1037.71 sec -2013-09-18 00:17:36,167 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 1037.71 sec -2013-09-18 00:17:37,173 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 1037.71 sec -2013-09-18 00:17:38,180 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 1037.71 sec -2013-09-18 00:17:39,186 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 1037.71 sec -2013-09-18 00:17:40,198 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 1047.34 sec -2013-09-18 00:17:41,214 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 1064.75 sec -2013-09-18 00:17:42,227 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 1113.1 sec -2013-09-18 00:17:43,233 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 1183.03 sec -2013-09-18 00:17:44,240 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 1236.75 sec -2013-09-18 00:17:45,256 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1266.7 sec -2013-09-18 00:17:46,269 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1290.62 sec -2013-09-18 00:17:47,277 Stage-1 map = 99%, reduce = 1%, Cumulative CPU 1290.62 sec -2013-09-18 00:17:48,283 Stage-1 map = 99%, reduce = 3%, Cumulative CPU 1304.6 sec -2013-09-18 00:17:49,289 Stage-1 map = 100%, reduce = 11%, Cumulative CPU 1320.25 sec -2013-09-18 00:17:50,296 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 1320.25 sec -2013-09-18 00:17:51,313 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 1320.25 sec -2013-09-18 00:17:52,319 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 1320.25 sec -2013-09-18 00:17:53,325 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1320.25 sec -2013-09-18 00:17:54,331 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1320.25 sec -2013-09-18 00:17:55,338 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1320.25 sec -2013-09-18 00:17:56,344 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1320.25 sec -2013-09-18 00:17:57,350 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1320.25 sec -2013-09-18 00:17:58,356 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1320.25 sec -2013-09-18 00:17:59,362 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:00,368 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:01,374 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:02,381 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:03,387 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:04,392 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:07,247 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:08,252 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:09,259 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:10,266 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:11,272 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:12,762 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:13,767 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:14,772 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:15,778 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:18,513 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:20,464 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:21,470 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:22,477 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:23,674 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:24,680 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:26,496 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:27,502 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:28,508 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:31,532 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:32,538 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:33,543 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1320.25 sec -2013-09-18 00:18:34,555 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1721.3 sec -2013-09-18 00:18:36,736 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1721.3 sec -2013-09-18 00:18:37,742 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1721.3 sec -2013-09-18 00:18:38,747 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1721.3 sec -2013-09-18 00:18:39,753 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1721.3 sec -2013-09-18 00:18:40,759 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1724.11 sec -2013-09-18 00:18:41,765 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1724.11 sec -2013-09-18 00:18:42,770 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1724.11 sec -2013-09-18 00:18:43,776 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1724.11 sec -2013-09-18 00:18:44,782 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1743.27 sec -2013-09-18 00:18:45,788 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1754.65 sec -2013-09-18 00:18:48,727 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1754.65 sec -2013-09-18 00:18:49,733 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1786.03 sec -2013-09-18 00:18:50,739 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1786.03 sec -2013-09-18 00:18:51,754 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1786.03 sec -MapReduce Total cumulative CPU time: 29 minutes 46 seconds 30 msec -Ended Job = job_201309172235_0092 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0093 -Hadoop job information for Stage-2: number of mappers: 9; number of reducers: 1 -2013-09-18 00:18:55,132 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:19:13,198 Stage-2 map = 33%, reduce = 0% -2013-09-18 00:19:21,228 Stage-2 map = 33%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:22,233 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:23,238 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:24,243 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:25,248 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:26,252 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:27,257 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:28,262 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:29,267 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:30,271 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:31,276 Stage-2 map = 56%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:32,280 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:33,285 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:34,290 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 22.55 sec -2013-09-18 00:19:35,295 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 398.54 sec -2013-09-18 00:19:36,299 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 398.54 sec -2013-09-18 00:19:37,305 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 398.54 sec -2013-09-18 00:19:38,310 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 398.54 sec -2013-09-18 00:19:39,315 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 398.54 sec -2013-09-18 00:19:40,320 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 398.54 sec -2013-09-18 00:19:41,326 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 398.54 sec -2013-09-18 00:19:42,331 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 398.54 sec -2013-09-18 00:19:43,349 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 398.54 sec -2013-09-18 00:19:44,354 Stage-2 map = 78%, reduce = 4%, Cumulative CPU 398.54 sec -2013-09-18 00:19:46,244 Stage-2 map = 84%, reduce = 4%, Cumulative CPU 423.18 sec -2013-09-18 00:19:47,248 Stage-2 map = 94%, reduce = 4%, Cumulative CPU 476.82 sec -2013-09-18 00:19:48,253 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 505.58 sec -2013-09-18 00:19:49,258 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 505.58 sec -2013-09-18 00:19:50,262 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 505.58 sec -2013-09-18 00:19:51,267 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 505.58 sec -2013-09-18 00:19:52,273 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 505.58 sec -2013-09-18 00:19:53,277 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 505.58 sec -2013-09-18 00:19:54,281 Stage-2 map = 100%, reduce = 4%, Cumulative CPU 505.58 sec -2013-09-18 00:19:55,285 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 505.58 sec -2013-09-18 00:19:56,289 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 505.58 sec -2013-09-18 00:19:57,293 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 505.58 sec -2013-09-18 00:19:58,296 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 505.58 sec -2013-09-18 00:19:59,300 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 505.58 sec -2013-09-18 00:20:00,305 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 505.58 sec -2013-09-18 00:20:01,309 Stage-2 map = 100%, reduce = 22%, Cumulative CPU 505.58 sec -2013-09-18 00:20:02,313 Stage-2 map = 100%, reduce = 22%, Cumulative CPU 505.58 sec -2013-09-18 00:20:03,317 Stage-2 map = 100%, reduce = 22%, Cumulative CPU 505.58 sec -2013-09-18 00:20:04,322 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 505.58 sec -2013-09-18 00:20:05,326 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 505.58 sec -2013-09-18 00:20:06,329 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 505.58 sec -2013-09-18 00:20:07,334 Stage-2 map = 100%, reduce = 30%, Cumulative CPU 505.58 sec -2013-09-18 00:20:08,338 Stage-2 map = 100%, reduce = 30%, Cumulative CPU 505.58 sec -2013-09-18 00:20:09,342 Stage-2 map = 100%, reduce = 30%, Cumulative CPU 505.58 sec -2013-09-18 00:20:10,347 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:11,357 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:12,362 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:13,366 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:14,371 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:15,376 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:16,381 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:17,386 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:18,391 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:19,396 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:20,401 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:21,405 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:22,410 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:24,399 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:25,403 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:26,407 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 505.58 sec -2013-09-18 00:20:27,761 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 505.58 sec -2013-09-18 00:20:28,766 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 505.58 sec -2013-09-18 00:20:29,947 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 505.58 sec -2013-09-18 00:20:30,952 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 505.58 sec -2013-09-18 00:20:31,956 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 505.58 sec -2013-09-18 00:20:32,961 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 505.58 sec -2013-09-18 00:20:33,966 Stage-2 map = 100%, reduce = 71%, Cumulative CPU 505.58 sec -2013-09-18 00:20:34,970 Stage-2 map = 100%, reduce = 71%, Cumulative CPU 544.29 sec -2013-09-18 00:20:35,975 Stage-2 map = 100%, reduce = 71%, Cumulative CPU 544.29 sec -2013-09-18 00:20:36,979 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 544.29 sec -2013-09-18 00:20:37,984 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 544.29 sec -2013-09-18 00:20:38,988 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 544.29 sec -2013-09-18 00:20:39,999 Stage-2 map = 100%, reduce = 76%, Cumulative CPU 544.29 sec -2013-09-18 00:20:41,004 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 555.06 sec -2013-09-18 00:20:42,008 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 555.06 sec -MapReduce Total cumulative CPU time: 9 minutes 15 seconds 60 msec -Ended Job = job_201309172235_0093 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1786.03 sec HDFS Read: 843455069 HDFS Write: 2238428518 SUCCESS -Job 1: Map: 9 Reduce: 1 Cumulative CPU: 555.06 sec HDFS Read: 2238513761 HDFS Write: 278 SUCCESS -Total MapReduce CPU Time Spent: 39 minutes 1 seconds 90 msec -OK -Time taken: 234.271 seconds, Fetched: 10 row(s) -hive> quit; --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - - -times: 1 -query: SELECT UserID FROM hits_100m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7682@mturlrep13_201309180020_537514801.txt -hive> SELECT UserID FROM hits_100m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0094 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 0 -2013-09-18 00:20:59,984 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:21:09,101 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:21:10,107 Stage-1 map = 16%, reduce = 0% -2013-09-18 00:21:14,125 Stage-1 map = 32%, reduce = 0% -2013-09-18 00:21:15,130 Stage-1 map = 34%, reduce = 0% -2013-09-18 00:21:17,153 Stage-1 map = 35%, reduce = 0%, Cumulative CPU 4.53 sec -2013-09-18 00:21:19,500 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 15.48 sec -2013-09-18 00:21:20,507 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 35.24 sec -2013-09-18 00:21:21,514 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 170.31 sec -2013-09-18 00:21:22,521 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 199.34 sec -2013-09-18 00:21:23,526 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 199.34 sec -MapReduce Total cumulative CPU time: 3 minutes 19 seconds 340 msec -Ended Job = job_201309172235_0094 -MapReduce Jobs Launched: -Job 0: Map: 37 Cumulative CPU: 199.34 sec HDFS Read: 663537300 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 19 seconds 340 msec -OK -Time taken: 34.019 seconds -hive> quit; - -times: 2 -query: SELECT UserID FROM hits_100m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12482@mturlrep13_201309180021_1171483805.txt -hive> SELECT UserID FROM hits_100m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0095 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 0 -2013-09-18 00:21:32,957 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:21:43,193 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 9.02 sec -2013-09-18 00:21:44,200 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 148.9 sec -2013-09-18 00:21:45,208 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 193.53 sec -2013-09-18 00:21:46,214 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 193.53 sec -2013-09-18 00:21:47,220 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 193.53 sec -MapReduce Total cumulative CPU time: 3 minutes 13 seconds 530 msec -Ended Job = job_201309172235_0095 -MapReduce Jobs Launched: -Job 0: Map: 37 Cumulative CPU: 193.53 sec HDFS Read: 663537300 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 13 seconds 530 msec -OK -Time taken: 21.834 seconds -hive> quit; - -times: 3 -query: SELECT UserID FROM hits_100m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17377@mturlrep13_201309180021_991857587.txt -hive> SELECT UserID FROM hits_100m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0096 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 0 -2013-09-18 00:21:56,800 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:22:07,146 Stage-1 map = 26%, reduce = 0%, Cumulative CPU 4.76 sec -2013-09-18 00:22:08,183 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 190.17 sec -2013-09-18 00:22:09,190 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 196.69 sec -2013-09-18 00:22:10,195 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 196.69 sec -MapReduce Total cumulative CPU time: 3 minutes 16 seconds 690 msec -Ended Job = job_201309172235_0096 -MapReduce Jobs Launched: -Job 0: Map: 37 Cumulative CPU: 196.69 sec HDFS Read: 663537300 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 16 seconds 690 msec -OK -Time taken: 21.08 seconds -hive> quit; --- мощная фильтрация по столбцу типа UInt64.; - - -times: 1 -query: SELECT count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22377@mturlrep13_201309180022_1306519242.txt -hive> SELECT count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0097 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:22:28,083 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:22:37,160 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:22:38,170 Stage-1 map = 6%, reduce = 0% -2013-09-18 00:22:42,194 Stage-1 map = 20%, reduce = 0% -2013-09-18 00:22:46,226 Stage-1 map = 46%, reduce = 0% -2013-09-18 00:22:47,241 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 10.63 sec -2013-09-18 00:22:48,259 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.62 sec -2013-09-18 00:22:49,271 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 38.64 sec -2013-09-18 00:22:50,278 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 61.43 sec -2013-09-18 00:22:51,542 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 244.69 sec -2013-09-18 00:22:52,548 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 321.75 sec -2013-09-18 00:22:53,555 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 417.5 sec -2013-09-18 00:22:54,560 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 417.5 sec -2013-09-18 00:22:55,566 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 417.5 sec -2013-09-18 00:22:56,572 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 417.5 sec -2013-09-18 00:22:57,581 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 420.38 sec -2013-09-18 00:22:58,586 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 420.38 sec -MapReduce Total cumulative CPU time: 7 minutes 0 seconds 380 msec -Ended Job = job_201309172235_0097 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 420.38 sec HDFS Read: 882764316 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 0 seconds 380 msec -OK -5238 -Time taken: 41.195 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27348@mturlrep13_201309180023_143827785.txt -hive> SELECT count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0098 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:23:09,675 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:23:19,749 Stage-1 map = 10%, reduce = 0% -2013-09-18 00:23:20,754 Stage-1 map = 12%, reduce = 0% -2013-09-18 00:23:21,760 Stage-1 map = 13%, reduce = 0% -2013-09-18 00:23:22,765 Stage-1 map = 30%, reduce = 0% -2013-09-18 00:23:23,779 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 8.29 sec -2013-09-18 00:23:24,790 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 8.29 sec -2013-09-18 00:23:25,805 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 38.21 sec -2013-09-18 00:23:26,811 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 125.77 sec -2013-09-18 00:23:27,819 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 297.36 sec -2013-09-18 00:23:28,825 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 425.9 sec -2013-09-18 00:23:29,830 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 439.17 sec -2013-09-18 00:23:30,836 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 439.17 sec -2013-09-18 00:23:31,841 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 439.17 sec -2013-09-18 00:23:32,847 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 439.17 sec -2013-09-18 00:23:33,853 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 439.17 sec -2013-09-18 00:23:34,858 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 439.17 sec -2013-09-18 00:23:35,865 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 441.94 sec -2013-09-18 00:23:36,871 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 441.94 sec -MapReduce Total cumulative CPU time: 7 minutes 21 seconds 940 msec -Ended Job = job_201309172235_0098 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 441.94 sec HDFS Read: 882764316 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 21 seconds 940 msec -OK -5238 -Time taken: 36.443 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32317@mturlrep13_201309180023_1607167986.txt -hive> SELECT count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0099 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:23:47,464 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:23:56,520 Stage-1 map = 3%, reduce = 0% -2013-09-18 00:23:57,526 Stage-1 map = 11%, reduce = 0% -2013-09-18 00:23:59,535 Stage-1 map = 15%, reduce = 0% -2013-09-18 00:24:00,541 Stage-1 map = 38%, reduce = 0% -2013-09-18 00:24:01,559 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 8.66 sec -2013-09-18 00:24:02,572 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 8.66 sec -2013-09-18 00:24:03,580 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 50.46 sec -2013-09-18 00:24:04,591 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 152.61 sec -2013-09-18 00:24:05,597 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 361.23 sec -2013-09-18 00:24:06,603 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 427.75 sec -2013-09-18 00:24:07,609 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 441.29 sec -2013-09-18 00:24:08,613 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 441.29 sec -2013-09-18 00:24:09,618 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 441.29 sec -2013-09-18 00:24:10,624 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 441.29 sec -2013-09-18 00:24:11,696 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 441.29 sec -2013-09-18 00:24:12,703 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 444.08 sec -2013-09-18 00:24:13,709 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 444.08 sec -2013-09-18 00:24:14,715 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 444.08 sec -MapReduce Total cumulative CPU time: 7 minutes 24 seconds 80 msec -Ended Job = job_201309172235_0099 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 444.08 sec HDFS Read: 882764316 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 24 seconds 80 msec -OK -5238 -Time taken: 35.979 seconds, Fetched: 1 row(s) -hive> quit; --- фильтрация по поиску подстроки в строке.; - - -times: 1 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4931@mturlrep13_201309180024_377998774.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0100 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:24:32,532 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:24:41,635 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:24:42,650 Stage-1 map = 4%, reduce = 0% -2013-09-18 00:24:46,694 Stage-1 map = 15%, reduce = 0% -2013-09-18 00:24:49,790 Stage-1 map = 39%, reduce = 0% -2013-09-18 00:24:50,805 Stage-1 map = 40%, reduce = 0%, Cumulative CPU 12.57 sec -2013-09-18 00:24:54,895 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 60.14 sec -2013-09-18 00:24:55,919 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 143.58 sec -2013-09-18 00:24:56,938 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 219.04 sec -2013-09-18 00:24:57,944 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 404.21 sec -2013-09-18 00:24:58,950 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 468.52 sec -2013-09-18 00:24:59,956 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 468.52 sec -2013-09-18 00:25:00,962 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 468.52 sec -2013-09-18 00:25:01,971 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 471.46 sec -2013-09-18 00:25:02,978 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 474.57 sec -2013-09-18 00:25:03,988 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 490.52 sec -2013-09-18 00:25:04,995 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 500.12 sec -2013-09-18 00:25:06,001 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 500.12 sec -2013-09-18 00:25:07,008 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 503.18 sec -2013-09-18 00:25:08,019 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 503.18 sec -MapReduce Total cumulative CPU time: 8 minutes 23 seconds 180 msec -Ended Job = job_201309172235_0100 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0101 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 00:25:11,520 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:25:13,528 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 00:25:14,534 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 00:25:15,540 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 00:25:16,545 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 00:25:17,549 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 00:25:18,554 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 00:25:19,559 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 00:25:20,564 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.73 sec -2013-09-18 00:25:21,569 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.17 sec -2013-09-18 00:25:22,574 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.17 sec -MapReduce Total cumulative CPU time: 2 seconds 170 msec -Ended Job = job_201309172235_0101 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 503.18 sec HDFS Read: 1060099081 HDFS Write: 11615 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.17 sec HDFS Read: 15192 HDFS Write: 1152 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 25 seconds 350 msec -OK -Time taken: 60.385 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12874@mturlrep13_201309180025_385714226.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0102 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:25:33,396 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:25:43,564 Stage-1 map = 9%, reduce = 0% -2013-09-18 00:25:45,601 Stage-1 map = 10%, reduce = 0% -2013-09-18 00:25:46,613 Stage-1 map = 33%, reduce = 0% -2013-09-18 00:25:47,620 Stage-1 map = 37%, reduce = 0% -2013-09-18 00:25:48,634 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 9.38 sec -2013-09-18 00:25:49,642 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 9.38 sec -2013-09-18 00:25:50,658 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 9.38 sec -2013-09-18 00:25:51,664 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 21.89 sec -2013-09-18 00:25:52,694 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 218.15 sec -2013-09-18 00:25:53,701 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 479.2 sec -2013-09-18 00:25:54,708 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 493.05 sec -2013-09-18 00:25:55,715 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 493.05 sec -2013-09-18 00:25:56,721 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 493.05 sec -2013-09-18 00:25:57,727 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 493.05 sec -2013-09-18 00:25:58,733 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 493.05 sec -2013-09-18 00:25:59,740 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 493.05 sec -2013-09-18 00:26:00,748 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 493.05 sec -2013-09-18 00:26:01,758 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 510.23 sec -2013-09-18 00:26:02,768 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 524.01 sec -2013-09-18 00:26:03,775 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 530.35 sec -2013-09-18 00:26:04,781 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 530.35 sec -MapReduce Total cumulative CPU time: 8 minutes 50 seconds 350 msec -Ended Job = job_201309172235_0102 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0103 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 00:26:08,247 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:26:09,253 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:26:10,258 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:26:11,263 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:26:12,269 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:26:13,275 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:26:14,280 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:26:15,284 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:26:16,289 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:26:17,294 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -2013-09-18 00:26:18,299 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -2013-09-18 00:26:19,304 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -MapReduce Total cumulative CPU time: 2 seconds 350 msec -Ended Job = job_201309172235_0103 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 530.35 sec HDFS Read: 1060099081 HDFS Write: 11615 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.35 sec HDFS Read: 15192 HDFS Write: 1152 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 52 seconds 700 msec -OK -Time taken: 54.888 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19768@mturlrep13_201309180026_1068991002.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0104 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:26:30,060 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:26:39,192 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:26:40,200 Stage-1 map = 9%, reduce = 0% -2013-09-18 00:26:41,206 Stage-1 map = 10%, reduce = 0% -2013-09-18 00:26:42,212 Stage-1 map = 11%, reduce = 0% -2013-09-18 00:26:43,218 Stage-1 map = 29%, reduce = 0% -2013-09-18 00:26:44,224 Stage-1 map = 35%, reduce = 0% -2013-09-18 00:26:45,239 Stage-1 map = 40%, reduce = 0%, Cumulative CPU 9.36 sec -2013-09-18 00:26:46,268 Stage-1 map = 45%, reduce = 0%, Cumulative CPU 9.36 sec -2013-09-18 00:26:47,275 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 9.36 sec -2013-09-18 00:26:48,316 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 21.78 sec -2013-09-18 00:26:49,334 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 217.46 sec -2013-09-18 00:26:50,341 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 406.39 sec -2013-09-18 00:26:51,348 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 477.62 sec -2013-09-18 00:26:52,356 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 491.94 sec -2013-09-18 00:26:53,362 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 491.94 sec -2013-09-18 00:26:54,368 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 491.94 sec -2013-09-18 00:26:55,374 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 491.94 sec -2013-09-18 00:26:56,380 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 491.94 sec -2013-09-18 00:26:57,388 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 491.94 sec -2013-09-18 00:26:58,397 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 504.87 sec -2013-09-18 00:26:59,404 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 518.69 sec -2013-09-18 00:27:00,410 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 528.53 sec -2013-09-18 00:27:01,416 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 528.53 sec -2013-09-18 00:27:02,423 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 528.53 sec -MapReduce Total cumulative CPU time: 8 minutes 48 seconds 530 msec -Ended Job = job_201309172235_0104 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0105 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 00:27:06,499 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:27:07,505 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 00:27:08,522 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 00:27:09,527 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 00:27:10,532 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 00:27:11,538 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 00:27:12,543 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 00:27:13,547 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 00:27:14,552 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 00:27:15,558 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.26 sec -2013-09-18 00:27:16,563 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.26 sec -2013-09-18 00:27:17,568 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.26 sec -MapReduce Total cumulative CPU time: 2 seconds 260 msec -Ended Job = job_201309172235_0105 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 528.53 sec HDFS Read: 1060099081 HDFS Write: 11615 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.26 sec HDFS Read: 15192 HDFS Write: 1152 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 50 seconds 790 msec -OK -Time taken: 56.541 seconds, Fetched: 10 row(s) -hive> quit; --- вынимаем большие столбцы, фильтрация по строке.; - - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26625@mturlrep13_201309180027_303480636.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0106 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:27:35,507 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:27:45,586 Stage-1 map = 2%, reduce = 0% -2013-09-18 00:27:48,661 Stage-1 map = 10%, reduce = 0% -2013-09-18 00:27:50,957 Stage-1 map = 14%, reduce = 0% -2013-09-18 00:27:51,973 Stage-1 map = 20%, reduce = 0% -2013-09-18 00:27:53,983 Stage-1 map = 22%, reduce = 0% -2013-09-18 00:27:56,028 Stage-1 map = 35%, reduce = 0% -2013-09-18 00:27:58,127 Stage-1 map = 40%, reduce = 0%, Cumulative CPU 12.89 sec -2013-09-18 00:27:59,135 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 12.89 sec -2013-09-18 00:28:00,141 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.18 sec -2013-09-18 00:28:01,159 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 27.18 sec -2013-09-18 00:28:02,167 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 59.55 sec -2013-09-18 00:28:04,208 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 92.71 sec -2013-09-18 00:28:05,214 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 175.04 sec -2013-09-18 00:28:06,329 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 341.42 sec -2013-09-18 00:28:07,336 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 556.8 sec -2013-09-18 00:28:08,342 Stage-1 map = 100%, reduce = 2%, Cumulative CPU 608.33 sec -2013-09-18 00:28:09,348 Stage-1 map = 100%, reduce = 2%, Cumulative CPU 608.33 sec -2013-09-18 00:28:10,354 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 608.33 sec -2013-09-18 00:28:11,362 Stage-1 map = 100%, reduce = 20%, Cumulative CPU 611.74 sec -2013-09-18 00:28:12,369 Stage-1 map = 100%, reduce = 47%, Cumulative CPU 622.43 sec -2013-09-18 00:28:13,376 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 632.58 sec -2013-09-18 00:28:14,383 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 632.58 sec -2013-09-18 00:28:15,389 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 643.54 sec -2013-09-18 00:28:16,396 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 647.01 sec -2013-09-18 00:28:17,401 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 647.01 sec -MapReduce Total cumulative CPU time: 10 minutes 47 seconds 10 msec -Ended Job = job_201309172235_0106 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0107 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 00:28:19,874 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:28:21,883 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.61 sec -2013-09-18 00:28:22,887 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.61 sec -2013-09-18 00:28:23,892 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.61 sec -2013-09-18 00:28:24,896 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.61 sec -2013-09-18 00:28:25,900 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.61 sec -2013-09-18 00:28:26,904 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.61 sec -2013-09-18 00:28:27,908 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.61 sec -2013-09-18 00:28:28,912 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.61 sec -2013-09-18 00:28:29,917 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.61 sec -2013-09-18 00:28:30,923 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.4 sec -2013-09-18 00:28:31,928 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.4 sec -MapReduce Total cumulative CPU time: 3 seconds 400 msec -Ended Job = job_201309172235_0107 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 647.01 sec HDFS Read: 2479854627 HDFS Write: 1092954 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.4 sec HDFS Read: 1096531 HDFS Write: 1307 SUCCESS -Total MapReduce CPU Time Spent: 10 minutes 50 seconds 410 msec -OK -Time taken: 67.257 seconds, Fetched: 10 row(s) -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1658@mturlrep13_201309180028_324429197.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0108 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:28:42,812 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:28:52,951 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:28:54,963 Stage-1 map = 11%, reduce = 0% -2013-09-18 00:28:55,970 Stage-1 map = 18%, reduce = 0% -2013-09-18 00:28:56,983 Stage-1 map = 19%, reduce = 0% -2013-09-18 00:28:57,990 Stage-1 map = 26%, reduce = 0% -2013-09-18 00:28:59,003 Stage-1 map = 29%, reduce = 0% -2013-09-18 00:29:00,009 Stage-1 map = 32%, reduce = 0% -2013-09-18 00:29:01,027 Stage-1 map = 39%, reduce = 0% -2013-09-18 00:29:02,034 Stage-1 map = 47%, reduce = 0% -2013-09-18 00:29:03,043 Stage-1 map = 51%, reduce = 0% -2013-09-18 00:29:04,091 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 30.86 sec -2013-09-18 00:29:05,114 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 63.18 sec -2013-09-18 00:29:06,133 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 233.25 sec -2013-09-18 00:29:07,141 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 514.05 sec -2013-09-18 00:29:08,148 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 641.77 sec -2013-09-18 00:29:09,154 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 641.77 sec -2013-09-18 00:29:10,160 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 641.77 sec -2013-09-18 00:29:11,166 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 641.77 sec -2013-09-18 00:29:12,173 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 641.77 sec -2013-09-18 00:29:13,179 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 641.77 sec -2013-09-18 00:29:14,186 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 641.77 sec -2013-09-18 00:29:15,211 Stage-1 map = 100%, reduce = 45%, Cumulative CPU 648.85 sec -2013-09-18 00:29:16,218 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 679.72 sec -2013-09-18 00:29:17,224 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 683.49 sec -2013-09-18 00:29:18,230 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 683.49 sec -MapReduce Total cumulative CPU time: 11 minutes 23 seconds 490 msec -Ended Job = job_201309172235_0108 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0109 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 00:29:20,736 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:29:23,749 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-18 00:29:24,754 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-18 00:29:25,759 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-18 00:29:26,763 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-18 00:29:27,767 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-18 00:29:28,772 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-18 00:29:29,776 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-18 00:29:30,782 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.6 sec -2013-09-18 00:29:31,787 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.35 sec -2013-09-18 00:29:32,792 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.35 sec -MapReduce Total cumulative CPU time: 3 seconds 350 msec -Ended Job = job_201309172235_0109 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 683.49 sec HDFS Read: 2479854627 HDFS Write: 1092954 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.35 sec HDFS Read: 1096531 HDFS Write: 1307 SUCCESS -Total MapReduce CPU Time Spent: 11 minutes 26 seconds 840 msec -OK -Time taken: 58.995 seconds, Fetched: 10 row(s) -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9465@mturlrep13_201309180029_1752618536.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0110 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:29:42,551 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:29:53,652 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:29:56,667 Stage-1 map = 16%, reduce = 0% -2013-09-18 00:29:57,673 Stage-1 map = 21%, reduce = 0% -2013-09-18 00:29:58,678 Stage-1 map = 26%, reduce = 0% -2013-09-18 00:29:59,684 Stage-1 map = 28%, reduce = 0% -2013-09-18 00:30:00,690 Stage-1 map = 31%, reduce = 0% -2013-09-18 00:30:01,707 Stage-1 map = 38%, reduce = 0% -2013-09-18 00:30:02,717 Stage-1 map = 43%, reduce = 0% -2013-09-18 00:30:03,733 Stage-1 map = 51%, reduce = 0% -2013-09-18 00:30:04,774 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 14.24 sec -2013-09-18 00:30:05,800 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 45.73 sec -2013-09-18 00:30:06,808 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 128.94 sec -2013-09-18 00:30:07,818 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 335.92 sec -2013-09-18 00:30:08,826 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 549.46 sec -2013-09-18 00:30:09,832 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 641.18 sec -2013-09-18 00:30:10,838 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 641.18 sec -2013-09-18 00:30:11,844 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 641.18 sec -2013-09-18 00:30:12,850 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 641.18 sec -2013-09-18 00:30:13,856 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 641.18 sec -2013-09-18 00:30:14,863 Stage-1 map = 100%, reduce = 15%, Cumulative CPU 641.18 sec -2013-09-18 00:30:15,874 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 644.72 sec -2013-09-18 00:30:16,882 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 667.58 sec -2013-09-18 00:30:17,888 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 682.62 sec -2013-09-18 00:30:18,895 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 682.62 sec -2013-09-18 00:30:19,901 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 682.62 sec -MapReduce Total cumulative CPU time: 11 minutes 22 seconds 620 msec -Ended Job = job_201309172235_0110 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0111 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 00:30:23,450 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:30:25,458 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.57 sec -2013-09-18 00:30:26,463 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.57 sec -2013-09-18 00:30:27,468 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.57 sec -2013-09-18 00:30:28,472 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.57 sec -2013-09-18 00:30:29,476 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.57 sec -2013-09-18 00:30:30,480 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.57 sec -2013-09-18 00:30:31,484 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.57 sec -2013-09-18 00:30:32,489 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.57 sec -2013-09-18 00:30:33,494 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.37 sec -2013-09-18 00:30:34,500 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.37 sec -MapReduce Total cumulative CPU time: 3 seconds 370 msec -Ended Job = job_201309172235_0111 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 682.62 sec HDFS Read: 2479854627 HDFS Write: 1092954 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.37 sec HDFS Read: 1096531 HDFS Write: 1307 SUCCESS -Total MapReduce CPU Time Spent: 11 minutes 25 seconds 990 msec -OK -Time taken: 59.828 seconds, Fetched: 10 row(s) -hive> quit; --- чуть больше столбцы.; - - -times: 1 -query: SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17569@mturlrep13_201309180030_564255388.txt -hive> SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0112 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:30:52,920 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:31:12,034 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:31:15,535 Stage-1 map = 3%, reduce = 0% -2013-09-18 00:31:16,540 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:31:17,545 Stage-1 map = 7%, reduce = 0% -2013-09-18 00:31:18,551 Stage-1 map = 10%, reduce = 0% -2013-09-18 00:31:19,555 Stage-1 map = 13%, reduce = 0% -2013-09-18 00:31:21,568 Stage-1 map = 15%, reduce = 0% -2013-09-18 00:31:22,572 Stage-1 map = 17%, reduce = 0% -2013-09-18 00:31:23,581 Stage-1 map = 19%, reduce = 0% -2013-09-18 00:31:24,586 Stage-1 map = 20%, reduce = 0% -2013-09-18 00:31:25,609 Stage-1 map = 25%, reduce = 0% -2013-09-18 00:31:26,614 Stage-1 map = 27%, reduce = 0% -2013-09-18 00:31:27,633 Stage-1 map = 28%, reduce = 0% -2013-09-18 00:31:28,641 Stage-1 map = 30%, reduce = 0% -2013-09-18 00:31:29,675 Stage-1 map = 32%, reduce = 0% -2013-09-18 00:31:30,698 Stage-1 map = 33%, reduce = 0% -2013-09-18 00:31:32,058 Stage-1 map = 36%, reduce = 0% -2013-09-18 00:31:33,079 Stage-1 map = 38%, reduce = 0% -2013-09-18 00:31:34,102 Stage-1 map = 40%, reduce = 0% -2013-09-18 00:31:35,117 Stage-1 map = 44%, reduce = 0% -2013-09-18 00:31:36,122 Stage-1 map = 45%, reduce = 0% -2013-09-18 00:31:37,152 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 1131.99 sec -2013-09-18 00:31:38,167 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 1131.99 sec -2013-09-18 00:31:39,186 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:40,202 Stage-1 map = 51%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:41,208 Stage-1 map = 54%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:42,221 Stage-1 map = 58%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:43,226 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:44,286 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:45,292 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:46,298 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:47,305 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:48,325 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:49,332 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:50,340 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 1135.59 sec -2013-09-18 00:31:51,346 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 1150.66 sec -2013-09-18 00:31:52,357 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 1150.66 sec -2013-09-18 00:31:53,363 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 1201.45 sec -2013-09-18 00:31:54,370 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 1286.34 sec -2013-09-18 00:31:55,375 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 1340.63 sec -2013-09-18 00:31:56,384 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 1412.26 sec -2013-09-18 00:31:57,396 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 1529.38 sec -2013-09-18 00:31:58,401 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 1690.25 sec -2013-09-18 00:31:59,406 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1753.24 sec -2013-09-18 00:32:00,412 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1777.14 sec -2013-09-18 00:32:01,416 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1828.2 sec -2013-09-18 00:32:02,421 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1828.2 sec -2013-09-18 00:32:03,426 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1828.2 sec -2013-09-18 00:32:04,430 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1828.2 sec -2013-09-18 00:32:05,435 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1828.2 sec -2013-09-18 00:32:06,439 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1828.2 sec -2013-09-18 00:32:07,444 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1828.2 sec -2013-09-18 00:32:08,451 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1831.77 sec -2013-09-18 00:32:09,457 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1831.77 sec -MapReduce Total cumulative CPU time: 30 minutes 31 seconds 770 msec -Ended Job = job_201309172235_0112 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 1831.77 sec HDFS Read: 10081441390 HDFS Write: 5802 SUCCESS -Total MapReduce CPU Time Spent: 30 minutes 31 seconds 770 msec -OK -Time taken: 87.697 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22644@mturlrep13_201309180032_2112140807.txt -hive> SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0113 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:32:23,647 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:32:44,899 Stage-1 map = 1%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:45,910 Stage-1 map = 4%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:46,921 Stage-1 map = 5%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:47,927 Stage-1 map = 7%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:48,933 Stage-1 map = 9%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:49,944 Stage-1 map = 12%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:50,950 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:51,968 Stage-1 map = 16%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:52,974 Stage-1 map = 18%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:53,981 Stage-1 map = 19%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:54,996 Stage-1 map = 21%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:56,002 Stage-1 map = 26%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:57,007 Stage-1 map = 28%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:58,206 Stage-1 map = 30%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:32:59,212 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:00,218 Stage-1 map = 33%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:01,225 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:02,242 Stage-1 map = 37%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:03,248 Stage-1 map = 40%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:04,253 Stage-1 map = 41%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:05,260 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:06,266 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:07,271 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:08,282 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:09,287 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 144.44 sec -2013-09-18 00:33:10,300 Stage-1 map = 54%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:11,313 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:12,329 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:13,349 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:14,361 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:15,370 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:16,376 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:17,382 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:18,389 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:19,406 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:20,411 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:21,417 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:22,426 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 174.77 sec -2013-09-18 00:33:23,432 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 345.6 sec -2013-09-18 00:33:24,439 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 435.05 sec -2013-09-18 00:33:25,444 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 477.09 sec -2013-09-18 00:33:26,450 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 930.38 sec -2013-09-18 00:33:27,467 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 1352.12 sec -2013-09-18 00:33:28,477 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 1492.34 sec -2013-09-18 00:33:29,483 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 1684.33 sec -2013-09-18 00:33:30,488 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1684.33 sec -2013-09-18 00:33:31,494 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1784.31 sec -2013-09-18 00:33:32,499 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1839.12 sec -2013-09-18 00:33:33,504 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1839.12 sec -2013-09-18 00:33:34,509 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1839.12 sec -2013-09-18 00:33:35,515 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1839.12 sec -2013-09-18 00:33:36,519 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1839.12 sec -2013-09-18 00:33:37,525 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1839.93 sec -2013-09-18 00:33:38,530 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1839.93 sec -2013-09-18 00:33:39,537 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1842.69 sec -2013-09-18 00:33:40,543 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1842.69 sec -MapReduce Total cumulative CPU time: 30 minutes 42 seconds 690 msec -Ended Job = job_201309172235_0113 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 1842.69 sec HDFS Read: 10081441390 HDFS Write: 5802 SUCCESS -Total MapReduce CPU Time Spent: 30 minutes 42 seconds 690 msec -OK -Time taken: 88.521 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27658@mturlrep13_201309180033_1797641182.txt -hive> SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0114 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:33:53,935 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:34:14,165 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:34:16,361 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:34:17,366 Stage-1 map = 6%, reduce = 0% -2013-09-18 00:34:18,371 Stage-1 map = 7%, reduce = 0% -2013-09-18 00:34:19,376 Stage-1 map = 10%, reduce = 0% -2013-09-18 00:34:20,389 Stage-1 map = 14%, reduce = 0% -2013-09-18 00:34:21,394 Stage-1 map = 15%, reduce = 0% -2013-09-18 00:34:22,404 Stage-1 map = 18%, reduce = 0% -2013-09-18 00:34:23,409 Stage-1 map = 19%, reduce = 0% -2013-09-18 00:34:25,642 Stage-1 map = 22%, reduce = 0% -2013-09-18 00:34:26,649 Stage-1 map = 27%, reduce = 0% -2013-09-18 00:34:28,674 Stage-1 map = 28%, reduce = 0% -2013-09-18 00:34:29,680 Stage-1 map = 31%, reduce = 0% -2013-09-18 00:34:30,686 Stage-1 map = 32%, reduce = 0% -2013-09-18 00:34:31,692 Stage-1 map = 34%, reduce = 0% -2013-09-18 00:34:32,697 Stage-1 map = 36%, reduce = 0% -2013-09-18 00:34:33,702 Stage-1 map = 39%, reduce = 0% -2013-09-18 00:34:34,719 Stage-1 map = 40%, reduce = 0% -2013-09-18 00:34:35,725 Stage-1 map = 42%, reduce = 0% -2013-09-18 00:34:36,730 Stage-1 map = 44%, reduce = 0% -2013-09-18 00:34:37,740 Stage-1 map = 45%, reduce = 0%, Cumulative CPU 1134.9 sec -2013-09-18 00:34:38,752 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 1134.9 sec -2013-09-18 00:34:39,768 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 1134.9 sec -2013-09-18 00:34:40,779 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 1134.9 sec -2013-09-18 00:34:41,816 Stage-1 map = 54%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:42,825 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:43,831 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:44,837 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:45,843 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:46,856 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:47,862 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:48,880 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:49,889 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:50,902 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:51,908 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 1140.46 sec -2013-09-18 00:34:52,913 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 1171.26 sec -2013-09-18 00:34:53,952 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 1186.68 sec -2013-09-18 00:34:54,957 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 1255.59 sec -2013-09-18 00:34:55,970 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 1310.59 sec -2013-09-18 00:34:56,976 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 1386.73 sec -2013-09-18 00:34:57,981 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 1506.92 sec -2013-09-18 00:34:58,986 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 1683.99 sec -2013-09-18 00:34:59,991 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 1723.75 sec -2013-09-18 00:35:00,997 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 1772.53 sec -2013-09-18 00:35:02,002 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1772.53 sec -2013-09-18 00:35:03,006 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1826.62 sec -2013-09-18 00:35:04,011 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1855.25 sec -2013-09-18 00:35:05,015 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1855.25 sec -2013-09-18 00:35:06,019 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1855.25 sec -2013-09-18 00:35:07,023 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1855.25 sec -2013-09-18 00:35:08,028 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1855.25 sec -2013-09-18 00:35:09,033 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1855.25 sec -2013-09-18 00:35:10,038 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1855.25 sec -2013-09-18 00:35:11,045 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1858.98 sec -2013-09-18 00:35:12,051 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1858.98 sec -MapReduce Total cumulative CPU time: 30 minutes 58 seconds 980 msec -Ended Job = job_201309172235_0114 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 1858.98 sec HDFS Read: 10081441390 HDFS Write: 5802 SUCCESS -Total MapReduce CPU Time Spent: 30 minutes 58 seconds 980 msec -OK -Time taken: 89.324 seconds, Fetched: 10 row(s) -hive> quit; --- плохой запрос - вынимаем все столбцы.; - - -times: 1 -query: SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_637@mturlrep13_201309180035_636558816.txt -hive> SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0115 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:35:29,995 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:35:41,126 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:35:42,132 Stage-1 map = 4%, reduce = 0% -2013-09-18 00:35:43,139 Stage-1 map = 12%, reduce = 0% -2013-09-18 00:35:44,146 Stage-1 map = 16%, reduce = 0% -2013-09-18 00:35:45,160 Stage-1 map = 18%, reduce = 0%, Cumulative CPU 8.89 sec -2013-09-18 00:35:46,172 Stage-1 map = 18%, reduce = 0%, Cumulative CPU 8.89 sec -2013-09-18 00:35:47,181 Stage-1 map = 18%, reduce = 0%, Cumulative CPU 8.89 sec -2013-09-18 00:35:48,823 Stage-1 map = 42%, reduce = 0%, Cumulative CPU 19.53 sec -2013-09-18 00:35:50,089 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 92.18 sec -2013-09-18 00:35:51,103 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 137.0 sec -2013-09-18 00:35:52,111 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 217.4 sec -2013-09-18 00:35:53,117 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 317.92 sec -2013-09-18 00:35:54,123 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 435.87 sec -2013-09-18 00:35:55,129 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 435.87 sec -2013-09-18 00:35:56,135 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 435.87 sec -2013-09-18 00:35:57,141 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 435.87 sec -2013-09-18 00:35:58,148 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 435.87 sec -2013-09-18 00:35:59,154 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 435.87 sec -2013-09-18 00:36:00,159 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 435.87 sec -2013-09-18 00:36:01,165 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 435.87 sec -2013-09-18 00:36:02,170 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 435.87 sec -2013-09-18 00:36:03,175 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 435.87 sec -2013-09-18 00:36:04,180 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 435.87 sec -2013-09-18 00:36:05,186 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 435.87 sec -2013-09-18 00:36:06,191 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 435.87 sec -2013-09-18 00:36:07,197 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 435.87 sec -2013-09-18 00:36:08,202 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 435.87 sec -2013-09-18 00:36:09,208 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 435.87 sec -2013-09-18 00:36:10,212 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 435.87 sec -2013-09-18 00:36:11,217 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 435.87 sec -2013-09-18 00:36:12,224 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 457.03 sec -2013-09-18 00:36:13,230 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 457.03 sec -MapReduce Total cumulative CPU time: 7 minutes 37 seconds 30 msec -Ended Job = job_201309172235_0115 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 457.03 sec HDFS Read: 184864723 HDFS Write: 747 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 37 seconds 30 msec -OK -Time taken: 53.796 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6758@mturlrep13_201309180036_334325394.txt -hive> SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0116 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:36:23,502 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:36:35,575 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:36:36,583 Stage-1 map = 9%, reduce = 0% -2013-09-18 00:36:37,595 Stage-1 map = 16%, reduce = 0%, Cumulative CPU 232.24 sec -2013-09-18 00:36:38,607 Stage-1 map = 24%, reduce = 0%, Cumulative CPU 232.24 sec -2013-09-18 00:36:39,625 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 232.24 sec -2013-09-18 00:36:40,671 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 246.94 sec -2013-09-18 00:36:41,688 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 294.67 sec -2013-09-18 00:36:42,699 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 362.37 sec -2013-09-18 00:36:43,705 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 448.5 sec -2013-09-18 00:36:44,711 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 448.5 sec -2013-09-18 00:36:45,717 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 448.5 sec -2013-09-18 00:36:46,723 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 448.5 sec -2013-09-18 00:36:47,729 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 448.5 sec -2013-09-18 00:36:48,735 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 448.5 sec -2013-09-18 00:36:49,741 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 448.5 sec -2013-09-18 00:36:50,747 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 448.5 sec -2013-09-18 00:36:51,753 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 448.5 sec -2013-09-18 00:36:52,759 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 448.5 sec -2013-09-18 00:36:53,764 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 448.5 sec -2013-09-18 00:36:54,770 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 448.5 sec -2013-09-18 00:36:55,776 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 448.5 sec -2013-09-18 00:36:56,782 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 448.5 sec -2013-09-18 00:36:57,789 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 448.5 sec -2013-09-18 00:36:58,796 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 448.5 sec -2013-09-18 00:36:59,802 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 448.5 sec -2013-09-18 00:37:00,808 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 448.5 sec -2013-09-18 00:37:01,814 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 448.5 sec -2013-09-18 00:37:02,820 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 448.5 sec -2013-09-18 00:37:03,828 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 469.6 sec -2013-09-18 00:37:04,834 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 469.6 sec -2013-09-18 00:37:05,840 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 469.6 sec -MapReduce Total cumulative CPU time: 7 minutes 49 seconds 600 msec -Ended Job = job_201309172235_0116 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 469.6 sec HDFS Read: 184864723 HDFS Write: 747 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 49 seconds 600 msec -OK -Time taken: 50.592 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12112@mturlrep13_201309180037_615192245.txt -hive> SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0117 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:37:16,411 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:37:27,531 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:37:28,541 Stage-1 map = 11%, reduce = 0% -2013-09-18 00:37:29,559 Stage-1 map = 19%, reduce = 0% -2013-09-18 00:37:30,564 Stage-1 map = 24%, reduce = 0% -2013-09-18 00:37:31,580 Stage-1 map = 51%, reduce = 0%, Cumulative CPU 9.16 sec -2013-09-18 00:37:32,592 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 47.94 sec -2013-09-18 00:37:33,600 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 167.83 sec -2013-09-18 00:37:34,607 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 256.71 sec -2013-09-18 00:37:35,612 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 445.74 sec -2013-09-18 00:37:36,617 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 445.74 sec -2013-09-18 00:37:37,622 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 445.74 sec -2013-09-18 00:37:38,627 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 445.74 sec -2013-09-18 00:37:39,632 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 445.74 sec -2013-09-18 00:37:40,637 Stage-1 map = 100%, reduce = 11%, Cumulative CPU 445.74 sec -2013-09-18 00:37:41,643 Stage-1 map = 100%, reduce = 11%, Cumulative CPU 445.74 sec -2013-09-18 00:37:42,648 Stage-1 map = 100%, reduce = 11%, Cumulative CPU 445.74 sec -2013-09-18 00:37:43,654 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 445.74 sec -2013-09-18 00:37:44,659 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 445.74 sec -2013-09-18 00:37:45,664 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 445.74 sec -2013-09-18 00:37:46,669 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 445.74 sec -2013-09-18 00:37:47,675 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 445.74 sec -2013-09-18 00:37:48,680 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 445.74 sec -2013-09-18 00:37:49,685 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 445.74 sec -2013-09-18 00:37:50,691 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 445.74 sec -2013-09-18 00:37:51,696 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 445.74 sec -2013-09-18 00:37:52,701 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 445.74 sec -2013-09-18 00:37:53,706 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 445.74 sec -2013-09-18 00:37:54,711 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 445.74 sec -2013-09-18 00:37:55,718 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 467.25 sec -2013-09-18 00:37:56,723 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 467.25 sec -2013-09-18 00:37:57,729 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 467.25 sec -MapReduce Total cumulative CPU time: 7 minutes 47 seconds 250 msec -Ended Job = job_201309172235_0117 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 467.25 sec HDFS Read: 184864723 HDFS Write: 747 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 47 seconds 250 msec -OK -Time taken: 50.118 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка.; - - -times: 1 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17453@mturlrep13_201309180038_1517091963.txt -hive> SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0118 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:38:15,283 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:38:25,343 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:38:27,409 Stage-1 map = 4%, reduce = 0% -2013-09-18 00:38:28,422 Stage-1 map = 14%, reduce = 0% -2013-09-18 00:38:29,428 Stage-1 map = 20%, reduce = 0% -2013-09-18 00:38:31,442 Stage-1 map = 46%, reduce = 0% -2013-09-18 00:38:32,447 Stage-1 map = 54%, reduce = 0% -2013-09-18 00:38:34,502 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 56.43 sec -2013-09-18 00:38:35,526 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 160.97 sec -2013-09-18 00:38:36,540 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 264.56 sec -2013-09-18 00:38:37,546 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 421.84 sec -2013-09-18 00:38:38,552 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 433.68 sec -2013-09-18 00:38:39,557 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 442.48 sec -2013-09-18 00:38:40,562 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 442.48 sec -2013-09-18 00:38:41,567 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 442.48 sec -2013-09-18 00:38:42,573 Stage-1 map = 100%, reduce = 5%, Cumulative CPU 442.48 sec -2013-09-18 00:38:43,579 Stage-1 map = 100%, reduce = 5%, Cumulative CPU 442.48 sec -2013-09-18 00:38:44,585 Stage-1 map = 100%, reduce = 5%, Cumulative CPU 442.48 sec -2013-09-18 00:38:45,590 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 442.48 sec -2013-09-18 00:38:46,596 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 442.48 sec -2013-09-18 00:38:47,602 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 442.48 sec -2013-09-18 00:38:48,607 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 442.48 sec -2013-09-18 00:38:49,613 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 442.48 sec -2013-09-18 00:38:50,618 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 442.48 sec -2013-09-18 00:38:51,624 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 442.48 sec -2013-09-18 00:38:52,629 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 442.48 sec -2013-09-18 00:38:53,635 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 442.48 sec -2013-09-18 00:38:54,640 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 442.48 sec -2013-09-18 00:38:55,645 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 442.48 sec -2013-09-18 00:38:56,649 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 442.48 sec -2013-09-18 00:38:57,654 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 442.48 sec -2013-09-18 00:38:58,658 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 442.48 sec -2013-09-18 00:38:59,663 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 442.48 sec -2013-09-18 00:39:00,668 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 442.48 sec -2013-09-18 00:39:01,673 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 442.48 sec -2013-09-18 00:39:02,677 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 442.48 sec -2013-09-18 00:39:03,682 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 442.48 sec -2013-09-18 00:39:04,689 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 477.8 sec -2013-09-18 00:39:08,034 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 477.8 sec -2013-09-18 00:39:09,045 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 477.8 sec -2013-09-18 00:39:10,050 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 477.8 sec -2013-09-18 00:39:11,056 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 477.8 sec -2013-09-18 00:39:12,062 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 477.8 sec -MapReduce Total cumulative CPU time: 7 minutes 57 seconds 800 msec -Ended Job = job_201309172235_0118 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 477.8 sec HDFS Read: 182281719 HDFS Write: 509 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 57 seconds 800 msec -OK - снип 2.07.01-89 - 19105-79 -! 20 самых ужасные пыток в истории человечества. видео -! bt -! гобилен желтый купить ) -! жизнь в тюрьме россии - -! минимальное значение измеряемой величины? -! новый htc 603e -! о чем говорит огромное количество антител в крови к кишечной палочке -Time taken: 68.786 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22553@mturlrep13_201309180039_34144715.txt -hive> SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0119 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:39:23,346 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:39:35,450 Stage-1 map = 2%, reduce = 0% -2013-09-18 00:39:36,456 Stage-1 map = 19%, reduce = 0% -2013-09-18 00:39:37,472 Stage-1 map = 33%, reduce = 0%, Cumulative CPU 259.71 sec -2013-09-18 00:39:38,480 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 259.71 sec -2013-09-18 00:39:39,499 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 267.32 sec -2013-09-18 00:39:40,516 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 282.8 sec -2013-09-18 00:39:41,528 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 308.59 sec -2013-09-18 00:39:42,536 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 377.66 sec -2013-09-18 00:39:43,542 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 452.74 sec -2013-09-18 00:39:44,548 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 452.74 sec -2013-09-18 00:39:45,553 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 452.74 sec -2013-09-18 00:39:46,559 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 452.74 sec -2013-09-18 00:39:47,565 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 452.74 sec -2013-09-18 00:39:48,572 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 452.74 sec -2013-09-18 00:39:49,578 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 452.74 sec -2013-09-18 00:39:50,584 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 452.74 sec -2013-09-18 00:39:51,590 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 452.74 sec -2013-09-18 00:39:52,596 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 452.74 sec -2013-09-18 00:39:53,602 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 452.74 sec -2013-09-18 00:39:54,607 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.74 sec -2013-09-18 00:39:55,613 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.74 sec -2013-09-18 00:39:56,619 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.74 sec -2013-09-18 00:39:57,625 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.74 sec -2013-09-18 00:39:58,630 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.74 sec -2013-09-18 00:39:59,636 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.74 sec -2013-09-18 00:40:00,642 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.74 sec -2013-09-18 00:40:01,648 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.74 sec -2013-09-18 00:40:02,655 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.74 sec -2013-09-18 00:40:03,660 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 452.74 sec -2013-09-18 00:40:04,665 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 452.74 sec -2013-09-18 00:40:05,670 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 452.74 sec -2013-09-18 00:40:06,675 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 452.74 sec -2013-09-18 00:40:07,680 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 452.74 sec -2013-09-18 00:40:08,686 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 452.74 sec -2013-09-18 00:40:09,691 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 452.74 sec -2013-09-18 00:40:10,699 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 487.91 sec -2013-09-18 00:40:11,704 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 487.91 sec -2013-09-18 00:40:12,710 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 487.91 sec -MapReduce Total cumulative CPU time: 8 minutes 7 seconds 910 msec -Ended Job = job_201309172235_0119 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 487.91 sec HDFS Read: 182281719 HDFS Write: 509 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 7 seconds 910 msec -OK - снип 2.07.01-89 - 19105-79 -! 20 самых ужасные пыток в истории человечества. видео -! bt -! гобилен желтый купить ) -! жизнь в тюрьме россии - -! минимальное значение измеряемой величины? -! новый htc 603e -! о чем говорит огромное количество антител в крови к кишечной палочке -Time taken: 63.993 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27974@mturlrep13_201309180040_814853077.txt -hive> SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0120 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:40:30,016 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:40:42,091 Stage-1 map = 12%, reduce = 0% -2013-09-18 00:40:43,103 Stage-1 map = 28%, reduce = 0% -2013-09-18 00:40:44,113 Stage-1 map = 34%, reduce = 0% -2013-09-18 00:40:45,135 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 18.32 sec -2013-09-18 00:40:46,146 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 77.03 sec -2013-09-18 00:40:47,154 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 151.48 sec -2013-09-18 00:40:48,161 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 201.55 sec -2013-09-18 00:40:49,167 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 395.4 sec -2013-09-18 00:40:50,173 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 452.5 sec -2013-09-18 00:40:51,178 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 452.5 sec -2013-09-18 00:40:52,184 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 452.5 sec -2013-09-18 00:40:53,259 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 452.5 sec -2013-09-18 00:40:54,265 Stage-1 map = 100%, reduce = 7%, Cumulative CPU 452.5 sec -2013-09-18 00:40:55,271 Stage-1 map = 100%, reduce = 7%, Cumulative CPU 452.5 sec -2013-09-18 00:40:56,281 Stage-1 map = 100%, reduce = 7%, Cumulative CPU 452.5 sec -2013-09-18 00:40:57,288 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 452.5 sec -2013-09-18 00:40:58,293 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 452.5 sec -2013-09-18 00:40:59,298 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 452.5 sec -2013-09-18 00:41:00,303 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.5 sec -2013-09-18 00:41:01,307 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.5 sec -2013-09-18 00:41:02,312 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.5 sec -2013-09-18 00:41:03,317 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.5 sec -2013-09-18 00:41:04,322 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.5 sec -2013-09-18 00:41:05,326 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.5 sec -2013-09-18 00:41:06,331 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.5 sec -2013-09-18 00:41:07,336 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.5 sec -2013-09-18 00:41:08,341 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 452.5 sec -2013-09-18 00:41:09,346 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 452.5 sec -2013-09-18 00:41:10,351 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 452.5 sec -2013-09-18 00:41:11,356 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 452.5 sec -2013-09-18 00:41:12,360 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 452.5 sec -2013-09-18 00:41:13,365 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 452.5 sec -2013-09-18 00:41:14,369 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 452.5 sec -2013-09-18 00:41:15,374 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 452.5 sec -2013-09-18 00:41:16,381 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 487.48 sec -2013-09-18 00:41:17,387 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 487.48 sec -2013-09-18 00:41:18,392 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 487.48 sec -MapReduce Total cumulative CPU time: 8 minutes 7 seconds 480 msec -Ended Job = job_201309172235_0120 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 487.48 sec HDFS Read: 182281719 HDFS Write: 509 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 7 seconds 480 msec -OK - снип 2.07.01-89 - 19105-79 -! 20 самых ужасные пыток в истории человечества. видео -! bt -! гобилен желтый купить ) -! жизнь в тюрьме россии - -! минимальное значение измеряемой величины? -! новый htc 603e -! о чем говорит огромное количество антител в крови к кишечной палочке -Time taken: 62.886 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка по строкам.; - - -times: 1 -query: SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_964@mturlrep13_201309180041_1295667471.txt -hive> SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0121 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:41:41,880 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:41:53,981 Stage-1 map = 3%, reduce = 0% -2013-09-18 00:41:54,989 Stage-1 map = 10%, reduce = 0% -2013-09-18 00:41:55,998 Stage-1 map = 16%, reduce = 0% -2013-09-18 00:41:57,005 Stage-1 map = 24%, reduce = 0% -2013-09-18 00:41:58,010 Stage-1 map = 34%, reduce = 0% -2013-09-18 00:41:59,015 Stage-1 map = 39%, reduce = 0% -2013-09-18 00:42:00,845 Stage-1 map = 51%, reduce = 0% -2013-09-18 00:42:02,853 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 116.0 sec -2013-09-18 00:42:03,921 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 228.22 sec -2013-09-18 00:42:04,929 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 296.35 sec -2013-09-18 00:42:05,935 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 412.74 sec -2013-09-18 00:42:06,941 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 470.11 sec -2013-09-18 00:42:07,947 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 470.11 sec -2013-09-18 00:42:08,953 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 470.11 sec -2013-09-18 00:42:09,962 Stage-1 map = 100%, reduce = 5%, Cumulative CPU 470.11 sec -2013-09-18 00:42:10,968 Stage-1 map = 100%, reduce = 5%, Cumulative CPU 470.11 sec -2013-09-18 00:42:11,974 Stage-1 map = 100%, reduce = 5%, Cumulative CPU 470.11 sec -2013-09-18 00:42:12,980 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 470.11 sec -2013-09-18 00:42:13,985 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 470.11 sec -2013-09-18 00:42:14,991 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 470.11 sec -2013-09-18 00:42:15,996 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 470.11 sec -2013-09-18 00:42:17,001 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 470.11 sec -2013-09-18 00:42:18,007 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 470.11 sec -2013-09-18 00:42:19,012 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 470.11 sec -2013-09-18 00:42:20,018 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 470.11 sec -2013-09-18 00:42:21,024 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 470.11 sec -2013-09-18 00:42:22,030 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 470.11 sec -2013-09-18 00:42:23,036 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 470.11 sec -2013-09-18 00:42:24,040 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 470.11 sec -2013-09-18 00:42:25,044 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 470.11 sec -2013-09-18 00:42:26,049 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 470.11 sec -2013-09-18 00:42:27,054 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 470.11 sec -2013-09-18 00:42:28,059 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 470.11 sec -2013-09-18 00:42:29,064 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 470.11 sec -2013-09-18 00:42:30,071 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 503.72 sec -2013-09-18 00:42:31,077 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 503.72 sec -2013-09-18 00:42:32,083 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 503.72 sec -MapReduce Total cumulative CPU time: 8 minutes 23 seconds 720 msec -Ended Job = job_201309172235_0121 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 503.72 sec HDFS Read: 184864723 HDFS Write: 758 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 23 seconds 720 msec -OK -Time taken: 60.715 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6757@mturlrep13_201309180042_1836094631.txt -hive> SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0122 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:42:42,848 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:42:53,935 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:42:54,967 Stage-1 map = 7%, reduce = 0% -2013-09-18 00:42:55,986 Stage-1 map = 15%, reduce = 0% -2013-09-18 00:42:56,991 Stage-1 map = 19%, reduce = 0% -2013-09-18 00:42:58,003 Stage-1 map = 37%, reduce = 0% -2013-09-18 00:42:59,028 Stage-1 map = 58%, reduce = 0%, Cumulative CPU 20.21 sec -2013-09-18 00:43:00,042 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 95.48 sec -2013-09-18 00:43:01,060 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 203.64 sec -2013-09-18 00:43:02,067 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 316.18 sec -2013-09-18 00:43:03,074 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 451.06 sec -2013-09-18 00:43:04,079 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 482.61 sec -2013-09-18 00:43:05,085 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 482.61 sec -2013-09-18 00:43:06,090 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 482.61 sec -2013-09-18 00:43:07,095 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 482.61 sec -2013-09-18 00:43:08,100 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 482.61 sec -2013-09-18 00:43:09,105 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 482.61 sec -2013-09-18 00:43:10,112 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 482.61 sec -2013-09-18 00:43:11,119 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 482.61 sec -2013-09-18 00:43:12,124 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 482.61 sec -2013-09-18 00:43:13,130 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 482.61 sec -2013-09-18 00:43:14,136 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 482.61 sec -2013-09-18 00:43:15,141 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 482.61 sec -2013-09-18 00:43:16,147 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 482.61 sec -2013-09-18 00:43:17,152 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 482.61 sec -2013-09-18 00:43:18,158 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 482.61 sec -2013-09-18 00:43:19,162 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 482.61 sec -2013-09-18 00:43:20,167 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 482.61 sec -2013-09-18 00:43:21,172 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 482.61 sec -2013-09-18 00:43:22,177 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 482.61 sec -2013-09-18 00:43:23,182 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 482.61 sec -2013-09-18 00:43:24,186 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 482.61 sec -2013-09-18 00:43:25,190 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 482.61 sec -2013-09-18 00:43:26,195 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 482.61 sec -2013-09-18 00:43:27,200 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 482.61 sec -2013-09-18 00:43:28,206 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 517.47 sec -2013-09-18 00:43:29,827 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 517.47 sec -MapReduce Total cumulative CPU time: 8 minutes 37 seconds 470 msec -Ended Job = job_201309172235_0122 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 517.47 sec HDFS Read: 184864723 HDFS Write: 758 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 37 seconds 470 msec -OK -Time taken: 56.14 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12116@mturlrep13_201309180043_688174400.txt -hive> SELECT SearchPhrase, EventTime FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0123 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:43:39,885 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:43:52,084 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:43:53,089 Stage-1 map = 9%, reduce = 0% -2013-09-18 00:43:54,103 Stage-1 map = 21%, reduce = 0% -2013-09-18 00:43:55,109 Stage-1 map = 22%, reduce = 0% -2013-09-18 00:43:56,124 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 9.78 sec -2013-09-18 00:43:57,139 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 49.2 sec -2013-09-18 00:43:58,156 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 146.89 sec -2013-09-18 00:43:59,166 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 211.3 sec -2013-09-18 00:44:00,173 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 339.17 sec -2013-09-18 00:44:01,179 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 474.94 sec -2013-09-18 00:44:02,184 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 474.94 sec -2013-09-18 00:44:03,190 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 474.94 sec -2013-09-18 00:44:04,196 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 474.94 sec -2013-09-18 00:44:05,203 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 474.94 sec -2013-09-18 00:44:06,209 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 474.94 sec -2013-09-18 00:44:07,216 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 474.94 sec -2013-09-18 00:44:08,222 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 474.94 sec -2013-09-18 00:44:09,228 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 474.94 sec -2013-09-18 00:44:10,235 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 474.94 sec -2013-09-18 00:44:11,241 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 474.94 sec -2013-09-18 00:44:12,247 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 474.94 sec -2013-09-18 00:44:13,253 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 474.94 sec -2013-09-18 00:44:14,258 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 474.94 sec -2013-09-18 00:44:15,264 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 474.94 sec -2013-09-18 00:44:16,270 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 474.94 sec -2013-09-18 00:44:17,277 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 474.94 sec -2013-09-18 00:44:18,283 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 474.94 sec -2013-09-18 00:44:19,290 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 474.94 sec -2013-09-18 00:44:20,296 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 474.94 sec -2013-09-18 00:44:21,301 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 474.94 sec -2013-09-18 00:44:22,306 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 474.94 sec -2013-09-18 00:44:23,312 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 474.94 sec -2013-09-18 00:44:24,318 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 474.94 sec -2013-09-18 00:44:25,324 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 474.94 sec -2013-09-18 00:44:26,332 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 511.22 sec -2013-09-18 00:44:27,337 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 511.22 sec -MapReduce Total cumulative CPU time: 8 minutes 31 seconds 220 msec -Ended Job = job_201309172235_0123 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 511.22 sec HDFS Read: 184864723 HDFS Write: 758 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 31 seconds 220 msec -OK -Time taken: 55.303 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка по кортежу.; - - -times: 1 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17460@mturlrep13_201309180044_876855052.txt -hive> SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0124 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:44:53,239 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:45:09,346 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:45:10,353 Stage-1 map = 7%, reduce = 0% -2013-09-18 00:45:11,358 Stage-1 map = 8%, reduce = 0% -2013-09-18 00:45:12,365 Stage-1 map = 11%, reduce = 0% -2013-09-18 00:45:14,374 Stage-1 map = 14%, reduce = 0% -2013-09-18 00:45:15,389 Stage-1 map = 17%, reduce = 0% -2013-09-18 00:45:16,396 Stage-1 map = 21%, reduce = 0% -2013-09-18 00:45:17,410 Stage-1 map = 25%, reduce = 0% -2013-09-18 00:45:18,415 Stage-1 map = 28%, reduce = 0% -2013-09-18 00:45:19,438 Stage-1 map = 33%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:20,445 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:22,432 Stage-1 map = 40%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:23,470 Stage-1 map = 45%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:24,485 Stage-1 map = 45%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:25,493 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:27,100 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:28,112 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:29,121 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:30,130 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:31,442 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:32,466 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 14.93 sec -2013-09-18 00:45:33,482 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 72.87 sec -2013-09-18 00:45:34,504 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 152.52 sec -2013-09-18 00:45:35,718 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 324.07 sec -2013-09-18 00:45:36,726 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 615.07 sec -2013-09-18 00:45:37,738 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 861.87 sec -2013-09-18 00:45:38,744 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1085.12 sec -2013-09-18 00:45:39,750 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:40,756 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:41,761 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:42,772 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:43,781 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:44,801 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:45,818 Stage-1 map = 100%, reduce = 15%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:46,825 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:47,831 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:48,838 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:49,845 Stage-1 map = 100%, reduce = 44%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:50,852 Stage-1 map = 100%, reduce = 47%, Cumulative CPU 1087.12 sec -2013-09-18 00:45:51,861 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 1098.67 sec -2013-09-18 00:45:52,867 Stage-1 map = 100%, reduce = 60%, Cumulative CPU 1098.67 sec -2013-09-18 00:45:53,875 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1098.67 sec -2013-09-18 00:45:54,882 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1098.67 sec -2013-09-18 00:45:55,889 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1114.66 sec -2013-09-18 00:45:56,895 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 1114.66 sec -2013-09-18 00:45:57,902 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1132.03 sec -2013-09-18 00:45:58,908 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 1132.03 sec -2013-09-18 00:45:59,915 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1132.03 sec -2013-09-18 00:46:00,922 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1175.97 sec -2013-09-18 00:46:01,929 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1199.97 sec -2013-09-18 00:46:02,936 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1199.97 sec -2013-09-18 00:46:03,943 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1199.97 sec -2013-09-18 00:46:04,949 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1199.97 sec -2013-09-18 00:46:05,955 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1199.97 sec -2013-09-18 00:46:10,612 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1224.95 sec -2013-09-18 00:46:11,618 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1224.95 sec -2013-09-18 00:46:13,271 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1250.63 sec -2013-09-18 00:46:14,276 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1250.63 sec -2013-09-18 00:46:15,344 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1277.48 sec -2013-09-18 00:46:16,350 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1333.41 sec -2013-09-18 00:46:17,356 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1333.41 sec -2013-09-18 00:46:20,396 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1333.41 sec -MapReduce Total cumulative CPU time: 22 minutes 13 seconds 410 msec -Ended Job = job_201309172235_0124 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0125 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 00:46:24,314 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:46:25,319 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:46:26,325 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:46:27,330 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:46:28,335 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:46:29,340 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:46:30,345 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:46:31,350 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:46:32,355 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 00:46:33,360 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -2013-09-18 00:46:34,366 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -2013-09-18 00:46:35,371 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -MapReduce Total cumulative CPU time: 2 seconds 290 msec -Ended Job = job_201309172235_0125 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1333.41 sec HDFS Read: 978257057 HDFS Write: 4324 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.29 sec HDFS Read: 7901 HDFS Write: 790 SUCCESS -Total MapReduce CPU Time Spent: 22 minutes 15 seconds 700 msec -OK -Time taken: 112.58 seconds, Fetched: 25 row(s) -hive> quit; - -times: 2 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25268@mturlrep13_201309180046_2058731462.txt -hive> SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0126 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:46:46,178 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:46:59,296 Stage-1 map = 4%, reduce = 0% -2013-09-18 00:47:00,310 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:47:01,316 Stage-1 map = 6%, reduce = 0% -2013-09-18 00:47:03,326 Stage-1 map = 8%, reduce = 0% -2013-09-18 00:47:04,331 Stage-1 map = 13%, reduce = 0% -2013-09-18 00:47:05,337 Stage-1 map = 16%, reduce = 0% -2013-09-18 00:47:06,343 Stage-1 map = 19%, reduce = 0% -2013-09-18 00:47:07,364 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:08,406 Stage-1 map = 30%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:09,418 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:10,425 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:11,459 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:12,470 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:13,476 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:14,517 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:15,536 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:16,546 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:17,560 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:18,584 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:19,602 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 14.43 sec -2013-09-18 00:47:20,610 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 40.87 sec -2013-09-18 00:47:21,619 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 69.23 sec -2013-09-18 00:47:22,628 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 237.25 sec -2013-09-18 00:47:23,638 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 408.41 sec -2013-09-18 00:47:24,670 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 645.37 sec -2013-09-18 00:47:25,677 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 955.16 sec -2013-09-18 00:47:26,685 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 987.34 sec -2013-09-18 00:47:27,691 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:28,696 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:29,702 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:30,722 Stage-1 map = 100%, reduce = 4%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:31,729 Stage-1 map = 100%, reduce = 15%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:32,735 Stage-1 map = 100%, reduce = 16%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:33,741 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:34,747 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:35,753 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:36,759 Stage-1 map = 100%, reduce = 40%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:37,765 Stage-1 map = 100%, reduce = 45%, Cumulative CPU 1085.3 sec -2013-09-18 00:47:38,774 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 1196.28 sec -2013-09-18 00:47:39,781 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 1196.28 sec -2013-09-18 00:47:40,787 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1196.28 sec -2013-09-18 00:47:41,794 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1196.28 sec -2013-09-18 00:47:42,800 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1201.93 sec -2013-09-18 00:47:43,806 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1201.93 sec -2013-09-18 00:47:44,820 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 1209.07 sec -2013-09-18 00:47:45,826 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 1209.07 sec -2013-09-18 00:47:46,832 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 1209.07 sec -2013-09-18 00:47:47,838 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1220.62 sec -2013-09-18 00:47:48,844 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1220.62 sec -2013-09-18 00:47:49,850 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1246.48 sec -2013-09-18 00:47:50,856 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1260.92 sec -2013-09-18 00:47:51,862 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1276.65 sec -2013-09-18 00:47:52,868 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1293.47 sec -2013-09-18 00:47:53,873 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:47:56,015 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:47:57,021 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:47:58,026 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:47:59,031 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:48:00,036 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:48:01,041 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:48:02,046 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:48:03,366 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:48:04,371 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:48:05,376 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1293.47 sec -2013-09-18 00:48:06,382 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1328.31 sec -2013-09-18 00:48:09,722 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1328.31 sec -MapReduce Total cumulative CPU time: 22 minutes 8 seconds 310 msec -Ended Job = job_201309172235_0126 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0127 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 00:48:13,466 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:48:15,474 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-18 00:48:16,478 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-18 00:48:17,481 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-18 00:48:18,485 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-18 00:48:19,489 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-18 00:48:20,494 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-18 00:48:21,497 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-18 00:48:22,502 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.38 sec -2013-09-18 00:48:23,508 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.38 sec -2013-09-18 00:48:24,513 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.38 sec -MapReduce Total cumulative CPU time: 2 seconds 380 msec -Ended Job = job_201309172235_0127 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1328.31 sec HDFS Read: 978257057 HDFS Write: 4324 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.38 sec HDFS Read: 7901 HDFS Write: 790 SUCCESS -Total MapReduce CPU Time Spent: 22 minutes 10 seconds 690 msec -OK -Time taken: 107.297 seconds, Fetched: 25 row(s) -hive> quit; - -times: 3 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32317@mturlrep13_201309180048_1314756439.txt -hive> SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0128 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:48:35,184 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:48:48,323 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:48:49,329 Stage-1 map = 7%, reduce = 0% -2013-09-18 00:48:53,356 Stage-1 map = 11%, reduce = 0% -2013-09-18 00:48:54,367 Stage-1 map = 15%, reduce = 0% -2013-09-18 00:48:55,377 Stage-1 map = 18%, reduce = 0% -2013-09-18 00:48:56,389 Stage-1 map = 23%, reduce = 0% -2013-09-18 00:48:57,404 Stage-1 map = 29%, reduce = 0% -2013-09-18 00:48:58,435 Stage-1 map = 32%, reduce = 0% -2013-09-18 00:48:59,450 Stage-1 map = 37%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:00,470 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:01,482 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:02,500 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:03,514 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:04,524 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:05,530 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:06,542 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:07,548 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:08,560 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:09,568 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 15.22 sec -2013-09-18 00:49:10,574 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 69.2 sec -2013-09-18 00:49:11,591 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 154.6 sec -2013-09-18 00:49:12,600 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 386.29 sec -2013-09-18 00:49:13,616 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 684.84 sec -2013-09-18 00:49:14,628 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 904.62 sec -2013-09-18 00:49:15,634 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 999.79 sec -2013-09-18 00:49:16,640 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:17,645 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:18,651 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:19,657 Stage-1 map = 100%, reduce = 2%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:20,664 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:21,693 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:22,703 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:23,709 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:24,715 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:25,721 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:26,728 Stage-1 map = 100%, reduce = 41%, Cumulative CPU 1097.36 sec -2013-09-18 00:49:27,736 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 1108.08 sec -2013-09-18 00:49:28,742 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 1108.08 sec -2013-09-18 00:49:29,748 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1108.08 sec -2013-09-18 00:49:30,754 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1108.08 sec -2013-09-18 00:49:31,760 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 1124.02 sec -2013-09-18 00:49:32,766 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1139.64 sec -2013-09-18 00:49:33,772 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 1139.64 sec -2013-09-18 00:49:34,778 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 1139.64 sec -2013-09-18 00:49:35,783 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 1139.64 sec -2013-09-18 00:49:36,789 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1160.79 sec -2013-09-18 00:49:37,795 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1160.79 sec -2013-09-18 00:49:38,801 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1313.18 sec -2013-09-18 00:49:39,807 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1314.93 sec -2013-09-18 00:49:40,813 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1321.52 sec -2013-09-18 00:49:41,819 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1321.52 sec -2013-09-18 00:49:42,825 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1332.94 sec -2013-09-18 00:49:43,911 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1332.94 sec -2013-09-18 00:49:44,917 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1332.94 sec -2013-09-18 00:49:45,922 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1332.94 sec -2013-09-18 00:49:46,926 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1332.94 sec -2013-09-18 00:49:47,931 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1340.83 sec -2013-09-18 00:49:50,236 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1340.83 sec -2013-09-18 00:49:51,240 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1340.83 sec -2013-09-18 00:49:52,245 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1340.83 sec -2013-09-18 00:49:53,256 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1340.83 sec -MapReduce Total cumulative CPU time: 22 minutes 20 seconds 830 msec -Ended Job = job_201309172235_0128 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0129 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 00:50:01,798 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:50:03,805 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 00:50:04,810 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 00:50:05,814 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 00:50:06,819 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 00:50:07,823 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 00:50:08,828 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 00:50:09,832 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 00:50:10,837 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.79 sec -2013-09-18 00:50:11,843 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.4 sec -2013-09-18 00:50:12,847 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.4 sec -MapReduce Total cumulative CPU time: 2 seconds 400 msec -Ended Job = job_201309172235_0129 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1340.83 sec HDFS Read: 978257057 HDFS Write: 4324 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.4 sec HDFS Read: 7901 HDFS Write: 790 SUCCESS -Total MapReduce CPU Time Spent: 22 minutes 23 seconds 230 msec -OK -Time taken: 106.493 seconds, Fetched: 25 row(s) -hive> quit; --- считаем средние длины URL для крупных счётчиков.; - - -times: 1 -query: SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8110@mturlrep13_201309180050_337201518.txt -hive> SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -FAILED: SemanticException [Error 10011]: Line 1:337 Invalid function 'GREATEST' -hive> quit; - -times: 2 -query: SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8329@mturlrep13_201309180050_1201800197.txt -hive> SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -FAILED: SemanticException [Error 10011]: Line 1:337 Invalid function 'GREATEST' -hive> quit; - -times: 3 -query: SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8542@mturlrep13_201309180050_468390903.txt -hive> SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -FAILED: SemanticException [Error 10011]: Line 1:337 Invalid function 'GREATEST' -hive> quit; --- то же самое, но с разбивкой по доменам.; - - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8757@mturlrep13_201309180050_2105633958.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0130 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:50:56,035 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:51:21,228 Stage-1 map = 1%, reduce = 0% -2013-09-18 00:51:22,233 Stage-1 map = 4%, reduce = 0% -2013-09-18 00:51:23,242 Stage-1 map = 5%, reduce = 0% -2013-09-18 00:51:24,247 Stage-1 map = 6%, reduce = 0% -2013-09-18 00:51:25,251 Stage-1 map = 11%, reduce = 0% -2013-09-18 00:51:26,256 Stage-1 map = 13%, reduce = 0% -2013-09-18 00:51:28,265 Stage-1 map = 14%, reduce = 0% -2013-09-18 00:51:34,323 Stage-1 map = 17%, reduce = 0% -2013-09-18 00:51:35,333 Stage-1 map = 19%, reduce = 0% -2013-09-18 00:51:36,342 Stage-1 map = 20%, reduce = 0% -2013-09-18 00:51:37,347 Stage-1 map = 21%, reduce = 0% -2013-09-18 00:51:38,352 Stage-1 map = 25%, reduce = 0% -2013-09-18 00:51:39,363 Stage-1 map = 26%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:40,382 Stage-1 map = 26%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:41,395 Stage-1 map = 27%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:42,410 Stage-1 map = 27%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:43,416 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:44,421 Stage-1 map = 31%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:45,427 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:46,437 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:47,444 Stage-1 map = 33%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:48,450 Stage-1 map = 33%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:49,455 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:50,461 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:51,466 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:52,472 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:53,478 Stage-1 map = 41%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:54,483 Stage-1 map = 42%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:55,489 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:56,494 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:57,500 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:58,535 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:51:59,548 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:00,553 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:01,575 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:02,586 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:03,591 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:04,597 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:05,602 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:06,608 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:07,614 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:08,653 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:09,659 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:10,687 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:11,702 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:13,057 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:14,078 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:15,103 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:16,131 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:17,155 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:18,174 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:19,198 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 1178.1 sec -2013-09-18 00:52:20,210 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 1212.03 sec -2013-09-18 00:52:21,248 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 1212.03 sec -2013-09-18 00:52:22,262 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 1212.03 sec -2013-09-18 00:52:23,274 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 1212.03 sec -2013-09-18 00:52:24,280 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 1212.03 sec -2013-09-18 00:52:25,290 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 1212.03 sec -2013-09-18 00:52:26,311 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 1212.03 sec -2013-09-18 00:52:27,322 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 1257.13 sec -2013-09-18 00:52:28,327 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 1347.07 sec -2013-09-18 00:52:29,333 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 1347.07 sec -2013-09-18 00:52:30,346 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 1393.25 sec -2013-09-18 00:52:31,358 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 1483.18 sec -2013-09-18 00:52:32,373 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 1674.0 sec -2013-09-18 00:52:33,378 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 1768.61 sec -2013-09-18 00:52:34,383 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 1815.43 sec -2013-09-18 00:52:35,389 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 1865.46 sec -2013-09-18 00:52:36,395 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 2120.58 sec -2013-09-18 00:52:37,401 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 2333.45 sec -2013-09-18 00:52:38,406 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 2333.45 sec -2013-09-18 00:52:39,412 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 3023.48 sec -2013-09-18 00:52:40,418 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 3027.43 sec -2013-09-18 00:52:41,424 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 3039.12 sec -2013-09-18 00:52:42,429 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 3047.25 sec -2013-09-18 00:52:43,434 Stage-1 map = 98%, reduce = 25%, Cumulative CPU 3052.91 sec -2013-09-18 00:52:44,440 Stage-1 map = 99%, reduce = 25%, Cumulative CPU 3052.91 sec -2013-09-18 00:52:45,453 Stage-1 map = 99%, reduce = 25%, Cumulative CPU 3052.91 sec -2013-09-18 00:52:46,459 Stage-1 map = 99%, reduce = 25%, Cumulative CPU 3061.93 sec -2013-09-18 00:52:47,464 Stage-1 map = 99%, reduce = 25%, Cumulative CPU 3061.93 sec -2013-09-18 00:52:48,469 Stage-1 map = 99%, reduce = 25%, Cumulative CPU 3061.93 sec -2013-09-18 00:52:49,474 Stage-1 map = 99%, reduce = 25%, Cumulative CPU 3061.93 sec -2013-09-18 00:52:50,480 Stage-1 map = 99%, reduce = 25%, Cumulative CPU 3061.93 sec -2013-09-18 00:52:51,485 Stage-1 map = 99%, reduce = 25%, Cumulative CPU 3061.93 sec -2013-09-18 00:52:52,490 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 3091.79 sec -2013-09-18 00:52:53,495 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 3091.79 sec -2013-09-18 00:52:54,500 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 3091.79 sec -2013-09-18 00:52:55,504 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 3091.79 sec -2013-09-18 00:52:56,509 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 3091.79 sec -2013-09-18 00:52:57,514 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 3091.79 sec -2013-09-18 00:52:58,521 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 3095.82 sec -2013-09-18 00:52:59,529 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 3095.82 sec -2013-09-18 00:53:00,534 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 3095.82 sec -MapReduce Total cumulative CPU time: 51 minutes 35 seconds 820 msec -Ended Job = job_201309172235_0130 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 3095.82 sec HDFS Read: 77419054 HDFS Write: 1170 SUCCESS -Total MapReduce CPU Time Spent: 51 minutes 35 seconds 820 msec -OK -Time taken: 136.36 seconds, Fetched: 1 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13963@mturlrep13_201309180053_177355133.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0131 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:53:11,219 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:53:36,459 Stage-1 map = 2%, reduce = 0% -2013-09-18 00:53:38,478 Stage-1 map = 4%, reduce = 0% -2013-09-18 00:53:39,508 Stage-1 map = 7%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:40,526 Stage-1 map = 10%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:41,534 Stage-1 map = 13%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:42,546 Stage-1 map = 13%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:43,564 Stage-1 map = 13%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:44,570 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:45,576 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:46,582 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:47,588 Stage-1 map = 15%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:48,594 Stage-1 map = 15%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:49,600 Stage-1 map = 17%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:50,610 Stage-1 map = 20%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:51,667 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:52,674 Stage-1 map = 23%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:53,680 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:54,685 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:55,691 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:56,703 Stage-1 map = 27%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:57,708 Stage-1 map = 28%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:58,720 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:53:59,732 Stage-1 map = 30%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:00,738 Stage-1 map = 31%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:01,743 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:02,749 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:03,754 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:04,759 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:05,766 Stage-1 map = 37%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:06,771 Stage-1 map = 38%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:07,784 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:08,789 Stage-1 map = 40%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:09,795 Stage-1 map = 42%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:10,800 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:11,806 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:12,811 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:13,823 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:14,829 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:15,834 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:16,844 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:17,850 Stage-1 map = 51%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:18,858 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:19,871 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:20,878 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:21,883 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:22,888 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:23,894 Stage-1 map = 58%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:24,901 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:25,930 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:27,189 Stage-1 map = 62%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:28,194 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:29,201 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:30,206 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:31,220 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 679.85 sec -2013-09-18 00:54:32,227 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 726.21 sec -2013-09-18 00:54:33,241 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 726.21 sec -2013-09-18 00:54:34,246 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 726.21 sec -2013-09-18 00:54:35,255 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 726.21 sec -2013-09-18 00:54:36,261 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 726.21 sec -2013-09-18 00:54:37,266 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 726.21 sec -2013-09-18 00:54:38,275 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 726.21 sec -2013-09-18 00:54:39,280 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 2591.6 sec -2013-09-18 00:54:40,290 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 2591.6 sec -2013-09-18 00:54:41,295 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 2591.6 sec -2013-09-18 00:54:42,301 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 2596.52 sec -2013-09-18 00:54:43,327 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 2606.89 sec -2013-09-18 00:54:44,338 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 2623.82 sec -2013-09-18 00:54:45,343 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 2631.82 sec -2013-09-18 00:54:46,349 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 2631.82 sec -2013-09-18 00:54:47,355 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 2647.97 sec -2013-09-18 00:54:48,360 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 2657.08 sec -2013-09-18 00:54:49,367 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 2679.32 sec -2013-09-18 00:54:50,372 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 2712.4 sec -2013-09-18 00:54:51,377 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 2738.4 sec -2013-09-18 00:54:52,383 Stage-1 map = 92%, reduce = 14%, Cumulative CPU 2779.62 sec -2013-09-18 00:54:53,388 Stage-1 map = 93%, reduce = 14%, Cumulative CPU 2809.67 sec -2013-09-18 00:54:54,399 Stage-1 map = 95%, reduce = 14%, Cumulative CPU 2872.97 sec -2013-09-18 00:54:55,404 Stage-1 map = 96%, reduce = 14%, Cumulative CPU 2923.72 sec -2013-09-18 00:54:56,410 Stage-1 map = 97%, reduce = 14%, Cumulative CPU 2923.72 sec -2013-09-18 00:54:57,414 Stage-1 map = 97%, reduce = 14%, Cumulative CPU 2942.34 sec -2013-09-18 00:54:58,419 Stage-1 map = 98%, reduce = 14%, Cumulative CPU 2962.2 sec -2013-09-18 00:54:59,424 Stage-1 map = 98%, reduce = 14%, Cumulative CPU 2983.52 sec -2013-09-18 00:55:00,429 Stage-1 map = 98%, reduce = 14%, Cumulative CPU 3004.99 sec -2013-09-18 00:55:01,435 Stage-1 map = 98%, reduce = 27%, Cumulative CPU 3004.99 sec -2013-09-18 00:55:02,440 Stage-1 map = 99%, reduce = 27%, Cumulative CPU 3004.99 sec -2013-09-18 00:55:03,445 Stage-1 map = 99%, reduce = 27%, Cumulative CPU 3029.77 sec -2013-09-18 00:55:04,450 Stage-1 map = 99%, reduce = 27%, Cumulative CPU 3029.77 sec -2013-09-18 00:55:05,454 Stage-1 map = 99%, reduce = 27%, Cumulative CPU 3029.77 sec -2013-09-18 00:55:06,459 Stage-1 map = 99%, reduce = 27%, Cumulative CPU 3057.4 sec -2013-09-18 00:55:07,464 Stage-1 map = 99%, reduce = 32%, Cumulative CPU 3057.4 sec -2013-09-18 00:55:08,469 Stage-1 map = 99%, reduce = 32%, Cumulative CPU 3057.4 sec -2013-09-18 00:55:09,474 Stage-1 map = 99%, reduce = 32%, Cumulative CPU 3057.4 sec -2013-09-18 00:55:10,479 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 3089.43 sec -2013-09-18 00:55:11,484 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 3089.43 sec -2013-09-18 00:55:12,489 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 3089.43 sec -2013-09-18 00:55:13,495 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 3093.95 sec -2013-09-18 00:55:14,501 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 3093.95 sec -2013-09-18 00:55:15,506 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 3093.95 sec -MapReduce Total cumulative CPU time: 51 minutes 33 seconds 950 msec -Ended Job = job_201309172235_0131 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 3093.95 sec HDFS Read: 77419054 HDFS Write: 1170 SUCCESS -Total MapReduce CPU Time Spent: 51 minutes 33 seconds 950 msec -OK -Time taken: 133.102 seconds, Fetched: 1 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19854@mturlrep13_201309180055_1882171438.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0132 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 1 -2013-09-18 00:55:27,376 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:55:51,539 Stage-1 map = 1%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:55:52,555 Stage-1 map = 2%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:55:53,565 Stage-1 map = 5%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:55:54,571 Stage-1 map = 7%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:55:55,576 Stage-1 map = 10%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:55:56,580 Stage-1 map = 12%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:55:57,586 Stage-1 map = 13%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:55:58,602 Stage-1 map = 13%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:55:59,608 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:00,614 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:01,619 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:02,625 Stage-1 map = 15%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:03,631 Stage-1 map = 15%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:04,639 Stage-1 map = 17%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:05,645 Stage-1 map = 19%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:06,650 Stage-1 map = 20%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:07,655 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:08,661 Stage-1 map = 24%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:09,666 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:10,671 Stage-1 map = 27%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:11,676 Stage-1 map = 28%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:12,681 Stage-1 map = 28%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:13,687 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:14,692 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:15,697 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:16,703 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:17,708 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:18,797 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:19,802 Stage-1 map = 35%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:20,808 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:21,813 Stage-1 map = 37%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:22,818 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:23,824 Stage-1 map = 40%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:24,829 Stage-1 map = 42%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:25,834 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:26,840 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:27,845 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:28,850 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:29,855 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:30,861 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:31,866 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:32,871 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:33,876 Stage-1 map = 51%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:34,881 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:35,887 Stage-1 map = 54%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:36,899 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:38,073 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:39,078 Stage-1 map = 58%, reduce = 0%, Cumulative CPU 28.25 sec -2013-09-18 00:56:40,093 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 2125.98 sec -2013-09-18 00:56:41,098 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 2125.98 sec -2013-09-18 00:56:42,104 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 2125.98 sec -2013-09-18 00:56:43,109 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 2125.98 sec -2013-09-18 00:56:44,116 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:45,121 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:46,127 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:47,136 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:48,141 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:49,147 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:50,152 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:51,158 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:52,163 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:53,168 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:54,174 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 2129.75 sec -2013-09-18 00:56:55,179 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 2144.3 sec -2013-09-18 00:56:56,184 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 2144.3 sec -2013-09-18 00:56:57,189 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 2160.94 sec -2013-09-18 00:56:58,195 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 2178.2 sec -2013-09-18 00:56:59,200 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 2178.2 sec -2013-09-18 00:57:00,205 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 2199.66 sec -2013-09-18 00:57:01,210 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 2218.05 sec -2013-09-18 00:57:02,216 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 2282.19 sec -2013-09-18 00:57:03,221 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 2304.47 sec -2013-09-18 00:57:04,227 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 2349.8 sec -2013-09-18 00:57:05,233 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 2372.02 sec -2013-09-18 00:57:06,242 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 2427.89 sec -2013-09-18 00:57:07,248 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 2503.41 sec -2013-09-18 00:57:08,254 Stage-1 map = 94%, reduce = 11%, Cumulative CPU 2613.26 sec -2013-09-18 00:57:09,259 Stage-1 map = 95%, reduce = 11%, Cumulative CPU 2701.51 sec -2013-09-18 00:57:10,265 Stage-1 map = 96%, reduce = 11%, Cumulative CPU 2758.3 sec -2013-09-18 00:57:11,271 Stage-1 map = 97%, reduce = 22%, Cumulative CPU 2875.86 sec -2013-09-18 00:57:12,278 Stage-1 map = 97%, reduce = 22%, Cumulative CPU 2875.86 sec -2013-09-18 00:57:13,284 Stage-1 map = 98%, reduce = 22%, Cumulative CPU 2941.83 sec -2013-09-18 00:57:14,290 Stage-1 map = 99%, reduce = 22%, Cumulative CPU 3011.19 sec -2013-09-18 00:57:15,296 Stage-1 map = 99%, reduce = 22%, Cumulative CPU 3011.19 sec -2013-09-18 00:57:16,302 Stage-1 map = 99%, reduce = 22%, Cumulative CPU 3011.19 sec -2013-09-18 00:57:17,307 Stage-1 map = 99%, reduce = 22%, Cumulative CPU 3011.19 sec -2013-09-18 00:57:18,313 Stage-1 map = 99%, reduce = 22%, Cumulative CPU 3011.19 sec -2013-09-18 00:57:19,319 Stage-1 map = 99%, reduce = 22%, Cumulative CPU 3011.19 sec -2013-09-18 00:57:20,325 Stage-1 map = 99%, reduce = 32%, Cumulative CPU 3011.19 sec -2013-09-18 00:57:21,330 Stage-1 map = 99%, reduce = 32%, Cumulative CPU 3011.19 sec -2013-09-18 00:57:22,335 Stage-1 map = 99%, reduce = 32%, Cumulative CPU 3011.19 sec -2013-09-18 00:57:23,340 Stage-1 map = 99%, reduce = 32%, Cumulative CPU 3053.74 sec -2013-09-18 00:57:24,344 Stage-1 map = 99%, reduce = 32%, Cumulative CPU 3053.74 sec -2013-09-18 00:57:25,348 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 3099.46 sec -2013-09-18 00:57:26,352 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 3099.46 sec -2013-09-18 00:57:27,356 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 3099.46 sec -2013-09-18 00:57:28,359 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 3099.46 sec -2013-09-18 00:57:29,364 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 3099.46 sec -2013-09-18 00:57:30,368 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 3099.46 sec -2013-09-18 00:57:31,373 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 3099.46 sec -2013-09-18 00:57:32,380 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 3104.28 sec -2013-09-18 00:57:33,385 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 3104.28 sec -MapReduce Total cumulative CPU time: 51 minutes 44 seconds 280 msec -Ended Job = job_201309172235_0132 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 1 Cumulative CPU: 3104.28 sec HDFS Read: 77419054 HDFS Write: 1170 SUCCESS -Total MapReduce CPU Time Spent: 51 minutes 44 seconds 280 msec -OK -Time taken: 135.896 seconds, Fetched: 1 row(s) -hive> quit; --- много тупых агрегатных функций.; - - -times: 1 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25111@mturlrep13_201309180057_1190943875.txt -hive> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0133 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:57:50,216 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:58:00,383 Stage-1 map = 2%, reduce = 0% -2013-09-18 00:58:04,415 Stage-1 map = 8%, reduce = 0% -2013-09-18 00:58:07,430 Stage-1 map = 24%, reduce = 0% -2013-09-18 00:58:10,044 Stage-1 map = 33%, reduce = 0% -2013-09-18 00:58:11,136 Stage-1 map = 37%, reduce = 0% -2013-09-18 00:58:13,987 Stage-1 map = 45%, reduce = 0%, Cumulative CPU 10.4 sec -2013-09-18 00:58:14,998 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 10.4 sec -2013-09-18 00:58:16,016 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 44.12 sec -2013-09-18 00:58:17,029 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 44.12 sec -2013-09-18 00:58:19,269 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 44.12 sec -2013-09-18 00:58:20,277 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 54.05 sec -2013-09-18 00:58:21,283 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 54.05 sec -2013-09-18 00:58:22,290 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 217.63 sec -2013-09-18 00:58:23,297 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 293.68 sec -2013-09-18 00:58:24,303 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 354.53 sec -2013-09-18 00:58:25,310 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 431.46 sec -2013-09-18 00:58:26,316 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 532.5 sec -2013-09-18 00:58:27,322 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 532.5 sec -2013-09-18 00:58:28,329 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 532.5 sec -2013-09-18 00:58:29,337 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 532.5 sec -2013-09-18 00:58:30,344 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 532.5 sec -2013-09-18 00:58:31,351 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 532.5 sec -2013-09-18 00:58:32,359 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 532.5 sec -2013-09-18 00:58:33,367 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 532.5 sec -2013-09-18 00:58:34,374 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 532.5 sec -2013-09-18 00:58:35,379 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 532.5 sec -2013-09-18 00:58:36,387 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 600.15 sec -2013-09-18 00:58:37,394 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 600.15 sec -2013-09-18 00:58:38,402 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 622.76 sec -2013-09-18 00:58:39,408 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 656.89 sec -2013-09-18 00:58:40,415 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 656.89 sec -MapReduce Total cumulative CPU time: 10 minutes 56 seconds 890 msec -Ended Job = job_201309172235_0133 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0134 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 00:58:43,889 Stage-2 map = 0%, reduce = 0% -2013-09-18 00:58:50,910 Stage-2 map = 9%, reduce = 0% -2013-09-18 00:58:53,919 Stage-2 map = 18%, reduce = 0% -2013-09-18 00:58:56,928 Stage-2 map = 27%, reduce = 0% -2013-09-18 00:58:59,938 Stage-2 map = 45%, reduce = 0% -2013-09-18 00:59:02,948 Stage-2 map = 55%, reduce = 0% -2013-09-18 00:59:05,958 Stage-2 map = 64%, reduce = 0% -2013-09-18 00:59:08,967 Stage-2 map = 73%, reduce = 0% -2013-09-18 00:59:11,976 Stage-2 map = 82%, reduce = 0% -2013-09-18 00:59:14,991 Stage-2 map = 91%, reduce = 0% -2013-09-18 00:59:20,012 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.91 sec -2013-09-18 00:59:21,017 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.91 sec -2013-09-18 00:59:22,021 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.91 sec -2013-09-18 00:59:23,025 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.91 sec -2013-09-18 00:59:24,029 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.91 sec -2013-09-18 00:59:25,033 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.91 sec -2013-09-18 00:59:26,038 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.91 sec -2013-09-18 00:59:27,043 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 43.91 sec -2013-09-18 00:59:28,047 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 43.91 sec -2013-09-18 00:59:29,052 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 43.91 sec -2013-09-18 00:59:30,056 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 43.91 sec -2013-09-18 00:59:31,065 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 43.91 sec -2013-09-18 00:59:32,070 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.31 sec -2013-09-18 00:59:33,075 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.31 sec -MapReduce Total cumulative CPU time: 50 seconds 310 msec -Ended Job = job_201309172235_0134 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 656.89 sec HDFS Read: 613013229 HDFS Write: 191007594 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 50.31 sec HDFS Read: 191011171 HDFS Write: 399 SUCCESS -Total MapReduce CPU Time Spent: 11 minutes 47 seconds 200 msec -OK -Time taken: 113.348 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32655@mturlrep13_201309180059_1219302235.txt -hive> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0135 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 00:59:42,709 Stage-1 map = 0%, reduce = 0% -2013-09-18 00:59:54,021 Stage-1 map = 2%, reduce = 0% -2013-09-18 00:59:55,026 Stage-1 map = 6%, reduce = 0% -2013-09-18 00:59:56,031 Stage-1 map = 13%, reduce = 0% -2013-09-18 00:59:57,047 Stage-1 map = 19%, reduce = 0% -2013-09-18 00:59:58,067 Stage-1 map = 26%, reduce = 0% -2013-09-18 00:59:59,081 Stage-1 map = 39%, reduce = 0% -2013-09-18 01:00:00,095 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 11.06 sec -2013-09-18 01:00:01,103 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.66 sec -2013-09-18 01:00:02,114 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 93.97 sec -2013-09-18 01:00:03,127 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 133.19 sec -2013-09-18 01:00:04,152 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 186.98 sec -2013-09-18 01:00:05,168 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 248.68 sec -2013-09-18 01:00:06,178 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 281.8 sec -2013-09-18 01:00:07,185 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 545.59 sec -2013-09-18 01:00:08,191 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 563.72 sec -2013-09-18 01:00:09,197 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 563.72 sec -2013-09-18 01:00:10,203 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 563.72 sec -2013-09-18 01:00:11,210 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 563.72 sec -2013-09-18 01:00:12,218 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 563.72 sec -2013-09-18 01:00:13,233 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 563.72 sec -2013-09-18 01:00:14,240 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 563.72 sec -2013-09-18 01:00:15,247 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 563.72 sec -2013-09-18 01:00:16,253 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 563.72 sec -2013-09-18 01:00:17,261 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 563.72 sec -2013-09-18 01:00:18,270 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 574.8 sec -2013-09-18 01:00:19,278 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 609.56 sec -2013-09-18 01:00:20,285 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 690.28 sec -2013-09-18 01:00:21,291 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 690.28 sec -2013-09-18 01:00:22,299 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 690.28 sec -MapReduce Total cumulative CPU time: 11 minutes 30 seconds 280 msec -Ended Job = job_201309172235_0135 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0136 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 01:00:25,765 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:00:32,789 Stage-2 map = 9%, reduce = 0% -2013-09-18 01:00:35,800 Stage-2 map = 18%, reduce = 0% -2013-09-18 01:00:38,811 Stage-2 map = 27%, reduce = 0% -2013-09-18 01:00:41,823 Stage-2 map = 45%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:42,828 Stage-2 map = 45%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:43,834 Stage-2 map = 45%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:44,839 Stage-2 map = 55%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:45,844 Stage-2 map = 55%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:46,848 Stage-2 map = 55%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:47,853 Stage-2 map = 64%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:48,859 Stage-2 map = 64%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:49,864 Stage-2 map = 64%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:50,869 Stage-2 map = 73%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:51,874 Stage-2 map = 73%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:52,879 Stage-2 map = 73%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:53,885 Stage-2 map = 91%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:54,890 Stage-2 map = 91%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:55,895 Stage-2 map = 91%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:56,901 Stage-2 map = 91%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:57,906 Stage-2 map = 91%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:58,911 Stage-2 map = 91%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:00:59,916 Stage-2 map = 91%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:01:00,921 Stage-2 map = 91%, reduce = 0%, Cumulative CPU 17.28 sec -2013-09-18 01:01:01,926 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 01:01:02,936 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 01:01:03,941 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 01:01:04,945 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 01:01:05,949 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 01:01:06,954 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 01:01:07,958 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 01:01:08,963 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 43.57 sec -2013-09-18 01:01:09,968 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 43.57 sec -2013-09-18 01:01:10,972 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 43.57 sec -2013-09-18 01:01:11,976 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 43.57 sec -2013-09-18 01:01:12,981 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 43.57 sec -2013-09-18 01:01:13,986 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.9 sec -2013-09-18 01:01:14,991 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.9 sec -MapReduce Total cumulative CPU time: 49 seconds 900 msec -Ended Job = job_201309172235_0136 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 690.28 sec HDFS Read: 613013229 HDFS Write: 191007594 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 49.9 sec HDFS Read: 191011171 HDFS Write: 399 SUCCESS -Total MapReduce CPU Time Spent: 12 minutes 20 seconds 180 msec -OK -Time taken: 100.032 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8933@mturlrep13_201309180101_1921637373.txt -hive> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0137 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:01:24,685 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:01:35,760 Stage-1 map = 2%, reduce = 0% -2013-09-18 01:01:36,769 Stage-1 map = 7%, reduce = 0% -2013-09-18 01:01:37,774 Stage-1 map = 12%, reduce = 0% -2013-09-18 01:01:38,803 Stage-1 map = 17%, reduce = 0% -2013-09-18 01:01:39,847 Stage-1 map = 23%, reduce = 0% -2013-09-18 01:01:40,868 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 286.36 sec -2013-09-18 01:01:41,883 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 286.36 sec -2013-09-18 01:01:42,895 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 289.4 sec -2013-09-18 01:01:43,902 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 301.74 sec -2013-09-18 01:01:44,920 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 341.22 sec -2013-09-18 01:01:45,934 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 365.65 sec -2013-09-18 01:01:46,942 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 373.98 sec -2013-09-18 01:01:47,949 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 460.32 sec -2013-09-18 01:01:48,957 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 557.15 sec -2013-09-18 01:01:49,963 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 557.15 sec -2013-09-18 01:01:50,969 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 557.15 sec -2013-09-18 01:01:51,974 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 557.15 sec -2013-09-18 01:01:52,986 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 557.15 sec -2013-09-18 01:01:53,993 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 557.15 sec -2013-09-18 01:01:55,000 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 557.15 sec -2013-09-18 01:01:56,006 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 557.15 sec -2013-09-18 01:01:57,013 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 557.15 sec -2013-09-18 01:01:58,020 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 557.15 sec -2013-09-18 01:01:59,026 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 557.15 sec -2013-09-18 01:02:00,035 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 568.49 sec -2013-09-18 01:02:01,042 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 603.17 sec -2013-09-18 01:02:02,048 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 671.6 sec -2013-09-18 01:02:03,056 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 682.9 sec -2013-09-18 01:02:04,062 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 682.9 sec -MapReduce Total cumulative CPU time: 11 minutes 22 seconds 900 msec -Ended Job = job_201309172235_0137 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0138 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 01:02:07,547 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:02:14,570 Stage-2 map = 9%, reduce = 0% -2013-09-18 01:02:17,581 Stage-2 map = 18%, reduce = 0% -2013-09-18 01:02:20,591 Stage-2 map = 27%, reduce = 0% -2013-09-18 01:02:23,602 Stage-2 map = 45%, reduce = 0% -2013-09-18 01:02:26,613 Stage-2 map = 55%, reduce = 0% -2013-09-18 01:02:29,623 Stage-2 map = 64%, reduce = 0% -2013-09-18 01:02:32,634 Stage-2 map = 73%, reduce = 0% -2013-09-18 01:02:35,643 Stage-2 map = 91%, reduce = 0% -2013-09-18 01:02:43,669 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.8 sec -2013-09-18 01:02:44,674 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.8 sec -2013-09-18 01:02:45,679 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.8 sec -2013-09-18 01:02:46,684 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.8 sec -2013-09-18 01:02:47,688 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.8 sec -2013-09-18 01:02:48,693 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.8 sec -2013-09-18 01:02:49,698 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.8 sec -2013-09-18 01:02:50,704 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 42.8 sec -2013-09-18 01:02:51,708 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 42.8 sec -2013-09-18 01:02:52,713 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 42.8 sec -2013-09-18 01:02:53,717 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 42.8 sec -2013-09-18 01:02:54,722 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.05 sec -2013-09-18 01:02:55,728 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.05 sec -2013-09-18 01:02:56,733 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.05 sec -MapReduce Total cumulative CPU time: 49 seconds 50 msec -Ended Job = job_201309172235_0138 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 682.9 sec HDFS Read: 613013229 HDFS Write: 191007594 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 49.05 sec HDFS Read: 191011171 HDFS Write: 399 SUCCESS -Total MapReduce CPU Time Spent: 12 minutes 11 seconds 950 msec -OK -Time taken: 99.905 seconds, Fetched: 10 row(s) -hive> quit; --- сложная агрегация, для больших таблиц может не хватить оперативки.; - - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16458@mturlrep13_201309180103_1014086590.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0139 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:03:15,821 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:03:25,881 Stage-1 map = 2%, reduce = 0% -2013-09-18 01:03:28,896 Stage-1 map = 3%, reduce = 0% -2013-09-18 01:03:30,685 Stage-1 map = 11%, reduce = 0% -2013-09-18 01:03:31,692 Stage-1 map = 12%, reduce = 0% -2013-09-18 01:03:33,713 Stage-1 map = 24%, reduce = 0% -2013-09-18 01:03:34,719 Stage-1 map = 25%, reduce = 0% -2013-09-18 01:03:41,834 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 361.72 sec -2013-09-18 01:03:42,856 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 361.72 sec -2013-09-18 01:03:43,882 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 363.16 sec -2013-09-18 01:03:44,896 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 378.03 sec -2013-09-18 01:03:45,904 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 403.06 sec -2013-09-18 01:03:46,911 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 418.77 sec -2013-09-18 01:03:48,058 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 434.02 sec -2013-09-18 01:03:49,066 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 460.07 sec -2013-09-18 01:03:50,076 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 509.72 sec -2013-09-18 01:03:51,083 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 547.13 sec -2013-09-18 01:03:52,089 Stage-1 map = 100%, reduce = 2%, Cumulative CPU 547.13 sec -2013-09-18 01:03:53,096 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 547.13 sec -2013-09-18 01:03:54,105 Stage-1 map = 100%, reduce = 20%, Cumulative CPU 547.13 sec -2013-09-18 01:03:55,112 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 547.13 sec -2013-09-18 01:03:56,118 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 547.13 sec -2013-09-18 01:03:57,125 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 547.13 sec -2013-09-18 01:03:58,132 Stage-1 map = 100%, reduce = 65%, Cumulative CPU 547.13 sec -2013-09-18 01:03:59,138 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 547.13 sec -2013-09-18 01:04:00,145 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 547.13 sec -2013-09-18 01:04:01,154 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 560.31 sec -2013-09-18 01:04:02,162 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 573.82 sec -2013-09-18 01:04:03,169 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 600.97 sec -2013-09-18 01:04:04,176 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 683.89 sec -2013-09-18 01:04:05,182 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 683.89 sec -2013-09-18 01:04:06,188 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 697.41 sec -2013-09-18 01:04:07,193 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 697.41 sec -MapReduce Total cumulative CPU time: 11 minutes 37 seconds 410 msec -Ended Job = job_201309172235_0139 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0140 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 01:04:10,746 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:04:20,776 Stage-2 map = 10%, reduce = 0% -2013-09-18 01:04:23,786 Stage-2 map = 18%, reduce = 0% -2013-09-18 01:04:26,796 Stage-2 map = 28%, reduce = 0% -2013-09-18 01:04:29,806 Stage-2 map = 37%, reduce = 0% -2013-09-18 01:04:32,817 Stage-2 map = 55%, reduce = 0% -2013-09-18 01:04:38,835 Stage-2 map = 73%, reduce = 0% -2013-09-18 01:04:44,858 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 69.67 sec -2013-09-18 01:04:45,863 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 69.67 sec -2013-09-18 01:04:46,868 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 69.67 sec -2013-09-18 01:04:47,873 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 69.67 sec -2013-09-18 01:04:48,879 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 69.67 sec -2013-09-18 01:04:49,884 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 81.11 sec -2013-09-18 01:04:50,889 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 81.11 sec -2013-09-18 01:04:51,894 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 81.11 sec -2013-09-18 01:04:52,898 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 81.11 sec -2013-09-18 01:04:53,903 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 81.11 sec -2013-09-18 01:04:54,907 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 81.11 sec -2013-09-18 01:04:55,912 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 81.11 sec -2013-09-18 01:04:56,917 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 100.5 sec -2013-09-18 01:04:57,927 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 100.5 sec -2013-09-18 01:04:58,931 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 100.5 sec -2013-09-18 01:04:59,935 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 100.5 sec -2013-09-18 01:05:00,939 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 100.5 sec -2013-09-18 01:05:01,943 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 100.5 sec -2013-09-18 01:05:02,947 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 100.5 sec -2013-09-18 01:05:03,952 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 100.5 sec -2013-09-18 01:05:04,956 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 100.5 sec -2013-09-18 01:05:05,960 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 100.5 sec -2013-09-18 01:05:06,965 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 100.5 sec -2013-09-18 01:05:07,969 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 113.63 sec -2013-09-18 01:05:08,973 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 113.63 sec -MapReduce Total cumulative CPU time: 1 minutes 53 seconds 630 msec -Ended Job = job_201309172235_0140 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 697.41 sec HDFS Read: 1046532092 HDFS Write: 545445520 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 113.63 sec HDFS Read: 545449242 HDFS Write: 418 SUCCESS -Total MapReduce CPU Time Spent: 13 minutes 31 seconds 40 msec -OK -Time taken: 124.002 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24725@mturlrep13_201309180105_1494771182.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0141 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:05:19,764 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:05:29,107 Stage-1 map = 1%, reduce = 0% -2013-09-18 01:05:30,113 Stage-1 map = 4%, reduce = 0% -2013-09-18 01:05:31,124 Stage-1 map = 5%, reduce = 0% -2013-09-18 01:05:32,129 Stage-1 map = 12%, reduce = 0% -2013-09-18 01:05:33,137 Stage-1 map = 20%, reduce = 0% -2013-09-18 01:05:34,151 Stage-1 map = 22%, reduce = 0% -2013-09-18 01:05:35,158 Stage-1 map = 36%, reduce = 0% -2013-09-18 01:05:36,163 Stage-1 map = 39%, reduce = 0% -2013-09-18 01:05:37,173 Stage-1 map = 43%, reduce = 0% -2013-09-18 01:05:38,187 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 34.53 sec -2013-09-18 01:05:39,204 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 108.67 sec -2013-09-18 01:05:40,224 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 150.31 sec -2013-09-18 01:05:41,238 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 497.43 sec -2013-09-18 01:05:42,253 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 506.05 sec -2013-09-18 01:05:43,260 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 533.06 sec -2013-09-18 01:05:44,266 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 577.12 sec -2013-09-18 01:05:45,271 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 577.12 sec -2013-09-18 01:05:46,277 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 577.12 sec -2013-09-18 01:05:47,282 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 577.12 sec -2013-09-18 01:05:48,288 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 577.12 sec -2013-09-18 01:05:49,309 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 577.12 sec -2013-09-18 01:05:50,315 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 577.12 sec -2013-09-18 01:05:51,322 Stage-1 map = 100%, reduce = 43%, Cumulative CPU 577.12 sec -2013-09-18 01:05:52,329 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 577.12 sec -2013-09-18 01:05:53,335 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 577.12 sec -2013-09-18 01:05:54,342 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 577.12 sec -2013-09-18 01:05:55,349 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 577.12 sec -2013-09-18 01:05:56,355 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 577.12 sec -2013-09-18 01:05:57,364 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 590.71 sec -2013-09-18 01:05:58,370 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 646.07 sec -2013-09-18 01:05:59,388 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 729.12 sec -2013-09-18 01:06:00,394 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 729.12 sec -2013-09-18 01:06:01,400 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 729.12 sec -MapReduce Total cumulative CPU time: 12 minutes 9 seconds 120 msec -Ended Job = job_201309172235_0141 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0142 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 01:06:03,914 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:06:14,948 Stage-2 map = 8%, reduce = 0% -2013-09-18 01:06:17,957 Stage-2 map = 18%, reduce = 0% -2013-09-18 01:06:20,967 Stage-2 map = 37%, reduce = 0% -2013-09-18 01:06:26,985 Stage-2 map = 55%, reduce = 0% -2013-09-18 01:06:33,003 Stage-2 map = 73%, reduce = 0% -2013-09-18 01:06:39,021 Stage-2 map = 82%, reduce = 0% -2013-09-18 01:06:44,041 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 87.48 sec -2013-09-18 01:06:45,046 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 87.48 sec -2013-09-18 01:06:46,051 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 87.48 sec -2013-09-18 01:06:47,056 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 87.48 sec -2013-09-18 01:06:48,061 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 87.48 sec -2013-09-18 01:06:49,066 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 87.48 sec -2013-09-18 01:06:50,071 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 99.29 sec -2013-09-18 01:06:51,076 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 99.29 sec -2013-09-18 01:06:52,092 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 99.29 sec -2013-09-18 01:06:53,097 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 99.29 sec -2013-09-18 01:06:54,101 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 99.29 sec -2013-09-18 01:06:55,106 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 99.29 sec -2013-09-18 01:06:56,110 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 99.29 sec -2013-09-18 01:06:57,115 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 99.29 sec -2013-09-18 01:06:58,119 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 99.29 sec -2013-09-18 01:06:59,123 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 99.29 sec -2013-09-18 01:07:00,127 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 99.29 sec -2013-09-18 01:07:01,132 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 99.29 sec -2013-09-18 01:07:02,136 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 99.29 sec -2013-09-18 01:07:03,140 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 99.29 sec -2013-09-18 01:07:04,144 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 99.29 sec -2013-09-18 01:07:05,148 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 99.29 sec -2013-09-18 01:07:06,154 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 112.67 sec -2013-09-18 01:07:07,159 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 112.67 sec -2013-09-18 01:07:08,164 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 112.67 sec -MapReduce Total cumulative CPU time: 1 minutes 52 seconds 670 msec -Ended Job = job_201309172235_0142 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 729.12 sec HDFS Read: 1046532092 HDFS Write: 545445520 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 112.67 sec HDFS Read: 545449242 HDFS Write: 418 SUCCESS -Total MapReduce CPU Time Spent: 14 minutes 1 seconds 790 msec -OK -Time taken: 117.366 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_583@mturlrep13_201309180107_219916780.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0143 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:07:18,186 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:07:29,269 Stage-1 map = 4%, reduce = 0% -2013-09-18 01:07:30,388 Stage-1 map = 9%, reduce = 0% -2013-09-18 01:07:31,400 Stage-1 map = 11%, reduce = 0% -2013-09-18 01:07:32,410 Stage-1 map = 19%, reduce = 0% -2013-09-18 01:07:33,417 Stage-1 map = 20%, reduce = 0% -2013-09-18 01:07:34,436 Stage-1 map = 36%, reduce = 0% -2013-09-18 01:07:35,442 Stage-1 map = 39%, reduce = 0% -2013-09-18 01:07:36,448 Stage-1 map = 42%, reduce = 0% -2013-09-18 01:07:37,463 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 36.86 sec -2013-09-18 01:07:38,488 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 89.65 sec -2013-09-18 01:07:39,524 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 170.2 sec -2013-09-18 01:07:40,544 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 229.86 sec -2013-09-18 01:07:41,573 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 524.54 sec -2013-09-18 01:07:42,584 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 543.69 sec -2013-09-18 01:07:43,591 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 583.3 sec -2013-09-18 01:07:44,597 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 583.3 sec -2013-09-18 01:07:45,602 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 583.3 sec -2013-09-18 01:07:46,609 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 583.3 sec -2013-09-18 01:07:47,617 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 583.3 sec -2013-09-18 01:07:48,627 Stage-1 map = 100%, reduce = 23%, Cumulative CPU 583.3 sec -2013-09-18 01:07:49,634 Stage-1 map = 100%, reduce = 35%, Cumulative CPU 583.3 sec -2013-09-18 01:07:50,641 Stage-1 map = 100%, reduce = 38%, Cumulative CPU 583.3 sec -2013-09-18 01:07:51,648 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 583.3 sec -2013-09-18 01:07:52,655 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 583.3 sec -2013-09-18 01:07:53,662 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 583.3 sec -2013-09-18 01:07:54,669 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 583.3 sec -2013-09-18 01:07:55,676 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 583.3 sec -2013-09-18 01:07:56,686 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 610.88 sec -2013-09-18 01:07:57,693 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 665.19 sec -2013-09-18 01:07:58,698 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 735.33 sec -2013-09-18 01:07:59,704 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 735.33 sec -MapReduce Total cumulative CPU time: 12 minutes 15 seconds 330 msec -Ended Job = job_201309172235_0143 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0144 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 01:08:02,196 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:08:13,229 Stage-2 map = 18%, reduce = 0% -2013-09-18 01:08:19,247 Stage-2 map = 37%, reduce = 0% -2013-09-18 01:08:26,358 Stage-2 map = 55%, reduce = 0% -2013-09-18 01:08:31,375 Stage-2 map = 73%, reduce = 0% -2013-09-18 01:08:34,385 Stage-2 map = 82%, reduce = 0% -2013-09-18 01:08:40,407 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 45.23 sec -2013-09-18 01:08:41,412 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 90.45 sec -2013-09-18 01:08:42,416 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 90.45 sec -2013-09-18 01:08:43,421 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 90.45 sec -2013-09-18 01:08:44,425 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 90.45 sec -2013-09-18 01:08:45,430 Stage-2 map = 92%, reduce = 0%, Cumulative CPU 90.45 sec -2013-09-18 01:08:46,435 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 96.92 sec -2013-09-18 01:08:47,441 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 96.92 sec -2013-09-18 01:08:48,446 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 96.92 sec -2013-09-18 01:08:49,451 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 96.92 sec -2013-09-18 01:08:50,457 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 96.92 sec -2013-09-18 01:08:51,461 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 96.92 sec -2013-09-18 01:08:52,467 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 96.92 sec -2013-09-18 01:08:53,472 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 96.92 sec -2013-09-18 01:08:54,477 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 96.92 sec -2013-09-18 01:08:55,482 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 96.92 sec -2013-09-18 01:08:56,488 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 96.92 sec -2013-09-18 01:08:57,499 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 96.92 sec -2013-09-18 01:08:58,504 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 109.55 sec -2013-09-18 01:08:59,510 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 109.55 sec -MapReduce Total cumulative CPU time: 1 minutes 49 seconds 550 msec -Ended Job = job_201309172235_0144 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 735.33 sec HDFS Read: 1046532092 HDFS Write: 545445520 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 109.55 sec HDFS Read: 545449242 HDFS Write: 418 SUCCESS -Total MapReduce CPU Time Spent: 14 minutes 4 seconds 880 msec -OK -Time taken: 109.524 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9002@mturlrep13_201309180109_1127417909.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0145 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:09:21,157 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:09:34,238 Stage-1 map = 1%, reduce = 0% -2013-09-18 01:09:37,271 Stage-1 map = 2%, reduce = 0% -2013-09-18 01:09:38,281 Stage-1 map = 5%, reduce = 0% -2013-09-18 01:09:39,287 Stage-1 map = 6%, reduce = 0% -2013-09-18 01:09:40,293 Stage-1 map = 11%, reduce = 0% -2013-09-18 01:09:41,307 Stage-1 map = 15%, reduce = 0% -2013-09-18 01:09:42,344 Stage-1 map = 16%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:43,369 Stage-1 map = 18%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:44,396 Stage-1 map = 21%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:45,410 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:46,422 Stage-1 map = 26%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:47,429 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:48,440 Stage-1 map = 34%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:49,454 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:50,470 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:51,477 Stage-1 map = 45%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:52,487 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:53,494 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:54,510 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:55,539 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:56,564 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:57,574 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:58,586 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:09:59,593 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:10:00,605 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:10:01,613 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 424.15 sec -2013-09-18 01:10:02,631 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 461.38 sec -2013-09-18 01:10:03,659 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 461.38 sec -2013-09-18 01:10:04,670 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 504.24 sec -2013-09-18 01:10:05,681 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 504.24 sec -2013-09-18 01:10:06,688 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 526.72 sec -2013-09-18 01:10:07,709 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 618.33 sec -2013-09-18 01:10:08,722 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 708.66 sec -2013-09-18 01:10:09,737 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 906.45 sec -2013-09-18 01:10:10,743 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 956.72 sec -2013-09-18 01:10:11,750 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 1148.71 sec -2013-09-18 01:10:12,756 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1258.63 sec -2013-09-18 01:10:13,762 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1258.63 sec -2013-09-18 01:10:14,770 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1288.05 sec -2013-09-18 01:10:15,777 Stage-1 map = 100%, reduce = 2%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:16,784 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:17,807 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:18,813 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:19,819 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:20,825 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:21,831 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:22,837 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:26,141 Stage-1 map = 100%, reduce = 45%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:27,147 Stage-1 map = 100%, reduce = 45%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:28,153 Stage-1 map = 100%, reduce = 48%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:29,328 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:30,334 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:31,340 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:32,346 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:33,352 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:34,359 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:36,108 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:37,114 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:38,121 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:39,470 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:40,477 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:41,483 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1356.32 sec -2013-09-18 01:10:42,629 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:45,862 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:48,964 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:49,969 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:50,976 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:51,982 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:52,988 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:53,993 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:54,999 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:56,005 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:57,010 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:58,016 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1569.69 sec -2013-09-18 01:10:59,021 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:00,027 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:01,033 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:02,038 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:03,043 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:04,049 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:06,499 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:07,504 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:08,509 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:09,513 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:10,519 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:12,086 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:13,091 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:14,097 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:15,102 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:16,108 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:17,635 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:18,641 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:21,528 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:24,908 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:25,914 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:26,919 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:27,926 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:28,931 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:29,959 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:30,966 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:31,971 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:33,303 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:34,309 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:35,315 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1569.69 sec -2013-09-18 01:11:36,321 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1607.57 sec -2013-09-18 01:11:37,327 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1607.57 sec -2013-09-18 01:11:38,336 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1728.83 sec -2013-09-18 01:11:39,342 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1812.48 sec -2013-09-18 01:11:40,348 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1812.48 sec -2013-09-18 01:11:41,354 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1812.48 sec -2013-09-18 01:11:42,360 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 2013.45 sec -2013-09-18 01:11:43,365 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 2030.96 sec -2013-09-18 01:11:44,371 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 2030.96 sec -MapReduce Total cumulative CPU time: 33 minutes 50 seconds 960 msec -Ended Job = job_201309172235_0145 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0146 -Hadoop job information for Stage-2: number of mappers: 14; number of reducers: 1 -2013-09-18 01:11:47,866 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:12:01,932 Stage-2 map = 2%, reduce = 0% -2013-09-18 01:12:04,943 Stage-2 map = 15%, reduce = 0% -2013-09-18 01:12:07,952 Stage-2 map = 19%, reduce = 0% -2013-09-18 01:12:09,062 Stage-2 map = 22%, reduce = 0% -2013-09-18 01:12:11,068 Stage-2 map = 26%, reduce = 0% -2013-09-18 01:12:15,081 Stage-2 map = 28%, reduce = 0% -2013-09-18 01:12:17,087 Stage-2 map = 34%, reduce = 0% -2013-09-18 01:12:18,091 Stage-2 map = 43%, reduce = 0% -2013-09-18 01:12:21,102 Stage-2 map = 48%, reduce = 0% -2013-09-18 01:12:23,108 Stage-2 map = 52%, reduce = 0% -2013-09-18 01:12:24,112 Stage-2 map = 64%, reduce = 0% -2013-09-18 01:12:30,131 Stage-2 map = 73%, reduce = 0% -2013-09-18 01:12:33,143 Stage-2 map = 78%, reduce = 0% -2013-09-18 01:12:35,159 Stage-2 map = 79%, reduce = 0%, Cumulative CPU 54.68 sec -2013-09-18 01:12:36,935 Stage-2 map = 79%, reduce = 0%, Cumulative CPU 54.68 sec -2013-09-18 01:12:37,940 Stage-2 map = 79%, reduce = 0%, Cumulative CPU 54.68 sec -2013-09-18 01:12:38,945 Stage-2 map = 81%, reduce = 0%, Cumulative CPU 111.92 sec -2013-09-18 01:12:39,950 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 231.05 sec -2013-09-18 01:12:40,956 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 231.05 sec -2013-09-18 01:12:41,961 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 231.05 sec -2013-09-18 01:12:42,966 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 842.33 sec -2013-09-18 01:12:43,971 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 842.33 sec -2013-09-18 01:12:45,166 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 842.33 sec -2013-09-18 01:12:46,171 Stage-2 map = 86%, reduce = 10%, Cumulative CPU 845.72 sec -2013-09-18 01:12:47,176 Stage-2 map = 95%, reduce = 10%, Cumulative CPU 870.78 sec -2013-09-18 01:12:48,180 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 887.76 sec -2013-09-18 01:12:49,184 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 887.76 sec -2013-09-18 01:12:50,188 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 887.76 sec -2013-09-18 01:12:53,214 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 887.76 sec -2013-09-18 01:12:55,469 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 887.76 sec -2013-09-18 01:12:56,473 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 887.76 sec -2013-09-18 01:12:57,477 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 887.76 sec -2013-09-18 01:12:58,480 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 887.76 sec -2013-09-18 01:12:59,485 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 887.76 sec -2013-09-18 01:13:00,557 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 887.76 sec -2013-09-18 01:13:01,561 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 887.76 sec -2013-09-18 01:13:02,566 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 887.76 sec -2013-09-18 01:13:03,571 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 887.76 sec -2013-09-18 01:13:04,575 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 887.76 sec -2013-09-18 01:13:05,579 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 887.76 sec -2013-09-18 01:13:06,583 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 887.76 sec -2013-09-18 01:13:10,271 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 887.76 sec -2013-09-18 01:13:11,275 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 887.76 sec -2013-09-18 01:13:12,279 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 887.76 sec -2013-09-18 01:13:13,283 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.76 sec -2013-09-18 01:13:14,287 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.76 sec -2013-09-18 01:13:15,291 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.76 sec -2013-09-18 01:13:16,295 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.76 sec -2013-09-18 01:13:17,544 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.76 sec -2013-09-18 01:13:18,548 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.76 sec -2013-09-18 01:13:19,552 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.76 sec -2013-09-18 01:13:21,226 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 887.76 sec -2013-09-18 01:13:22,398 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 887.76 sec -2013-09-18 01:13:26,427 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 887.76 sec -2013-09-18 01:13:27,431 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 887.76 sec -2013-09-18 01:13:28,435 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 887.76 sec -2013-09-18 01:13:29,440 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 887.76 sec -2013-09-18 01:13:30,447 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 887.76 sec -2013-09-18 01:13:31,451 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 887.76 sec -2013-09-18 01:13:32,455 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 887.76 sec -2013-09-18 01:13:33,459 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 887.76 sec -2013-09-18 01:13:34,463 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 887.76 sec -2013-09-18 01:13:35,467 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 887.76 sec -2013-09-18 01:13:36,471 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 887.76 sec -2013-09-18 01:13:37,475 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 887.76 sec -2013-09-18 01:13:38,520 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 887.76 sec -2013-09-18 01:13:39,524 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 887.76 sec -2013-09-18 01:13:40,528 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 887.76 sec -2013-09-18 01:13:41,533 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 887.76 sec -2013-09-18 01:13:42,538 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 947.26 sec -2013-09-18 01:13:43,543 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 947.26 sec -2013-09-18 01:13:44,548 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:45,553 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:46,557 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:47,561 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:48,566 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:49,570 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:50,574 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:51,662 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:52,666 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:57,873 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:58,877 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:13:59,880 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:14:00,884 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:14:01,888 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:14:02,892 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:14:03,895 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:14:04,899 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:14:05,903 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 947.26 sec -2013-09-18 01:14:10,178 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 976.07 sec -2013-09-18 01:14:11,182 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 976.07 sec -2013-09-18 01:14:12,187 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 976.07 sec -MapReduce Total cumulative CPU time: 16 minutes 16 seconds 70 msec -Ended Job = job_201309172235_0146 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 2030.96 sec HDFS Read: 869197327 HDFS Write: 4140713341 SUCCESS -Job 1: Map: 14 Reduce: 1 Cumulative CPU: 976.07 sec HDFS Read: 4140841812 HDFS Write: 415 SUCCESS -Total MapReduce CPU Time Spent: 50 minutes 7 seconds 30 msec -OK -Time taken: 301.77 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19253@mturlrep13_201309180114_1319337284.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0147 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:14:21,934 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:14:38,062 Stage-1 map = 3%, reduce = 0% -2013-09-18 01:14:39,067 Stage-1 map = 4%, reduce = 0% -2013-09-18 01:14:40,081 Stage-1 map = 5%, reduce = 0% -2013-09-18 01:14:41,089 Stage-1 map = 9%, reduce = 0% -2013-09-18 01:14:42,106 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:43,120 Stage-1 map = 15%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:44,128 Stage-1 map = 19%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:45,134 Stage-1 map = 27%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:46,141 Stage-1 map = 27%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:47,152 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:48,158 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:49,164 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:50,177 Stage-1 map = 38%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:51,183 Stage-1 map = 45%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:52,193 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:53,199 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:54,206 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:55,212 Stage-1 map = 58%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:56,218 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:57,225 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:58,236 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:14:59,256 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 456.6 sec -2013-09-18 01:15:00,265 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 472.96 sec -2013-09-18 01:15:01,273 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 472.96 sec -2013-09-18 01:15:02,279 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 472.96 sec -2013-09-18 01:15:03,289 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 472.96 sec -2013-09-18 01:15:04,295 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 472.96 sec -2013-09-18 01:15:05,301 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 535.78 sec -2013-09-18 01:15:06,312 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 600.59 sec -2013-09-18 01:15:07,321 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 709.5 sec -2013-09-18 01:15:08,347 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 776.98 sec -2013-09-18 01:15:09,362 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 945.32 sec -2013-09-18 01:15:10,368 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 1072.86 sec -2013-09-18 01:15:11,374 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 1125.81 sec -2013-09-18 01:15:12,400 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1285.77 sec -2013-09-18 01:15:13,406 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1285.77 sec -2013-09-18 01:15:14,411 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1285.77 sec -2013-09-18 01:15:15,418 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:16,425 Stage-1 map = 100%, reduce = 7%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:17,452 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:18,458 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:19,463 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:20,469 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:21,475 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:23,349 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:24,354 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:25,360 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:26,368 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:27,374 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:29,096 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:30,102 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:31,107 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:32,113 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:33,119 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:34,124 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:35,356 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:36,362 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:37,368 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:38,374 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:39,379 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:40,385 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:41,390 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1349.28 sec -2013-09-18 01:15:43,287 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:44,293 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:45,299 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:46,306 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:47,312 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:48,318 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:49,324 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:50,329 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:51,335 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:52,341 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:53,347 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:54,353 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:57,177 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:58,183 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1558.8 sec -2013-09-18 01:15:59,188 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1558.8 sec -2013-09-18 01:16:00,499 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:01,505 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:02,511 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:03,517 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:04,524 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:05,551 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:06,557 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:07,562 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:08,567 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:09,573 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:10,583 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:11,588 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:12,594 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:13,600 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:14,606 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:16,677 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:17,682 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:20,073 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:21,079 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:23,600 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:24,606 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:25,614 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:26,620 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:27,625 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:28,631 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:29,637 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:30,642 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:31,648 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:32,654 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:33,660 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:34,666 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1536.68 sec -2013-09-18 01:16:35,672 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1613.31 sec -2013-09-18 01:16:37,443 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1656.62 sec -2013-09-18 01:16:38,449 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1742.64 sec -2013-09-18 01:16:39,455 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1879.43 sec -2013-09-18 01:16:40,460 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1970.65 sec -2013-09-18 01:16:41,465 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1970.65 sec -2013-09-18 01:16:43,070 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:44,076 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:45,081 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:46,086 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:47,092 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:50,460 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:51,564 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:52,760 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:53,765 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:54,771 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:55,776 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:16:56,781 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:17:00,159 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:17:01,164 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:17:02,169 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:17:03,175 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:17:04,180 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:17:05,185 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:17:06,189 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:17:07,194 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:17:08,199 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1997.98 sec -2013-09-18 01:17:09,934 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 2024.84 sec -2013-09-18 01:17:10,938 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 2024.84 sec -MapReduce Total cumulative CPU time: 33 minutes 44 seconds 840 msec -Ended Job = job_201309172235_0147 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0148 -Hadoop job information for Stage-2: number of mappers: 14; number of reducers: 1 -2013-09-18 01:17:14,417 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:17:27,528 Stage-2 map = 1%, reduce = 0% -2013-09-18 01:17:28,533 Stage-2 map = 2%, reduce = 0% -2013-09-18 01:17:31,543 Stage-2 map = 22%, reduce = 0% -2013-09-18 01:17:37,560 Stage-2 map = 28%, reduce = 0% -2013-09-18 01:17:40,569 Stage-2 map = 43%, reduce = 0% -2013-09-18 01:17:46,588 Stage-2 map = 53%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:47,592 Stage-2 map = 53%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:48,597 Stage-2 map = 53%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:49,601 Stage-2 map = 64%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:50,606 Stage-2 map = 64%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:51,611 Stage-2 map = 64%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:52,616 Stage-2 map = 70%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:53,621 Stage-2 map = 70%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:55,731 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:56,736 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:57,740 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 446.67 sec -2013-09-18 01:17:58,745 Stage-2 map = 79%, reduce = 0%, Cumulative CPU 467.91 sec -2013-09-18 01:17:59,749 Stage-2 map = 79%, reduce = 0%, Cumulative CPU 467.91 sec -2013-09-18 01:18:00,757 Stage-2 map = 79%, reduce = 0%, Cumulative CPU 467.91 sec -2013-09-18 01:18:01,762 Stage-2 map = 79%, reduce = 0%, Cumulative CPU 467.91 sec -2013-09-18 01:18:02,767 Stage-2 map = 79%, reduce = 0%, Cumulative CPU 467.91 sec -2013-09-18 01:18:03,771 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 547.4 sec -2013-09-18 01:18:04,776 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 547.4 sec -2013-09-18 01:18:05,781 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 547.4 sec -2013-09-18 01:18:06,786 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 547.4 sec -2013-09-18 01:18:07,791 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 547.4 sec -2013-09-18 01:18:09,462 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 547.4 sec -2013-09-18 01:18:10,467 Stage-2 map = 88%, reduce = 10%, Cumulative CPU 613.28 sec -2013-09-18 01:18:11,472 Stage-2 map = 95%, reduce = 10%, Cumulative CPU 782.49 sec -2013-09-18 01:18:12,482 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 886.7 sec -2013-09-18 01:18:13,487 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 886.7 sec -2013-09-18 01:18:14,491 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 886.7 sec -2013-09-18 01:18:15,496 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 886.7 sec -2013-09-18 01:18:16,501 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 886.7 sec -2013-09-18 01:18:17,505 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 886.7 sec -2013-09-18 01:18:18,510 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 886.7 sec -2013-09-18 01:18:19,566 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 886.7 sec -2013-09-18 01:18:20,571 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 886.7 sec -2013-09-18 01:18:21,574 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 886.7 sec -2013-09-18 01:18:22,578 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 886.7 sec -2013-09-18 01:18:23,583 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 886.7 sec -2013-09-18 01:18:24,587 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 886.7 sec -2013-09-18 01:18:25,592 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 886.7 sec -2013-09-18 01:18:26,597 Stage-2 map = 100%, reduce = 14%, Cumulative CPU 886.7 sec -2013-09-18 01:18:27,680 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 886.7 sec -2013-09-18 01:18:28,684 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 886.7 sec -2013-09-18 01:18:29,689 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 886.7 sec -2013-09-18 01:18:30,693 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 886.7 sec -2013-09-18 01:18:31,698 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 886.7 sec -2013-09-18 01:18:32,702 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 886.7 sec -2013-09-18 01:18:34,644 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 886.7 sec -2013-09-18 01:18:35,648 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 886.7 sec -2013-09-18 01:18:36,653 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 886.7 sec -2013-09-18 01:18:37,657 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 886.7 sec -2013-09-18 01:18:38,662 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 886.7 sec -2013-09-18 01:18:39,666 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 886.7 sec -2013-09-18 01:18:40,670 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 886.7 sec -2013-09-18 01:18:41,674 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 886.7 sec -2013-09-18 01:18:42,679 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 924.39 sec -2013-09-18 01:18:43,683 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 924.39 sec -2013-09-18 01:18:44,687 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 924.39 sec -2013-09-18 01:18:49,434 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 924.39 sec -2013-09-18 01:18:50,438 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 924.39 sec -2013-09-18 01:18:51,443 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 924.39 sec -2013-09-18 01:18:52,448 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 924.39 sec -2013-09-18 01:18:53,452 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 924.39 sec -2013-09-18 01:18:54,457 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 924.39 sec -2013-09-18 01:18:55,461 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:18:56,465 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:18:57,469 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:18:58,473 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:18:59,477 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:19:00,481 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:19:05,293 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:19:06,297 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:19:07,301 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:19:08,306 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:19:09,310 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:19:10,314 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 924.39 sec -2013-09-18 01:19:11,319 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:12,323 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:13,331 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:14,336 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:16,921 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:17,927 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:18,931 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:19,935 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:20,939 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:21,943 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:22,947 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:23,951 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:24,955 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:25,959 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:26,963 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:27,966 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:28,970 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:29,974 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:30,977 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:31,981 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:32,985 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:33,989 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:34,992 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:35,996 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 924.39 sec -2013-09-18 01:19:37,001 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 976.38 sec -2013-09-18 01:19:38,006 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 976.38 sec -2013-09-18 01:19:39,010 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 976.38 sec -MapReduce Total cumulative CPU time: 16 minutes 16 seconds 380 msec -Ended Job = job_201309172235_0148 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 2024.84 sec HDFS Read: 869197327 HDFS Write: 4140713341 SUCCESS -Job 1: Map: 14 Reduce: 1 Cumulative CPU: 976.38 sec HDFS Read: 4140841812 HDFS Write: 417 SUCCESS -Total MapReduce CPU Time Spent: 50 minutes 1 seconds 220 msec -OK -Time taken: 324.867 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29643@mturlrep13_201309180119_1843662809.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0149 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:19:49,884 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:20:04,981 Stage-1 map = 2%, reduce = 0% -2013-09-18 01:20:05,998 Stage-1 map = 3%, reduce = 0% -2013-09-18 01:20:08,008 Stage-1 map = 9%, reduce = 0% -2013-09-18 01:20:09,017 Stage-1 map = 14%, reduce = 0% -2013-09-18 01:20:11,035 Stage-1 map = 16%, reduce = 0% -2013-09-18 01:20:12,040 Stage-1 map = 26%, reduce = 0% -2013-09-18 01:20:13,052 Stage-1 map = 27%, reduce = 0% -2013-09-18 01:20:14,066 Stage-1 map = 28%, reduce = 0% -2013-09-18 01:20:15,083 Stage-1 map = 35%, reduce = 0% -2013-09-18 01:20:16,111 Stage-1 map = 36%, reduce = 0% -2013-09-18 01:20:17,123 Stage-1 map = 37%, reduce = 0% -2013-09-18 01:20:18,128 Stage-1 map = 45%, reduce = 0% -2013-09-18 01:20:19,164 Stage-1 map = 47%, reduce = 0% -2013-09-18 01:20:21,208 Stage-1 map = 55%, reduce = 0% -2013-09-18 01:20:22,215 Stage-1 map = 58%, reduce = 0% -2013-09-18 01:20:24,224 Stage-1 map = 65%, reduce = 0% -2013-09-18 01:20:25,234 Stage-1 map = 68%, reduce = 0% -2013-09-18 01:20:27,288 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 28.16 sec -2013-09-18 01:20:28,302 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 28.16 sec -2013-09-18 01:20:29,308 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 28.16 sec -2013-09-18 01:20:30,314 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 92.89 sec -2013-09-18 01:20:31,331 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 92.89 sec -2013-09-18 01:20:32,346 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 193.14 sec -2013-09-18 01:20:33,367 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 259.68 sec -2013-09-18 01:20:34,384 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 362.61 sec -2013-09-18 01:20:35,420 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 431.76 sec -2013-09-18 01:20:36,426 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 683.06 sec -2013-09-18 01:20:37,432 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 832.82 sec -2013-09-18 01:20:38,438 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 1057.56 sec -2013-09-18 01:20:39,444 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1253.19 sec -2013-09-18 01:20:40,455 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 1253.19 sec -2013-09-18 01:20:41,462 Stage-1 map = 99%, reduce = 1%, Cumulative CPU 1253.19 sec -2013-09-18 01:20:42,468 Stage-1 map = 99%, reduce = 4%, Cumulative CPU 1297.77 sec -2013-09-18 01:20:43,475 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:44,481 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:45,487 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:46,493 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:47,499 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:48,505 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:49,534 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:50,540 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:51,546 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:52,552 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:53,962 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:54,968 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:55,973 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:56,979 Stage-1 map = 100%, reduce = 48%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:57,988 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1357.14 sec -2013-09-18 01:20:59,065 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:00,071 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:01,077 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:02,083 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:03,089 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:04,198 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:05,204 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:06,211 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:07,217 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:08,223 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:09,720 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:10,726 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:11,732 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:12,738 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:13,744 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:16,687 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:17,693 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:18,698 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:19,707 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:20,715 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:21,721 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:25,701 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:26,707 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:28,607 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:29,613 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:30,619 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:31,625 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:32,630 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:33,636 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:34,641 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:35,646 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:37,031 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:38,037 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:39,043 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:40,049 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:41,055 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:42,060 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1357.14 sec -2013-09-18 01:21:43,068 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1852.58 sec -2013-09-18 01:21:44,074 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1852.58 sec -2013-09-18 01:21:45,080 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1852.58 sec -2013-09-18 01:21:46,963 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1852.58 sec -2013-09-18 01:21:47,969 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1852.58 sec -2013-09-18 01:21:48,975 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1852.58 sec -2013-09-18 01:21:52,859 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1852.58 sec -2013-09-18 01:21:54,438 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1852.58 sec -2013-09-18 01:21:55,444 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1852.58 sec -2013-09-18 01:21:57,237 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1852.58 sec -2013-09-18 01:21:58,259 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1873.66 sec -2013-09-18 01:21:59,265 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1873.66 sec -2013-09-18 01:22:00,270 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1873.66 sec -2013-09-18 01:22:01,276 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1886.44 sec -2013-09-18 01:22:02,282 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1886.44 sec -2013-09-18 01:22:03,288 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1961.16 sec -2013-09-18 01:22:04,293 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1978.53 sec -2013-09-18 01:22:05,299 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 2014.08 sec -2013-09-18 01:22:06,305 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 2014.08 sec -2013-09-18 01:22:07,310 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 2014.08 sec -MapReduce Total cumulative CPU time: 33 minutes 34 seconds 80 msec -Ended Job = job_201309172235_0149 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0150 -Hadoop job information for Stage-2: number of mappers: 14; number of reducers: 1 -2013-09-18 01:22:10,699 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:22:27,319 Stage-2 map = 5%, reduce = 0% -2013-09-18 01:22:30,330 Stage-2 map = 22%, reduce = 0% -2013-09-18 01:22:33,340 Stage-2 map = 25%, reduce = 0% -2013-09-18 01:22:36,352 Stage-2 map = 31%, reduce = 0% -2013-09-18 01:22:39,362 Stage-2 map = 43%, reduce = 0% -2013-09-18 01:22:42,371 Stage-2 map = 52%, reduce = 0% -2013-09-18 01:22:45,406 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 517.66 sec -2013-09-18 01:22:46,420 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 517.66 sec -2013-09-18 01:22:47,425 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 517.66 sec -2013-09-18 01:22:48,431 Stage-2 map = 64%, reduce = 0%, Cumulative CPU 517.66 sec -2013-09-18 01:22:49,436 Stage-2 map = 64%, reduce = 0%, Cumulative CPU 517.66 sec -2013-09-18 01:22:50,441 Stage-2 map = 64%, reduce = 0%, Cumulative CPU 517.66 sec -2013-09-18 01:22:51,446 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 517.66 sec -2013-09-18 01:22:54,870 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 517.66 sec -2013-09-18 01:22:55,875 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 517.66 sec -2013-09-18 01:22:56,880 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 517.66 sec -2013-09-18 01:22:57,885 Stage-2 map = 79%, reduce = 0%, Cumulative CPU 533.57 sec -2013-09-18 01:23:00,883 Stage-2 map = 79%, reduce = 0%, Cumulative CPU 533.57 sec -2013-09-18 01:23:01,889 Stage-2 map = 81%, reduce = 0%, Cumulative CPU 553.42 sec -2013-09-18 01:23:02,895 Stage-2 map = 81%, reduce = 0%, Cumulative CPU 553.42 sec -2013-09-18 01:23:03,900 Stage-2 map = 81%, reduce = 0%, Cumulative CPU 553.42 sec -2013-09-18 01:23:04,905 Stage-2 map = 81%, reduce = 0%, Cumulative CPU 553.42 sec -2013-09-18 01:23:06,469 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 597.6 sec -2013-09-18 01:23:07,474 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 597.6 sec -2013-09-18 01:23:08,479 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 597.6 sec -2013-09-18 01:23:09,483 Stage-2 map = 88%, reduce = 10%, Cumulative CPU 653.66 sec -2013-09-18 01:23:10,488 Stage-2 map = 92%, reduce = 10%, Cumulative CPU 738.42 sec -2013-09-18 01:23:11,492 Stage-2 map = 98%, reduce = 10%, Cumulative CPU 857.27 sec -2013-09-18 01:23:12,884 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 887.38 sec -2013-09-18 01:23:14,634 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 887.38 sec -2013-09-18 01:23:15,638 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 887.38 sec -2013-09-18 01:23:16,643 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 887.38 sec -2013-09-18 01:23:17,648 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 887.38 sec -2013-09-18 01:23:18,652 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 887.38 sec -2013-09-18 01:23:19,657 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 887.38 sec -2013-09-18 01:23:20,662 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 887.38 sec -2013-09-18 01:23:21,667 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 887.38 sec -2013-09-18 01:23:22,671 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 887.38 sec -2013-09-18 01:23:23,797 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 887.38 sec -2013-09-18 01:23:24,801 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 887.38 sec -2013-09-18 01:23:25,806 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 887.38 sec -2013-09-18 01:23:26,811 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 887.38 sec -2013-09-18 01:23:27,815 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 887.38 sec -2013-09-18 01:23:28,820 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 887.38 sec -2013-09-18 01:23:29,825 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 887.38 sec -2013-09-18 01:23:31,274 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 887.38 sec -2013-09-18 01:23:32,278 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 887.38 sec -2013-09-18 01:23:33,283 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 887.38 sec -2013-09-18 01:23:34,287 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 887.38 sec -2013-09-18 01:23:35,291 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 887.38 sec -2013-09-18 01:23:36,295 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 887.38 sec -2013-09-18 01:23:37,300 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.38 sec -2013-09-18 01:23:38,305 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.38 sec -2013-09-18 01:23:39,310 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.38 sec -2013-09-18 01:23:41,159 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.38 sec -2013-09-18 01:23:42,164 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.38 sec -2013-09-18 01:23:43,170 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 887.38 sec -2013-09-18 01:23:44,175 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 923.46 sec -2013-09-18 01:23:47,083 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 923.46 sec -2013-09-18 01:23:53,300 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 923.46 sec -2013-09-18 01:23:54,304 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 923.46 sec -2013-09-18 01:23:55,307 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 923.46 sec -2013-09-18 01:23:56,311 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:23:57,314 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:23:58,318 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:23:59,322 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:24:00,325 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:24:01,329 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:24:02,333 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:24:03,337 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:24:08,423 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:24:09,427 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:24:10,431 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 923.46 sec -2013-09-18 01:24:11,435 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:12,439 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:13,442 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:14,446 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:15,450 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:16,454 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:17,458 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:18,462 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:19,893 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:20,896 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:21,900 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:22,904 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:23,908 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:24,918 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:25,922 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:26,926 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:27,929 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:28,932 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:29,936 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:30,939 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:31,943 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:32,947 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 923.46 sec -2013-09-18 01:24:33,952 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 975.78 sec -2013-09-18 01:24:34,957 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 975.78 sec -MapReduce Total cumulative CPU time: 16 minutes 15 seconds 780 msec -Ended Job = job_201309172235_0150 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 2014.08 sec HDFS Read: 869197327 HDFS Write: 4140713341 SUCCESS -Job 1: Map: 14 Reduce: 1 Cumulative CPU: 975.78 sec HDFS Read: 4140841812 HDFS Write: 419 SUCCESS -Total MapReduce CPU Time Spent: 49 minutes 49 seconds 860 msec -OK -Time taken: 294.034 seconds, Fetched: 10 row(s) -hive> quit; --- то же самое, но ещё и без фильтрации.; - - -times: 1 -query: SELECT URL, count(*) AS c FROM hits_100m GROUP BY URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7945@mturlrep13_201309180124_199584274.txt -hive> SELECT URL, count(*) AS c FROM hits_100m GROUP BY URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0151 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:24:53,603 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:25:03,695 Stage-1 map = 2%, reduce = 0% -2013-09-18 01:25:04,700 Stage-1 map = 3%, reduce = 0% -2013-09-18 01:25:09,065 Stage-1 map = 17%, reduce = 0% -2013-09-18 01:25:10,073 Stage-1 map = 22%, reduce = 0% -2013-09-18 01:25:11,078 Stage-1 map = 28%, reduce = 0% -2013-09-18 01:25:12,084 Stage-1 map = 31%, reduce = 0% -2013-09-18 01:25:13,091 Stage-1 map = 34%, reduce = 0% -2013-09-18 01:25:14,104 Stage-1 map = 44%, reduce = 0% -2013-09-18 01:25:15,116 Stage-1 map = 47%, reduce = 0% -2013-09-18 01:25:16,125 Stage-1 map = 50%, reduce = 0% -2013-09-18 01:25:17,131 Stage-1 map = 57%, reduce = 0% -2013-09-18 01:25:18,139 Stage-1 map = 61%, reduce = 0% -2013-09-18 01:25:20,170 Stage-1 map = 64%, reduce = 0% -2013-09-18 01:25:21,175 Stage-1 map = 66%, reduce = 0% -2013-09-18 01:25:22,187 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 17.87 sec -2013-09-18 01:25:23,203 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 38.23 sec -2013-09-18 01:25:24,213 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 80.85 sec -2013-09-18 01:25:25,223 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 123.43 sec -2013-09-18 01:25:26,229 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 191.82 sec -2013-09-18 01:25:27,236 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 236.59 sec -2013-09-18 01:25:28,248 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 236.59 sec -2013-09-18 01:25:29,255 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 284.0 sec -2013-09-18 01:25:30,261 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 284.0 sec -2013-09-18 01:25:31,273 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 284.0 sec -2013-09-18 01:25:32,278 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 284.0 sec -2013-09-18 01:25:33,284 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 284.0 sec -2013-09-18 01:25:34,290 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 428.41 sec -2013-09-18 01:25:35,759 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 428.41 sec -2013-09-18 01:25:36,772 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 428.41 sec -2013-09-18 01:25:37,778 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 428.41 sec -2013-09-18 01:25:38,785 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 565.42 sec -2013-09-18 01:25:41,597 Stage-1 map = 94%, reduce = 15%, Cumulative CPU 565.42 sec -2013-09-18 01:25:45,549 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 1109.57 sec -2013-09-18 01:25:46,775 Stage-1 map = 99%, reduce = 17%, Cumulative CPU 1121.14 sec -2013-09-18 01:25:47,781 Stage-1 map = 99%, reduce = 17%, Cumulative CPU 1123.73 sec -2013-09-18 01:25:48,787 Stage-1 map = 99%, reduce = 17%, Cumulative CPU 1123.73 sec -2013-09-18 01:25:49,792 Stage-1 map = 99%, reduce = 17%, Cumulative CPU 1123.73 sec -2013-09-18 01:25:50,799 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 1123.73 sec -2013-09-18 01:25:51,805 Stage-1 map = 99%, reduce = 19%, Cumulative CPU 1123.73 sec -2013-09-18 01:25:52,811 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 1130.32 sec -2013-09-18 01:25:53,816 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 1130.32 sec -2013-09-18 01:25:55,206 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 1130.32 sec -2013-09-18 01:25:56,212 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 1130.32 sec -2013-09-18 01:25:57,221 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1130.32 sec -2013-09-18 01:25:58,227 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1130.32 sec -2013-09-18 01:25:59,260 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:00,267 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:01,273 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:02,279 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:03,286 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:04,292 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:09,638 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:10,644 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:14,140 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:15,145 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:16,152 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:17,158 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:18,165 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:19,170 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:20,177 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:21,183 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:22,189 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:23,194 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:24,203 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:25,208 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:26,409 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:27,415 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:28,420 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:29,426 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:30,432 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:31,439 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:33,621 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:34,626 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:35,631 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:36,637 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:37,643 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:38,648 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:40,155 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:41,161 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:43,159 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1130.32 sec -2013-09-18 01:26:44,308 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1494.96 sec -2013-09-18 01:26:45,314 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1494.96 sec -2013-09-18 01:26:46,518 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1494.96 sec -2013-09-18 01:26:47,523 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1494.96 sec -2013-09-18 01:26:48,532 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1494.96 sec -2013-09-18 01:26:49,538 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1526.79 sec -2013-09-18 01:26:50,543 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1526.79 sec -MapReduce Total cumulative CPU time: 25 minutes 26 seconds 790 msec -Ended Job = job_201309172235_0151 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0152 -Hadoop job information for Stage-2: number of mappers: 14; number of reducers: 1 -2013-09-18 01:27:03,639 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:27:11,681 Stage-2 map = 9%, reduce = 0% -2013-09-18 01:27:12,685 Stage-2 map = 12%, reduce = 0% -2013-09-18 01:27:15,339 Stage-2 map = 32%, reduce = 0% -2013-09-18 01:27:18,350 Stage-2 map = 59%, reduce = 0% -2013-09-18 01:27:21,361 Stage-2 map = 78%, reduce = 0% -2013-09-18 01:27:23,369 Stage-2 map = 81%, reduce = 0%, Cumulative CPU 47.02 sec -2013-09-18 01:27:24,374 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 142.27 sec -2013-09-18 01:27:25,378 Stage-2 map = 90%, reduce = 0%, Cumulative CPU 190.85 sec -2013-09-18 01:27:26,383 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.83 sec -2013-09-18 01:27:27,388 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.83 sec -2013-09-18 01:27:28,392 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.83 sec -2013-09-18 01:27:29,400 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.83 sec -2013-09-18 01:27:30,405 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.83 sec -2013-09-18 01:27:31,411 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.83 sec -2013-09-18 01:27:32,416 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.83 sec -2013-09-18 01:27:33,421 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 350.83 sec -2013-09-18 01:27:34,425 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 350.83 sec -2013-09-18 01:27:35,430 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 350.83 sec -2013-09-18 01:27:36,435 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 350.83 sec -2013-09-18 01:27:37,440 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 350.83 sec -2013-09-18 01:27:38,445 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 350.83 sec -2013-09-18 01:27:39,453 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 350.83 sec -2013-09-18 01:27:40,457 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 350.83 sec -2013-09-18 01:27:41,461 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 350.83 sec -2013-09-18 01:27:42,466 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 350.83 sec -2013-09-18 01:27:43,471 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 350.83 sec -2013-09-18 01:27:44,508 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 368.28 sec -2013-09-18 01:27:45,513 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 368.28 sec -2013-09-18 01:27:46,517 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 368.28 sec -2013-09-18 01:27:47,522 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 368.28 sec -2013-09-18 01:27:48,527 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 368.28 sec -2013-09-18 01:27:50,078 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 368.28 sec -2013-09-18 01:27:51,083 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 368.28 sec -2013-09-18 01:27:52,091 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 368.28 sec -2013-09-18 01:27:53,096 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 368.28 sec -2013-09-18 01:27:54,100 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 368.28 sec -2013-09-18 01:27:55,105 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 368.28 sec -2013-09-18 01:27:56,109 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 368.28 sec -2013-09-18 01:27:58,555 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 368.28 sec -2013-09-18 01:27:59,560 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 368.28 sec -2013-09-18 01:28:00,564 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 368.28 sec -2013-09-18 01:28:02,601 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 368.28 sec -2013-09-18 01:28:03,606 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 368.28 sec -2013-09-18 01:28:04,611 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 368.28 sec -2013-09-18 01:28:05,616 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 368.28 sec -2013-09-18 01:28:06,621 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 368.28 sec -2013-09-18 01:28:07,625 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 368.28 sec -2013-09-18 01:28:08,630 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 368.28 sec -2013-09-18 01:28:09,635 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:10,639 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:11,644 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:12,648 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:13,653 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:14,657 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:15,662 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:16,666 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:17,671 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:18,675 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:19,679 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:20,684 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:21,688 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:22,692 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:23,696 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:24,700 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:29,545 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:31,146 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:32,150 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:33,154 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:34,158 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 368.28 sec -2013-09-18 01:28:35,163 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 400.78 sec -2013-09-18 01:28:36,169 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 400.78 sec -2013-09-18 01:28:37,173 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 400.78 sec -MapReduce Total cumulative CPU time: 6 minutes 40 seconds 780 msec -Ended Job = job_201309172235_0152 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1526.79 sec HDFS Read: 882764316 HDFS Write: 4155848594 SUCCESS -Job 1: Map: 14 Reduce: 1 Cumulative CPU: 400.78 sec HDFS Read: 4156020610 HDFS Write: 399 SUCCESS -Total MapReduce CPU Time Spent: 32 minutes 7 seconds 570 msec -OK -Time taken: 233.937 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT URL, count(*) AS c FROM hits_100m GROUP BY URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17325@mturlrep13_201309180128_811787741.txt -hive> SELECT URL, count(*) AS c FROM hits_100m GROUP BY URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0153 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:28:48,068 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:28:58,179 Stage-1 map = 7%, reduce = 0% -2013-09-18 01:29:01,193 Stage-1 map = 12%, reduce = 0% -2013-09-18 01:29:02,199 Stage-1 map = 16%, reduce = 0% -2013-09-18 01:29:03,232 Stage-1 map = 18%, reduce = 0% -2013-09-18 01:29:04,245 Stage-1 map = 25%, reduce = 0% -2013-09-18 01:29:05,251 Stage-1 map = 34%, reduce = 0% -2013-09-18 01:29:06,256 Stage-1 map = 38%, reduce = 0% -2013-09-18 01:29:07,261 Stage-1 map = 44%, reduce = 0% -2013-09-18 01:29:08,269 Stage-1 map = 50%, reduce = 0% -2013-09-18 01:29:09,284 Stage-1 map = 56%, reduce = 0% -2013-09-18 01:29:10,289 Stage-1 map = 57%, reduce = 0% -2013-09-18 01:29:11,317 Stage-1 map = 60%, reduce = 0% -2013-09-18 01:29:12,323 Stage-1 map = 63%, reduce = 0% -2013-09-18 01:29:13,344 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 17.02 sec -2013-09-18 01:29:14,380 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 17.02 sec -2013-09-18 01:29:15,404 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 57.53 sec -2013-09-18 01:29:16,412 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 98.75 sec -2013-09-18 01:29:17,422 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 119.1 sec -2013-09-18 01:29:18,433 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 183.33 sec -2013-09-18 01:29:19,440 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 229.01 sec -2013-09-18 01:29:20,451 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 229.01 sec -2013-09-18 01:29:21,457 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 229.01 sec -2013-09-18 01:29:22,468 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 276.95 sec -2013-09-18 01:29:23,483 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 276.95 sec -2013-09-18 01:29:24,490 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 276.95 sec -2013-09-18 01:29:25,502 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 362.44 sec -2013-09-18 01:29:26,510 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 362.44 sec -2013-09-18 01:29:29,051 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 392.77 sec -2013-09-18 01:29:35,351 Stage-1 map = 94%, reduce = 15%, Cumulative CPU 521.09 sec -2013-09-18 01:29:37,211 Stage-1 map = 98%, reduce = 15%, Cumulative CPU 907.91 sec -2013-09-18 01:29:38,217 Stage-1 map = 99%, reduce = 15%, Cumulative CPU 946.27 sec -2013-09-18 01:29:39,222 Stage-1 map = 99%, reduce = 15%, Cumulative CPU 946.27 sec -2013-09-18 01:29:40,228 Stage-1 map = 99%, reduce = 16%, Cumulative CPU 946.27 sec -2013-09-18 01:29:41,234 Stage-1 map = 99%, reduce = 17%, Cumulative CPU 946.27 sec -2013-09-18 01:29:42,239 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 946.27 sec -2013-09-18 01:29:43,245 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 946.27 sec -2013-09-18 01:29:44,252 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 1126.33 sec -2013-09-18 01:29:45,257 Stage-1 map = 100%, reduce = 19%, Cumulative CPU 1126.92 sec -2013-09-18 01:29:46,262 Stage-1 map = 100%, reduce = 19%, Cumulative CPU 1126.92 sec -2013-09-18 01:29:47,268 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 1126.92 sec -2013-09-18 01:29:48,275 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 1126.92 sec -2013-09-18 01:29:49,281 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 1126.92 sec -2013-09-18 01:29:50,287 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1126.92 sec -2013-09-18 01:29:51,294 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1126.92 sec -2013-09-18 01:29:54,543 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1126.92 sec -2013-09-18 01:29:55,549 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1126.92 sec -2013-09-18 01:29:56,556 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:01,304 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:02,310 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:03,316 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:04,324 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:05,852 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:06,858 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:08,923 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:09,929 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:11,956 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:12,962 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:13,969 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:14,974 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:15,978 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:16,984 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:21,891 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:22,897 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:23,902 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:24,908 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:26,217 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:27,223 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:28,229 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:29,235 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:30,240 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:31,246 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:32,252 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:33,258 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:34,263 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:35,269 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:36,274 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:38,613 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:39,619 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:40,625 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:41,630 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:42,636 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:43,641 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:44,647 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:45,651 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:46,656 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:47,661 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:48,666 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:49,671 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:50,676 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:51,681 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:52,685 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:53,691 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1126.92 sec -2013-09-18 01:30:54,698 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:30:55,703 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:30:56,708 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:30:57,714 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:30:58,719 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:30:59,724 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:31:00,730 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:31:01,739 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:31:02,744 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:31:03,750 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:31:04,755 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:31:05,759 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1511.04 sec -2013-09-18 01:31:09,986 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1512.0 sec -2013-09-18 01:31:11,626 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1513.29 sec -2013-09-18 01:31:12,632 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1513.29 sec -2013-09-18 01:31:14,618 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1513.29 sec -2013-09-18 01:31:15,624 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1513.29 sec -2013-09-18 01:31:16,629 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1515.81 sec -2013-09-18 01:31:20,387 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1518.52 sec -2013-09-18 01:31:21,393 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1518.52 sec -MapReduce Total cumulative CPU time: 25 minutes 18 seconds 520 msec -Ended Job = job_201309172235_0153 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0154 -Hadoop job information for Stage-2: number of mappers: 14; number of reducers: 1 -2013-09-18 01:31:35,149 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:31:42,177 Stage-2 map = 13%, reduce = 0% -2013-09-18 01:31:45,189 Stage-2 map = 19%, reduce = 0% -2013-09-18 01:31:46,194 Stage-2 map = 36%, reduce = 0% -2013-09-18 01:31:49,205 Stage-2 map = 73%, reduce = 0% -2013-09-18 01:31:52,217 Stage-2 map = 82%, reduce = 0%, Cumulative CPU 45.62 sec -2013-09-18 01:31:53,223 Stage-2 map = 89%, reduce = 0%, Cumulative CPU 162.4 sec -2013-09-18 01:31:54,251 Stage-2 map = 94%, reduce = 0%, Cumulative CPU 239.19 sec -2013-09-18 01:31:55,256 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 346.31 sec -2013-09-18 01:31:56,260 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 346.31 sec -2013-09-18 01:31:57,265 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 346.31 sec -2013-09-18 01:31:58,270 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 346.31 sec -2013-09-18 01:31:59,275 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 346.31 sec -2013-09-18 01:32:00,280 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 346.31 sec -2013-09-18 01:32:01,285 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 346.31 sec -2013-09-18 01:32:02,291 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 346.31 sec -2013-09-18 01:32:03,296 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 346.31 sec -2013-09-18 01:32:04,301 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 346.31 sec -2013-09-18 01:32:05,307 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 346.31 sec -2013-09-18 01:32:06,312 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 346.31 sec -2013-09-18 01:32:07,318 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 346.31 sec -2013-09-18 01:32:08,323 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 346.31 sec -2013-09-18 01:32:09,328 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 346.31 sec -2013-09-18 01:32:10,332 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 346.31 sec -2013-09-18 01:32:11,338 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 346.31 sec -2013-09-18 01:32:12,343 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 346.31 sec -2013-09-18 01:32:13,348 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 346.31 sec -2013-09-18 01:32:14,353 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 346.31 sec -2013-09-18 01:32:15,358 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 346.31 sec -2013-09-18 01:32:16,363 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 346.31 sec -2013-09-18 01:32:17,368 Stage-2 map = 100%, reduce = 26%, Cumulative CPU 346.31 sec -2013-09-18 01:32:18,372 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 346.31 sec -2013-09-18 01:32:19,377 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 346.31 sec -2013-09-18 01:32:20,382 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 346.31 sec -2013-09-18 01:32:21,387 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 346.31 sec -2013-09-18 01:32:23,213 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 346.31 sec -2013-09-18 01:32:24,218 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:25,222 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:28,762 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:29,767 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:30,772 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:31,777 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:34,832 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:35,836 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:36,841 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:38,072 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:39,077 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 346.31 sec -2013-09-18 01:32:40,083 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 346.31 sec -2013-09-18 01:32:41,087 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 346.31 sec -2013-09-18 01:32:42,093 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 346.31 sec -2013-09-18 01:32:44,011 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:32:45,016 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:32:46,021 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:32:47,025 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:32:48,030 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:32:49,034 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:32:50,039 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:32:51,044 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:32:52,048 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:32:53,052 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:32:54,064 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 346.31 sec -2013-09-18 01:33:02,924 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 389.68 sec -2013-09-18 01:33:03,928 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 389.68 sec -2013-09-18 01:33:06,050 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 389.68 sec -2013-09-18 01:33:07,054 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 389.68 sec -2013-09-18 01:33:08,058 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 389.68 sec -2013-09-18 01:33:09,063 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 395.95 sec -2013-09-18 01:33:10,068 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 395.95 sec -2013-09-18 01:33:11,072 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 395.95 sec -MapReduce Total cumulative CPU time: 6 minutes 35 seconds 950 msec -Ended Job = job_201309172235_0154 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1518.52 sec HDFS Read: 882764316 HDFS Write: 4155848594 SUCCESS -Job 1: Map: 14 Reduce: 1 Cumulative CPU: 395.95 sec HDFS Read: 4156020676 HDFS Write: 399 SUCCESS -Total MapReduce CPU Time Spent: 31 minutes 54 seconds 470 msec -OK -Time taken: 272.053 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT URL, count(*) AS c FROM hits_100m GROUP BY URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26732@mturlrep13_201309180133_998177234.txt -hive> SELECT URL, count(*) AS c FROM hits_100m GROUP BY URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0155 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:33:21,912 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:33:30,987 Stage-1 map = 2%, reduce = 0% -2013-09-18 01:33:31,993 Stage-1 map = 5%, reduce = 0% -2013-09-18 01:33:32,999 Stage-1 map = 6%, reduce = 0% -2013-09-18 01:33:34,004 Stage-1 map = 7%, reduce = 0% -2013-09-18 01:33:35,010 Stage-1 map = 12%, reduce = 0% -2013-09-18 01:33:36,017 Stage-1 map = 15%, reduce = 0% -2013-09-18 01:33:37,023 Stage-1 map = 19%, reduce = 0% -2013-09-18 01:33:38,030 Stage-1 map = 27%, reduce = 0% -2013-09-18 01:33:39,035 Stage-1 map = 32%, reduce = 0% -2013-09-18 01:33:40,041 Stage-1 map = 36%, reduce = 0% -2013-09-18 01:33:41,047 Stage-1 map = 45%, reduce = 0% -2013-09-18 01:33:42,053 Stage-1 map = 49%, reduce = 0% -2013-09-18 01:33:43,073 Stage-1 map = 55%, reduce = 0% -2013-09-18 01:33:44,079 Stage-1 map = 58%, reduce = 0% -2013-09-18 01:33:45,084 Stage-1 map = 61%, reduce = 0% -2013-09-18 01:33:46,177 Stage-1 map = 63%, reduce = 0% -2013-09-18 01:33:47,185 Stage-1 map = 65%, reduce = 0% -2013-09-18 01:33:48,199 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 17.75 sec -2013-09-18 01:33:49,213 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 17.75 sec -2013-09-18 01:33:50,227 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 58.6 sec -2013-09-18 01:33:51,238 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 121.99 sec -2013-09-18 01:33:52,254 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 143.37 sec -2013-09-18 01:33:53,270 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 234.85 sec -2013-09-18 01:33:54,282 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 282.41 sec -2013-09-18 01:33:55,289 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 875.12 sec -2013-09-18 01:33:56,303 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 877.62 sec -2013-09-18 01:33:57,310 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 877.62 sec -2013-09-18 01:33:58,319 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 882.85 sec -2013-09-18 01:33:59,327 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 888.05 sec -2013-09-18 01:34:00,333 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 888.05 sec -2013-09-18 01:34:03,143 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 892.75 sec -2013-09-18 01:34:05,074 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 892.75 sec -2013-09-18 01:34:07,380 Stage-1 map = 94%, reduce = 15%, Cumulative CPU 917.16 sec -2013-09-18 01:34:08,387 Stage-1 map = 94%, reduce = 15%, Cumulative CPU 917.16 sec -2013-09-18 01:34:09,405 Stage-1 map = 94%, reduce = 15%, Cumulative CPU 917.16 sec -2013-09-18 01:34:12,449 Stage-1 map = 97%, reduce = 16%, Cumulative CPU 991.88 sec -2013-09-18 01:34:13,455 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 1042.76 sec -2013-09-18 01:34:14,461 Stage-1 map = 99%, reduce = 19%, Cumulative CPU 1042.76 sec -2013-09-18 01:34:15,467 Stage-1 map = 99%, reduce = 19%, Cumulative CPU 1042.76 sec -2013-09-18 01:34:16,473 Stage-1 map = 99%, reduce = 19%, Cumulative CPU 1085.14 sec -2013-09-18 01:34:17,478 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:18,484 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:19,489 Stage-1 map = 100%, reduce = 23%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:20,495 Stage-1 map = 100%, reduce = 23%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:21,500 Stage-1 map = 100%, reduce = 23%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:22,513 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:23,989 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:24,995 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:26,001 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:27,007 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:28,013 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:29,816 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:30,821 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:31,827 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:32,834 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:33,839 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:34,845 Stage-1 map = 100%, reduce = 42%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:38,795 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:40,768 Stage-1 map = 100%, reduce = 64%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:43,249 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:44,255 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:45,262 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:46,269 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:47,275 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:48,396 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:49,403 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:50,409 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:51,415 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:52,420 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:53,426 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:54,433 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:55,438 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1095.25 sec -2013-09-18 01:34:57,179 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 1415.82 sec -2013-09-18 01:34:58,185 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1415.82 sec -2013-09-18 01:34:59,192 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1415.82 sec -2013-09-18 01:35:00,198 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1415.82 sec -2013-09-18 01:35:01,205 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1415.82 sec -2013-09-18 01:35:02,210 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 1415.82 sec -2013-09-18 01:35:03,309 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1415.82 sec -2013-09-18 01:35:04,314 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1415.82 sec -2013-09-18 01:35:05,319 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1415.82 sec -2013-09-18 01:35:06,326 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1418.06 sec -2013-09-18 01:35:07,332 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1418.06 sec -2013-09-18 01:35:08,338 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1418.06 sec -2013-09-18 01:35:09,344 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1418.06 sec -2013-09-18 01:35:11,606 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1472.49 sec -2013-09-18 01:35:12,612 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1472.49 sec -2013-09-18 01:35:16,063 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1515.38 sec -2013-09-18 01:35:17,068 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1515.38 sec -MapReduce Total cumulative CPU time: 25 minutes 15 seconds 380 msec -Ended Job = job_201309172235_0155 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0156 -Hadoop job information for Stage-2: number of mappers: 14; number of reducers: 1 -2013-09-18 01:35:35,260 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:35:46,317 Stage-2 map = 17%, reduce = 0% -2013-09-18 01:35:49,327 Stage-2 map = 53%, reduce = 0% -2013-09-18 01:35:52,338 Stage-2 map = 78%, reduce = 0% -2013-09-18 01:35:55,352 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 310.42 sec -2013-09-18 01:35:56,358 Stage-2 map = 89%, reduce = 0%, Cumulative CPU 312.75 sec -2013-09-18 01:35:57,366 Stage-2 map = 97%, reduce = 0%, Cumulative CPU 320.21 sec -2013-09-18 01:35:58,371 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 325.09 sec -2013-09-18 01:35:59,376 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 325.09 sec -2013-09-18 01:36:00,380 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 325.09 sec -2013-09-18 01:36:01,386 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 325.09 sec -2013-09-18 01:36:02,391 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 325.09 sec -2013-09-18 01:36:03,397 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 325.09 sec -2013-09-18 01:36:04,402 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 325.09 sec -2013-09-18 01:36:05,407 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 325.09 sec -2013-09-18 01:36:06,412 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 325.09 sec -2013-09-18 01:36:07,417 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 325.09 sec -2013-09-18 01:36:08,422 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 325.09 sec -2013-09-18 01:36:09,427 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 325.09 sec -2013-09-18 01:36:10,433 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 325.09 sec -2013-09-18 01:36:11,438 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 325.09 sec -2013-09-18 01:36:12,444 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 325.09 sec -2013-09-18 01:36:13,449 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 325.09 sec -2013-09-18 01:36:14,453 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 325.09 sec -2013-09-18 01:36:15,458 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 325.09 sec -2013-09-18 01:36:16,463 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 325.09 sec -2013-09-18 01:36:17,468 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 325.09 sec -2013-09-18 01:36:18,473 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 325.09 sec -2013-09-18 01:36:19,477 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 325.09 sec -2013-09-18 01:36:20,482 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 325.09 sec -2013-09-18 01:36:21,486 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 325.09 sec -2013-09-18 01:36:25,199 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 325.09 sec -2013-09-18 01:36:26,204 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 325.09 sec -2013-09-18 01:36:27,208 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 325.09 sec -2013-09-18 01:36:28,213 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 325.09 sec -2013-09-18 01:36:29,217 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 325.09 sec -2013-09-18 01:36:32,866 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 325.09 sec -2013-09-18 01:36:33,870 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:34,875 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:35,880 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:36,885 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:37,888 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:38,892 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:39,896 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:40,900 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:41,904 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:43,207 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:44,211 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:45,216 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:46,220 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 325.09 sec -2013-09-18 01:36:47,225 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 325.09 sec -2013-09-18 01:36:48,377 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 325.09 sec -2013-09-18 01:36:49,382 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 325.09 sec -2013-09-18 01:36:50,386 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 325.09 sec -2013-09-18 01:36:51,390 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 325.09 sec -2013-09-18 01:36:52,394 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 325.09 sec -2013-09-18 01:36:53,398 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 325.09 sec -2013-09-18 01:36:54,402 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 325.09 sec -2013-09-18 01:36:55,406 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 325.09 sec -2013-09-18 01:37:04,189 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 369.99 sec -2013-09-18 01:37:05,193 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 369.99 sec -2013-09-18 01:37:06,197 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 369.99 sec -2013-09-18 01:37:07,202 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 369.99 sec -2013-09-18 01:37:08,206 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 369.99 sec -2013-09-18 01:37:09,212 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 369.99 sec -2013-09-18 01:37:10,216 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 376.06 sec -2013-09-18 01:37:11,220 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 376.06 sec -MapReduce Total cumulative CPU time: 6 minutes 16 seconds 60 msec -Ended Job = job_201309172235_0156 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1515.38 sec HDFS Read: 882764316 HDFS Write: 4155848594 SUCCESS -Job 1: Map: 14 Reduce: 1 Cumulative CPU: 376.06 sec HDFS Read: 4156020676 HDFS Write: 399 SUCCESS -Total MapReduce CPU Time Spent: 31 minutes 31 seconds 440 msec -OK -Time taken: 238.167 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по URL.; - - -times: 1 -query: SELECT 1, URL, count(*) AS c FROM hits_100m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3752@mturlrep13_201309180137_1035052608.txt -hive> SELECT 1, URL, count(*) AS c FROM hits_100m GROUP BY 1, URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0157 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:37:29,808 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:37:39,901 Stage-1 map = 5%, reduce = 0% -2013-09-18 01:37:42,922 Stage-1 map = 8%, reduce = 0% -2013-09-18 01:37:43,935 Stage-1 map = 11%, reduce = 0% -2013-09-18 01:37:44,947 Stage-1 map = 13%, reduce = 0% -2013-09-18 01:37:45,953 Stage-1 map = 18%, reduce = 0% -2013-09-18 01:37:46,968 Stage-1 map = 25%, reduce = 0% -2013-09-18 01:37:47,979 Stage-1 map = 32%, reduce = 0% -2013-09-18 01:37:48,989 Stage-1 map = 36%, reduce = 0% -2013-09-18 01:37:50,002 Stage-1 map = 43%, reduce = 0% -2013-09-18 01:37:51,007 Stage-1 map = 48%, reduce = 0% -2013-09-18 01:37:52,029 Stage-1 map = 49%, reduce = 0% -2013-09-18 01:37:53,038 Stage-1 map = 54%, reduce = 0% -2013-09-18 01:37:54,051 Stage-1 map = 58%, reduce = 0% -2013-09-18 01:37:55,056 Stage-1 map = 59%, reduce = 0% -2013-09-18 01:37:56,063 Stage-1 map = 61%, reduce = 0% -2013-09-18 01:37:57,079 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 669.47 sec -2013-09-18 01:37:58,095 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 671.87 sec -2013-09-18 01:37:59,110 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 674.43 sec -2013-09-18 01:38:00,117 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 680.69 sec -2013-09-18 01:38:01,130 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 692.13 sec -2013-09-18 01:38:02,144 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 697.59 sec -2013-09-18 01:38:03,151 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 703.09 sec -2013-09-18 01:38:04,158 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 715.71 sec -2013-09-18 01:38:05,171 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 724.69 sec -2013-09-18 01:38:06,182 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 733.5 sec -2013-09-18 01:38:07,188 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 733.5 sec -2013-09-18 01:38:12,135 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 799.0 sec -2013-09-18 01:38:15,432 Stage-1 map = 94%, reduce = 5%, Cumulative CPU 849.63 sec -2013-09-18 01:38:18,103 Stage-1 map = 94%, reduce = 16%, Cumulative CPU 849.63 sec -2013-09-18 01:38:21,194 Stage-1 map = 94%, reduce = 17%, Cumulative CPU 849.63 sec -2013-09-18 01:38:22,199 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 1126.64 sec -2013-09-18 01:38:23,206 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 1126.64 sec -2013-09-18 01:38:24,212 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 1149.35 sec -2013-09-18 01:38:25,218 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 1149.35 sec -2013-09-18 01:38:26,223 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 1149.35 sec -2013-09-18 01:38:27,229 Stage-1 map = 99%, reduce = 19%, Cumulative CPU 1149.35 sec -2013-09-18 01:38:28,235 Stage-1 map = 99%, reduce = 19%, Cumulative CPU 1149.35 sec -2013-09-18 01:38:29,242 Stage-1 map = 99%, reduce = 20%, Cumulative CPU 1149.35 sec -2013-09-18 01:38:30,940 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:31,946 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:32,952 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:33,959 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:34,965 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:37,627 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:38,638 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:39,961 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:40,968 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:41,978 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:42,985 Stage-1 map = 100%, reduce = 49%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:43,991 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:45,341 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:46,346 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:47,352 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:49,208 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:50,214 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:51,221 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:52,229 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:53,235 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:54,242 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:55,248 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:56,281 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:57,288 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:58,296 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1172.28 sec -2013-09-18 01:38:59,305 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:00,313 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:02,538 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:03,544 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:04,551 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:05,560 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:06,566 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:07,700 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:08,706 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:09,713 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:12,763 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:13,794 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:14,799 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:15,805 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:18,636 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:19,643 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1469.23 sec -2013-09-18 01:39:21,526 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1479.55 sec -2013-09-18 01:39:22,532 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1479.55 sec -2013-09-18 01:39:23,538 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1479.55 sec -2013-09-18 01:39:24,543 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1490.68 sec -2013-09-18 01:39:25,549 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1490.68 sec -2013-09-18 01:39:26,555 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1490.68 sec -2013-09-18 01:39:27,560 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1490.68 sec -2013-09-18 01:39:28,565 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1490.68 sec -2013-09-18 01:39:29,571 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1490.68 sec -2013-09-18 01:39:30,577 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1490.68 sec -2013-09-18 01:39:31,766 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1504.37 sec -2013-09-18 01:39:32,771 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1504.37 sec -2013-09-18 01:39:33,777 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1504.37 sec -2013-09-18 01:39:34,782 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1543.5 sec -2013-09-18 01:39:35,788 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1543.5 sec -2013-09-18 01:39:36,793 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1559.47 sec -2013-09-18 01:39:37,798 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1559.47 sec -2013-09-18 01:39:38,803 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1572.01 sec -2013-09-18 01:39:39,808 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1572.01 sec -2013-09-18 01:39:42,552 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1585.06 sec -2013-09-18 01:39:43,556 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1585.06 sec -2013-09-18 01:39:44,715 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1611.38 sec -2013-09-18 01:39:45,727 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1611.38 sec -2013-09-18 01:39:46,732 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1611.38 sec -2013-09-18 01:39:47,738 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1611.38 sec -2013-09-18 01:39:50,360 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1611.38 sec -2013-09-18 01:39:51,394 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1611.38 sec -2013-09-18 01:39:52,399 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1611.38 sec -MapReduce Total cumulative CPU time: 26 minutes 51 seconds 380 msec -Ended Job = job_201309172235_0157 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0158 -Hadoop job information for Stage-2: number of mappers: 14; number of reducers: 1 -2013-09-18 01:39:56,709 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:40:04,747 Stage-2 map = 15%, reduce = 0% -2013-09-18 01:40:07,758 Stage-2 map = 38%, reduce = 0% -2013-09-18 01:40:10,768 Stage-2 map = 73%, reduce = 0% -2013-09-18 01:40:13,811 Stage-2 map = 78%, reduce = 0% -2013-09-18 01:40:14,816 Stage-2 map = 86%, reduce = 0%, Cumulative CPU 115.95 sec -2013-09-18 01:40:15,821 Stage-2 map = 91%, reduce = 0%, Cumulative CPU 189.93 sec -2013-09-18 01:40:16,826 Stage-2 map = 95%, reduce = 0%, Cumulative CPU 268.74 sec -2013-09-18 01:40:17,830 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.63 sec -2013-09-18 01:40:18,835 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.63 sec -2013-09-18 01:40:19,839 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.63 sec -2013-09-18 01:40:20,843 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.63 sec -2013-09-18 01:40:21,848 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.63 sec -2013-09-18 01:40:22,853 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.63 sec -2013-09-18 01:40:23,857 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 350.63 sec -2013-09-18 01:40:24,862 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 350.63 sec -2013-09-18 01:40:25,866 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 350.63 sec -2013-09-18 01:40:26,871 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 350.63 sec -2013-09-18 01:40:27,875 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 350.63 sec -2013-09-18 01:40:28,880 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 350.63 sec -2013-09-18 01:40:29,884 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 350.63 sec -2013-09-18 01:40:30,889 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 350.63 sec -2013-09-18 01:40:31,893 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 350.63 sec -2013-09-18 01:40:32,897 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 350.63 sec -2013-09-18 01:40:33,901 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 350.63 sec -2013-09-18 01:40:34,905 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 350.63 sec -2013-09-18 01:40:35,909 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 350.63 sec -2013-09-18 01:40:36,914 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 350.63 sec -2013-09-18 01:40:37,918 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 350.63 sec -2013-09-18 01:40:38,923 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 350.63 sec -2013-09-18 01:40:39,927 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 350.63 sec -2013-09-18 01:40:40,931 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 350.63 sec -2013-09-18 01:40:41,935 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 350.63 sec -2013-09-18 01:40:42,939 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 350.63 sec -2013-09-18 01:40:43,943 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 350.63 sec -2013-09-18 01:40:44,947 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 350.63 sec -2013-09-18 01:40:45,951 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 350.63 sec -2013-09-18 01:40:46,956 Stage-2 map = 100%, reduce = 31%, Cumulative CPU 350.63 sec -2013-09-18 01:40:52,079 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 350.63 sec -2013-09-18 01:40:53,087 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 350.63 sec -2013-09-18 01:40:54,091 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 350.63 sec -2013-09-18 01:40:55,095 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 350.63 sec -2013-09-18 01:40:56,100 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 350.63 sec -2013-09-18 01:40:57,104 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 350.63 sec -2013-09-18 01:40:58,109 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 350.63 sec -2013-09-18 01:40:59,113 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:00,117 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:01,220 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:02,224 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:03,228 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:04,232 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:05,237 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:07,415 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:08,419 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:09,423 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:10,427 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:11,431 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:12,435 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:13,439 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:14,522 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:15,526 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:16,530 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:17,534 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:18,538 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 392.86 sec -2013-09-18 01:41:19,543 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 400.39 sec -2013-09-18 01:41:20,547 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 400.39 sec -MapReduce Total cumulative CPU time: 6 minutes 40 seconds 390 msec -Ended Job = job_201309172235_0158 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1611.38 sec HDFS Read: 882764316 HDFS Write: 4177941697 SUCCESS -Job 1: Map: 14 Reduce: 1 Cumulative CPU: 400.39 sec HDFS Read: 4178102019 HDFS Write: 419 SUCCESS -Total MapReduce CPU Time Spent: 33 minutes 31 seconds 770 msec -OK -Time taken: 241.283 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT 1, URL, count(*) AS c FROM hits_100m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13514@mturlrep13_201309180141_1043268364.txt -hive> SELECT 1, URL, count(*) AS c FROM hits_100m GROUP BY 1, URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0159 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:41:31,551 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:41:40,616 Stage-1 map = 1%, reduce = 0% -2013-09-18 01:41:41,622 Stage-1 map = 4%, reduce = 0% -2013-09-18 01:41:42,627 Stage-1 map = 5%, reduce = 0% -2013-09-18 01:41:44,636 Stage-1 map = 13%, reduce = 0% -2013-09-18 01:41:45,642 Stage-1 map = 15%, reduce = 0% -2013-09-18 01:41:46,663 Stage-1 map = 17%, reduce = 0% -2013-09-18 01:41:47,669 Stage-1 map = 23%, reduce = 0% -2013-09-18 01:41:48,691 Stage-1 map = 32%, reduce = 0% -2013-09-18 01:41:49,697 Stage-1 map = 34%, reduce = 0% -2013-09-18 01:41:50,703 Stage-1 map = 37%, reduce = 0% -2013-09-18 01:41:51,708 Stage-1 map = 47%, reduce = 0% -2013-09-18 01:41:52,719 Stage-1 map = 52%, reduce = 0% -2013-09-18 01:41:53,724 Stage-1 map = 54%, reduce = 0% -2013-09-18 01:41:54,734 Stage-1 map = 57%, reduce = 0% -2013-09-18 01:41:55,745 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 18.08 sec -2013-09-18 01:41:56,770 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 18.08 sec -2013-09-18 01:41:57,780 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 18.08 sec -2013-09-18 01:41:58,786 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 18.08 sec -2013-09-18 01:41:59,801 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 706.43 sec -2013-09-18 01:42:00,824 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 708.46 sec -2013-09-18 01:42:01,830 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 716.88 sec -2013-09-18 01:42:02,842 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 720.42 sec -2013-09-18 01:42:03,848 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 736.35 sec -2013-09-18 01:42:04,853 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 752.31 sec -2013-09-18 01:42:05,863 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 752.31 sec -2013-09-18 01:42:06,870 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 759.4 sec -2013-09-18 01:42:07,876 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 759.4 sec -2013-09-18 01:42:08,882 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 759.4 sec -2013-09-18 01:42:12,723 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 801.07 sec -2013-09-18 01:42:13,730 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 825.56 sec -2013-09-18 01:42:18,187 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 825.56 sec -2013-09-18 01:42:19,197 Stage-1 map = 93%, reduce = 13%, Cumulative CPU 825.56 sec -2013-09-18 01:42:20,202 Stage-1 map = 93%, reduce = 13%, Cumulative CPU 825.56 sec -2013-09-18 01:42:21,215 Stage-1 map = 93%, reduce = 13%, Cumulative CPU 825.56 sec -2013-09-18 01:42:22,230 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 976.34 sec -2013-09-18 01:42:23,237 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 994.69 sec -2013-09-18 01:42:25,056 Stage-1 map = 99%, reduce = 18%, Cumulative CPU 1066.61 sec -2013-09-18 01:42:26,062 Stage-1 map = 99%, reduce = 19%, Cumulative CPU 1066.61 sec -2013-09-18 01:42:27,068 Stage-1 map = 99%, reduce = 19%, Cumulative CPU 1088.21 sec -2013-09-18 01:42:28,074 Stage-1 map = 99%, reduce = 21%, Cumulative CPU 1088.21 sec -2013-09-18 01:42:30,283 Stage-1 map = 99%, reduce = 26%, Cumulative CPU 1088.21 sec -2013-09-18 01:42:31,294 Stage-1 map = 99%, reduce = 29%, Cumulative CPU 1131.05 sec -2013-09-18 01:42:32,300 Stage-1 map = 99%, reduce = 29%, Cumulative CPU 1131.05 sec -2013-09-18 01:42:33,318 Stage-1 map = 99%, reduce = 29%, Cumulative CPU 1131.05 sec -2013-09-18 01:42:34,323 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:35,330 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:36,336 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:37,343 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:38,350 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:39,356 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:40,361 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:42,047 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:43,052 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:44,058 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:45,064 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:46,070 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:48,082 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:51,512 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:52,518 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:53,525 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:56,648 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:57,653 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:58,659 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1155.52 sec -2013-09-18 01:42:59,667 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:00,674 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:01,680 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:02,686 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:03,695 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:04,701 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:05,707 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:06,713 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:08,485 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:09,490 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:10,495 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:11,501 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:12,507 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:13,772 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:14,777 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:15,782 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:16,788 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:17,794 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:18,800 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:19,806 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:21,139 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:22,145 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:23,151 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:26,220 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:30,331 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:31,337 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:32,343 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:33,349 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1351.1 sec -2013-09-18 01:43:34,355 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1392.71 sec -2013-09-18 01:43:35,367 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1597.17 sec -2013-09-18 01:43:36,372 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1597.17 sec -MapReduce Total cumulative CPU time: 26 minutes 37 seconds 170 msec -Ended Job = job_201309172235_0159 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0160 -Hadoop job information for Stage-2: number of mappers: 14; number of reducers: 1 -2013-09-18 01:43:38,868 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:43:49,917 Stage-2 map = 6%, reduce = 0% -2013-09-18 01:43:55,242 Stage-2 map = 16%, reduce = 0% -2013-09-18 01:43:58,254 Stage-2 map = 58%, reduce = 0% -2013-09-18 01:44:01,267 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 232.96 sec -2013-09-18 01:44:02,272 Stage-2 map = 80%, reduce = 0%, Cumulative CPU 236.94 sec -2013-09-18 01:44:03,277 Stage-2 map = 85%, reduce = 0%, Cumulative CPU 254.07 sec -2013-09-18 01:44:04,282 Stage-2 map = 91%, reduce = 0%, Cumulative CPU 282.18 sec -2013-09-18 01:44:05,288 Stage-2 map = 95%, reduce = 0%, Cumulative CPU 324.72 sec -2013-09-18 01:44:06,294 Stage-2 map = 98%, reduce = 0%, Cumulative CPU 324.72 sec -2013-09-18 01:44:07,299 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 336.02 sec -2013-09-18 01:44:08,304 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 336.02 sec -2013-09-18 01:44:09,309 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 336.02 sec -2013-09-18 01:44:10,314 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 336.02 sec -2013-09-18 01:44:11,319 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 336.02 sec -2013-09-18 01:44:12,324 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 336.02 sec -2013-09-18 01:44:13,329 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 336.02 sec -2013-09-18 01:44:14,334 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 336.02 sec -2013-09-18 01:44:15,338 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 336.02 sec -2013-09-18 01:44:16,343 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 336.02 sec -2013-09-18 01:44:17,347 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 336.02 sec -2013-09-18 01:44:18,351 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 336.02 sec -2013-09-18 01:44:19,356 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 336.02 sec -2013-09-18 01:44:20,365 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 336.02 sec -2013-09-18 01:44:21,370 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 336.02 sec -2013-09-18 01:44:22,374 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 336.02 sec -2013-09-18 01:44:23,379 Stage-2 map = 100%, reduce = 19%, Cumulative CPU 336.02 sec -2013-09-18 01:44:24,383 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 336.02 sec -2013-09-18 01:44:25,388 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 336.02 sec -2013-09-18 01:44:26,392 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 336.02 sec -2013-09-18 01:44:27,396 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 336.02 sec -2013-09-18 01:44:28,401 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 336.02 sec -2013-09-18 01:44:29,405 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 336.02 sec -2013-09-18 01:44:30,409 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 336.02 sec -2013-09-18 01:44:31,413 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 336.02 sec -2013-09-18 01:44:32,417 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 336.02 sec -2013-09-18 01:44:33,421 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 336.02 sec -2013-09-18 01:44:34,425 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 336.02 sec -2013-09-18 01:44:37,987 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 336.02 sec -2013-09-18 01:44:38,993 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 336.02 sec -2013-09-18 01:44:39,997 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 336.02 sec -2013-09-18 01:44:41,001 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 336.02 sec -2013-09-18 01:44:42,006 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 336.02 sec -2013-09-18 01:44:43,010 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 336.02 sec -2013-09-18 01:44:44,083 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 336.02 sec -2013-09-18 01:44:45,087 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 336.02 sec -2013-09-18 01:44:46,091 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 336.02 sec -2013-09-18 01:44:47,095 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 336.02 sec -2013-09-18 01:44:48,099 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 336.02 sec -2013-09-18 01:44:49,103 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 336.02 sec -2013-09-18 01:44:50,106 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 336.02 sec -2013-09-18 01:44:51,478 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 336.02 sec -2013-09-18 01:44:52,482 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 336.02 sec -2013-09-18 01:44:53,485 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 336.02 sec -2013-09-18 01:44:54,489 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 336.02 sec -2013-09-18 01:44:55,493 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 336.02 sec -2013-09-18 01:44:56,497 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 336.02 sec -2013-09-18 01:44:57,500 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 336.02 sec -2013-09-18 01:44:58,503 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 336.02 sec -2013-09-18 01:44:59,507 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 381.91 sec -2013-09-18 01:45:00,511 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 381.91 sec -2013-09-18 01:45:01,514 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 381.91 sec -2013-09-18 01:45:02,518 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 381.91 sec -2013-09-18 01:45:03,522 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 381.91 sec -2013-09-18 01:45:04,526 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 381.91 sec -2013-09-18 01:45:05,530 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 381.91 sec -2013-09-18 01:45:06,534 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 381.91 sec -2013-09-18 01:45:07,538 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 381.91 sec -2013-09-18 01:45:08,543 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 386.71 sec -2013-09-18 01:45:09,548 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 386.71 sec -2013-09-18 01:45:10,553 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 386.71 sec -MapReduce Total cumulative CPU time: 6 minutes 26 seconds 710 msec -Ended Job = job_201309172235_0160 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1597.17 sec HDFS Read: 882764316 HDFS Write: 4177941697 SUCCESS -Job 1: Map: 14 Reduce: 1 Cumulative CPU: 386.71 sec HDFS Read: 4178102019 HDFS Write: 419 SUCCESS -Total MapReduce CPU Time Spent: 33 minutes 3 seconds 880 msec -OK -Time taken: 228.198 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT 1, URL, count(*) AS c FROM hits_100m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22530@mturlrep13_201309180145_1699071486.txt -hive> SELECT 1, URL, count(*) AS c FROM hits_100m GROUP BY 1, URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0161 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:45:20,932 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:45:30,999 Stage-1 map = 2%, reduce = 0% -2013-09-18 01:45:32,006 Stage-1 map = 4%, reduce = 0% -2013-09-18 01:45:35,020 Stage-1 map = 9%, reduce = 0% -2013-09-18 01:45:36,034 Stage-1 map = 16%, reduce = 0% -2013-09-18 01:45:37,039 Stage-1 map = 17%, reduce = 0% -2013-09-18 01:45:38,045 Stage-1 map = 20%, reduce = 0% -2013-09-18 01:45:39,054 Stage-1 map = 33%, reduce = 0% -2013-09-18 01:45:41,072 Stage-1 map = 36%, reduce = 0% -2013-09-18 01:45:42,085 Stage-1 map = 50%, reduce = 0% -2013-09-18 01:45:44,098 Stage-1 map = 54%, reduce = 0% -2013-09-18 01:45:45,104 Stage-1 map = 55%, reduce = 0% -2013-09-18 01:45:46,112 Stage-1 map = 60%, reduce = 0% -2013-09-18 01:45:48,123 Stage-1 map = 62%, reduce = 0% -2013-09-18 01:45:49,139 Stage-1 map = 65%, reduce = 0%, Cumulative CPU 19.16 sec -2013-09-18 01:45:50,153 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 61.29 sec -2013-09-18 01:45:51,160 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 82.27 sec -2013-09-18 01:45:52,178 Stage-1 map = 73%, reduce = 0%, Cumulative CPU 104.66 sec -2013-09-18 01:45:53,184 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 149.39 sec -2013-09-18 01:45:54,190 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 196.78 sec -2013-09-18 01:45:55,197 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 221.14 sec -2013-09-18 01:45:56,210 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 271.7 sec -2013-09-18 01:45:57,219 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 297.77 sec -2013-09-18 01:45:58,225 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 297.77 sec -2013-09-18 01:45:59,232 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 326.46 sec -2013-09-18 01:46:02,398 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 976.26 sec -2013-09-18 01:46:03,406 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 978.66 sec -2013-09-18 01:46:07,469 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 981.5 sec -2013-09-18 01:46:08,492 Stage-1 map = 93%, reduce = 9%, Cumulative CPU 985.55 sec -2013-09-18 01:46:09,498 Stage-1 map = 93%, reduce = 9%, Cumulative CPU 990.67 sec -2013-09-18 01:46:12,267 Stage-1 map = 94%, reduce = 9%, Cumulative CPU 999.32 sec -2013-09-18 01:46:13,634 Stage-1 map = 96%, reduce = 16%, Cumulative CPU 1050.69 sec -2013-09-18 01:46:14,648 Stage-1 map = 98%, reduce = 16%, Cumulative CPU 1108.43 sec -2013-09-18 01:46:15,653 Stage-1 map = 99%, reduce = 16%, Cumulative CPU 1125.98 sec -2013-09-18 01:46:16,659 Stage-1 map = 100%, reduce = 16%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:17,666 Stage-1 map = 100%, reduce = 16%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:18,672 Stage-1 map = 100%, reduce = 16%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:20,622 Stage-1 map = 100%, reduce = 19%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:21,629 Stage-1 map = 100%, reduce = 22%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:22,635 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:23,643 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:24,982 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:25,989 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:26,996 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:28,002 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:30,536 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:31,543 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:32,549 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:35,134 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:36,140 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:39,099 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:40,105 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:41,113 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:42,118 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:43,185 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:44,192 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:45,198 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:46,205 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:47,211 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:48,217 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:49,748 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:50,753 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:51,759 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:52,765 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:53,772 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:54,777 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:56,274 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:57,278 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:58,286 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 1161.21 sec -2013-09-18 01:46:59,292 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1161.21 sec -2013-09-18 01:47:00,301 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1508.73 sec -2013-09-18 01:47:01,307 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 1508.73 sec -2013-09-18 01:47:02,711 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1508.73 sec -2013-09-18 01:47:03,717 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1508.73 sec -2013-09-18 01:47:04,723 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 1508.73 sec -2013-09-18 01:47:05,730 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1508.73 sec -2013-09-18 01:47:09,415 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 1508.73 sec -2013-09-18 01:47:13,308 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1508.73 sec -2013-09-18 01:47:14,314 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1516.27 sec -2013-09-18 01:47:15,320 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1516.27 sec -2013-09-18 01:47:16,326 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:17,332 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:18,337 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:19,342 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:20,348 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:21,353 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:22,358 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:23,363 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:24,368 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:25,373 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:26,379 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1524.18 sec -2013-09-18 01:47:27,385 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1541.34 sec -2013-09-18 01:47:28,390 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1541.34 sec -2013-09-18 01:47:29,396 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1541.34 sec -2013-09-18 01:47:30,401 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1541.34 sec -2013-09-18 01:47:31,406 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1555.39 sec -2013-09-18 01:47:32,411 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1555.39 sec -2013-09-18 01:47:33,417 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1555.39 sec -2013-09-18 01:47:34,423 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1555.39 sec -2013-09-18 01:47:35,428 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1555.39 sec -2013-09-18 01:47:36,433 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1555.39 sec -2013-09-18 01:47:37,855 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1555.39 sec -2013-09-18 01:47:38,866 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1555.39 sec -2013-09-18 01:47:39,872 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1555.39 sec -2013-09-18 01:47:40,877 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1555.39 sec -2013-09-18 01:47:41,881 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1565.2 sec -2013-09-18 01:47:42,886 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1565.2 sec -2013-09-18 01:47:44,926 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1565.2 sec -2013-09-18 01:47:45,931 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1565.2 sec -2013-09-18 01:47:46,936 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 1565.2 sec -2013-09-18 01:47:47,942 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1621.8 sec -2013-09-18 01:47:51,089 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 1621.8 sec -MapReduce Total cumulative CPU time: 27 minutes 1 seconds 800 msec -Ended Job = job_201309172235_0161 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0162 -Hadoop job information for Stage-2: number of mappers: 14; number of reducers: 1 -2013-09-18 01:47:56,428 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:48:04,465 Stage-2 map = 7%, reduce = 0% -2013-09-18 01:48:05,469 Stage-2 map = 10%, reduce = 0% -2013-09-18 01:48:07,477 Stage-2 map = 21%, reduce = 0% -2013-09-18 01:48:08,482 Stage-2 map = 34%, reduce = 0% -2013-09-18 01:48:10,489 Stage-2 map = 43%, reduce = 0% -2013-09-18 01:48:11,494 Stage-2 map = 70%, reduce = 0% -2013-09-18 01:48:14,505 Stage-2 map = 78%, reduce = 0% -2013-09-18 01:48:15,510 Stage-2 map = 89%, reduce = 0%, Cumulative CPU 167.69 sec -2013-09-18 01:48:16,516 Stage-2 map = 89%, reduce = 0%, Cumulative CPU 167.69 sec -2013-09-18 01:48:17,535 Stage-2 map = 94%, reduce = 0%, Cumulative CPU 246.5 sec -2013-09-18 01:48:18,541 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 356.26 sec -2013-09-18 01:48:19,545 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 356.26 sec -2013-09-18 01:48:20,551 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 356.26 sec -2013-09-18 01:48:21,555 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 356.26 sec -2013-09-18 01:48:22,560 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 356.26 sec -2013-09-18 01:48:23,565 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 356.26 sec -2013-09-18 01:48:24,570 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 356.26 sec -2013-09-18 01:48:25,574 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 356.26 sec -2013-09-18 01:48:26,578 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 356.26 sec -2013-09-18 01:48:27,582 Stage-2 map = 100%, reduce = 10%, Cumulative CPU 356.26 sec -2013-09-18 01:48:28,587 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 356.26 sec -2013-09-18 01:48:29,591 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 356.26 sec -2013-09-18 01:48:30,595 Stage-2 map = 100%, reduce = 12%, Cumulative CPU 356.26 sec -2013-09-18 01:48:31,599 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 356.26 sec -2013-09-18 01:48:32,603 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 356.26 sec -2013-09-18 01:48:33,606 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 356.26 sec -2013-09-18 01:48:34,610 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 356.26 sec -2013-09-18 01:48:35,615 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 356.26 sec -2013-09-18 01:48:36,619 Stage-2 map = 100%, reduce = 21%, Cumulative CPU 356.26 sec -2013-09-18 01:48:37,624 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 356.26 sec -2013-09-18 01:48:38,628 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 356.26 sec -2013-09-18 01:48:39,632 Stage-2 map = 100%, reduce = 24%, Cumulative CPU 356.26 sec -2013-09-18 01:48:40,636 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 356.26 sec -2013-09-18 01:48:41,641 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 356.26 sec -2013-09-18 01:48:42,645 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 356.26 sec -2013-09-18 01:48:43,649 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 356.26 sec -2013-09-18 01:48:44,653 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 356.26 sec -2013-09-18 01:48:45,658 Stage-2 map = 100%, reduce = 29%, Cumulative CPU 356.26 sec -2013-09-18 01:48:46,662 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 356.26 sec -2013-09-18 01:48:52,547 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 356.26 sec -2013-09-18 01:48:53,551 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 356.26 sec -2013-09-18 01:48:54,555 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 356.26 sec -2013-09-18 01:48:55,649 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 356.26 sec -2013-09-18 01:48:56,653 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 356.26 sec -2013-09-18 01:48:57,657 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 356.26 sec -2013-09-18 01:48:58,661 Stage-2 map = 100%, reduce = 70%, Cumulative CPU 356.26 sec -2013-09-18 01:49:00,401 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:01,405 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:02,409 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:03,413 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:04,417 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:05,421 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:06,424 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:07,428 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:08,431 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:09,435 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:10,439 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:11,443 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:12,447 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:13,452 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:14,456 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:15,461 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:16,465 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:17,470 Stage-2 map = 100%, reduce = 78%, Cumulative CPU 399.92 sec -2013-09-18 01:49:18,477 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 406.76 sec -2013-09-18 01:49:19,481 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 406.76 sec -MapReduce Total cumulative CPU time: 6 minutes 46 seconds 760 msec -Ended Job = job_201309172235_0162 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 1621.8 sec HDFS Read: 882764316 HDFS Write: 4177941697 SUCCESS -Job 1: Map: 14 Reduce: 1 Cumulative CPU: 406.76 sec HDFS Read: 4178102019 HDFS Write: 419 SUCCESS -Total MapReduce CPU Time Spent: 33 minutes 48 seconds 560 msec -OK -Time taken: 246.999 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по URL и числу.; - - -times: 1 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31570@mturlrep13_201309180149_1050651506.txt -hive> SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0163 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:49:38,118 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:49:51,275 Stage-1 map = 2%, reduce = 0% -2013-09-18 01:49:52,282 Stage-1 map = 6%, reduce = 0% -2013-09-18 01:49:53,293 Stage-1 map = 8%, reduce = 0% -2013-09-18 01:49:54,299 Stage-1 map = 25%, reduce = 0% -2013-09-18 01:49:55,305 Stage-1 map = 31%, reduce = 0% -2013-09-18 01:49:56,314 Stage-1 map = 33%, reduce = 0% -2013-09-18 01:49:57,321 Stage-1 map = 49%, reduce = 0% -2013-09-18 01:49:58,338 Stage-1 map = 55%, reduce = 0% -2013-09-18 01:49:59,351 Stage-1 map = 60%, reduce = 0%, Cumulative CPU 13.34 sec -2013-09-18 01:50:00,378 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 494.57 sec -2013-09-18 01:50:01,385 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 494.57 sec -2013-09-18 01:50:02,403 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 498.86 sec -2013-09-18 01:50:03,411 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 506.12 sec -2013-09-18 01:50:04,419 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 517.77 sec -2013-09-18 01:50:05,426 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 529.34 sec -2013-09-18 01:50:06,432 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 536.01 sec -2013-09-18 01:50:07,440 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 574.29 sec -2013-09-18 01:50:08,448 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 643.5 sec -2013-09-18 01:50:09,455 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 690.76 sec -2013-09-18 01:50:10,461 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 701.69 sec -2013-09-18 01:50:11,468 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 701.69 sec -2013-09-18 01:50:12,491 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 716.61 sec -2013-09-18 01:50:13,497 Stage-1 map = 99%, reduce = 4%, Cumulative CPU 758.65 sec -2013-09-18 01:50:14,502 Stage-1 map = 99%, reduce = 19%, Cumulative CPU 758.65 sec -2013-09-18 01:50:15,508 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 774.74 sec -2013-09-18 01:50:16,515 Stage-1 map = 100%, reduce = 26%, Cumulative CPU 774.74 sec -2013-09-18 01:50:17,521 Stage-1 map = 100%, reduce = 28%, Cumulative CPU 774.74 sec -2013-09-18 01:50:18,526 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 774.74 sec -2013-09-18 01:50:19,533 Stage-1 map = 100%, reduce = 32%, Cumulative CPU 774.74 sec -2013-09-18 01:50:20,539 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 774.74 sec -2013-09-18 01:50:21,545 Stage-1 map = 100%, reduce = 39%, Cumulative CPU 774.74 sec -2013-09-18 01:50:22,551 Stage-1 map = 100%, reduce = 42%, Cumulative CPU 774.74 sec -2013-09-18 01:50:23,557 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 774.74 sec -2013-09-18 01:50:24,562 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 774.74 sec -2013-09-18 01:50:25,568 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 774.74 sec -2013-09-18 01:50:26,574 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 774.74 sec -2013-09-18 01:50:27,583 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 802.67 sec -2013-09-18 01:50:28,589 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 874.81 sec -2013-09-18 01:50:29,594 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 934.02 sec -2013-09-18 01:50:30,617 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 934.02 sec -MapReduce Total cumulative CPU time: 15 minutes 34 seconds 20 msec -Ended Job = job_201309172235_0163 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0164 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 01:50:34,091 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:50:44,120 Stage-2 map = 23%, reduce = 0% -2013-09-18 01:50:47,129 Stage-2 map = 46%, reduce = 0% -2013-09-18 01:50:51,142 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 23.2 sec -2013-09-18 01:50:52,147 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 23.2 sec -2013-09-18 01:50:53,152 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 23.2 sec -2013-09-18 01:50:54,157 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 23.2 sec -2013-09-18 01:50:55,162 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 23.2 sec -2013-09-18 01:50:56,167 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 23.2 sec -2013-09-18 01:50:58,116 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 23.2 sec -2013-09-18 01:50:59,121 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 23.2 sec -2013-09-18 01:51:00,129 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 23.2 sec -2013-09-18 01:51:01,134 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:02,139 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:03,144 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:04,149 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:05,154 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:06,158 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:07,163 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:08,172 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:09,177 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:10,182 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:11,186 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:12,191 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:13,195 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:14,200 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:15,205 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:16,209 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.42 sec -2013-09-18 01:51:17,214 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.78 sec -2013-09-18 01:51:18,218 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.78 sec -2013-09-18 01:51:19,223 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.78 sec -2013-09-18 01:51:20,227 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.78 sec -2013-09-18 01:51:21,232 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.78 sec -2013-09-18 01:51:22,236 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.78 sec -2013-09-18 01:51:23,241 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.78 sec -2013-09-18 01:51:24,245 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.78 sec -2013-09-18 01:51:25,250 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.78 sec -2013-09-18 01:51:26,254 Stage-2 map = 100%, reduce = 89%, Cumulative CPU 77.78 sec -2013-09-18 01:51:27,259 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 87.04 sec -2013-09-18 01:51:28,264 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 87.04 sec -MapReduce Total cumulative CPU time: 1 minutes 27 seconds 40 msec -Ended Job = job_201309172235_0164 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 934.02 sec HDFS Read: 339202346 HDFS Write: 374792539 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 87.04 sec HDFS Read: 374796261 HDFS Write: 498 SUCCESS -Total MapReduce CPU Time Spent: 17 minutes 1 seconds 60 msec -OK -Time taken: 120.684 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7222@mturlrep13_201309180151_945335118.txt -hive> SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0165 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:51:37,818 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:51:49,905 Stage-1 map = 2%, reduce = 0% -2013-09-18 01:51:50,923 Stage-1 map = 6%, reduce = 0% -2013-09-18 01:51:52,936 Stage-1 map = 15%, reduce = 0% -2013-09-18 01:51:53,959 Stage-1 map = 33%, reduce = 0% -2013-09-18 01:51:54,965 Stage-1 map = 36%, reduce = 0% -2013-09-18 01:51:55,979 Stage-1 map = 41%, reduce = 0% -2013-09-18 01:51:57,007 Stage-1 map = 62%, reduce = 0% -2013-09-18 01:51:58,018 Stage-1 map = 63%, reduce = 0% -2013-09-18 01:51:59,032 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 13.91 sec -2013-09-18 01:52:00,049 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 13.91 sec -2013-09-18 01:52:01,062 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 546.5 sec -2013-09-18 01:52:02,075 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 546.5 sec -2013-09-18 01:52:03,086 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 553.11 sec -2013-09-18 01:52:04,094 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 570.16 sec -2013-09-18 01:52:05,112 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 591.73 sec -2013-09-18 01:52:06,122 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 606.61 sec -2013-09-18 01:52:07,134 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 689.49 sec -2013-09-18 01:52:08,140 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 704.24 sec -2013-09-18 01:52:09,146 Stage-1 map = 95%, reduce = 0%, Cumulative CPU 704.24 sec -2013-09-18 01:52:10,152 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 704.24 sec -2013-09-18 01:52:11,158 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 704.24 sec -2013-09-18 01:52:12,165 Stage-1 map = 98%, reduce = 4%, Cumulative CPU 717.42 sec -2013-09-18 01:52:13,170 Stage-1 map = 99%, reduce = 23%, Cumulative CPU 745.46 sec -2013-09-18 01:52:14,176 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 775.29 sec -2013-09-18 01:52:15,181 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 775.29 sec -2013-09-18 01:52:16,187 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 775.29 sec -2013-09-18 01:52:17,193 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 775.29 sec -2013-09-18 01:52:18,199 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 775.29 sec -2013-09-18 01:52:19,204 Stage-1 map = 100%, reduce = 31%, Cumulative CPU 775.29 sec -2013-09-18 01:52:20,209 Stage-1 map = 100%, reduce = 34%, Cumulative CPU 775.29 sec -2013-09-18 01:52:21,215 Stage-1 map = 100%, reduce = 43%, Cumulative CPU 775.29 sec -2013-09-18 01:52:22,221 Stage-1 map = 100%, reduce = 60%, Cumulative CPU 775.29 sec -2013-09-18 01:52:23,226 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 775.29 sec -2013-09-18 01:52:24,232 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 775.29 sec -2013-09-18 01:52:25,238 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 775.29 sec -2013-09-18 01:52:26,245 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 789.78 sec -2013-09-18 01:52:27,251 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 848.54 sec -2013-09-18 01:52:28,257 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 936.77 sec -2013-09-18 01:52:29,275 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 936.77 sec -2013-09-18 01:52:30,282 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 936.77 sec -MapReduce Total cumulative CPU time: 15 minutes 36 seconds 770 msec -Ended Job = job_201309172235_0165 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0166 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 01:52:33,818 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:52:43,850 Stage-2 map = 23%, reduce = 0% -2013-09-18 01:52:46,861 Stage-2 map = 46%, reduce = 0% -2013-09-18 01:52:49,871 Stage-2 map = 52%, reduce = 0% -2013-09-18 01:52:51,880 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 24.11 sec -2013-09-18 01:52:52,885 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 24.11 sec -2013-09-18 01:52:53,890 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 24.11 sec -2013-09-18 01:52:54,896 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 24.11 sec -2013-09-18 01:52:55,901 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 24.11 sec -2013-09-18 01:52:56,907 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 24.11 sec -2013-09-18 01:52:57,912 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 24.11 sec -2013-09-18 01:52:58,918 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 24.11 sec -2013-09-18 01:52:59,923 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 24.11 sec -2013-09-18 01:53:00,928 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:01,933 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:02,938 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:03,943 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:04,948 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:05,959 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:06,963 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:07,968 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:08,973 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:09,979 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:10,984 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:11,989 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:12,995 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:14,000 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:15,006 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 58.35 sec -2013-09-18 01:53:16,010 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.21 sec -2013-09-18 01:53:17,014 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.21 sec -2013-09-18 01:53:18,018 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.21 sec -2013-09-18 01:53:19,023 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.21 sec -2013-09-18 01:53:20,027 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.21 sec -2013-09-18 01:53:21,031 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.21 sec -2013-09-18 01:53:22,035 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.21 sec -2013-09-18 01:53:23,039 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.21 sec -2013-09-18 01:53:24,044 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 78.21 sec -2013-09-18 01:53:25,048 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 78.21 sec -2013-09-18 01:53:26,053 Stage-2 map = 100%, reduce = 83%, Cumulative CPU 78.21 sec -2013-09-18 01:53:27,058 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 87.43 sec -2013-09-18 01:53:28,063 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 87.43 sec -2013-09-18 01:53:29,068 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 87.43 sec -MapReduce Total cumulative CPU time: 1 minutes 27 seconds 430 msec -Ended Job = job_201309172235_0166 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 936.77 sec HDFS Read: 339202346 HDFS Write: 374792539 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 87.43 sec HDFS Read: 374796261 HDFS Write: 498 SUCCESS -Total MapReduce CPU Time Spent: 17 minutes 4 seconds 200 msec -OK -Time taken: 118.948 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14110@mturlrep13_201309180153_1737492136.txt -hive> SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0167 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:53:39,704 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:53:50,787 Stage-1 map = 1%, reduce = 0% -2013-09-18 01:53:51,799 Stage-1 map = 4%, reduce = 0% -2013-09-18 01:53:52,813 Stage-1 map = 5%, reduce = 0% -2013-09-18 01:53:53,831 Stage-1 map = 14%, reduce = 0% -2013-09-18 01:53:54,840 Stage-1 map = 34%, reduce = 0% -2013-09-18 01:53:55,846 Stage-1 map = 35%, reduce = 0% -2013-09-18 01:53:56,851 Stage-1 map = 39%, reduce = 0% -2013-09-18 01:53:57,860 Stage-1 map = 59%, reduce = 0% -2013-09-18 01:53:58,883 Stage-1 map = 63%, reduce = 0% -2013-09-18 01:53:59,905 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 13.8 sec -2013-09-18 01:54:00,943 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 13.8 sec -2013-09-18 01:54:01,952 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 535.34 sec -2013-09-18 01:54:02,959 Stage-1 map = 80%, reduce = 0%, Cumulative CPU 537.7 sec -2013-09-18 01:54:03,966 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 540.16 sec -2013-09-18 01:54:04,973 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 551.46 sec -2013-09-18 01:54:05,979 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 573.16 sec -2013-09-18 01:54:07,008 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 605.0 sec -2013-09-18 01:54:08,027 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 654.51 sec -2013-09-18 01:54:09,034 Stage-1 map = 94%, reduce = 0%, Cumulative CPU 691.64 sec -2013-09-18 01:54:10,040 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 700.45 sec -2013-09-18 01:54:11,047 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 700.45 sec -2013-09-18 01:54:12,053 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 700.45 sec -2013-09-18 01:54:13,060 Stage-1 map = 98%, reduce = 3%, Cumulative CPU 700.45 sec -2013-09-18 01:54:14,066 Stage-1 map = 99%, reduce = 21%, Cumulative CPU 745.67 sec -2013-09-18 01:54:15,072 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 776.9 sec -2013-09-18 01:54:16,078 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 776.9 sec -2013-09-18 01:54:17,084 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 776.9 sec -2013-09-18 01:54:18,094 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 776.9 sec -2013-09-18 01:54:19,103 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 776.9 sec -2013-09-18 01:54:20,109 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 776.9 sec -2013-09-18 01:54:21,115 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 776.9 sec -2013-09-18 01:54:22,125 Stage-1 map = 100%, reduce = 43%, Cumulative CPU 776.9 sec -2013-09-18 01:54:23,131 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 776.9 sec -2013-09-18 01:54:24,137 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 776.9 sec -2013-09-18 01:54:25,144 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 776.9 sec -2013-09-18 01:54:26,149 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 776.9 sec -2013-09-18 01:54:27,155 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 776.9 sec -2013-09-18 01:54:28,164 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 862.86 sec -2013-09-18 01:54:29,177 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 934.72 sec -2013-09-18 01:54:30,183 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 934.72 sec -2013-09-18 01:54:31,188 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 934.72 sec -MapReduce Total cumulative CPU time: 15 minutes 34 seconds 720 msec -Ended Job = job_201309172235_0167 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0168 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 01:54:34,764 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:54:44,797 Stage-2 map = 23%, reduce = 0% -2013-09-18 01:54:47,807 Stage-2 map = 46%, reduce = 0% -2013-09-18 01:54:50,817 Stage-2 map = 52%, reduce = 0% -2013-09-18 01:54:52,826 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-18 01:54:53,831 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-18 01:54:54,836 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-18 01:54:55,841 Stage-2 map = 69%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-18 01:54:56,847 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-18 01:54:57,852 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-18 01:54:58,857 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-18 01:54:59,863 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 23.57 sec -2013-09-18 01:55:00,868 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 23.57 sec -2013-09-18 01:55:01,874 Stage-2 map = 81%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:02,879 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:03,883 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:04,888 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:05,898 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:06,903 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:07,908 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:08,913 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:09,918 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:10,923 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:11,928 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:12,933 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:13,938 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:14,942 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:15,947 Stage-2 map = 94%, reduce = 17%, Cumulative CPU 57.88 sec -2013-09-18 01:55:16,952 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.06 sec -2013-09-18 01:55:17,957 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.06 sec -2013-09-18 01:55:18,962 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.06 sec -2013-09-18 01:55:19,967 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.06 sec -2013-09-18 01:55:20,972 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.06 sec -2013-09-18 01:55:21,977 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.06 sec -2013-09-18 01:55:22,982 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.06 sec -2013-09-18 01:55:23,987 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.06 sec -2013-09-18 01:55:24,992 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.06 sec -2013-09-18 01:55:25,997 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.06 sec -2013-09-18 01:55:27,002 Stage-2 map = 100%, reduce = 89%, Cumulative CPU 77.06 sec -2013-09-18 01:55:28,008 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 86.19 sec -2013-09-18 01:55:29,013 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 86.19 sec -MapReduce Total cumulative CPU time: 1 minutes 26 seconds 190 msec -Ended Job = job_201309172235_0168 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 934.72 sec HDFS Read: 339202346 HDFS Write: 374792539 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 86.19 sec HDFS Read: 374796261 HDFS Write: 498 SUCCESS -Total MapReduce CPU Time Spent: 17 minutes 0 seconds 910 msec -OK -Time taken: 118.18 seconds, Fetched: 10 row(s) -hive> quit; - -times: 1 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21734@mturlrep13_201309180155_15887287.txt -hive> ; -hive> quit; - -times: 2 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21941@mturlrep13_201309180155_1922409770.txt -hive> ; -hive> quit; - -times: 3 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22141@mturlrep13_201309180155_818952348.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22349@mturlrep13_201309180155_129320467.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0169 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:55:58,117 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:56:11,233 Stage-1 map = 2%, reduce = 0% -2013-09-18 01:56:12,984 Stage-1 map = 3%, reduce = 0% -2013-09-18 01:56:13,990 Stage-1 map = 8%, reduce = 0% -2013-09-18 01:56:14,996 Stage-1 map = 14%, reduce = 0% -2013-09-18 01:56:16,314 Stage-1 map = 16%, reduce = 0% -2013-09-18 01:56:21,716 Stage-1 map = 36%, reduce = 0% -2013-09-18 01:56:22,731 Stage-1 map = 38%, reduce = 0% -2013-09-18 01:56:24,752 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 22.96 sec -2013-09-18 01:56:25,760 Stage-1 map = 58%, reduce = 0%, Cumulative CPU 22.96 sec -2013-09-18 01:56:26,767 Stage-1 map = 58%, reduce = 0%, Cumulative CPU 22.96 sec -2013-09-18 01:56:28,182 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 44.33 sec -2013-09-18 01:56:29,188 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 44.33 sec -2013-09-18 01:56:30,404 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 90.25 sec -2013-09-18 01:56:31,813 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 335.9 sec -2013-09-18 01:56:32,821 Stage-1 map = 98%, reduce = 0%, Cumulative CPU 404.3 sec -2013-09-18 01:56:33,827 Stage-1 map = 100%, reduce = 2%, Cumulative CPU 427.67 sec -2013-09-18 01:56:34,833 Stage-1 map = 100%, reduce = 11%, Cumulative CPU 427.67 sec -2013-09-18 01:56:35,840 Stage-1 map = 100%, reduce = 14%, Cumulative CPU 427.67 sec -2013-09-18 01:56:36,846 Stage-1 map = 100%, reduce = 23%, Cumulative CPU 427.67 sec -2013-09-18 01:56:37,854 Stage-1 map = 100%, reduce = 41%, Cumulative CPU 436.66 sec -2013-09-18 01:56:38,862 Stage-1 map = 100%, reduce = 47%, Cumulative CPU 436.66 sec -2013-09-18 01:56:39,869 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 446.83 sec -2013-09-18 01:56:40,875 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 454.28 sec -2013-09-18 01:56:41,881 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 463.94 sec -2013-09-18 01:56:42,887 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 463.94 sec -MapReduce Total cumulative CPU time: 7 minutes 43 seconds 940 msec -Ended Job = job_201309172235_0169 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0170 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 01:56:46,407 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:56:48,416 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.68 sec -2013-09-18 01:56:49,421 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.68 sec -2013-09-18 01:56:50,426 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.68 sec -2013-09-18 01:56:51,430 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.68 sec -2013-09-18 01:56:52,435 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.68 sec -2013-09-18 01:56:53,439 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.68 sec -2013-09-18 01:56:54,444 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.68 sec -2013-09-18 01:56:55,450 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -2013-09-18 01:56:56,455 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -2013-09-18 01:56:57,460 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -MapReduce Total cumulative CPU time: 2 seconds 140 msec -Ended Job = job_201309172235_0170 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 463.94 sec HDFS Read: 993495605 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.14 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 46 seconds 80 msec -OK -Time taken: 70.292 seconds -hive> quit; - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29743@mturlrep13_201309180156_208621189.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0171 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:57:08,252 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:57:17,362 Stage-1 map = 1%, reduce = 0% -2013-09-18 01:57:18,402 Stage-1 map = 8%, reduce = 0% -2013-09-18 01:57:19,415 Stage-1 map = 9%, reduce = 0% -2013-09-18 01:57:20,427 Stage-1 map = 13%, reduce = 0% -2013-09-18 01:57:21,432 Stage-1 map = 36%, reduce = 0% -2013-09-18 01:57:22,447 Stage-1 map = 40%, reduce = 0%, Cumulative CPU 8.67 sec -2013-09-18 01:57:23,455 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 8.67 sec -2013-09-18 01:57:24,469 Stage-1 map = 52%, reduce = 0%, Cumulative CPU 8.67 sec -2013-09-18 01:57:25,476 Stage-1 map = 64%, reduce = 0%, Cumulative CPU 33.09 sec -2013-09-18 01:57:26,484 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 276.39 sec -2013-09-18 01:57:27,491 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 454.8 sec -2013-09-18 01:57:28,497 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 454.8 sec -2013-09-18 01:57:29,504 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 454.8 sec -2013-09-18 01:57:30,510 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 454.8 sec -2013-09-18 01:57:31,516 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 454.8 sec -2013-09-18 01:57:32,522 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 454.8 sec -2013-09-18 01:57:33,529 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 454.8 sec -2013-09-18 01:57:34,536 Stage-1 map = 100%, reduce = 18%, Cumulative CPU 454.8 sec -2013-09-18 01:57:35,549 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 469.58 sec -2013-09-18 01:57:36,556 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 487.12 sec -2013-09-18 01:57:37,564 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 493.48 sec -2013-09-18 01:57:38,570 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 493.48 sec -2013-09-18 01:57:39,576 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 493.48 sec -MapReduce Total cumulative CPU time: 8 minutes 13 seconds 480 msec -Ended Job = job_201309172235_0171 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0172 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 01:57:43,495 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:57:44,501 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 01:57:45,506 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 01:57:46,511 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 01:57:47,516 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 01:57:48,520 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 01:57:49,524 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 01:57:50,529 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 01:57:51,533 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 01:57:52,538 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -2013-09-18 01:57:53,544 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -2013-09-18 01:57:54,549 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -MapReduce Total cumulative CPU time: 2 seconds 250 msec -Ended Job = job_201309172235_0172 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 493.48 sec HDFS Read: 993495605 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.25 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 15 seconds 730 msec -OK -Time taken: 55.211 seconds -hive> quit; - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4778@mturlrep13_201309180157_2071772922.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0173 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:58:05,573 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:58:14,649 Stage-1 map = 1%, reduce = 0% -2013-09-18 01:58:15,664 Stage-1 map = 9%, reduce = 0% -2013-09-18 01:58:17,723 Stage-1 map = 12%, reduce = 0% -2013-09-18 01:58:18,734 Stage-1 map = 36%, reduce = 0% -2013-09-18 01:58:19,756 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 8.38 sec -2013-09-18 01:58:20,765 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 8.38 sec -2013-09-18 01:58:21,773 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 8.38 sec -2013-09-18 01:58:22,797 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 32.26 sec -2013-09-18 01:58:23,804 Stage-1 map = 87%, reduce = 0%, Cumulative CPU 254.83 sec -2013-09-18 01:58:24,812 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 460.38 sec -2013-09-18 01:58:25,818 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 460.38 sec -2013-09-18 01:58:26,825 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 460.38 sec -2013-09-18 01:58:27,831 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 460.38 sec -2013-09-18 01:58:28,836 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 460.38 sec -2013-09-18 01:58:29,842 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 460.38 sec -2013-09-18 01:58:30,847 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 460.38 sec -2013-09-18 01:58:31,854 Stage-1 map = 100%, reduce = 15%, Cumulative CPU 460.38 sec -2013-09-18 01:58:32,864 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 474.21 sec -2013-09-18 01:58:33,870 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 487.03 sec -2013-09-18 01:58:34,876 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 496.96 sec -2013-09-18 01:58:35,882 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 496.96 sec -2013-09-18 01:58:36,888 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 496.96 sec -MapReduce Total cumulative CPU time: 8 minutes 16 seconds 960 msec -Ended Job = job_201309172235_0173 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0174 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 01:58:40,381 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:58:41,387 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 01:58:42,393 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 01:58:43,398 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 01:58:44,403 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 01:58:45,408 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 01:58:46,413 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 01:58:47,418 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 01:58:48,423 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 01:58:49,429 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -2013-09-18 01:58:50,435 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -2013-09-18 01:58:51,440 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -MapReduce Total cumulative CPU time: 2 seconds 200 msec -Ended Job = job_201309172235_0174 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 496.96 sec HDFS Read: 993495605 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.2 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 19 seconds 160 msec -OK -Time taken: 55.076 seconds -hive> quit; - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12570@mturlrep13_201309180158_2097047961.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0175 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 01:59:09,511 Stage-1 map = 0%, reduce = 0% -2013-09-18 01:59:19,576 Stage-1 map = 5%, reduce = 0% -2013-09-18 01:59:22,593 Stage-1 map = 15%, reduce = 0% -2013-09-18 01:59:25,607 Stage-1 map = 29%, reduce = 0% -2013-09-18 01:59:26,612 Stage-1 map = 30%, reduce = 0% -2013-09-18 01:59:28,833 Stage-1 map = 42%, reduce = 0% -2013-09-18 01:59:34,330 Stage-1 map = 67%, reduce = 0%, Cumulative CPU 122.09 sec -2013-09-18 01:59:35,337 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 176.65 sec -2013-09-18 01:59:36,349 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 230.21 sec -2013-09-18 01:59:37,356 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 395.83 sec -2013-09-18 01:59:38,362 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 407.07 sec -2013-09-18 01:59:39,368 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 407.07 sec -2013-09-18 01:59:40,374 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 407.07 sec -2013-09-18 01:59:41,380 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 407.07 sec -2013-09-18 01:59:42,386 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 407.07 sec -2013-09-18 01:59:43,395 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 413.7 sec -2013-09-18 01:59:44,403 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 423.63 sec -2013-09-18 01:59:45,410 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 434.03 sec -2013-09-18 01:59:46,417 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 443.68 sec -2013-09-18 01:59:47,424 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 443.68 sec -MapReduce Total cumulative CPU time: 7 minutes 23 seconds 680 msec -Ended Job = job_201309172235_0175 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0176 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 01:59:50,937 Stage-2 map = 0%, reduce = 0% -2013-09-18 01:59:52,947 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 01:59:53,953 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 01:59:54,958 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 01:59:55,963 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 01:59:56,969 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 01:59:57,974 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 01:59:58,981 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 01:59:59,987 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.72 sec -2013-09-18 02:00:00,993 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -2013-09-18 02:00:01,999 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -MapReduce Total cumulative CPU time: 2 seconds 250 msec -Ended Job = job_201309172235_0176 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 443.68 sec HDFS Read: 876843443 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.25 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 25 seconds 930 msec -OK -Time taken: 63.4 seconds -hive> quit; - -times: 2 -query: SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19761@mturlrep13_201309180200_1379051620.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0177 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:00:13,526 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:00:23,659 Stage-1 map = 11%, reduce = 0% -2013-09-18 02:00:26,730 Stage-1 map = 48%, reduce = 0% -2013-09-18 02:00:28,741 Stage-1 map = 50%, reduce = 0% -2013-09-18 02:00:29,755 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 48.44 sec -2013-09-18 02:00:30,772 Stage-1 map = 77%, reduce = 0%, Cumulative CPU 100.32 sec -2013-09-18 02:00:31,784 Stage-1 map = 91%, reduce = 0%, Cumulative CPU 281.43 sec -2013-09-18 02:00:32,792 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 435.17 sec -2013-09-18 02:00:33,799 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 435.17 sec -2013-09-18 02:00:34,806 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 435.17 sec -2013-09-18 02:00:35,812 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 435.17 sec -2013-09-18 02:00:36,818 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 435.17 sec -2013-09-18 02:00:37,824 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 435.17 sec -2013-09-18 02:00:38,831 Stage-1 map = 100%, reduce = 15%, Cumulative CPU 435.17 sec -2013-09-18 02:00:39,843 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 448.52 sec -2013-09-18 02:00:40,850 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 470.07 sec -2013-09-18 02:00:41,857 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 473.43 sec -2013-09-18 02:00:42,864 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 473.43 sec -MapReduce Total cumulative CPU time: 7 minutes 53 seconds 430 msec -Ended Job = job_201309172235_0177 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0178 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:00:45,348 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:00:47,357 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:00:48,363 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:00:49,367 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:00:50,372 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:00:51,376 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:00:52,381 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:00:53,385 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:00:54,390 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.71 sec -2013-09-18 02:00:55,396 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -2013-09-18 02:00:56,403 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -MapReduce Total cumulative CPU time: 2 seconds 190 msec -Ended Job = job_201309172235_0178 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 473.43 sec HDFS Read: 876843443 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.19 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 55 seconds 620 msec -OK -Time taken: 52.093 seconds -hive> quit; - -times: 3 -query: SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27543@mturlrep13_201309180200_2073175542.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0179 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:01:07,075 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:01:16,231 Stage-1 map = 4%, reduce = 0% -2013-09-18 02:01:17,250 Stage-1 map = 10%, reduce = 0% -2013-09-18 02:01:19,265 Stage-1 map = 22%, reduce = 0% -2013-09-18 02:01:20,281 Stage-1 map = 48%, reduce = 0% -2013-09-18 02:01:21,287 Stage-1 map = 50%, reduce = 0% -2013-09-18 02:01:22,302 Stage-1 map = 54%, reduce = 0%, Cumulative CPU 28.59 sec -2013-09-18 02:01:23,344 Stage-1 map = 69%, reduce = 0%, Cumulative CPU 58.45 sec -2013-09-18 02:01:24,364 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 92.94 sec -2013-09-18 02:01:25,375 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 322.31 sec -2013-09-18 02:01:26,382 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 437.99 sec -2013-09-18 02:01:27,387 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 437.99 sec -2013-09-18 02:01:28,392 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 437.99 sec -2013-09-18 02:01:29,398 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 437.99 sec -2013-09-18 02:01:30,404 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 437.99 sec -2013-09-18 02:01:31,410 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 437.99 sec -2013-09-18 02:01:32,417 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 437.99 sec -2013-09-18 02:01:33,427 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 448.33 sec -2013-09-18 02:01:34,433 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 477.42 sec -2013-09-18 02:01:35,440 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 477.42 sec -2013-09-18 02:01:36,447 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 477.42 sec -MapReduce Total cumulative CPU time: 7 minutes 57 seconds 420 msec -Ended Job = job_201309172235_0179 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0180 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:01:39,908 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:01:40,913 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:01:41,919 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:01:42,924 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:01:43,928 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:01:44,933 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:01:46,032 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:01:47,037 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:01:48,042 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.79 sec -2013-09-18 02:01:49,048 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-18 02:01:50,053 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-18 02:01:51,058 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -MapReduce Total cumulative CPU time: 2 seconds 210 msec -Ended Job = job_201309172235_0180 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 477.42 sec HDFS Read: 876843443 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.21 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 59 seconds 630 msec -OK -Time taken: 52.895 seconds -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2326@mturlrep13_201309180201_303775162.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0181 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:02:08,645 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:02:18,769 Stage-1 map = 1%, reduce = 0% -2013-09-18 02:02:22,796 Stage-1 map = 5%, reduce = 0% -2013-09-18 02:02:23,803 Stage-1 map = 6%, reduce = 0% -2013-09-18 02:02:27,959 Stage-1 map = 24%, reduce = 0% -2013-09-18 02:02:32,561 Stage-1 map = 44%, reduce = 0% -2013-09-18 02:02:33,569 Stage-1 map = 54%, reduce = 0% -2013-09-18 02:02:34,585 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 56.79 sec -2013-09-18 02:02:35,594 Stage-1 map = 62%, reduce = 0%, Cumulative CPU 80.27 sec -2013-09-18 02:02:38,969 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 148.88 sec -2013-09-18 02:02:39,974 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 219.04 sec -2013-09-18 02:02:40,981 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 315.44 sec -2013-09-18 02:02:41,987 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 427.42 sec -2013-09-18 02:02:42,993 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 438.91 sec -2013-09-18 02:02:44,000 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 438.91 sec -2013-09-18 02:02:45,006 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 438.91 sec -2013-09-18 02:02:46,014 Stage-1 map = 100%, reduce = 45%, Cumulative CPU 454.78 sec -2013-09-18 02:02:47,020 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 454.78 sec -2013-09-18 02:02:48,027 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 465.13 sec -2013-09-18 02:02:49,033 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 474.77 sec -2013-09-18 02:02:50,040 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 474.77 sec -2013-09-18 02:02:51,046 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 474.77 sec -MapReduce Total cumulative CPU time: 7 minutes 54 seconds 770 msec -Ended Job = job_201309172235_0181 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0182 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:02:54,504 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:02:56,511 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:02:57,516 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:02:58,520 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:02:59,524 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:00,529 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:01,533 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:02,538 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:03,543 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -2013-09-18 02:03:04,547 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -2013-09-18 02:03:05,553 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -MapReduce Total cumulative CPU time: 2 seconds 200 msec -Ended Job = job_201309172235_0182 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 474.77 sec HDFS Read: 988828092 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.2 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 56 seconds 970 msec -OK -Time taken: 67.483 seconds -hive> quit; - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10101@mturlrep13_201309180203_1414891537.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0183 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:03:15,331 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:03:25,419 Stage-1 map = 2%, reduce = 0% -2013-09-18 02:03:26,435 Stage-1 map = 10%, reduce = 0% -2013-09-18 02:03:28,457 Stage-1 map = 14%, reduce = 0% -2013-09-18 02:03:29,465 Stage-1 map = 37%, reduce = 0% -2013-09-18 02:03:30,480 Stage-1 map = 41%, reduce = 0%, Cumulative CPU 8.3 sec -2013-09-18 02:03:31,488 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 8.3 sec -2013-09-18 02:03:32,496 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 19.5 sec -2013-09-18 02:03:33,503 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 31.5 sec -2013-09-18 02:03:34,511 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 192.2 sec -2013-09-18 02:03:35,518 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 435.25 sec -2013-09-18 02:03:36,525 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 463.51 sec -2013-09-18 02:03:37,531 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 463.51 sec -2013-09-18 02:03:38,536 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 463.51 sec -2013-09-18 02:03:39,542 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 463.51 sec -2013-09-18 02:03:40,548 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 463.51 sec -2013-09-18 02:03:41,554 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 463.51 sec -2013-09-18 02:03:42,560 Stage-1 map = 100%, reduce = 24%, Cumulative CPU 463.51 sec -2013-09-18 02:03:43,568 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 487.09 sec -2013-09-18 02:03:44,575 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 500.52 sec -2013-09-18 02:03:45,581 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 500.52 sec -2013-09-18 02:03:46,587 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 500.52 sec -MapReduce Total cumulative CPU time: 8 minutes 20 seconds 520 msec -Ended Job = job_201309172235_0183 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0184 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:03:49,037 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:03:51,046 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:52,051 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:53,056 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:54,061 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:55,066 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:56,071 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:57,076 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:58,080 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 02:03:59,086 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.34 sec -2013-09-18 02:04:00,091 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.34 sec -2013-09-18 02:04:01,096 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.34 sec -MapReduce Total cumulative CPU time: 2 seconds 340 msec -Ended Job = job_201309172235_0184 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 500.52 sec HDFS Read: 988828092 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.34 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 22 seconds 860 msec -OK -Time taken: 53.704 seconds -hive> quit; - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17498@mturlrep13_201309180204_309695720.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0185 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:04:11,776 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:04:20,938 Stage-1 map = 3%, reduce = 0% -2013-09-18 02:04:21,952 Stage-1 map = 11%, reduce = 0% -2013-09-18 02:04:23,979 Stage-1 map = 13%, reduce = 0% -2013-09-18 02:04:24,984 Stage-1 map = 37%, reduce = 0% -2013-09-18 02:04:25,990 Stage-1 map = 44%, reduce = 0% -2013-09-18 02:04:28,019 Stage-1 map = 53%, reduce = 0%, Cumulative CPU 8.73 sec -2013-09-18 02:04:29,048 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 43.13 sec -2013-09-18 02:04:30,071 Stage-1 map = 84%, reduce = 0%, Cumulative CPU 227.84 sec -2013-09-18 02:04:31,079 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 459.59 sec -2013-09-18 02:04:32,089 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 459.59 sec -2013-09-18 02:04:33,095 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 459.59 sec -2013-09-18 02:04:34,101 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 459.59 sec -2013-09-18 02:04:35,106 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 459.59 sec -2013-09-18 02:04:36,112 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 459.59 sec -2013-09-18 02:04:37,119 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 459.59 sec -2013-09-18 02:04:38,128 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 462.59 sec -2013-09-18 02:04:39,135 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 476.43 sec -2013-09-18 02:04:40,142 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 490.11 sec -2013-09-18 02:04:41,149 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 496.41 sec -2013-09-18 02:04:42,156 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 496.41 sec -2013-09-18 02:04:43,162 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 496.41 sec -MapReduce Total cumulative CPU time: 8 minutes 16 seconds 410 msec -Ended Job = job_201309172235_0185 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0186 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:04:45,666 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:04:47,674 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:04:48,680 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:04:49,685 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:04:50,689 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:04:51,694 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:04:52,699 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:04:53,703 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:04:54,707 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.7 sec -2013-09-18 02:04:55,712 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.15 sec -2013-09-18 02:04:56,718 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.15 sec -2013-09-18 02:04:57,723 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.15 sec -MapReduce Total cumulative CPU time: 2 seconds 150 msec -Ended Job = job_201309172235_0186 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 496.41 sec HDFS Read: 988828092 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.15 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 8 minutes 18 seconds 560 msec -OK -Time taken: 54.814 seconds -hive> quit; - -times: 1 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24933@mturlrep13_201309180205_1377920242.txt -hive> SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; ; -FAILED: SemanticException [Error 10128]: Line 1:415 Not yet supported place for UDAF 'count' -hive> quit; - -times: 2 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25460@mturlrep13_201309180205_1132498226.txt -hive> SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; ; -FAILED: SemanticException [Error 10128]: Line 1:415 Not yet supported place for UDAF 'count' -hive> quit; - -times: 3 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26035@mturlrep13_201309180205_978107887.txt -hive> SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; ; -FAILED: SemanticException [Error 10128]: Line 1:415 Not yet supported place for UDAF 'count' -hive> quit; - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26250@mturlrep13_201309180205_637154344.txt -hive> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0187 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:05:41,227 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:05:51,363 Stage-1 map = 5%, reduce = 0% -2013-09-18 02:05:54,380 Stage-1 map = 16%, reduce = 0% -2013-09-18 02:05:57,395 Stage-1 map = 27%, reduce = 0% -2013-09-18 02:05:58,400 Stage-1 map = 30%, reduce = 0% -2013-09-18 02:06:00,410 Stage-1 map = 42%, reduce = 0% -2013-09-18 02:06:01,424 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 17.15 sec -2013-09-18 02:06:02,445 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 262.15 sec -2013-09-18 02:06:04,340 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 262.15 sec -2013-09-18 02:06:05,365 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 262.15 sec -2013-09-18 02:06:06,379 Stage-1 map = 68%, reduce = 0%, Cumulative CPU 285.59 sec -2013-09-18 02:06:07,390 Stage-1 map = 71%, reduce = 0%, Cumulative CPU 291.15 sec -2013-09-18 02:06:08,397 Stage-1 map = 90%, reduce = 0%, Cumulative CPU 320.44 sec -2013-09-18 02:06:09,404 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 364.61 sec -2013-09-18 02:06:10,410 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 373.66 sec -2013-09-18 02:06:11,417 Stage-1 map = 100%, reduce = 3%, Cumulative CPU 373.66 sec -2013-09-18 02:06:12,424 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 373.66 sec -2013-09-18 02:06:13,433 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 379.8 sec -2013-09-18 02:06:14,439 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 396.64 sec -2013-09-18 02:06:15,446 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 407.2 sec -2013-09-18 02:06:16,453 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 407.2 sec -2013-09-18 02:06:17,460 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 410.44 sec -2013-09-18 02:06:18,468 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 410.44 sec -MapReduce Total cumulative CPU time: 6 minutes 50 seconds 440 msec -Ended Job = job_201309172235_0187 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0188 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:06:21,966 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:06:22,972 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:06:23,978 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:06:24,984 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:06:25,990 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:06:26,994 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:06:27,999 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:06:29,004 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:06:30,008 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:06:31,015 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.41 sec -2013-09-18 02:06:32,021 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.41 sec -2013-09-18 02:06:33,027 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.41 sec -MapReduce Total cumulative CPU time: 2 seconds 410 msec -Ended Job = job_201309172235_0188 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 410.44 sec HDFS Read: 1326437326 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.41 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 6 minutes 52 seconds 850 msec -OK -Time taken: 62.644 seconds -hive> quit; - -times: 2 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1255@mturlrep13_201309180206_866108725.txt -hive> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0189 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:06:42,810 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:06:53,890 Stage-1 map = 1%, reduce = 0% -2013-09-18 02:06:54,899 Stage-1 map = 3%, reduce = 0% -2013-09-18 02:06:55,912 Stage-1 map = 12%, reduce = 0% -2013-09-18 02:06:56,943 Stage-1 map = 19%, reduce = 0% -2013-09-18 02:06:57,951 Stage-1 map = 20%, reduce = 0% -2013-09-18 02:06:58,970 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 9.16 sec -2013-09-18 02:07:00,000 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 83.33 sec -2013-09-18 02:07:01,028 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 218.77 sec -2013-09-18 02:07:02,042 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 399.52 sec -2013-09-18 02:07:03,049 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 425.85 sec -2013-09-18 02:07:04,055 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 425.85 sec -2013-09-18 02:07:05,061 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 425.85 sec -2013-09-18 02:07:06,067 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 425.85 sec -2013-09-18 02:07:07,073 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 425.85 sec -2013-09-18 02:07:08,080 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 425.85 sec -2013-09-18 02:07:09,086 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 425.85 sec -2013-09-18 02:07:10,096 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 446.81 sec -2013-09-18 02:07:11,102 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 459.98 sec -2013-09-18 02:07:12,109 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 463.17 sec -2013-09-18 02:07:13,117 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 463.17 sec -MapReduce Total cumulative CPU time: 7 minutes 43 seconds 170 msec -Ended Job = job_201309172235_0189 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0190 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:07:15,611 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:07:17,620 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:07:18,626 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:07:19,631 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:07:20,636 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:07:21,641 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:07:22,650 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:07:23,655 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:07:24,660 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.73 sec -2013-09-18 02:07:25,666 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.32 sec -2013-09-18 02:07:26,675 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.32 sec -MapReduce Total cumulative CPU time: 2 seconds 320 msec -Ended Job = job_201309172235_0190 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 463.17 sec HDFS Read: 1326437326 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.32 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 45 seconds 490 msec -OK -Time taken: 51.812 seconds -hive> quit; - -times: 3 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9097@mturlrep13_201309180207_752390467.txt -hive> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0191 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:07:36,443 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:07:48,547 Stage-1 map = 3%, reduce = 0% -2013-09-18 02:07:49,583 Stage-1 map = 13%, reduce = 0% -2013-09-18 02:07:50,619 Stage-1 map = 15%, reduce = 0% -2013-09-18 02:07:51,643 Stage-1 map = 22%, reduce = 0% -2013-09-18 02:07:52,690 Stage-1 map = 54%, reduce = 0%, Cumulative CPU 9.35 sec -2013-09-18 02:07:53,721 Stage-1 map = 62%, reduce = 0%, Cumulative CPU 50.99 sec -2013-09-18 02:07:54,737 Stage-1 map = 81%, reduce = 0%, Cumulative CPU 242.49 sec -2013-09-18 02:07:55,745 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 412.77 sec -2013-09-18 02:07:56,751 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 426.23 sec -2013-09-18 02:07:57,758 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 426.23 sec -2013-09-18 02:07:58,763 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 426.23 sec -2013-09-18 02:07:59,770 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 426.23 sec -2013-09-18 02:08:00,776 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 426.23 sec -2013-09-18 02:08:01,781 Stage-1 map = 100%, reduce = 9%, Cumulative CPU 426.23 sec -2013-09-18 02:08:02,791 Stage-1 map = 100%, reduce = 27%, Cumulative CPU 433.68 sec -2013-09-18 02:08:03,799 Stage-1 map = 100%, reduce = 61%, Cumulative CPU 444.3 sec -2013-09-18 02:08:04,806 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 460.63 sec -2013-09-18 02:08:05,812 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 464.02 sec -2013-09-18 02:08:06,819 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 464.02 sec -MapReduce Total cumulative CPU time: 7 minutes 44 seconds 20 msec -Ended Job = job_201309172235_0191 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0192 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:08:09,707 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:08:11,717 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:08:12,723 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:08:13,729 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:08:14,734 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:08:15,740 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:08:16,744 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:08:17,749 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:08:18,754 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.74 sec -2013-09-18 02:08:19,760 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -2013-09-18 02:08:20,766 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -2013-09-18 02:08:21,771 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -MapReduce Total cumulative CPU time: 2 seconds 290 msec -Ended Job = job_201309172235_0192 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 464.02 sec HDFS Read: 1326437326 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.29 sec HDFS Read: 4622 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 46 seconds 310 msec -OK -Time taken: 53.23 seconds -hive> quit; - -times: 1 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16494@mturlrep13_201309180208_1768463196.txt -hive> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0193 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:08:39,476 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:08:52,587 Stage-1 map = 2%, reduce = 0% -2013-09-18 02:08:53,593 Stage-1 map = 3%, reduce = 0% -2013-09-18 02:08:58,633 Stage-1 map = 18%, reduce = 0% -2013-09-18 02:08:59,640 Stage-1 map = 22%, reduce = 0% -2013-09-18 02:09:01,649 Stage-1 map = 37%, reduce = 0% -2013-09-18 02:09:02,654 Stage-1 map = 38%, reduce = 0% -2013-09-18 02:09:04,909 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 242.14 sec -2013-09-18 02:09:05,923 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 243.18 sec -2013-09-18 02:09:06,929 Stage-1 map = 49%, reduce = 0%, Cumulative CPU 243.18 sec -2013-09-18 02:09:07,941 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 244.77 sec -2013-09-18 02:09:08,962 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 244.77 sec -2013-09-18 02:09:10,398 Stage-1 map = 66%, reduce = 0%, Cumulative CPU 250.0 sec -2013-09-18 02:09:11,404 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 278.26 sec -2013-09-18 02:09:12,413 Stage-1 map = 76%, reduce = 0%, Cumulative CPU 278.26 sec -2013-09-18 02:09:13,428 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 289.83 sec -2013-09-18 02:09:14,435 Stage-1 map = 97%, reduce = 1%, Cumulative CPU 362.43 sec -2013-09-18 02:09:15,442 Stage-1 map = 100%, reduce = 1%, Cumulative CPU 399.93 sec -2013-09-18 02:09:16,449 Stage-1 map = 100%, reduce = 4%, Cumulative CPU 399.93 sec -2013-09-18 02:09:17,455 Stage-1 map = 100%, reduce = 15%, Cumulative CPU 399.93 sec -2013-09-18 02:09:18,463 Stage-1 map = 100%, reduce = 45%, Cumulative CPU 412.86 sec -2013-09-18 02:09:19,469 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 419.48 sec -2013-09-18 02:09:20,476 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 419.48 sec -2013-09-18 02:09:21,483 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 434.29 sec -2013-09-18 02:09:22,490 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 437.36 sec -2013-09-18 02:09:23,496 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 437.36 sec -MapReduce Total cumulative CPU time: 7 minutes 17 seconds 360 msec -Ended Job = job_201309172235_0193 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0194 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:09:27,006 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:09:28,011 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:09:29,015 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:09:30,021 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:09:31,024 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:09:32,028 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:09:33,032 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:09:34,037 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:09:35,041 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.7 sec -2013-09-18 02:09:36,047 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-18 02:09:37,052 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-18 02:09:38,057 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -MapReduce Total cumulative CPU time: 2 seconds 210 msec -Ended Job = job_201309172235_0194 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 437.36 sec HDFS Read: 1004716597 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.21 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 19 seconds 570 msec -OK -Time taken: 69.068 seconds -hive> quit; - -times: 2 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24195@mturlrep13_201309180209_485949400.txt -hive> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0195 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:09:47,707 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:09:58,784 Stage-1 map = 1%, reduce = 0% -2013-09-18 02:09:59,803 Stage-1 map = 8%, reduce = 0% -2013-09-18 02:10:00,817 Stage-1 map = 13%, reduce = 0% -2013-09-18 02:10:01,836 Stage-1 map = 18%, reduce = 0% -2013-09-18 02:10:02,908 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 279.26 sec -2013-09-18 02:10:03,925 Stage-1 map = 55%, reduce = 0%, Cumulative CPU 286.13 sec -2013-09-18 02:10:04,949 Stage-1 map = 62%, reduce = 0%, Cumulative CPU 292.13 sec -2013-09-18 02:10:05,955 Stage-1 map = 83%, reduce = 0%, Cumulative CPU 359.01 sec -2013-09-18 02:10:06,963 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 409.34 sec -2013-09-18 02:10:07,970 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 422.22 sec -2013-09-18 02:10:08,976 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 422.22 sec -2013-09-18 02:10:09,981 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 422.22 sec -2013-09-18 02:10:10,987 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 422.22 sec -2013-09-18 02:10:11,993 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 422.22 sec -2013-09-18 02:10:13,002 Stage-1 map = 100%, reduce = 12%, Cumulative CPU 422.22 sec -2013-09-18 02:10:14,012 Stage-1 map = 100%, reduce = 30%, Cumulative CPU 425.38 sec -2013-09-18 02:10:15,028 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 448.26 sec -2013-09-18 02:10:16,035 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 462.58 sec -2013-09-18 02:10:17,041 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 462.58 sec -2013-09-18 02:10:18,047 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 462.58 sec -MapReduce Total cumulative CPU time: 7 minutes 42 seconds 580 msec -Ended Job = job_201309172235_0195 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0196 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:10:22,087 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:10:24,096 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:10:25,101 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:10:26,106 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:10:27,111 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:10:28,116 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:10:29,121 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:10:30,126 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:10:31,131 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -2013-09-18 02:10:32,137 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -2013-09-18 02:10:33,142 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -MapReduce Total cumulative CPU time: 2 seconds 250 msec -Ended Job = job_201309172235_0196 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 462.58 sec HDFS Read: 1004716597 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.25 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 44 seconds 830 msec -OK -Time taken: 53.234 seconds -hive> quit; - -times: 3 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32545@mturlrep13_201309180210_1142487279.txt -hive> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 11 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0197 -Hadoop job information for Stage-1: number of mappers: 37; number of reducers: 11 -2013-09-18 02:10:43,786 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:10:53,999 Stage-1 map = 1%, reduce = 0% -2013-09-18 02:10:55,007 Stage-1 map = 4%, reduce = 0% -2013-09-18 02:10:56,013 Stage-1 map = 11%, reduce = 0% -2013-09-18 02:10:57,019 Stage-1 map = 17%, reduce = 0% -2013-09-18 02:10:58,030 Stage-1 map = 28%, reduce = 0% -2013-09-18 02:10:59,047 Stage-1 map = 48%, reduce = 0% -2013-09-18 02:11:00,066 Stage-1 map = 61%, reduce = 0%, Cumulative CPU 51.25 sec -2013-09-18 02:11:01,076 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 185.97 sec -2013-09-18 02:11:02,087 Stage-1 map = 99%, reduce = 0%, Cumulative CPU 391.71 sec -2013-09-18 02:11:03,100 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 432.05 sec -2013-09-18 02:11:04,107 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 432.05 sec -2013-09-18 02:11:05,113 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 432.05 sec -2013-09-18 02:11:06,118 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 432.05 sec -2013-09-18 02:11:07,124 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 432.05 sec -2013-09-18 02:11:08,130 Stage-1 map = 100%, reduce = 6%, Cumulative CPU 432.05 sec -2013-09-18 02:11:09,140 Stage-1 map = 100%, reduce = 36%, Cumulative CPU 438.09 sec -2013-09-18 02:11:10,147 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 452.13 sec -2013-09-18 02:11:11,154 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 466.84 sec -2013-09-18 02:11:12,160 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 470.1 sec -2013-09-18 02:11:13,166 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 470.1 sec -MapReduce Total cumulative CPU time: 7 minutes 50 seconds 100 msec -Ended Job = job_201309172235_0197 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0198 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:11:16,669 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:11:17,675 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:11:18,680 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:11:19,686 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:11:20,691 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:11:21,696 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:11:22,700 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:11:23,705 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:11:24,711 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-18 02:11:25,716 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.26 sec -2013-09-18 02:11:26,721 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.26 sec -2013-09-18 02:11:27,727 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.26 sec -MapReduce Total cumulative CPU time: 2 seconds 260 msec -Ended Job = job_201309172235_0198 -MapReduce Jobs Launched: -Job 0: Map: 37 Reduce: 11 Cumulative CPU: 470.1 sec HDFS Read: 1004716597 HDFS Write: 1056 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.26 sec HDFS Read: 4633 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 7 minutes 52 seconds 360 msec -OK -Time taken: 52.853 seconds -hive> quit; - -times: 1 -query: SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8277@mturlrep13_201309180211_1471622.txt -hive> SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; ; -hive> quit; - -times: 2 -query: SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8496@mturlrep13_201309180211_1815785181.txt -hive> SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; ; -hive> quit; - -times: 3 -query: SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8703@mturlrep13_201309180211_1151158687.txt -hive> SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_100m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; ; -hive> quit; -stop time: Ср. сент. 18 02:11:52 MSK 2013 diff --git a/benchmark/hive/log/log_10m/log_10m_ b/benchmark/hive/log/log_10m/log_10m_ deleted file mode 100644 index 691327c9a38..00000000000 --- a/benchmark/hive/log/log_10m/log_10m_ +++ /dev/null @@ -1,5506 +0,0 @@ -start time: Вт. сент. 10 19:59:13 MSK 2013 -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30031@mturlrep13_201309101959_1531819947.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30467@mturlrep13_201309101959_742405451.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0102 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 19:59:33,010 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:59:40,041 Stage-1 map = 7%, reduce = 0% -2013-09-10 19:59:46,067 Stage-1 map = 14%, reduce = 0% -2013-09-10 19:59:49,084 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 19:59:50,090 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 19:59:51,096 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 19:59:52,102 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 19:59:53,107 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 19:59:54,112 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 19:59:55,145 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 19:59:56,150 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 19:59:57,155 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 19:59:58,160 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 19:59:59,165 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 20:00:00,170 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 45.21 sec -2013-09-10 20:00:01,177 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 70.99 sec -2013-09-10 20:00:02,183 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 70.99 sec -2013-09-10 20:00:03,189 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 70.99 sec -2013-09-10 20:00:04,194 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 70.99 sec -2013-09-10 20:00:05,221 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 70.99 sec -2013-09-10 20:00:06,226 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 70.99 sec -2013-09-10 20:00:07,231 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 70.99 sec -2013-09-10 20:00:08,237 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:09,242 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:10,247 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:11,253 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:12,258 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:13,264 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:14,269 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:15,303 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:16,308 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:17,313 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:18,317 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:19,322 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:20,327 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:21,332 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:22,337 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:23,342 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:24,347 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:25,353 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:26,358 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 70.99 sec -2013-09-10 20:00:27,362 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 107.19 sec -2013-09-10 20:00:28,367 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 107.19 sec -2013-09-10 20:00:29,371 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 145.24 sec -2013-09-10 20:00:30,376 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 145.24 sec -2013-09-10 20:00:31,380 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 145.24 sec -2013-09-10 20:00:32,385 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 145.24 sec -2013-09-10 20:00:33,390 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 145.24 sec -2013-09-10 20:00:34,397 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.71 sec -2013-09-10 20:00:35,402 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.71 sec -MapReduce Total cumulative CPU time: 2 minutes 27 seconds 710 msec -Ended Job = job_201309101627_0102 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 147.71 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 27 seconds 710 msec -OK -10000000 -Time taken: 72.111 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32544@mturlrep13_201309102000_1646880380.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_518@mturlrep13_201309102000_754325810.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0103 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:00:48,452 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:00:56,488 Stage-1 map = 7%, reduce = 0% -2013-09-10 20:00:59,500 Stage-1 map = 14%, reduce = 0% -2013-09-10 20:01:05,526 Stage-1 map = 22%, reduce = 0% -2013-09-10 20:01:08,538 Stage-1 map = 29%, reduce = 0% -2013-09-10 20:01:11,550 Stage-1 map = 36%, reduce = 0% -2013-09-10 20:01:14,561 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:01:16,575 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 80.56 sec -2013-09-10 20:01:17,582 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 80.56 sec -2013-09-10 20:01:18,587 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 80.56 sec -2013-09-10 20:01:19,593 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 80.56 sec -2013-09-10 20:01:20,598 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 80.56 sec -2013-09-10 20:01:21,603 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 80.56 sec -2013-09-10 20:01:22,608 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 80.56 sec -2013-09-10 20:01:23,613 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 80.56 sec -2013-09-10 20:01:24,619 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:25,624 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:26,629 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:27,634 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:28,639 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:29,645 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:30,650 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:31,654 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:32,659 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:33,665 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:34,670 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:35,675 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:36,680 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:37,685 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:38,690 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:39,695 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 80.56 sec -2013-09-10 20:01:40,700 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 110.63 sec -2013-09-10 20:01:41,705 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 110.63 sec -2013-09-10 20:01:42,710 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 110.63 sec -2013-09-10 20:01:43,715 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 145.16 sec -2013-09-10 20:01:44,719 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 145.16 sec -2013-09-10 20:01:45,723 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 145.16 sec -2013-09-10 20:01:46,727 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 145.16 sec -2013-09-10 20:01:47,732 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 145.16 sec -2013-09-10 20:01:48,737 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 145.16 sec -2013-09-10 20:01:49,744 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.56 sec -2013-09-10 20:01:50,761 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.56 sec -2013-09-10 20:01:51,766 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.56 sec -MapReduce Total cumulative CPU time: 2 minutes 27 seconds 560 msec -Ended Job = job_201309101627_0103 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 147.56 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 27 seconds 560 msec -OK -10000000 -Time taken: 70.619 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2229@mturlrep13_201309102001_150357587.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2633@mturlrep13_201309102001_764470607.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0104 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:02:04,867 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:02:12,918 Stage-1 map = 7%, reduce = 0% -2013-09-10 20:02:15,932 Stage-1 map = 14%, reduce = 0% -2013-09-10 20:02:21,957 Stage-1 map = 22%, reduce = 0% -2013-09-10 20:02:24,969 Stage-1 map = 29%, reduce = 0% -2013-09-10 20:02:27,981 Stage-1 map = 36%, reduce = 0% -2013-09-10 20:02:30,995 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:02:32,005 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 34.44 sec -2013-09-10 20:02:33,013 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.6 sec -2013-09-10 20:02:34,019 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.6 sec -2013-09-10 20:02:35,032 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.6 sec -2013-09-10 20:02:36,044 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.6 sec -2013-09-10 20:02:37,049 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.6 sec -2013-09-10 20:02:38,055 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.6 sec -2013-09-10 20:02:39,060 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.6 sec -2013-09-10 20:02:40,065 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 76.6 sec -2013-09-10 20:02:41,071 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 76.6 sec -2013-09-10 20:02:42,076 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 76.6 sec -2013-09-10 20:02:43,082 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 76.6 sec -2013-09-10 20:02:44,087 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 76.6 sec -2013-09-10 20:02:45,092 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 76.6 sec -2013-09-10 20:02:46,112 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 76.6 sec -2013-09-10 20:02:47,116 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 76.6 sec -2013-09-10 20:02:48,131 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 76.6 sec -2013-09-10 20:02:49,136 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 76.6 sec -2013-09-10 20:02:50,142 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 122.35 sec -2013-09-10 20:02:51,147 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 122.35 sec -2013-09-10 20:02:52,151 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 122.35 sec -2013-09-10 20:02:53,156 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 122.35 sec -2013-09-10 20:02:54,161 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 122.35 sec -2013-09-10 20:02:55,167 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 122.35 sec -2013-09-10 20:02:56,195 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 122.35 sec -2013-09-10 20:02:57,200 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 122.35 sec -2013-09-10 20:02:58,205 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 137.05 sec -2013-09-10 20:02:59,210 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 137.05 sec -2013-09-10 20:03:00,215 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 149.43 sec -2013-09-10 20:03:01,220 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 149.43 sec -2013-09-10 20:03:02,224 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 149.43 sec -2013-09-10 20:03:03,228 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 149.43 sec -2013-09-10 20:03:04,232 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 149.43 sec -2013-09-10 20:03:05,239 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 151.13 sec -2013-09-10 20:03:06,246 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 151.13 sec -2013-09-10 20:03:07,252 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 151.13 sec -MapReduce Total cumulative CPU time: 2 minutes 31 seconds 130 msec -Ended Job = job_201309101627_0104 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 151.13 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 31 seconds 130 msec -OK -10000000 -Time taken: 69.712 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4005@mturlrep13_201309102003_1205264534.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4453@mturlrep13_201309102003_994011847.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0105 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:03:27,324 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:03:32,353 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.26 sec -2013-09-10 20:03:33,361 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.26 sec -2013-09-10 20:03:34,368 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.26 sec -2013-09-10 20:03:35,374 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.26 sec -2013-09-10 20:03:36,380 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.26 sec -2013-09-10 20:03:37,386 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 17.88 sec -2013-09-10 20:03:38,392 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.98 sec -2013-09-10 20:03:39,398 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.98 sec -2013-09-10 20:03:40,407 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.98 sec -2013-09-10 20:03:41,413 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.98 sec -2013-09-10 20:03:42,420 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.98 sec -MapReduce Total cumulative CPU time: 25 seconds 980 msec -Ended Job = job_201309101627_0105 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 25.98 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 25 seconds 980 msec -OK -171127 -Time taken: 24.908 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5972@mturlrep13_201309102003_1892761219.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6381@mturlrep13_201309102003_2036843139.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0106 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:03:55,419 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:04:00,446 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 20:04:01,454 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 20:04:02,462 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 20:04:03,469 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 20:04:04,475 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 20:04:05,481 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-10 20:04:06,487 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-10 20:04:07,493 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-10 20:04:08,501 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.71 sec -2013-09-10 20:04:09,508 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.71 sec -2013-09-10 20:04:10,515 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.71 sec -MapReduce Total cumulative CPU time: 25 seconds 710 msec -Ended Job = job_201309101627_0106 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 25.71 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 25 seconds 710 msec -OK -171127 -Time taken: 22.567 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7629@mturlrep13_201309102004_1501464055.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8043@mturlrep13_201309102004_2101165091.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0107 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:04:23,561 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:04:28,586 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.66 sec -2013-09-10 20:04:29,594 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.66 sec -2013-09-10 20:04:30,602 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.66 sec -2013-09-10 20:04:31,608 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.66 sec -2013-09-10 20:04:32,613 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.66 sec -2013-09-10 20:04:33,618 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.37 sec -2013-09-10 20:04:34,623 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.37 sec -2013-09-10 20:04:35,628 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.37 sec -2013-09-10 20:04:36,635 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.28 sec -2013-09-10 20:04:37,641 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.28 sec -2013-09-10 20:04:38,648 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.28 sec -MapReduce Total cumulative CPU time: 25 seconds 280 msec -Ended Job = job_201309101627_0107 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 25.28 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 25 seconds 280 msec -OK -171127 -Time taken: 22.452 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9266@mturlrep13_201309102004_1013038557.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9693@mturlrep13_201309102004_1030085538.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0108 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:04:59,466 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:05:06,500 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-10 20:05:07,507 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-10 20:05:08,515 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-10 20:05:09,520 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-10 20:05:10,526 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-10 20:05:11,531 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-10 20:05:12,538 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-10 20:05:13,544 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 23.47 sec -2013-09-10 20:05:14,550 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.35 sec -2013-09-10 20:05:15,554 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.35 sec -2013-09-10 20:05:16,559 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.35 sec -2013-09-10 20:05:17,564 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.35 sec -2013-09-10 20:05:18,569 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.35 sec -2013-09-10 20:05:19,576 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.46 sec -2013-09-10 20:05:20,581 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.46 sec -2013-09-10 20:05:21,587 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.46 sec -MapReduce Total cumulative CPU time: 33 seconds 460 msec -Ended Job = job_201309101627_0108 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 33.46 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 460 msec -OK -Time taken: 32.261 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11697@mturlrep13_201309102005_1481049011.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12122@mturlrep13_201309102005_1165569106.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0109 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:05:35,519 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:05:41,547 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.67 sec -2013-09-10 20:05:42,554 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.67 sec -2013-09-10 20:05:43,560 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.67 sec -2013-09-10 20:05:44,565 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.67 sec -2013-09-10 20:05:45,570 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.67 sec -2013-09-10 20:05:46,575 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.67 sec -2013-09-10 20:05:47,581 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.67 sec -2013-09-10 20:05:48,586 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.39 sec -2013-09-10 20:05:49,591 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.39 sec -2013-09-10 20:05:50,597 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.98 sec -2013-09-10 20:05:51,602 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.98 sec -2013-09-10 20:05:52,607 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.98 sec -2013-09-10 20:05:53,611 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.98 sec -2013-09-10 20:05:54,618 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.53 sec -2013-09-10 20:05:55,624 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.53 sec -2013-09-10 20:05:56,629 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.53 sec -MapReduce Total cumulative CPU time: 34 seconds 530 msec -Ended Job = job_201309101627_0109 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 34.53 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 34 seconds 530 msec -OK -Time taken: 29.475 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13449@mturlrep13_201309102005_1900832859.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13873@mturlrep13_201309102006_801912719.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0110 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:06:10,558 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:06:16,584 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.41 sec -2013-09-10 20:06:17,590 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.41 sec -2013-09-10 20:06:18,599 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.41 sec -2013-09-10 20:06:19,606 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.41 sec -2013-09-10 20:06:20,611 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.41 sec -2013-09-10 20:06:21,616 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.41 sec -2013-09-10 20:06:22,622 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.41 sec -2013-09-10 20:06:23,627 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.36 sec -2013-09-10 20:06:24,632 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 32.36 sec -2013-09-10 20:06:25,638 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 32.36 sec -2013-09-10 20:06:26,643 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 32.36 sec -2013-09-10 20:06:27,648 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 32.36 sec -2013-09-10 20:06:28,653 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 32.36 sec -2013-09-10 20:06:29,657 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 32.36 sec -2013-09-10 20:06:30,664 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.45 sec -2013-09-10 20:06:31,670 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.45 sec -MapReduce Total cumulative CPU time: 34 seconds 450 msec -Ended Job = job_201309101627_0110 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 34.45 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 34 seconds 450 msec -OK -Time taken: 29.445 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15232@mturlrep13_201309102006_1831985638.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15654@mturlrep13_201309102006_663118057.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0111 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:06:52,411 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:06:59,445 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.89 sec -2013-09-10 20:07:00,452 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.89 sec -2013-09-10 20:07:01,459 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.89 sec -2013-09-10 20:07:02,465 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.89 sec -2013-09-10 20:07:03,470 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.89 sec -2013-09-10 20:07:04,476 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.89 sec -2013-09-10 20:07:05,481 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 21.89 sec -2013-09-10 20:07:06,487 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 29.59 sec -2013-09-10 20:07:07,492 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 29.59 sec -2013-09-10 20:07:08,497 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 29.59 sec -2013-09-10 20:07:09,502 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 29.59 sec -2013-09-10 20:07:10,506 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 29.59 sec -2013-09-10 20:07:11,511 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 29.59 sec -2013-09-10 20:07:12,518 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.64 sec -2013-09-10 20:07:13,524 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.64 sec -MapReduce Total cumulative CPU time: 31 seconds 640 msec -Ended Job = job_201309101627_0111 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 31.64 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 31 seconds 640 msec -OK --4662894107982093709 -Time taken: 30.886 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16961@mturlrep13_201309102007_560489574.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17378@mturlrep13_201309102007_948891522.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0112 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:07:27,959 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:07:33,989 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.72 sec -2013-09-10 20:07:34,997 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.72 sec -2013-09-10 20:07:36,004 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.72 sec -2013-09-10 20:07:37,010 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.72 sec -2013-09-10 20:07:38,016 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.72 sec -2013-09-10 20:07:39,022 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.72 sec -2013-09-10 20:07:40,028 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.86 sec -2013-09-10 20:07:41,032 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 31.86 sec -2013-09-10 20:07:42,038 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 31.86 sec -2013-09-10 20:07:43,043 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 31.86 sec -2013-09-10 20:07:44,049 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 31.86 sec -2013-09-10 20:07:45,054 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 31.86 sec -2013-09-10 20:07:46,059 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 31.86 sec -2013-09-10 20:07:47,066 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.75 sec -2013-09-10 20:07:48,071 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.75 sec -MapReduce Total cumulative CPU time: 33 seconds 750 msec -Ended Job = job_201309101627_0112 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 33.75 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 750 msec -OK --4662894107982093709 -Time taken: 28.62 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18627@mturlrep13_201309102007_1570970698.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19048@mturlrep13_201309102007_265385188.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0113 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:08:01,232 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:08:07,261 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 7.12 sec -2013-09-10 20:08:08,269 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.41 sec -2013-09-10 20:08:09,277 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.41 sec -2013-09-10 20:08:10,282 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.41 sec -2013-09-10 20:08:11,288 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.41 sec -2013-09-10 20:08:12,294 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.41 sec -2013-09-10 20:08:13,300 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 21.76 sec -2013-09-10 20:08:14,304 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 29.62 sec -2013-09-10 20:08:15,309 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 29.62 sec -2013-09-10 20:08:16,315 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 29.62 sec -2013-09-10 20:08:17,320 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 29.62 sec -2013-09-10 20:08:18,325 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 29.62 sec -2013-09-10 20:08:19,329 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 29.62 sec -2013-09-10 20:08:20,334 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 29.62 sec -2013-09-10 20:08:21,341 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.69 sec -2013-09-10 20:08:22,347 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.69 sec -MapReduce Total cumulative CPU time: 31 seconds 690 msec -Ended Job = job_201309101627_0113 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 31.69 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 31 seconds 690 msec -OK --4662894107982093709 -Time taken: 28.461 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20298@mturlrep13_201309102008_320778143.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20705@mturlrep13_201309102008_862349350.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0114 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:08:42,650 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:08:49,678 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:08:52,698 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.9 sec -2013-09-10 20:08:53,705 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.9 sec -2013-09-10 20:08:54,712 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.9 sec -2013-09-10 20:08:55,718 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.9 sec -2013-09-10 20:08:56,723 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.9 sec -2013-09-10 20:08:57,729 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.9 sec -2013-09-10 20:08:58,734 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.9 sec -2013-09-10 20:08:59,739 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.9 sec -2013-09-10 20:09:00,744 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.9 sec -2013-09-10 20:09:01,749 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.9 sec -2013-09-10 20:09:02,754 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.85 sec -2013-09-10 20:09:03,758 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.85 sec -2013-09-10 20:09:04,763 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.85 sec -2013-09-10 20:09:05,768 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.85 sec -2013-09-10 20:09:06,774 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.85 sec -2013-09-10 20:09:07,779 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.85 sec -2013-09-10 20:09:08,784 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.85 sec -2013-09-10 20:09:09,790 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 54.85 sec -2013-09-10 20:09:10,797 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.65 sec -2013-09-10 20:09:11,802 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.65 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 650 msec -Ended Job = job_201309101627_0114 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 62.65 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 2 seconds 650 msec -OK -2037258 -Time taken: 39.152 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22014@mturlrep13_201309102009_2085887753.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22418@mturlrep13_201309102009_1216544523.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0115 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:09:25,056 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:09:33,088 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:09:35,102 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.41 sec -2013-09-10 20:09:36,108 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.41 sec -2013-09-10 20:09:37,115 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.41 sec -2013-09-10 20:09:38,121 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.41 sec -2013-09-10 20:09:39,127 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.41 sec -2013-09-10 20:09:40,134 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.41 sec -2013-09-10 20:09:41,140 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.41 sec -2013-09-10 20:09:42,150 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.41 sec -2013-09-10 20:09:43,155 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.41 sec -2013-09-10 20:09:44,160 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.41 sec -2013-09-10 20:09:45,165 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.74 sec -2013-09-10 20:09:46,171 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.74 sec -2013-09-10 20:09:47,176 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.74 sec -2013-09-10 20:09:48,182 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.74 sec -2013-09-10 20:09:49,187 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.74 sec -2013-09-10 20:09:50,193 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.57 sec -2013-09-10 20:09:51,198 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.57 sec -2013-09-10 20:09:52,205 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.64 sec -2013-09-10 20:09:53,211 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.64 sec -2013-09-10 20:09:54,217 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.64 sec -MapReduce Total cumulative CPU time: 1 minutes 3 seconds 640 msec -Ended Job = job_201309101627_0115 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 63.64 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 3 seconds 640 msec -OK -2037258 -Time taken: 36.46 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23691@mturlrep13_201309102009_809944135.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24111@mturlrep13_201309102009_812926126.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0116 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:10:08,163 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:10:15,213 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:10:17,227 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.08 sec -2013-09-10 20:10:18,233 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.08 sec -2013-09-10 20:10:19,240 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.08 sec -2013-09-10 20:10:20,246 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.08 sec -2013-09-10 20:10:21,252 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.08 sec -2013-09-10 20:10:22,258 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.08 sec -2013-09-10 20:10:23,265 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.08 sec -2013-09-10 20:10:24,270 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.08 sec -2013-09-10 20:10:25,275 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.08 sec -2013-09-10 20:10:26,280 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 39.75 sec -2013-09-10 20:10:27,285 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.02 sec -2013-09-10 20:10:28,290 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.02 sec -2013-09-10 20:10:29,295 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.02 sec -2013-09-10 20:10:30,300 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.02 sec -2013-09-10 20:10:31,305 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.02 sec -2013-09-10 20:10:32,311 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.02 sec -2013-09-10 20:10:33,315 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.02 sec -2013-09-10 20:10:34,321 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 54.02 sec -2013-09-10 20:10:35,327 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 61.73 sec -2013-09-10 20:10:36,333 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 61.73 sec -MapReduce Total cumulative CPU time: 1 minutes 1 seconds 730 msec -Ended Job = job_201309101627_0116 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 61.73 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 1 seconds 730 msec -OK -2037258 -Time taken: 36.551 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26095@mturlrep13_201309102010_1548501792.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26522@mturlrep13_201309102010_1522319778.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0117 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:10:57,776 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:11:04,804 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:11:05,815 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.53 sec -2013-09-10 20:11:06,821 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.53 sec -2013-09-10 20:11:07,829 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.53 sec -2013-09-10 20:11:08,835 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.53 sec -2013-09-10 20:11:09,840 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.53 sec -2013-09-10 20:11:10,848 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.53 sec -2013-09-10 20:11:11,854 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.53 sec -2013-09-10 20:11:12,861 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.53 sec -2013-09-10 20:11:13,866 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 21.53 sec -2013-09-10 20:11:14,871 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.24 sec -2013-09-10 20:11:15,876 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.24 sec -2013-09-10 20:11:16,880 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.24 sec -2013-09-10 20:11:17,885 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.24 sec -2013-09-10 20:11:18,890 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.24 sec -2013-09-10 20:11:19,895 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.24 sec -2013-09-10 20:11:20,899 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.24 sec -2013-09-10 20:11:21,904 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.24 sec -2013-09-10 20:11:22,909 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 42.24 sec -2013-09-10 20:11:23,916 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.0 sec -2013-09-10 20:11:24,922 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.0 sec -MapReduce Total cumulative CPU time: 50 seconds 0 msec -Ended Job = job_201309101627_0117 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 50.0 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 50 seconds 0 msec -OK -1110413 -Time taken: 36.986 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27838@mturlrep13_201309102011_1558608875.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28243@mturlrep13_201309102011_1322698871.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0118 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:11:37,820 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:11:45,856 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:11:46,868 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-10 20:11:47,874 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-10 20:11:48,881 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-10 20:11:49,887 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-10 20:11:50,893 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-10 20:11:51,899 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-10 20:11:52,906 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-10 20:11:53,911 Stage-1 map = 72%, reduce = 17%, Cumulative CPU 22.55 sec -2013-09-10 20:11:54,917 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 32.95 sec -2013-09-10 20:11:55,922 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:11:56,926 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:11:57,931 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:11:58,936 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:11:59,941 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:12:00,946 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:12:01,962 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:12:02,967 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 44.61 sec -2013-09-10 20:12:03,975 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.41 sec -2013-09-10 20:12:04,981 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.41 sec -MapReduce Total cumulative CPU time: 52 seconds 410 msec -Ended Job = job_201309101627_0118 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 52.41 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 52 seconds 410 msec -OK -1110413 -Time taken: 34.466 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29534@mturlrep13_201309102012_702635740.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29941@mturlrep13_201309102012_2027998571.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0119 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:12:17,991 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:12:26,021 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:12:27,031 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.05 sec -2013-09-10 20:12:28,038 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.05 sec -2013-09-10 20:12:29,045 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.05 sec -2013-09-10 20:12:30,051 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.05 sec -2013-09-10 20:12:31,057 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.05 sec -2013-09-10 20:12:32,063 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.05 sec -2013-09-10 20:12:33,070 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.05 sec -2013-09-10 20:12:34,076 Stage-1 map = 72%, reduce = 17%, Cumulative CPU 22.05 sec -2013-09-10 20:12:35,081 Stage-1 map = 99%, reduce = 17%, Cumulative CPU 33.57 sec -2013-09-10 20:12:36,086 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:12:37,091 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:12:38,096 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:12:39,101 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:12:40,107 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:12:41,112 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:12:42,117 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.61 sec -2013-09-10 20:12:43,122 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 44.61 sec -2013-09-10 20:12:44,130 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.1 sec -2013-09-10 20:12:45,135 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.1 sec -MapReduce Total cumulative CPU time: 52 seconds 100 msec -Ended Job = job_201309101627_0119 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 52.1 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 52 seconds 100 msec -OK -1110413 -Time taken: 34.424 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31270@mturlrep13_201309102012_1959011426.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31694@mturlrep13_201309102012_1837865750.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0120 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:13:04,874 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:13:10,902 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.53 sec -2013-09-10 20:13:11,910 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.53 sec -2013-09-10 20:13:12,917 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.53 sec -2013-09-10 20:13:13,923 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.53 sec -2013-09-10 20:13:14,928 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.53 sec -2013-09-10 20:13:15,934 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.53 sec -2013-09-10 20:13:16,939 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 21.15 sec -2013-09-10 20:13:17,945 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 28.11 sec -2013-09-10 20:13:18,950 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.11 sec -2013-09-10 20:13:19,956 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.11 sec -2013-09-10 20:13:20,961 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.11 sec -2013-09-10 20:13:21,965 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.11 sec -2013-09-10 20:13:22,970 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.11 sec -2013-09-10 20:13:23,975 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.11 sec -2013-09-10 20:13:24,982 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.21 sec -2013-09-10 20:13:25,988 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.21 sec -MapReduce Total cumulative CPU time: 30 seconds 210 msec -Ended Job = job_201309101627_0120 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 30.21 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 30 seconds 210 msec -OK -Time taken: 31.036 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_484@mturlrep13_201309102013_979592639.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_912@mturlrep13_201309102013_1524044255.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0121 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:13:40,111 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:13:45,139 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-10 20:13:46,146 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-10 20:13:47,154 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-10 20:13:48,159 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-10 20:13:49,165 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-10 20:13:50,171 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 19.88 sec -2013-09-10 20:13:51,176 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.39 sec -2013-09-10 20:13:52,181 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.39 sec -2013-09-10 20:13:53,188 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.17 sec -2013-09-10 20:13:54,195 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.17 sec -2013-09-10 20:13:55,201 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.17 sec -MapReduce Total cumulative CPU time: 28 seconds 170 msec -Ended Job = job_201309101627_0121 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 28.17 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 28 seconds 170 msec -OK -Time taken: 23.561 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2425@mturlrep13_201309102013_943607470.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2853@mturlrep13_201309102014_7623515.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0122 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:14:08,198 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:14:14,228 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.03 sec -2013-09-10 20:14:15,235 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.03 sec -2013-09-10 20:14:16,243 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.03 sec -2013-09-10 20:14:17,249 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.03 sec -2013-09-10 20:14:18,254 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.03 sec -2013-09-10 20:14:19,260 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.03 sec -2013-09-10 20:14:20,265 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 20.82 sec -2013-09-10 20:14:21,270 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 28.54 sec -2013-09-10 20:14:22,276 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 28.54 sec -2013-09-10 20:14:23,284 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.46 sec -2013-09-10 20:14:24,289 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.46 sec -MapReduce Total cumulative CPU time: 30 seconds 460 msec -Ended Job = job_201309101627_0122 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 30.46 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 30 seconds 460 msec -OK -Time taken: 23.42 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4095@mturlrep13_201309102014_942872156.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4560@mturlrep13_201309102014_1712040963.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0123 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:14:44,237 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:14:49,266 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.11 sec -2013-09-10 20:14:50,273 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.11 sec -2013-09-10 20:14:51,282 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.11 sec -2013-09-10 20:14:52,288 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.11 sec -2013-09-10 20:14:53,294 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.11 sec -2013-09-10 20:14:54,301 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.77 sec -2013-09-10 20:14:55,307 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.77 sec -2013-09-10 20:14:56,313 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 24.77 sec -2013-09-10 20:14:57,320 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.57 sec -2013-09-10 20:14:58,326 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.57 sec -2013-09-10 20:14:59,332 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.57 sec -MapReduce Total cumulative CPU time: 28 seconds 570 msec -Ended Job = job_201309101627_0123 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0124 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:15:01,785 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:15:03,793 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:15:04,798 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:15:05,809 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:15:06,814 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:15:07,820 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:15:08,825 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:15:09,830 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:15:10,835 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.82 sec -2013-09-10 20:15:11,840 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -2013-09-10 20:15:12,846 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -MapReduce Total cumulative CPU time: 2 seconds 230 msec -Ended Job = job_201309101627_0124 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 28.57 sec HDFS Read: 907716 HDFS Write: 384 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.23 sec HDFS Read: 1151 HDFS Write: 60 SUCCESS -Total MapReduce CPU Time Spent: 30 seconds 800 msec -OK -Time taken: 38.302 seconds, Fetched: 9 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7012@mturlrep13_201309102015_1321687480.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7736@mturlrep13_201309102015_1218789323.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0125 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:15:25,872 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:15:30,907 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 5.97 sec -2013-09-10 20:15:31,916 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.9 sec -2013-09-10 20:15:32,924 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.9 sec -2013-09-10 20:15:33,931 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.9 sec -2013-09-10 20:15:34,938 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.9 sec -2013-09-10 20:15:35,944 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 18.67 sec -2013-09-10 20:15:36,950 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.98 sec -2013-09-10 20:15:37,957 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.98 sec -2013-09-10 20:15:38,965 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 26.8 sec -2013-09-10 20:15:39,972 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.6 sec -2013-09-10 20:15:40,978 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.6 sec -MapReduce Total cumulative CPU time: 28 seconds 600 msec -Ended Job = job_201309101627_0125 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0126 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:15:44,517 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:15:45,522 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 20:15:46,528 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 20:15:47,534 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 20:15:48,539 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 20:15:49,544 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 20:15:50,549 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 20:15:51,555 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 20:15:52,561 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 20:15:53,567 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.24 sec -2013-09-10 20:15:54,573 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.24 sec -2013-09-10 20:15:55,578 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.24 sec -MapReduce Total cumulative CPU time: 2 seconds 240 msec -Ended Job = job_201309101627_0126 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 28.6 sec HDFS Read: 907716 HDFS Write: 384 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.24 sec HDFS Read: 1149 HDFS Write: 60 SUCCESS -Total MapReduce CPU Time Spent: 30 seconds 840 msec -OK -Time taken: 37.076 seconds, Fetched: 9 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9595@mturlrep13_201309102015_955881835.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10020@mturlrep13_201309102016_1021454413.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0127 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:16:08,481 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:16:13,508 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.5 sec -2013-09-10 20:16:14,515 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.5 sec -2013-09-10 20:16:15,522 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.5 sec -2013-09-10 20:16:16,528 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.5 sec -2013-09-10 20:16:17,535 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.5 sec -2013-09-10 20:16:18,542 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.29 sec -2013-09-10 20:16:19,547 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.29 sec -2013-09-10 20:16:20,553 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.29 sec -2013-09-10 20:16:21,560 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 25.17 sec -2013-09-10 20:16:22,567 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.08 sec -2013-09-10 20:16:23,574 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.08 sec -MapReduce Total cumulative CPU time: 27 seconds 80 msec -Ended Job = job_201309101627_0127 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0128 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:16:27,070 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:16:28,075 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 20:16:29,081 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 20:16:30,090 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 20:16:31,096 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 20:16:32,101 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 20:16:33,106 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 20:16:34,111 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 20:16:35,117 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 20:16:36,122 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.12 sec -2013-09-10 20:16:37,127 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.12 sec -2013-09-10 20:16:38,133 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.12 sec -MapReduce Total cumulative CPU time: 2 seconds 120 msec -Ended Job = job_201309101627_0128 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 27.08 sec HDFS Read: 907716 HDFS Write: 384 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.12 sec HDFS Read: 1153 HDFS Write: 60 SUCCESS -Total MapReduce CPU Time Spent: 29 seconds 200 msec -OK -Time taken: 37.118 seconds, Fetched: 9 row(s) -hive> quit; --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11865@mturlrep13_201309102016_504239459.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12310@mturlrep13_201309102016_364343400.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0129 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:16:59,009 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:17:06,036 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:17:10,057 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.36 sec -2013-09-10 20:17:11,064 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.36 sec -2013-09-10 20:17:12,072 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.36 sec -2013-09-10 20:17:13,079 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.36 sec -2013-09-10 20:17:14,086 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.36 sec -2013-09-10 20:17:15,092 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.36 sec -2013-09-10 20:17:16,099 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.36 sec -2013-09-10 20:17:17,105 Stage-1 map = 72%, reduce = 8%, Cumulative CPU 29.36 sec -2013-09-10 20:17:18,111 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 29.36 sec -2013-09-10 20:17:19,117 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 29.36 sec -2013-09-10 20:17:20,122 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 29.36 sec -2013-09-10 20:17:21,127 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.96 sec -2013-09-10 20:17:22,133 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.96 sec -2013-09-10 20:17:23,139 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.96 sec -2013-09-10 20:17:24,146 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.96 sec -2013-09-10 20:17:25,151 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.96 sec -2013-09-10 20:17:26,158 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 63.53 sec -2013-09-10 20:17:27,165 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.82 sec -2013-09-10 20:17:28,171 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.82 sec -MapReduce Total cumulative CPU time: 1 minutes 10 seconds 820 msec -Ended Job = job_201309101627_0129 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0130 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:17:30,606 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:17:32,615 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.51 sec -2013-09-10 20:17:33,621 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.51 sec -2013-09-10 20:17:34,626 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.51 sec -2013-09-10 20:17:35,631 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.51 sec -2013-09-10 20:17:36,637 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.51 sec -2013-09-10 20:17:37,642 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.51 sec -2013-09-10 20:17:38,648 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.51 sec -2013-09-10 20:17:39,653 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.51 sec -2013-09-10 20:17:40,659 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.05 sec -2013-09-10 20:17:41,665 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.05 sec -2013-09-10 20:17:42,670 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.05 sec -MapReduce Total cumulative CPU time: 3 seconds 50 msec -Ended Job = job_201309101627_0130 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 70.82 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.05 sec HDFS Read: 100911 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 13 seconds 870 msec -OK -Time taken: 53.485 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14188@mturlrep13_201309102017_126837512.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14629@mturlrep13_201309102017_1747557025.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0131 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:17:56,720 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:18:03,746 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:18:06,763 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-10 20:18:07,770 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-10 20:18:08,777 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-10 20:18:09,783 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-10 20:18:10,789 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-10 20:18:11,795 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-10 20:18:12,801 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-10 20:18:13,812 Stage-1 map = 96%, reduce = 8%, Cumulative CPU 28.46 sec -2013-09-10 20:18:14,818 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-10 20:18:15,823 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-10 20:18:16,829 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.8 sec -2013-09-10 20:18:17,834 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.8 sec -2013-09-10 20:18:18,839 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.8 sec -2013-09-10 20:18:19,845 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.8 sec -2013-09-10 20:18:20,850 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.8 sec -2013-09-10 20:18:21,855 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.8 sec -2013-09-10 20:18:22,863 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 63.32 sec -2013-09-10 20:18:23,869 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.39 sec -2013-09-10 20:18:24,875 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.39 sec -MapReduce Total cumulative CPU time: 1 minutes 10 seconds 390 msec -Ended Job = job_201309101627_0131 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0132 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:18:27,568 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:18:30,579 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.46 sec -2013-09-10 20:18:31,584 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.46 sec -2013-09-10 20:18:32,589 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.46 sec -2013-09-10 20:18:33,594 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.46 sec -2013-09-10 20:18:34,599 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.46 sec -2013-09-10 20:18:35,604 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.46 sec -2013-09-10 20:18:36,609 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.46 sec -2013-09-10 20:18:37,614 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.0 sec -2013-09-10 20:18:38,620 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.0 sec -2013-09-10 20:18:39,625 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.0 sec -MapReduce Total cumulative CPU time: 3 seconds 0 msec -Ended Job = job_201309101627_0132 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 70.39 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.0 sec HDFS Read: 100911 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 13 seconds 390 msec -OK -Time taken: 51.134 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16555@mturlrep13_201309102018_646695690.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16977@mturlrep13_201309102018_162818540.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0133 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:18:53,456 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:19:00,484 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:19:02,498 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-10 20:19:03,506 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-10 20:19:04,513 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-10 20:19:05,520 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-10 20:19:06,526 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-10 20:19:07,533 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-10 20:19:08,540 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-10 20:19:09,546 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-10 20:19:10,558 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.76 sec -2013-09-10 20:19:11,564 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.76 sec -2013-09-10 20:19:12,570 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 42.08 sec -2013-09-10 20:19:13,576 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.65 sec -2013-09-10 20:19:14,582 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.65 sec -2013-09-10 20:19:15,588 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.65 sec -2013-09-10 20:19:16,594 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.65 sec -2013-09-10 20:19:17,600 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.65 sec -2013-09-10 20:19:18,608 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 62.31 sec -2013-09-10 20:19:19,614 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 62.31 sec -2013-09-10 20:19:20,621 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.64 sec -2013-09-10 20:19:21,627 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.64 sec -MapReduce Total cumulative CPU time: 1 minutes 9 seconds 640 msec -Ended Job = job_201309101627_0133 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0134 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:19:25,083 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:19:27,101 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.52 sec -2013-09-10 20:19:28,107 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.52 sec -2013-09-10 20:19:29,112 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.52 sec -2013-09-10 20:19:30,117 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.52 sec -2013-09-10 20:19:31,122 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.52 sec -2013-09-10 20:19:32,127 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.52 sec -2013-09-10 20:19:33,133 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.52 sec -2013-09-10 20:19:34,138 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.52 sec -2013-09-10 20:19:35,143 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.06 sec -2013-09-10 20:19:36,149 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.06 sec -MapReduce Total cumulative CPU time: 3 seconds 60 msec -Ended Job = job_201309101627_0134 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 69.64 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.06 sec HDFS Read: 100911 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 12 seconds 700 msec -OK -Time taken: 50.893 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18929@mturlrep13_201309102019_1852281941.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19369@mturlrep13_201309102019_1237243258.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0135 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:19:56,153 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:20:03,183 Stage-1 map = 29%, reduce = 0% -2013-09-10 20:20:06,195 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:20:09,213 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-10 20:20:10,221 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-10 20:20:11,229 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-10 20:20:12,236 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-10 20:20:13,242 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-10 20:20:14,249 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-10 20:20:15,255 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-10 20:20:16,261 Stage-1 map = 64%, reduce = 8%, Cumulative CPU 34.32 sec -2013-09-10 20:20:17,275 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 34.32 sec -2013-09-10 20:20:18,328 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 34.32 sec -2013-09-10 20:20:19,334 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.32 sec -2013-09-10 20:20:20,340 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.32 sec -2013-09-10 20:20:21,347 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.32 sec -2013-09-10 20:20:22,353 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.37 sec -2013-09-10 20:20:23,358 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.37 sec -2013-09-10 20:20:24,363 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.37 sec -2013-09-10 20:20:25,369 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.37 sec -2013-09-10 20:20:26,375 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.37 sec -2013-09-10 20:20:27,380 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.37 sec -2013-09-10 20:20:28,386 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.37 sec -2013-09-10 20:20:29,391 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.37 sec -2013-09-10 20:20:30,397 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.37 sec -2013-09-10 20:20:31,404 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 74.61 sec -2013-09-10 20:20:32,409 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 74.61 sec -2013-09-10 20:20:33,415 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 83.92 sec -2013-09-10 20:20:34,421 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 83.92 sec -MapReduce Total cumulative CPU time: 1 minutes 23 seconds 920 msec -Ended Job = job_201309101627_0135 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0136 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:20:36,980 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:20:39,990 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.64 sec -2013-09-10 20:20:40,995 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.64 sec -2013-09-10 20:20:42,000 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.64 sec -2013-09-10 20:20:43,005 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.64 sec -2013-09-10 20:20:44,010 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.64 sec -2013-09-10 20:20:45,016 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.64 sec -2013-09-10 20:20:46,021 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.64 sec -2013-09-10 20:20:47,027 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.21 sec -2013-09-10 20:20:48,032 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.21 sec -2013-09-10 20:20:49,038 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.21 sec -MapReduce Total cumulative CPU time: 3 seconds 210 msec -Ended Job = job_201309101627_0136 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 83.92 sec HDFS Read: 74853201 HDFS Write: 148871 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.21 sec HDFS Read: 149640 HDFS Write: 414 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 27 seconds 130 msec -OK -Time taken: 62.752 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22157@mturlrep13_201309102020_335437932.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22568@mturlrep13_201309102020_882304972.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0137 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:21:02,043 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:21:10,075 Stage-1 map = 36%, reduce = 0% -2013-09-10 20:21:13,089 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:21:15,103 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.24 sec -2013-09-10 20:21:16,110 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.24 sec -2013-09-10 20:21:17,119 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.24 sec -2013-09-10 20:21:18,125 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.24 sec -2013-09-10 20:21:19,132 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.24 sec -2013-09-10 20:21:20,138 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.24 sec -2013-09-10 20:21:21,145 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.24 sec -2013-09-10 20:21:22,151 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.24 sec -2013-09-10 20:21:23,157 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.24 sec -2013-09-10 20:21:24,162 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.24 sec -2013-09-10 20:21:25,168 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.24 sec -2013-09-10 20:21:26,174 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.24 sec -2013-09-10 20:21:27,179 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 49.89 sec -2013-09-10 20:21:28,185 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.04 sec -2013-09-10 20:21:29,190 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.04 sec -2013-09-10 20:21:30,195 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.04 sec -2013-09-10 20:21:31,202 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.04 sec -2013-09-10 20:21:32,209 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.04 sec -2013-09-10 20:21:33,214 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.04 sec -2013-09-10 20:21:34,220 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.04 sec -2013-09-10 20:21:35,225 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.04 sec -2013-09-10 20:21:36,231 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.04 sec -2013-09-10 20:21:37,239 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 75.13 sec -2013-09-10 20:21:38,244 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 75.13 sec -2013-09-10 20:21:39,250 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 84.3 sec -2013-09-10 20:21:40,256 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 84.3 sec -2013-09-10 20:21:41,262 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 84.3 sec -MapReduce Total cumulative CPU time: 1 minutes 24 seconds 300 msec -Ended Job = job_201309101627_0137 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0138 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:21:43,758 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:21:45,767 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.81 sec -2013-09-10 20:21:46,772 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.81 sec -2013-09-10 20:21:47,776 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.81 sec -2013-09-10 20:21:48,781 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.81 sec -2013-09-10 20:21:49,785 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.81 sec -2013-09-10 20:21:50,789 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.81 sec -2013-09-10 20:21:51,793 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.81 sec -2013-09-10 20:21:52,798 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.81 sec -2013-09-10 20:21:53,803 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.33 sec -2013-09-10 20:21:54,809 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.33 sec -2013-09-10 20:21:55,814 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.33 sec -MapReduce Total cumulative CPU time: 3 seconds 330 msec -Ended Job = job_201309101627_0138 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 84.3 sec HDFS Read: 74853201 HDFS Write: 148871 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.33 sec HDFS Read: 149640 HDFS Write: 414 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 27 seconds 630 msec -OK -Time taken: 61.04 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24690@mturlrep13_201309102021_1595959377.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25093@mturlrep13_201309102022_1638369950.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0139 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:22:08,691 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:22:16,719 Stage-1 map = 36%, reduce = 0% -2013-09-10 20:22:19,729 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:22:21,743 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.81 sec -2013-09-10 20:22:22,750 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.81 sec -2013-09-10 20:22:23,757 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.81 sec -2013-09-10 20:22:24,763 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.81 sec -2013-09-10 20:22:25,772 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.81 sec -2013-09-10 20:22:26,789 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.81 sec -2013-09-10 20:22:27,794 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.81 sec -2013-09-10 20:22:28,800 Stage-1 map = 88%, reduce = 8%, Cumulative CPU 32.81 sec -2013-09-10 20:22:29,804 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 32.81 sec -2013-09-10 20:22:30,809 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 32.81 sec -2013-09-10 20:22:31,815 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 32.81 sec -2013-09-10 20:22:32,820 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 48.46 sec -2013-09-10 20:22:33,825 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.91 sec -2013-09-10 20:22:34,830 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.91 sec -2013-09-10 20:22:35,835 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.91 sec -2013-09-10 20:22:36,840 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.91 sec -2013-09-10 20:22:37,846 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.91 sec -2013-09-10 20:22:38,851 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.91 sec -2013-09-10 20:22:39,857 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.91 sec -2013-09-10 20:22:40,862 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.91 sec -2013-09-10 20:22:41,867 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.91 sec -2013-09-10 20:22:42,872 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.91 sec -2013-09-10 20:22:43,879 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 74.06 sec -2013-09-10 20:22:44,885 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 74.06 sec -2013-09-10 20:22:45,890 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 83.62 sec -2013-09-10 20:22:46,895 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 83.62 sec -MapReduce Total cumulative CPU time: 1 minutes 23 seconds 620 msec -Ended Job = job_201309101627_0139 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0140 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:22:49,470 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:22:52,481 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.58 sec -2013-09-10 20:22:53,486 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.58 sec -2013-09-10 20:22:54,491 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.58 sec -2013-09-10 20:22:55,496 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.58 sec -2013-09-10 20:22:56,501 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.58 sec -2013-09-10 20:22:57,505 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.58 sec -2013-09-10 20:22:58,510 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.58 sec -2013-09-10 20:22:59,516 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.1 sec -2013-09-10 20:23:00,521 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.1 sec -2013-09-10 20:23:01,527 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.1 sec -MapReduce Total cumulative CPU time: 3 seconds 100 msec -Ended Job = job_201309101627_0140 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 83.62 sec HDFS Read: 74853201 HDFS Write: 148871 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.1 sec HDFS Read: 149640 HDFS Write: 414 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 26 seconds 720 msec -OK -Time taken: 60.141 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27176@mturlrep13_201309102023_1934793750.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27614@mturlrep13_201309102023_1296494329.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0141 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:23:21,374 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:23:28,411 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.04 sec -2013-09-10 20:23:29,419 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.04 sec -2013-09-10 20:23:30,426 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.04 sec -2013-09-10 20:23:31,433 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.04 sec -2013-09-10 20:23:32,439 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.04 sec -2013-09-10 20:23:33,446 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.04 sec -2013-09-10 20:23:34,453 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 17.69 sec -2013-09-10 20:23:35,460 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.96 sec -2013-09-10 20:23:36,466 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.96 sec -2013-09-10 20:23:37,472 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.96 sec -2013-09-10 20:23:38,478 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.96 sec -2013-09-10 20:23:39,483 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.96 sec -2013-09-10 20:23:40,489 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.96 sec -2013-09-10 20:23:41,496 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.96 sec -2013-09-10 20:23:42,503 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.53 sec -2013-09-10 20:23:43,510 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.53 sec -MapReduce Total cumulative CPU time: 30 seconds 530 msec -Ended Job = job_201309101627_0141 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0142 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:23:46,001 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:23:48,011 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:23:49,017 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:23:50,022 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:23:51,028 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:23:52,033 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:23:53,038 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:23:54,043 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-10 20:23:55,049 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.82 sec -2013-09-10 20:23:56,055 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-10 20:23:57,060 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -MapReduce Total cumulative CPU time: 2 seconds 210 msec -Ended Job = job_201309101627_0142 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 30.53 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.21 sec HDFS Read: 21897 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 32 seconds 740 msec -OK -Time taken: 45.242 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29548@mturlrep13_201309102023_2146255552.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29955@mturlrep13_201309102024_1375294080.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0143 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:24:10,138 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:24:16,170 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.73 sec -2013-09-10 20:24:17,178 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.73 sec -2013-09-10 20:24:18,185 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.73 sec -2013-09-10 20:24:19,192 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.73 sec -2013-09-10 20:24:20,198 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.73 sec -2013-09-10 20:24:21,205 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.73 sec -2013-09-10 20:24:22,212 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 17.85 sec -2013-09-10 20:24:23,218 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.46 sec -2013-09-10 20:24:24,224 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.46 sec -2013-09-10 20:24:25,230 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.46 sec -2013-09-10 20:24:26,236 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.46 sec -2013-09-10 20:24:27,242 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.46 sec -2013-09-10 20:24:28,247 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.46 sec -2013-09-10 20:24:29,253 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.46 sec -2013-09-10 20:24:30,261 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.02 sec -2013-09-10 20:24:31,267 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.02 sec -2013-09-10 20:24:32,273 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.02 sec -MapReduce Total cumulative CPU time: 31 seconds 20 msec -Ended Job = job_201309101627_0143 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0144 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:24:35,773 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:24:37,782 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.02 sec -2013-09-10 20:24:38,787 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.02 sec -2013-09-10 20:24:39,792 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.02 sec -2013-09-10 20:24:40,797 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.02 sec -2013-09-10 20:24:41,801 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.02 sec -2013-09-10 20:24:42,806 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.02 sec -2013-09-10 20:24:43,812 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.02 sec -2013-09-10 20:24:44,817 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.49 sec -2013-09-10 20:24:45,823 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.49 sec -2013-09-10 20:24:46,829 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.49 sec -MapReduce Total cumulative CPU time: 2 seconds 490 msec -Ended Job = job_201309101627_0144 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 31.02 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.49 sec HDFS Read: 21897 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 510 msec -OK -Time taken: 44.206 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31926@mturlrep13_201309102024_227013922.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32329@mturlrep13_201309102024_666768593.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0145 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:25:00,728 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:25:05,756 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 5.9 sec -2013-09-10 20:25:06,765 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 20:25:07,772 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 20:25:08,779 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 20:25:09,786 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 20:25:10,792 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 20:25:11,798 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 20:25:12,804 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 28.24 sec -2013-09-10 20:25:13,810 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 28.24 sec -2013-09-10 20:25:14,818 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 31.4 sec -2013-09-10 20:25:15,824 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 31.4 sec -2013-09-10 20:25:16,831 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 31.4 sec -2013-09-10 20:25:17,837 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 31.4 sec -2013-09-10 20:25:18,843 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 31.4 sec -2013-09-10 20:25:19,849 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.7 sec -2013-09-10 20:25:20,855 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.7 sec -2013-09-10 20:25:21,861 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.7 sec -MapReduce Total cumulative CPU time: 34 seconds 700 msec -Ended Job = job_201309101627_0145 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0146 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:25:24,374 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:25:26,382 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 20:25:27,388 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 20:25:28,393 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 20:25:29,398 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 20:25:30,402 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 20:25:31,407 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 20:25:32,412 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 20:25:33,418 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.88 sec -2013-09-10 20:25:34,423 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.41 sec -2013-09-10 20:25:35,429 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.41 sec -MapReduce Total cumulative CPU time: 2 seconds 410 msec -Ended Job = job_201309101627_0146 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 34.7 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.41 sec HDFS Read: 21895 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 37 seconds 110 msec -OK -Time taken: 43.002 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по строкам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2839@mturlrep13_201309102025_1906588406.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3277@mturlrep13_201309102025_1190023510.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0147 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:25:56,480 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:26:02,515 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.68 sec -2013-09-10 20:26:03,523 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.68 sec -2013-09-10 20:26:04,531 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.68 sec -2013-09-10 20:26:05,538 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.68 sec -2013-09-10 20:26:06,545 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.68 sec -2013-09-10 20:26:07,552 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.68 sec -2013-09-10 20:26:08,558 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.68 sec -2013-09-10 20:26:09,564 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 23.83 sec -2013-09-10 20:26:10,570 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 23.83 sec -2013-09-10 20:26:11,577 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 23.83 sec -2013-09-10 20:26:12,583 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 23.83 sec -2013-09-10 20:26:13,589 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 23.83 sec -2013-09-10 20:26:14,595 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 23.83 sec -2013-09-10 20:26:15,601 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 23.83 sec -2013-09-10 20:26:16,609 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.35 sec -2013-09-10 20:26:17,636 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.35 sec -2013-09-10 20:26:18,643 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.35 sec -MapReduce Total cumulative CPU time: 31 seconds 350 msec -Ended Job = job_201309101627_0147 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0148 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:26:22,244 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:26:23,249 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 20:26:24,255 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 20:26:25,261 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 20:26:26,266 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 20:26:27,272 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 20:26:28,277 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 20:26:29,282 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 20:26:30,287 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 20:26:31,294 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.34 sec -2013-09-10 20:26:32,299 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.34 sec -2013-09-10 20:26:33,305 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.34 sec -MapReduce Total cumulative CPU time: 2 seconds 340 msec -Ended Job = job_201309101627_0148 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 31.35 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.34 sec HDFS Read: 23479 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 690 msec -OK -Time taken: 46.938 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5533@mturlrep13_201309102026_343654465.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5957@mturlrep13_201309102026_864824933.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0149 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:26:46,474 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:26:52,507 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.69 sec -2013-09-10 20:26:53,515 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.69 sec -2013-09-10 20:26:54,522 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.69 sec -2013-09-10 20:26:55,528 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.69 sec -2013-09-10 20:26:56,534 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.69 sec -2013-09-10 20:26:57,540 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.69 sec -2013-09-10 20:26:58,546 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.76 sec -2013-09-10 20:26:59,552 Stage-1 map = 100%, reduce = 13%, Cumulative CPU 23.76 sec -2013-09-10 20:27:00,559 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 23.76 sec -2013-09-10 20:27:01,568 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 27.2 sec -2013-09-10 20:27:02,574 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 27.2 sec -2013-09-10 20:27:03,580 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 27.2 sec -2013-09-10 20:27:04,586 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 27.2 sec -2013-09-10 20:27:05,592 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 27.2 sec -2013-09-10 20:27:06,599 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.79 sec -2013-09-10 20:27:07,605 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.79 sec -MapReduce Total cumulative CPU time: 30 seconds 790 msec -Ended Job = job_201309101627_0149 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0150 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:27:10,095 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:27:12,104 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 20:27:13,109 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 20:27:14,115 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 20:27:15,120 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 20:27:16,125 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 20:27:17,130 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 20:27:18,136 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 20:27:19,141 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 20:27:20,148 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -2013-09-10 20:27:21,154 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -2013-09-10 20:27:22,160 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -MapReduce Total cumulative CPU time: 2 seconds 350 msec -Ended Job = job_201309101627_0150 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 30.79 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.35 sec HDFS Read: 23479 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 140 msec -OK -Time taken: 43.048 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7889@mturlrep13_201309102027_433190922.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8295@mturlrep13_201309102027_581718614.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0151 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:27:35,347 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:27:41,379 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.72 sec -2013-09-10 20:27:42,387 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.72 sec -2013-09-10 20:27:43,395 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.72 sec -2013-09-10 20:27:44,401 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.72 sec -2013-09-10 20:27:45,408 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.72 sec -2013-09-10 20:27:46,415 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.72 sec -2013-09-10 20:27:47,422 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.93 sec -2013-09-10 20:27:48,428 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.93 sec -2013-09-10 20:27:49,434 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 23.93 sec -2013-09-10 20:27:50,442 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 27.74 sec -2013-09-10 20:27:51,449 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 28.35 sec -2013-09-10 20:27:52,455 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 28.35 sec -2013-09-10 20:27:53,461 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 28.35 sec -2013-09-10 20:27:54,467 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 28.35 sec -2013-09-10 20:27:55,473 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.22 sec -2013-09-10 20:27:56,479 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.22 sec -2013-09-10 20:27:57,486 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.22 sec -MapReduce Total cumulative CPU time: 31 seconds 220 msec -Ended Job = job_201309101627_0151 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0152 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:28:00,985 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:28:02,995 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.96 sec -2013-09-10 20:28:04,000 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.96 sec -2013-09-10 20:28:05,005 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.96 sec -2013-09-10 20:28:06,010 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.96 sec -2013-09-10 20:28:07,016 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.96 sec -2013-09-10 20:28:08,021 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.96 sec -2013-09-10 20:28:09,027 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.96 sec -2013-09-10 20:28:10,033 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.4 sec -2013-09-10 20:28:11,039 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.4 sec -2013-09-10 20:28:12,044 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.4 sec -MapReduce Total cumulative CPU time: 2 seconds 400 msec -Ended Job = job_201309101627_0152 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 31.22 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.4 sec HDFS Read: 23479 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 620 msec -OK -Time taken: 44.124 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10235@mturlrep13_201309102028_1893210303.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10677@mturlrep13_201309102028_1721692497.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0153 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:28:32,602 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:28:39,632 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:28:40,644 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.88 sec -2013-09-10 20:28:41,652 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.88 sec -2013-09-10 20:28:42,659 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.88 sec -2013-09-10 20:28:43,665 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.88 sec -2013-09-10 20:28:44,671 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.88 sec -2013-09-10 20:28:45,677 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.88 sec -2013-09-10 20:28:46,684 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.88 sec -2013-09-10 20:28:47,691 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.88 sec -2013-09-10 20:28:48,697 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.87 sec -2013-09-10 20:28:49,702 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.87 sec -2013-09-10 20:28:50,708 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.87 sec -2013-09-10 20:28:51,714 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.54 sec -2013-09-10 20:28:52,720 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.54 sec -2013-09-10 20:28:53,726 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.54 sec -2013-09-10 20:28:54,732 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.54 sec -2013-09-10 20:28:55,738 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.54 sec -2013-09-10 20:28:56,744 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.54 sec -2013-09-10 20:28:57,752 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.79 sec -2013-09-10 20:28:58,758 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.79 sec -2013-09-10 20:28:59,763 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.79 sec -MapReduce Total cumulative CPU time: 56 seconds 790 msec -Ended Job = job_201309101627_0153 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0154 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:29:03,287 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:29:10,313 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:29:12,321 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.87 sec -2013-09-10 20:29:13,327 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.87 sec -2013-09-10 20:29:14,332 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.87 sec -2013-09-10 20:29:15,336 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.87 sec -2013-09-10 20:29:16,340 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.87 sec -2013-09-10 20:29:17,345 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.87 sec -2013-09-10 20:29:18,350 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.87 sec -2013-09-10 20:29:19,354 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.87 sec -2013-09-10 20:29:20,360 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.87 sec -2013-09-10 20:29:21,365 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.4 sec -2013-09-10 20:29:22,370 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.4 sec -2013-09-10 20:29:23,375 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.4 sec -MapReduce Total cumulative CPU time: 18 seconds 400 msec -Ended Job = job_201309101627_0154 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 56.79 sec HDFS Read: 27820105 HDFS Write: 79726641 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.4 sec HDFS Read: 79727410 HDFS Write: 275 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 15 seconds 190 msec -OK -Time taken: 60.48 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12624@mturlrep13_201309102029_1863431716.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13040@mturlrep13_201309102029_973897390.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0155 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:29:37,380 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:29:44,415 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.02 sec -2013-09-10 20:29:45,423 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.02 sec -2013-09-10 20:29:46,431 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.02 sec -2013-09-10 20:29:47,438 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.02 sec -2013-09-10 20:29:48,445 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.02 sec -2013-09-10 20:29:49,452 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.02 sec -2013-09-10 20:29:50,458 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.02 sec -2013-09-10 20:29:51,465 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.02 sec -2013-09-10 20:29:52,472 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.52 sec -2013-09-10 20:29:53,478 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.52 sec -2013-09-10 20:29:54,484 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.52 sec -2013-09-10 20:29:55,490 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.52 sec -2013-09-10 20:29:56,496 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.52 sec -2013-09-10 20:29:57,502 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.52 sec -2013-09-10 20:29:58,508 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.52 sec -2013-09-10 20:29:59,514 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.52 sec -2013-09-10 20:30:00,520 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.52 sec -2013-09-10 20:30:01,529 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.86 sec -2013-09-10 20:30:02,535 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.86 sec -2013-09-10 20:30:03,541 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.86 sec -MapReduce Total cumulative CPU time: 56 seconds 860 msec -Ended Job = job_201309101627_0155 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0156 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:30:06,136 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:30:14,165 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:30:16,174 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.7 sec -2013-09-10 20:30:17,180 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.7 sec -2013-09-10 20:30:18,186 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.7 sec -2013-09-10 20:30:19,191 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.7 sec -2013-09-10 20:30:20,195 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.7 sec -2013-09-10 20:30:21,200 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.7 sec -2013-09-10 20:30:22,205 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.7 sec -2013-09-10 20:30:23,210 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.7 sec -2013-09-10 20:30:24,216 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.7 sec -2013-09-10 20:30:25,221 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.4 sec -2013-09-10 20:30:26,227 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.4 sec -2013-09-10 20:30:27,232 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.4 sec -MapReduce Total cumulative CPU time: 18 seconds 400 msec -Ended Job = job_201309101627_0156 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 56.86 sec HDFS Read: 27820105 HDFS Write: 79726641 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.4 sec HDFS Read: 79727410 HDFS Write: 275 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 15 seconds 260 msec -OK -Time taken: 58.248 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15653@mturlrep13_201309102030_1115123601.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16121@mturlrep13_201309102030_1858922257.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0157 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:30:40,546 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:30:48,583 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.04 sec -2013-09-10 20:30:49,592 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.04 sec -2013-09-10 20:30:50,600 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.04 sec -2013-09-10 20:30:51,608 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.04 sec -2013-09-10 20:30:52,614 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.04 sec -2013-09-10 20:30:53,621 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.04 sec -2013-09-10 20:30:54,628 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.04 sec -2013-09-10 20:30:55,635 Stage-1 map = 96%, reduce = 8%, Cumulative CPU 21.04 sec -2013-09-10 20:30:56,641 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.64 sec -2013-09-10 20:30:57,648 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.64 sec -2013-09-10 20:30:58,655 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.64 sec -2013-09-10 20:30:59,661 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.64 sec -2013-09-10 20:31:00,667 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.64 sec -2013-09-10 20:31:01,673 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.64 sec -2013-09-10 20:31:02,679 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.64 sec -2013-09-10 20:31:03,685 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.64 sec -2013-09-10 20:31:04,693 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 49.56 sec -2013-09-10 20:31:05,699 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.8 sec -2013-09-10 20:31:06,706 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.8 sec -MapReduce Total cumulative CPU time: 56 seconds 800 msec -Ended Job = job_201309101627_0157 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0158 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:31:10,220 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:31:17,244 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:31:20,255 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.82 sec -2013-09-10 20:31:21,260 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.82 sec -2013-09-10 20:31:22,265 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.82 sec -2013-09-10 20:31:23,269 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.82 sec -2013-09-10 20:31:24,273 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.82 sec -2013-09-10 20:31:25,277 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.82 sec -2013-09-10 20:31:26,281 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.82 sec -2013-09-10 20:31:27,286 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.82 sec -2013-09-10 20:31:28,291 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.82 sec -2013-09-10 20:31:29,296 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.28 sec -2013-09-10 20:31:30,301 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.28 sec -MapReduce Total cumulative CPU time: 18 seconds 280 msec -Ended Job = job_201309101627_0158 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 56.8 sec HDFS Read: 27820105 HDFS Write: 79726641 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.28 sec HDFS Read: 79727410 HDFS Write: 275 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 15 seconds 80 msec -OK -Time taken: 57.158 seconds, Fetched: 10 row(s) -hive> quit; --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18105@mturlrep13_201309102031_1802187609.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18522@mturlrep13_201309102031_2086673923.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0159 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:31:51,697 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:31:58,726 Stage-1 map = 36%, reduce = 0% -2013-09-10 20:32:00,742 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.86 sec -2013-09-10 20:32:01,750 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.86 sec -2013-09-10 20:32:02,758 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.86 sec -2013-09-10 20:32:03,765 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.86 sec -2013-09-10 20:32:04,772 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.86 sec -2013-09-10 20:32:05,778 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.86 sec -2013-09-10 20:32:06,785 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.86 sec -2013-09-10 20:32:07,792 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.86 sec -2013-09-10 20:32:08,798 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 21.86 sec -2013-09-10 20:32:09,804 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 34.11 sec -2013-09-10 20:32:10,810 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.7 sec -2013-09-10 20:32:11,815 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.7 sec -2013-09-10 20:32:12,821 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.7 sec -2013-09-10 20:32:13,827 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.7 sec -2013-09-10 20:32:14,833 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.7 sec -2013-09-10 20:32:15,838 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.7 sec -2013-09-10 20:32:16,845 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.7 sec -2013-09-10 20:32:17,852 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 53.79 sec -2013-09-10 20:32:18,858 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.45 sec -2013-09-10 20:32:19,864 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.45 sec -2013-09-10 20:32:20,870 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.45 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 450 msec -Ended Job = job_201309101627_0159 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0160 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:32:23,381 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:32:31,407 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:32:33,415 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.98 sec -2013-09-10 20:32:34,420 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.98 sec -2013-09-10 20:32:35,425 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.98 sec -2013-09-10 20:32:36,430 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.98 sec -2013-09-10 20:32:37,435 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.98 sec -2013-09-10 20:32:38,439 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.98 sec -2013-09-10 20:32:39,444 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.98 sec -2013-09-10 20:32:40,450 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.98 sec -2013-09-10 20:32:41,454 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.98 sec -2013-09-10 20:32:42,460 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.54 sec -2013-09-10 20:32:43,465 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.54 sec -2013-09-10 20:32:44,470 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.54 sec -MapReduce Total cumulative CPU time: 18 seconds 540 msec -Ended Job = job_201309101627_0160 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.45 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.54 sec HDFS Read: 79727313 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 20 seconds 990 msec -OK -Time taken: 62.742 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20555@mturlrep13_201309102032_475578712.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20972@mturlrep13_201309102032_588341028.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0161 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:32:58,361 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:33:05,390 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:33:06,404 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.79 sec -2013-09-10 20:33:07,412 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.79 sec -2013-09-10 20:33:08,419 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.79 sec -2013-09-10 20:33:09,426 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.79 sec -2013-09-10 20:33:10,433 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.79 sec -2013-09-10 20:33:11,439 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.79 sec -2013-09-10 20:33:12,445 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.79 sec -2013-09-10 20:33:13,453 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 22.79 sec -2013-09-10 20:33:14,459 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 34.01 sec -2013-09-10 20:33:15,464 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.13 sec -2013-09-10 20:33:16,469 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.13 sec -2013-09-10 20:33:17,474 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.13 sec -2013-09-10 20:33:18,480 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.13 sec -2013-09-10 20:33:19,487 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.13 sec -2013-09-10 20:33:20,493 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.13 sec -2013-09-10 20:33:21,500 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.13 sec -2013-09-10 20:33:22,506 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 46.13 sec -2013-09-10 20:33:23,514 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 54.52 sec -2013-09-10 20:33:24,520 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.25 sec -2013-09-10 20:33:25,526 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.25 sec -MapReduce Total cumulative CPU time: 1 minutes 3 seconds 250 msec -Ended Job = job_201309101627_0161 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0162 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:33:28,009 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:33:36,038 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:33:38,047 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.62 sec -2013-09-10 20:33:39,052 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.62 sec -2013-09-10 20:33:40,057 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.62 sec -2013-09-10 20:33:41,062 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.62 sec -2013-09-10 20:33:42,067 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.62 sec -2013-09-10 20:33:43,072 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.62 sec -2013-09-10 20:33:44,077 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.62 sec -2013-09-10 20:33:45,083 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 14.62 sec -2013-09-10 20:33:46,088 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 14.62 sec -2013-09-10 20:33:47,094 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.02 sec -2013-09-10 20:33:48,100 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.02 sec -2013-09-10 20:33:49,106 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.02 sec -MapReduce Total cumulative CPU time: 19 seconds 20 msec -Ended Job = job_201309101627_0162 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 63.25 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 19.02 sec HDFS Read: 79727313 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 22 seconds 270 msec -OK -Time taken: 59.043 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22965@mturlrep13_201309102033_23929963.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23374@mturlrep13_201309102033_541536373.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0163 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:34:02,415 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:34:10,447 Stage-1 map = 39%, reduce = 0% -2013-09-10 20:34:11,460 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.54 sec -2013-09-10 20:34:12,468 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.54 sec -2013-09-10 20:34:13,476 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.54 sec -2013-09-10 20:34:14,482 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.54 sec -2013-09-10 20:34:15,489 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.54 sec -2013-09-10 20:34:16,496 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.54 sec -2013-09-10 20:34:17,504 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.54 sec -2013-09-10 20:34:18,510 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.54 sec -2013-09-10 20:34:19,517 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 34.51 sec -2013-09-10 20:34:20,523 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.08 sec -2013-09-10 20:34:21,528 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.08 sec -2013-09-10 20:34:22,534 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.08 sec -2013-09-10 20:34:23,540 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.08 sec -2013-09-10 20:34:24,546 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.08 sec -2013-09-10 20:34:25,552 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.08 sec -2013-09-10 20:34:26,558 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.08 sec -2013-09-10 20:34:27,617 Stage-1 map = 100%, reduce = 53%, Cumulative CPU 46.08 sec -2013-09-10 20:34:28,624 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 46.08 sec -2013-09-10 20:34:29,632 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.7 sec -2013-09-10 20:34:30,639 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.7 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 700 msec -Ended Job = job_201309101627_0163 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0164 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:34:33,122 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:34:41,155 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:34:43,164 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.71 sec -2013-09-10 20:34:44,169 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.71 sec -2013-09-10 20:34:45,174 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.71 sec -2013-09-10 20:34:46,178 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.71 sec -2013-09-10 20:34:47,183 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.71 sec -2013-09-10 20:34:48,187 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.71 sec -2013-09-10 20:34:49,192 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.71 sec -2013-09-10 20:34:50,197 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.71 sec -2013-09-10 20:34:51,202 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 12.71 sec -2013-09-10 20:34:52,208 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.18 sec -2013-09-10 20:34:53,213 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.18 sec -2013-09-10 20:34:54,218 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.18 sec -MapReduce Total cumulative CPU time: 17 seconds 180 msec -Ended Job = job_201309101627_0164 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.7 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.18 sec HDFS Read: 79727313 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 19 seconds 880 msec -OK -Time taken: 59.131 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация чуть сложнее.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25402@mturlrep13_201309102035_1149364562.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25922@mturlrep13_201309102035_1476813424.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0165 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:35:15,161 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:35:22,191 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:35:23,203 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-10 20:35:24,211 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.67 sec -2013-09-10 20:35:25,219 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.67 sec -2013-09-10 20:35:26,225 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.67 sec -2013-09-10 20:35:27,231 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.67 sec -2013-09-10 20:35:28,238 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.67 sec -2013-09-10 20:35:29,244 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.67 sec -2013-09-10 20:35:30,251 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.67 sec -2013-09-10 20:35:31,257 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 23.67 sec -2013-09-10 20:35:32,262 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.57 sec -2013-09-10 20:35:33,268 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.57 sec -2013-09-10 20:35:34,273 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.57 sec -2013-09-10 20:35:35,279 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.57 sec -2013-09-10 20:35:36,285 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.57 sec -2013-09-10 20:35:37,290 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.57 sec -2013-09-10 20:35:38,296 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.57 sec -2013-09-10 20:35:39,302 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.57 sec -2013-09-10 20:35:40,308 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 49.57 sec -2013-09-10 20:35:41,315 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 65.45 sec -2013-09-10 20:35:42,321 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 65.45 sec -MapReduce Total cumulative CPU time: 1 minutes 5 seconds 450 msec -Ended Job = job_201309101627_0165 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0166 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:35:45,794 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:35:52,817 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:35:55,829 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.25 sec -2013-09-10 20:35:56,834 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.25 sec -2013-09-10 20:35:57,839 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.25 sec -2013-09-10 20:35:58,844 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.25 sec -2013-09-10 20:35:59,849 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.25 sec -2013-09-10 20:36:00,854 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.25 sec -2013-09-10 20:36:01,859 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.25 sec -2013-09-10 20:36:02,864 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.25 sec -2013-09-10 20:36:03,871 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 14.25 sec -2013-09-10 20:36:04,875 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.73 sec -2013-09-10 20:36:05,880 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.73 sec -2013-09-10 20:36:06,885 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.73 sec -MapReduce Total cumulative CPU time: 18 seconds 730 msec -Ended Job = job_201309101627_0166 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 65.45 sec HDFS Read: 30310112 HDFS Write: 84160093 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.73 sec HDFS Read: 84160862 HDFS Write: 297 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 24 seconds 180 msec -OK -Time taken: 61.752 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28465@mturlrep13_201309102036_191417244.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28883@mturlrep13_201309102036_2013031139.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0167 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:36:19,984 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:36:28,024 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 13.29 sec -2013-09-10 20:36:29,031 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.97 sec -2013-09-10 20:36:30,039 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.97 sec -2013-09-10 20:36:31,045 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.97 sec -2013-09-10 20:36:32,051 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.97 sec -2013-09-10 20:36:33,057 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.97 sec -2013-09-10 20:36:34,064 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.97 sec -2013-09-10 20:36:35,071 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.97 sec -2013-09-10 20:36:36,077 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.97 sec -2013-09-10 20:36:37,083 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-10 20:36:38,088 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-10 20:36:39,094 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-10 20:36:40,100 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-10 20:36:41,106 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-10 20:36:42,113 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-10 20:36:43,119 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-10 20:36:44,125 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-10 20:36:45,133 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 59.87 sec -2013-09-10 20:36:46,162 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 67.67 sec -2013-09-10 20:36:47,168 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 67.67 sec -MapReduce Total cumulative CPU time: 1 minutes 7 seconds 670 msec -Ended Job = job_201309101627_0167 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0168 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:36:50,673 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:36:57,699 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:37:00,711 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.57 sec -2013-09-10 20:37:01,717 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.57 sec -2013-09-10 20:37:02,722 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.57 sec -2013-09-10 20:37:03,727 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.57 sec -2013-09-10 20:37:04,732 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.57 sec -2013-09-10 20:37:05,737 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.57 sec -2013-09-10 20:37:06,742 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.57 sec -2013-09-10 20:37:07,747 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.57 sec -2013-09-10 20:37:08,753 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 14.57 sec -2013-09-10 20:37:09,759 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.06 sec -2013-09-10 20:37:10,766 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.06 sec -2013-09-10 20:37:11,771 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.06 sec -MapReduce Total cumulative CPU time: 19 seconds 60 msec -Ended Job = job_201309101627_0168 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 67.67 sec HDFS Read: 30310112 HDFS Write: 84160093 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 19.06 sec HDFS Read: 84160862 HDFS Write: 297 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 26 seconds 730 msec -OK -Time taken: 59.316 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30836@mturlrep13_201309102037_1364975139.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31259@mturlrep13_201309102037_375310228.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0169 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:37:26,291 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:37:33,329 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-10 20:37:34,338 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-10 20:37:35,345 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-10 20:37:36,353 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-10 20:37:37,359 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-10 20:37:38,366 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-10 20:37:39,373 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-10 20:37:40,379 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-10 20:37:41,386 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.79 sec -2013-09-10 20:37:42,392 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.79 sec -2013-09-10 20:37:43,397 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.79 sec -2013-09-10 20:37:44,402 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.79 sec -2013-09-10 20:37:45,408 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.79 sec -2013-09-10 20:37:46,414 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.79 sec -2013-09-10 20:37:47,430 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.79 sec -2013-09-10 20:37:48,436 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.79 sec -2013-09-10 20:37:49,442 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.79 sec -2013-09-10 20:37:50,450 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.25 sec -2013-09-10 20:37:51,457 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.25 sec -2013-09-10 20:37:52,463 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.25 sec -MapReduce Total cumulative CPU time: 57 seconds 250 msec -Ended Job = job_201309101627_0169 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0170 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:37:55,055 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:38:03,082 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:38:06,093 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.95 sec -2013-09-10 20:38:07,099 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.95 sec -2013-09-10 20:38:08,103 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.95 sec -2013-09-10 20:38:09,108 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.95 sec -2013-09-10 20:38:10,112 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.95 sec -2013-09-10 20:38:11,116 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.95 sec -2013-09-10 20:38:12,121 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.95 sec -2013-09-10 20:38:13,132 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.95 sec -2013-09-10 20:38:14,138 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.95 sec -2013-09-10 20:38:15,145 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.87 sec -2013-09-10 20:38:16,151 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.87 sec -MapReduce Total cumulative CPU time: 18 seconds 870 msec -Ended Job = job_201309101627_0170 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 57.25 sec HDFS Read: 30310112 HDFS Write: 84160093 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.87 sec HDFS Read: 84160862 HDFS Write: 297 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 16 seconds 120 msec -OK -Time taken: 58.243 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по числу и строке, большое количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_793@mturlrep13_201309102038_41120761.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1406@mturlrep13_201309102038_1726333638.txt -hive> SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0171 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:38:38,249 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:38:45,278 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:38:48,297 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.52 sec -2013-09-10 20:38:49,304 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.52 sec -2013-09-10 20:38:50,311 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.52 sec -2013-09-10 20:38:51,317 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.52 sec -2013-09-10 20:38:52,325 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.52 sec -2013-09-10 20:38:53,333 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.52 sec -2013-09-10 20:38:54,339 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.52 sec -2013-09-10 20:38:55,346 Stage-1 map = 72%, reduce = 8%, Cumulative CPU 27.52 sec -2013-09-10 20:38:56,352 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.52 sec -2013-09-10 20:38:57,358 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.52 sec -2013-09-10 20:38:58,365 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.52 sec -2013-09-10 20:38:59,371 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.77 sec -2013-09-10 20:39:00,377 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.77 sec -2013-09-10 20:39:01,383 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.77 sec -2013-09-10 20:39:02,390 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.77 sec -2013-09-10 20:39:03,396 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.77 sec -2013-09-10 20:39:04,401 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 55.77 sec -2013-09-10 20:39:05,409 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 62.77 sec -2013-09-10 20:39:06,415 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.11 sec -2013-09-10 20:39:07,421 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.11 sec -MapReduce Total cumulative CPU time: 1 minutes 10 seconds 110 msec -Ended Job = job_201309101627_0171 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0172 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:39:10,920 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:39:20,949 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:39:23,958 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 19.25 sec -2013-09-10 20:39:24,964 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 19.25 sec -2013-09-10 20:39:25,969 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 19.25 sec -2013-09-10 20:39:26,974 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 19.25 sec -2013-09-10 20:39:27,979 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 19.25 sec -2013-09-10 20:39:28,984 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 19.25 sec -2013-09-10 20:39:29,988 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 19.25 sec -2013-09-10 20:39:30,994 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 19.25 sec -2013-09-10 20:39:31,998 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 19.25 sec -2013-09-10 20:39:33,003 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 19.25 sec -2013-09-10 20:39:34,009 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.7 sec -2013-09-10 20:39:35,014 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.7 sec -MapReduce Total cumulative CPU time: 23 seconds 700 msec -Ended Job = job_201309101627_0172 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 70.11 sec HDFS Read: 57312623 HDFS Write: 55475412 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 23.7 sec HDFS Read: 55476181 HDFS Write: 246 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 33 seconds 810 msec -OK -Time taken: 66.769 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3462@mturlrep13_201309102039_1149451168.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3871@mturlrep13_201309102039_1735915084.txt -hive> SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0173 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:39:48,055 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:39:56,086 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:39:58,100 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-10 20:39:59,108 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-10 20:40:00,116 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-10 20:40:01,122 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-10 20:40:02,128 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-10 20:40:03,135 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-10 20:40:04,141 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-10 20:40:05,147 Stage-1 map = 72%, reduce = 8%, Cumulative CPU 27.32 sec -2013-09-10 20:40:06,152 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.32 sec -2013-09-10 20:40:07,158 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 39.83 sec -2013-09-10 20:40:08,163 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.14 sec -2013-09-10 20:40:09,168 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.14 sec -2013-09-10 20:40:10,174 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.14 sec -2013-09-10 20:40:11,179 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.14 sec -2013-09-10 20:40:12,185 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.14 sec -2013-09-10 20:40:13,190 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.14 sec -2013-09-10 20:40:14,196 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 55.14 sec -2013-09-10 20:40:15,218 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.62 sec -2013-09-10 20:40:16,223 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.62 sec -2013-09-10 20:40:17,229 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.62 sec -MapReduce Total cumulative CPU time: 1 minutes 9 seconds 620 msec -Ended Job = job_201309101627_0173 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0174 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:40:19,719 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:40:30,752 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:40:34,764 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 20:40:35,768 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 20:40:36,772 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 20:40:37,776 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 20:40:38,781 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 20:40:39,785 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 20:40:40,790 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 20:40:41,807 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.83 sec -2013-09-10 20:40:42,811 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.83 sec -2013-09-10 20:40:43,816 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.17 sec -2013-09-10 20:40:44,820 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.17 sec -MapReduce Total cumulative CPU time: 23 seconds 170 msec -Ended Job = job_201309101627_0174 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 69.62 sec HDFS Read: 57312623 HDFS Write: 55475412 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 23.17 sec HDFS Read: 55476181 HDFS Write: 246 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 32 seconds 790 msec -OK -Time taken: 64.078 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6839@mturlrep13_201309102040_775437631.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7242@mturlrep13_201309102040_2081233093.txt -hive> SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0175 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:40:57,539 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:41:05,573 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:41:07,588 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.59 sec -2013-09-10 20:41:08,596 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.59 sec -2013-09-10 20:41:09,603 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.59 sec -2013-09-10 20:41:10,609 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.59 sec -2013-09-10 20:41:11,616 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.59 sec -2013-09-10 20:41:12,623 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.59 sec -2013-09-10 20:41:13,629 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.59 sec -2013-09-10 20:41:14,635 Stage-1 map = 96%, reduce = 8%, Cumulative CPU 26.59 sec -2013-09-10 20:41:15,641 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 26.59 sec -2013-09-10 20:41:16,647 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 39.34 sec -2013-09-10 20:41:17,653 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.36 sec -2013-09-10 20:41:18,658 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.36 sec -2013-09-10 20:41:19,664 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.36 sec -2013-09-10 20:41:20,670 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.36 sec -2013-09-10 20:41:21,677 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.36 sec -2013-09-10 20:41:22,683 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.36 sec -2013-09-10 20:41:23,689 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 54.36 sec -2013-09-10 20:41:24,697 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.52 sec -2013-09-10 20:41:25,704 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.52 sec -2013-09-10 20:41:26,709 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.52 sec -MapReduce Total cumulative CPU time: 1 minutes 8 seconds 520 msec -Ended Job = job_201309101627_0175 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0176 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:41:29,212 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:41:40,246 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:41:44,259 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.47 sec -2013-09-10 20:41:45,264 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.47 sec -2013-09-10 20:41:46,268 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.47 sec -2013-09-10 20:41:47,273 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.47 sec -2013-09-10 20:41:48,278 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.47 sec -2013-09-10 20:41:49,283 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.47 sec -2013-09-10 20:41:50,287 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.47 sec -2013-09-10 20:41:51,293 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.47 sec -2013-09-10 20:41:52,298 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.47 sec -2013-09-10 20:41:53,303 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.87 sec -2013-09-10 20:41:54,309 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.87 sec -MapReduce Total cumulative CPU time: 22 seconds 870 msec -Ended Job = job_201309101627_0176 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 68.52 sec HDFS Read: 57312623 HDFS Write: 55475412 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 22.87 sec HDFS Read: 55476181 HDFS Write: 246 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 31 seconds 390 msec -OK -Time taken: 63.944 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9222@mturlrep13_201309102042_1508377166.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9653@mturlrep13_201309102042_1395015461.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0177 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:42:17,156 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:42:24,184 Stage-1 map = 36%, reduce = 0% -2013-09-10 20:42:27,198 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:42:29,212 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 16.85 sec -2013-09-10 20:42:30,219 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.6 sec -2013-09-10 20:42:31,227 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.6 sec -2013-09-10 20:42:32,234 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.6 sec -2013-09-10 20:42:33,240 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.6 sec -2013-09-10 20:42:34,246 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.6 sec -2013-09-10 20:42:35,251 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.6 sec -2013-09-10 20:42:36,257 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.6 sec -2013-09-10 20:42:37,262 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.6 sec -2013-09-10 20:42:38,272 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.6 sec -2013-09-10 20:42:39,277 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.6 sec -2013-09-10 20:42:40,283 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.6 sec -2013-09-10 20:42:41,289 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.6 sec -2013-09-10 20:42:42,295 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.15 sec -2013-09-10 20:42:43,301 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.15 sec -2013-09-10 20:42:44,306 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.15 sec -2013-09-10 20:42:45,312 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.15 sec -2013-09-10 20:42:46,318 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.15 sec -2013-09-10 20:42:47,324 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.15 sec -2013-09-10 20:42:48,329 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.15 sec -2013-09-10 20:42:49,336 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.15 sec -2013-09-10 20:42:50,341 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.15 sec -2013-09-10 20:42:51,347 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.15 sec -2013-09-10 20:42:52,352 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 68.15 sec -2013-09-10 20:42:53,360 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 84.75 sec -2013-09-10 20:42:54,366 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.79 sec -2013-09-10 20:42:55,371 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.79 sec -2013-09-10 20:42:56,377 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.79 sec -MapReduce Total cumulative CPU time: 1 minutes 29 seconds 790 msec -Ended Job = job_201309101627_0177 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0178 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:42:58,964 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:43:10,003 Stage-2 map = 46%, reduce = 0% -2013-09-10 20:43:13,012 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:43:16,023 Stage-2 map = 96%, reduce = 0% -2013-09-10 20:43:18,031 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.46 sec -2013-09-10 20:43:19,036 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.46 sec -2013-09-10 20:43:20,041 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.46 sec -2013-09-10 20:43:21,046 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.46 sec -2013-09-10 20:43:22,050 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.46 sec -2013-09-10 20:43:23,055 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.46 sec -2013-09-10 20:43:24,059 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.46 sec -2013-09-10 20:43:25,063 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.46 sec -2013-09-10 20:43:26,067 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.46 sec -2013-09-10 20:43:27,072 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.46 sec -2013-09-10 20:43:28,077 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.87 sec -2013-09-10 20:43:29,082 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.87 sec -2013-09-10 20:43:30,087 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.87 sec -MapReduce Total cumulative CPU time: 28 seconds 870 msec -Ended Job = job_201309101627_0178 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 89.79 sec HDFS Read: 84536695 HDFS Write: 146202868 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 28.87 sec HDFS Read: 146210123 HDFS Write: 256 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 58 seconds 660 msec -OK -Time taken: 82.627 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11739@mturlrep13_201309102043_755615207.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12158@mturlrep13_201309102043_667274494.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0179 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:43:44,421 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:43:51,451 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:43:55,477 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.5 sec -2013-09-10 20:43:56,485 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.5 sec -2013-09-10 20:43:57,492 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.5 sec -2013-09-10 20:43:58,500 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.5 sec -2013-09-10 20:43:59,508 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.5 sec -2013-09-10 20:44:00,514 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.5 sec -2013-09-10 20:44:01,521 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.5 sec -2013-09-10 20:44:02,527 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.5 sec -2013-09-10 20:44:03,534 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 33.5 sec -2013-09-10 20:44:04,541 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 33.5 sec -2013-09-10 20:44:05,547 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 33.5 sec -2013-09-10 20:44:06,553 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.5 sec -2013-09-10 20:44:07,559 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 49.48 sec -2013-09-10 20:44:08,565 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.45 sec -2013-09-10 20:44:09,570 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.45 sec -2013-09-10 20:44:10,575 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.45 sec -2013-09-10 20:44:11,580 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.45 sec -2013-09-10 20:44:12,586 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.45 sec -2013-09-10 20:44:13,594 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.45 sec -2013-09-10 20:44:14,600 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.45 sec -2013-09-10 20:44:15,606 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.45 sec -2013-09-10 20:44:16,612 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.45 sec -2013-09-10 20:44:17,618 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.45 sec -2013-09-10 20:44:18,624 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 67.45 sec -2013-09-10 20:44:19,630 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 67.45 sec -2013-09-10 20:44:20,638 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.72 sec -2013-09-10 20:44:21,644 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.72 sec -2013-09-10 20:44:22,650 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.72 sec -MapReduce Total cumulative CPU time: 1 minutes 29 seconds 720 msec -Ended Job = job_201309101627_0179 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0180 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:44:25,163 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:44:36,195 Stage-2 map = 46%, reduce = 0% -2013-09-10 20:44:39,205 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:44:42,214 Stage-2 map = 96%, reduce = 0% -2013-09-10 20:44:44,222 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-10 20:44:45,227 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-10 20:44:46,231 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-10 20:44:47,236 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-10 20:44:48,240 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-10 20:44:49,244 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-10 20:44:50,249 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-10 20:44:51,254 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.57 sec -2013-09-10 20:44:52,259 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.57 sec -2013-09-10 20:44:53,265 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.62 sec -2013-09-10 20:44:54,271 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.03 sec -2013-09-10 20:44:55,276 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.03 sec -2013-09-10 20:44:56,282 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.03 sec -MapReduce Total cumulative CPU time: 28 seconds 30 msec -Ended Job = job_201309101627_0180 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 89.72 sec HDFS Read: 84536695 HDFS Write: 146202868 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 28.03 sec HDFS Read: 146210123 HDFS Write: 256 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 57 seconds 750 msec -OK -Time taken: 80.369 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14245@mturlrep13_201309102044_1237091624.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14650@mturlrep13_201309102045_372590589.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0181 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:45:10,380 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:45:17,409 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:45:22,436 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.02 sec -2013-09-10 20:45:23,444 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.02 sec -2013-09-10 20:45:24,451 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.02 sec -2013-09-10 20:45:25,459 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.02 sec -2013-09-10 20:45:26,465 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.02 sec -2013-09-10 20:45:27,472 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.02 sec -2013-09-10 20:45:28,478 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.02 sec -2013-09-10 20:45:29,486 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 34.02 sec -2013-09-10 20:45:30,493 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 34.02 sec -2013-09-10 20:45:31,499 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 34.02 sec -2013-09-10 20:45:32,505 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.02 sec -2013-09-10 20:45:33,510 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 50.52 sec -2013-09-10 20:45:34,516 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.88 sec -2013-09-10 20:45:35,521 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.88 sec -2013-09-10 20:45:36,526 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.88 sec -2013-09-10 20:45:37,531 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.88 sec -2013-09-10 20:45:38,537 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.88 sec -2013-09-10 20:45:39,543 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.88 sec -2013-09-10 20:45:40,549 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.88 sec -2013-09-10 20:45:41,555 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.88 sec -2013-09-10 20:45:42,561 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.88 sec -2013-09-10 20:45:43,566 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.88 sec -2013-09-10 20:45:44,572 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 68.88 sec -2013-09-10 20:45:45,578 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 68.88 sec -2013-09-10 20:45:46,585 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.92 sec -2013-09-10 20:45:47,591 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.92 sec -2013-09-10 20:45:48,597 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.92 sec -MapReduce Total cumulative CPU time: 1 minutes 29 seconds 920 msec -Ended Job = job_201309101627_0181 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0182 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:45:52,108 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:46:02,139 Stage-2 map = 46%, reduce = 0% -2013-09-10 20:46:05,147 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:46:08,156 Stage-2 map = 96%, reduce = 0% -2013-09-10 20:46:10,163 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.28 sec -2013-09-10 20:46:11,168 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.28 sec -2013-09-10 20:46:12,173 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.28 sec -2013-09-10 20:46:13,177 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.28 sec -2013-09-10 20:46:14,181 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.28 sec -2013-09-10 20:46:15,186 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.28 sec -2013-09-10 20:46:16,190 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.28 sec -2013-09-10 20:46:17,195 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.28 sec -2013-09-10 20:46:18,200 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.28 sec -2013-09-10 20:46:19,205 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.28 sec -2013-09-10 20:46:20,210 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.28 sec -2013-09-10 20:46:21,215 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.79 sec -2013-09-10 20:46:22,220 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.79 sec -MapReduce Total cumulative CPU time: 28 seconds 790 msec -Ended Job = job_201309101627_0182 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 89.92 sec HDFS Read: 84536695 HDFS Write: 146202868 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 28.79 sec HDFS Read: 146210123 HDFS Write: 256 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 58 seconds 710 msec -OK -Time taken: 80.288 seconds, Fetched: 10 row(s) -hive> quit; --- ещё более сложная агрегация.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17569@mturlrep13_201309102046_1421699291.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17985@mturlrep13_201309102046_1356986531.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0183 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:46:44,167 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:46:51,195 Stage-1 map = 32%, reduce = 0% -2013-09-10 20:46:54,211 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 18.03 sec -2013-09-10 20:46:55,217 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 18.03 sec -2013-09-10 20:46:56,226 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 26.65 sec -2013-09-10 20:46:57,232 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.42 sec -2013-09-10 20:46:58,239 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.42 sec -2013-09-10 20:46:59,246 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.42 sec -2013-09-10 20:47:00,252 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.42 sec -2013-09-10 20:47:01,259 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.42 sec -2013-09-10 20:47:02,265 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.42 sec -2013-09-10 20:47:03,271 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.42 sec -2013-09-10 20:47:04,276 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 33.42 sec -2013-09-10 20:47:05,281 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 33.42 sec -2013-09-10 20:47:06,287 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 33.42 sec -2013-09-10 20:47:07,292 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.42 sec -2013-09-10 20:47:08,297 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.42 sec -2013-09-10 20:47:09,302 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.29 sec -2013-09-10 20:47:10,307 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.29 sec -2013-09-10 20:47:11,312 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.29 sec -2013-09-10 20:47:12,317 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.29 sec -2013-09-10 20:47:13,323 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.29 sec -2013-09-10 20:47:14,328 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.29 sec -2013-09-10 20:47:15,333 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 66.29 sec -2013-09-10 20:47:16,340 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 75.8 sec -2013-09-10 20:47:17,346 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 75.8 sec -2013-09-10 20:47:18,352 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 75.8 sec -MapReduce Total cumulative CPU time: 1 minutes 15 seconds 800 msec -Ended Job = job_201309101627_0183 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 75.8 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 15 seconds 800 msec -OK -Time taken: 44.05 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19485@mturlrep13_201309102047_352390381.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19906@mturlrep13_201309102047_307105282.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0184 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:47:32,371 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:47:39,398 Stage-1 map = 39%, reduce = 0% -2013-09-10 20:47:42,409 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:47:43,421 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 17.21 sec -2013-09-10 20:47:44,427 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.34 sec -2013-09-10 20:47:45,435 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.34 sec -2013-09-10 20:47:46,441 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.34 sec -2013-09-10 20:47:47,448 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.34 sec -2013-09-10 20:47:48,454 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.34 sec -2013-09-10 20:47:49,460 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.34 sec -2013-09-10 20:47:50,465 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.34 sec -2013-09-10 20:47:51,470 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 35.34 sec -2013-09-10 20:47:52,476 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 35.34 sec -2013-09-10 20:47:53,483 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 55.87 sec -2013-09-10 20:47:54,488 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 55.87 sec -2013-09-10 20:47:55,494 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 60.92 sec -2013-09-10 20:47:56,499 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.81 sec -2013-09-10 20:47:57,504 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.81 sec -2013-09-10 20:47:58,508 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.81 sec -2013-09-10 20:47:59,513 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.81 sec -2013-09-10 20:48:00,519 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.81 sec -2013-09-10 20:48:01,524 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.81 sec -2013-09-10 20:48:02,530 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.81 sec -2013-09-10 20:48:03,537 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.42 sec -2013-09-10 20:48:04,544 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.42 sec -2013-09-10 20:48:05,549 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.42 sec -MapReduce Total cumulative CPU time: 1 minutes 17 seconds 420 msec -Ended Job = job_201309101627_0184 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 77.42 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 420 msec -OK -Time taken: 41.574 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21404@mturlrep13_201309102048_126554804.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21810@mturlrep13_201309102048_1454592884.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0185 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:48:18,529 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:48:26,558 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:48:30,578 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.15 sec -2013-09-10 20:48:31,586 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.15 sec -2013-09-10 20:48:32,594 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.15 sec -2013-09-10 20:48:33,601 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.15 sec -2013-09-10 20:48:34,606 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.15 sec -2013-09-10 20:48:35,613 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.15 sec -2013-09-10 20:48:36,618 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.15 sec -2013-09-10 20:48:37,623 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.15 sec -2013-09-10 20:48:38,632 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 35.15 sec -2013-09-10 20:48:39,638 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 35.15 sec -2013-09-10 20:48:40,654 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 35.15 sec -2013-09-10 20:48:41,660 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 35.15 sec -2013-09-10 20:48:42,665 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 50.57 sec -2013-09-10 20:48:43,670 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.89 sec -2013-09-10 20:48:44,675 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.89 sec -2013-09-10 20:48:45,680 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.89 sec -2013-09-10 20:48:46,690 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.89 sec -2013-09-10 20:48:47,695 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 68.89 sec -2013-09-10 20:48:48,701 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 68.89 sec -2013-09-10 20:48:49,706 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 68.89 sec -2013-09-10 20:48:50,711 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 68.89 sec -2013-09-10 20:48:51,718 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 78.48 sec -2013-09-10 20:48:52,724 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 78.48 sec -MapReduce Total cumulative CPU time: 1 minutes 18 seconds 480 msec -Ended Job = job_201309101627_0185 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 78.48 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 18 seconds 480 msec -OK -Time taken: 41.516 seconds, Fetched: 10 row(s) -hive> quit; --- то же самое, но без сортировки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23317@mturlrep13_201309102048_633775434.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23745@mturlrep13_201309102049_424444339.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0186 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:49:13,392 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:49:20,429 Stage-1 map = 7%, reduce = 0% -2013-09-10 20:49:23,442 Stage-1 map = 22%, reduce = 0% -2013-09-10 20:49:26,455 Stage-1 map = 29%, reduce = 0% -2013-09-10 20:49:29,469 Stage-1 map = 36%, reduce = 0% -2013-09-10 20:49:32,480 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:49:34,495 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.74 sec -2013-09-10 20:49:35,502 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.74 sec -2013-09-10 20:49:36,510 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.74 sec -2013-09-10 20:49:37,517 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.74 sec -2013-09-10 20:49:38,523 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.74 sec -2013-09-10 20:49:39,529 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.74 sec -2013-09-10 20:49:40,534 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.74 sec -2013-09-10 20:49:41,540 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:42,545 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:43,551 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:44,557 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:45,562 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:46,567 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:47,571 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:48,576 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:49,581 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:50,585 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:51,590 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:52,596 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 54.74 sec -2013-09-10 20:49:53,603 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 104.41 sec -2013-09-10 20:49:54,608 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 109.4 sec -2013-09-10 20:49:55,613 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 109.4 sec -2013-09-10 20:49:56,619 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 109.4 sec -2013-09-10 20:49:57,624 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 109.4 sec -2013-09-10 20:49:58,630 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 109.4 sec -2013-09-10 20:49:59,636 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 109.4 sec -2013-09-10 20:50:00,642 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 109.4 sec -2013-09-10 20:50:01,648 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 109.4 sec -2013-09-10 20:50:02,654 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 109.4 sec -2013-09-10 20:50:03,660 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 109.4 sec -2013-09-10 20:50:04,665 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 109.4 sec -2013-09-10 20:50:06,204 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 109.4 sec -2013-09-10 20:50:07,210 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 109.4 sec -2013-09-10 20:50:08,216 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 109.4 sec -2013-09-10 20:50:09,222 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 109.4 sec -2013-09-10 20:50:10,228 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 109.4 sec -2013-09-10 20:50:11,233 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 109.4 sec -2013-09-10 20:50:12,238 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 109.4 sec -2013-09-10 20:50:13,245 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 109.4 sec -2013-09-10 20:50:14,251 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 128.24 sec -2013-09-10 20:50:15,258 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.79 sec -2013-09-10 20:50:16,263 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.79 sec -MapReduce Total cumulative CPU time: 2 minutes 27 seconds 790 msec -Ended Job = job_201309101627_0186 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0187 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:50:19,831 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:50:32,871 Stage-2 map = 28%, reduce = 0% -2013-09-10 20:50:38,890 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:50:48,919 Stage-2 map = 78%, reduce = 0% -2013-09-10 20:50:56,946 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.44 sec -2013-09-10 20:50:57,951 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.44 sec -2013-09-10 20:50:58,956 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.44 sec -2013-09-10 20:50:59,960 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.44 sec -2013-09-10 20:51:00,964 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.44 sec -2013-09-10 20:51:01,968 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.44 sec -2013-09-10 20:51:02,972 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.44 sec -2013-09-10 20:51:03,977 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.44 sec -2013-09-10 20:51:04,980 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.44 sec -2013-09-10 20:51:05,984 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.44 sec -2013-09-10 20:51:06,989 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 43.44 sec -2013-09-10 20:51:07,994 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.97 sec -2013-09-10 20:51:08,999 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.97 sec -2013-09-10 20:51:10,004 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.97 sec -MapReduce Total cumulative CPU time: 49 seconds 970 msec -Ended Job = job_201309101627_0187 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 147.79 sec HDFS Read: 84944733 HDFS Write: 241346048 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 49.97 sec HDFS Read: 241349358 HDFS Write: 268 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 17 seconds 760 msec -OK -Time taken: 126.359 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26780@mturlrep13_201309102051_75093307.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27199@mturlrep13_201309102051_564952806.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0188 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:51:23,085 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:51:31,119 Stage-1 map = 11%, reduce = 0% -2013-09-10 20:51:34,132 Stage-1 map = 22%, reduce = 0% -2013-09-10 20:51:37,144 Stage-1 map = 29%, reduce = 0% -2013-09-10 20:51:40,158 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:51:43,177 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.96 sec -2013-09-10 20:51:44,183 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.96 sec -2013-09-10 20:51:45,190 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.96 sec -2013-09-10 20:51:46,196 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.96 sec -2013-09-10 20:51:47,202 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.96 sec -2013-09-10 20:51:48,230 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.96 sec -2013-09-10 20:51:49,236 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.96 sec -2013-09-10 20:51:50,242 Stage-1 map = 54%, reduce = 8%, Cumulative CPU 52.96 sec -2013-09-10 20:51:51,247 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 52.96 sec -2013-09-10 20:51:52,253 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 52.96 sec -2013-09-10 20:51:53,259 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 81.09 sec -2013-09-10 20:51:54,265 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 81.09 sec -2013-09-10 20:51:55,271 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 81.09 sec -2013-09-10 20:51:56,276 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 81.09 sec -2013-09-10 20:51:57,282 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 81.09 sec -2013-09-10 20:51:58,288 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 81.09 sec -2013-09-10 20:51:59,294 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 81.09 sec -2013-09-10 20:52:00,300 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 81.09 sec -2013-09-10 20:52:01,305 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 81.09 sec -2013-09-10 20:52:02,311 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 95.63 sec -2013-09-10 20:52:03,315 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 95.63 sec -2013-09-10 20:52:04,320 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 109.67 sec -2013-09-10 20:52:05,325 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 109.67 sec -2013-09-10 20:52:06,330 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 109.67 sec -2013-09-10 20:52:07,334 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 109.67 sec -2013-09-10 20:52:08,339 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 109.67 sec -2013-09-10 20:52:09,344 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 109.67 sec -2013-09-10 20:52:10,349 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 109.67 sec -2013-09-10 20:52:11,355 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 109.67 sec -2013-09-10 20:52:12,360 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 109.67 sec -2013-09-10 20:52:13,366 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 109.67 sec -2013-09-10 20:52:14,371 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 109.67 sec -2013-09-10 20:52:15,377 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 109.67 sec -2013-09-10 20:52:16,382 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 109.67 sec -2013-09-10 20:52:17,388 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 109.67 sec -2013-09-10 20:52:18,393 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 109.67 sec -2013-09-10 20:52:19,399 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 109.67 sec -2013-09-10 20:52:20,404 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 109.67 sec -2013-09-10 20:52:21,410 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 109.67 sec -2013-09-10 20:52:22,417 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 148.66 sec -2013-09-10 20:52:23,428 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 148.66 sec -2013-09-10 20:52:24,433 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 148.66 sec -MapReduce Total cumulative CPU time: 2 minutes 28 seconds 660 msec -Ended Job = job_201309101627_0188 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0189 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:52:27,956 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:52:40,999 Stage-2 map = 28%, reduce = 0% -2013-09-10 20:52:47,512 Stage-2 map = 50%, reduce = 0% -2013-09-10 20:52:53,533 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 31.33 sec -2013-09-10 20:52:54,538 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 31.33 sec -2013-09-10 20:52:55,543 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 31.33 sec -2013-09-10 20:52:56,548 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 31.33 sec -2013-09-10 20:52:57,552 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 31.33 sec -2013-09-10 20:52:58,557 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 31.33 sec -2013-09-10 20:52:59,561 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 31.33 sec -2013-09-10 20:53:00,565 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 31.33 sec -2013-09-10 20:53:01,569 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 31.33 sec -2013-09-10 20:53:02,573 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 31.33 sec -2013-09-10 20:53:03,577 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:04,582 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:05,587 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:06,591 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:07,596 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:08,600 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:09,604 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:10,609 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:11,613 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:12,617 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:13,621 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.09 sec -2013-09-10 20:53:14,636 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 43.09 sec -2013-09-10 20:53:15,641 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.5 sec -2013-09-10 20:53:16,645 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.5 sec -2013-09-10 20:53:17,650 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.5 sec -MapReduce Total cumulative CPU time: 49 seconds 500 msec -Ended Job = job_201309101627_0189 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 148.66 sec HDFS Read: 84944733 HDFS Write: 241346048 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 49.5 sec HDFS Read: 241349358 HDFS Write: 268 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 18 seconds 160 msec -OK -Time taken: 121.853 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29500@mturlrep13_201309102053_2142354545.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29928@mturlrep13_201309102053_368055123.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0190 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:53:30,426 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:53:38,459 Stage-1 map = 14%, reduce = 0% -2013-09-10 20:53:41,471 Stage-1 map = 22%, reduce = 0% -2013-09-10 20:53:44,485 Stage-1 map = 32%, reduce = 0% -2013-09-10 20:53:47,498 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:53:50,516 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.53 sec -2013-09-10 20:53:51,523 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.53 sec -2013-09-10 20:53:52,538 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.53 sec -2013-09-10 20:53:53,544 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.53 sec -2013-09-10 20:53:54,550 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.53 sec -2013-09-10 20:53:55,556 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.53 sec -2013-09-10 20:53:56,562 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.53 sec -2013-09-10 20:53:57,568 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 52.53 sec -2013-09-10 20:53:58,574 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 52.53 sec -2013-09-10 20:53:59,579 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 52.53 sec -2013-09-10 20:54:00,585 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 52.53 sec -2013-09-10 20:54:01,592 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 52.53 sec -2013-09-10 20:54:02,598 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 52.53 sec -2013-09-10 20:54:03,604 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 52.53 sec -2013-09-10 20:54:04,610 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 52.53 sec -2013-09-10 20:54:05,616 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 52.53 sec -2013-09-10 20:54:06,622 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 52.53 sec -2013-09-10 20:54:07,628 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 52.53 sec -2013-09-10 20:54:08,634 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 77.54 sec -2013-09-10 20:54:09,640 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 77.54 sec -2013-09-10 20:54:10,645 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 104.77 sec -2013-09-10 20:54:11,650 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 104.77 sec -2013-09-10 20:54:12,655 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 104.77 sec -2013-09-10 20:54:13,661 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 104.77 sec -2013-09-10 20:54:14,666 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 104.77 sec -2013-09-10 20:54:15,671 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 104.77 sec -2013-09-10 20:54:16,677 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 104.77 sec -2013-09-10 20:54:17,682 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 104.77 sec -2013-09-10 20:54:18,688 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 104.77 sec -2013-09-10 20:54:19,693 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 104.77 sec -2013-09-10 20:54:20,699 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 104.77 sec -2013-09-10 20:54:21,704 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 104.77 sec -2013-09-10 20:54:23,614 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 104.77 sec -2013-09-10 20:54:24,620 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 104.77 sec -2013-09-10 20:54:25,626 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 104.77 sec -2013-09-10 20:54:26,632 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 104.77 sec -2013-09-10 20:54:27,638 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 104.77 sec -2013-09-10 20:54:28,643 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 104.77 sec -2013-09-10 20:54:29,649 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 104.77 sec -2013-09-10 20:54:30,656 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 146.68 sec -2013-09-10 20:54:31,662 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 146.68 sec -MapReduce Total cumulative CPU time: 2 minutes 26 seconds 680 msec -Ended Job = job_201309101627_0190 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0191 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:54:35,128 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:54:48,165 Stage-2 map = 28%, reduce = 0% -2013-09-10 20:54:54,183 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:54:55,188 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:54:56,192 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:54:57,197 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:54:58,202 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:54:59,207 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:00,211 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:01,216 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:02,220 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:03,224 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:04,229 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:05,233 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:06,238 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:07,242 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:08,247 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:09,251 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:10,256 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 20:55:11,261 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.54 sec -2013-09-10 20:55:12,265 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.54 sec -2013-09-10 20:55:13,270 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.54 sec -2013-09-10 20:55:14,274 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.54 sec -2013-09-10 20:55:15,286 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.54 sec -2013-09-10 20:55:16,291 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.54 sec -2013-09-10 20:55:17,296 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.54 sec -2013-09-10 20:55:18,301 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.54 sec -2013-09-10 20:55:19,307 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.54 sec -2013-09-10 20:55:20,311 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.54 sec -2013-09-10 20:55:21,316 Stage-2 map = 100%, reduce = 69%, Cumulative CPU 42.54 sec -2013-09-10 20:55:22,321 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 48.86 sec -2013-09-10 20:55:23,326 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 48.86 sec -2013-09-10 20:55:24,330 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 48.86 sec -MapReduce Total cumulative CPU time: 48 seconds 860 msec -Ended Job = job_201309101627_0191 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 146.68 sec HDFS Read: 84944733 HDFS Write: 241346048 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 48.86 sec HDFS Read: 241349358 HDFS Write: 268 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 15 seconds 540 msec -OK -Time taken: 121.047 seconds, Fetched: 10 row(s) -hive> quit; --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_506@mturlrep13_201309102055_4203966.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_956@mturlrep13_201309102055_26161103.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0192 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-10 20:55:43,857 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:55:48,887 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.59 sec -2013-09-10 20:55:49,894 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.59 sec -2013-09-10 20:55:50,902 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.59 sec -2013-09-10 20:55:51,908 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.59 sec -2013-09-10 20:55:52,914 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.59 sec -2013-09-10 20:55:53,920 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.27 sec -2013-09-10 20:55:54,925 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.27 sec -2013-09-10 20:55:55,931 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 15.27 sec -MapReduce Total cumulative CPU time: 15 seconds 270 msec -Ended Job = job_201309101627_0192 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 15.27 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 15 seconds 270 msec -OK -Time taken: 21.635 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2371@mturlrep13_201309102055_813101102.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2777@mturlrep13_201309102056_2011806893.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0193 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-10 20:56:09,574 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:56:13,596 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.59 sec -2013-09-10 20:56:14,604 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.59 sec -2013-09-10 20:56:15,610 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.59 sec -2013-09-10 20:56:16,616 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.59 sec -2013-09-10 20:56:17,621 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 11.18 sec -2013-09-10 20:56:18,626 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.35 sec -2013-09-10 20:56:19,632 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 15.35 sec -MapReduce Total cumulative CPU time: 15 seconds 350 msec -Ended Job = job_201309101627_0193 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 15.35 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 15 seconds 350 msec -OK -Time taken: 18.086 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3888@mturlrep13_201309102056_1592116097.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4301@mturlrep13_201309102056_499033559.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0194 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-10 20:56:32,600 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:56:37,626 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.67 sec -2013-09-10 20:56:38,633 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.67 sec -2013-09-10 20:56:39,640 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.67 sec -2013-09-10 20:56:40,645 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.67 sec -2013-09-10 20:56:41,651 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.06 sec -2013-09-10 20:56:42,656 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.06 sec -2013-09-10 20:56:43,661 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 15.06 sec -MapReduce Total cumulative CPU time: 15 seconds 60 msec -Ended Job = job_201309101627_0194 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 15.06 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 15 seconds 60 msec -OK -Time taken: 18.296 seconds -hive> quit; --- мощная фильтрация по столбцу типа UInt64.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5720@mturlrep13_201309102056_94435251.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6130@mturlrep13_201309102056_174777183.txt -hive> SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0195 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:57:03,409 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:57:10,436 Stage-1 map = 43%, reduce = 0% -2013-09-10 20:57:11,449 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.79 sec -2013-09-10 20:57:12,456 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.79 sec -2013-09-10 20:57:13,463 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.79 sec -2013-09-10 20:57:14,468 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.79 sec -2013-09-10 20:57:15,474 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.79 sec -2013-09-10 20:57:16,480 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.79 sec -2013-09-10 20:57:17,486 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.79 sec -2013-09-10 20:57:18,492 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 17.79 sec -2013-09-10 20:57:19,497 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.67 sec -2013-09-10 20:57:20,502 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.67 sec -2013-09-10 20:57:21,506 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.67 sec -2013-09-10 20:57:22,511 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.67 sec -2013-09-10 20:57:23,517 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.67 sec -2013-09-10 20:57:24,524 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.81 sec -2013-09-10 20:57:25,529 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.81 sec -MapReduce Total cumulative CPU time: 37 seconds 810 msec -Ended Job = job_201309101627_0195 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 37.81 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 37 seconds 810 msec -OK -8428 -Time taken: 32.044 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7386@mturlrep13_201309102057_110374524.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7806@mturlrep13_201309102057_1983544680.txt -hive> SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0196 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:57:38,517 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:57:46,554 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.33 sec -2013-09-10 20:57:47,562 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.33 sec -2013-09-10 20:57:48,570 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.33 sec -2013-09-10 20:57:49,576 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.33 sec -2013-09-10 20:57:50,582 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.33 sec -2013-09-10 20:57:51,588 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.33 sec -2013-09-10 20:57:52,594 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.33 sec -2013-09-10 20:57:53,601 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.87 sec -2013-09-10 20:57:54,605 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.87 sec -2013-09-10 20:57:55,610 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.87 sec -2013-09-10 20:57:56,615 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.87 sec -2013-09-10 20:57:57,620 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.87 sec -2013-09-10 20:57:58,625 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.87 sec -2013-09-10 20:57:59,632 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 36.29 sec -2013-09-10 20:58:00,637 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 36.29 sec -MapReduce Total cumulative CPU time: 36 seconds 290 msec -Ended Job = job_201309101627_0196 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 36.29 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 36 seconds 290 msec -OK -8428 -Time taken: 29.413 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9045@mturlrep13_201309102058_1522105504.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9464@mturlrep13_201309102058_1256270560.txt -hive> SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0197 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 20:58:13,484 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:58:21,525 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.01 sec -2013-09-10 20:58:22,534 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.01 sec -2013-09-10 20:58:23,540 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.01 sec -2013-09-10 20:58:24,546 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.01 sec -2013-09-10 20:58:25,551 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.01 sec -2013-09-10 20:58:26,556 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.01 sec -2013-09-10 20:58:27,562 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.01 sec -2013-09-10 20:58:28,568 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.97 sec -2013-09-10 20:58:29,572 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.97 sec -2013-09-10 20:58:30,577 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.97 sec -2013-09-10 20:58:31,582 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.97 sec -2013-09-10 20:58:32,587 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.97 sec -2013-09-10 20:58:33,591 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.97 sec -2013-09-10 20:58:34,599 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 36.93 sec -2013-09-10 20:58:35,604 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 36.93 sec -MapReduce Total cumulative CPU time: 36 seconds 930 msec -Ended Job = job_201309101627_0197 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 36.93 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 36 seconds 930 msec -OK -8428 -Time taken: 29.432 seconds, Fetched: 1 row(s) -hive> quit; --- фильтрация по поиску подстроки в строке.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10706@mturlrep13_201309102058_1880832226.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11132@mturlrep13_201309102058_286164766.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0198 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:58:55,183 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:59:02,210 Stage-1 map = 36%, reduce = 0% -2013-09-10 20:59:03,222 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 20:59:04,230 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 20:59:05,237 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 20:59:06,243 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 20:59:07,248 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 20:59:08,253 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 20:59:09,258 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 20:59:10,265 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 20:59:11,270 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 19.12 sec -2013-09-10 20:59:12,276 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.14 sec -2013-09-10 20:59:13,281 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.14 sec -2013-09-10 20:59:14,286 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.14 sec -2013-09-10 20:59:15,291 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.14 sec -2013-09-10 20:59:16,296 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.14 sec -2013-09-10 20:59:17,304 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.42 sec -2013-09-10 20:59:18,311 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.42 sec -MapReduce Total cumulative CPU time: 42 seconds 420 msec -Ended Job = job_201309101627_0198 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0199 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 20:59:20,834 Stage-2 map = 0%, reduce = 0% -2013-09-10 20:59:22,842 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-10 20:59:23,847 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-10 20:59:24,852 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-10 20:59:25,856 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-10 20:59:26,860 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-10 20:59:27,865 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-10 20:59:28,870 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-10 20:59:29,875 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.78 sec -2013-09-10 20:59:30,881 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.17 sec -2013-09-10 20:59:31,886 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.17 sec -MapReduce Total cumulative CPU time: 2 seconds 170 msec -Ended Job = job_201309101627_0199 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.42 sec HDFS Read: 136675723 HDFS Write: 5172 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.17 sec HDFS Read: 5941 HDFS Write: 984 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 590 msec -OK -Time taken: 46.47 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13069@mturlrep13_201309102059_1992501004.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13485@mturlrep13_201309102059_1401379353.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0200 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 20:59:44,971 Stage-1 map = 0%, reduce = 0% -2013-09-10 20:59:53,007 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.27 sec -2013-09-10 20:59:54,015 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.27 sec -2013-09-10 20:59:55,023 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.27 sec -2013-09-10 20:59:56,030 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.27 sec -2013-09-10 20:59:57,036 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.27 sec -2013-09-10 20:59:58,042 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.27 sec -2013-09-10 20:59:59,048 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.27 sec -2013-09-10 21:00:00,055 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.27 sec -2013-09-10 21:00:01,061 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.85 sec -2013-09-10 21:00:02,067 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.85 sec -2013-09-10 21:00:03,072 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.85 sec -2013-09-10 21:00:04,078 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.85 sec -2013-09-10 21:00:05,083 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.85 sec -2013-09-10 21:00:06,091 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 38.79 sec -2013-09-10 21:00:07,097 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.76 sec -2013-09-10 21:00:08,103 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.76 sec -MapReduce Total cumulative CPU time: 40 seconds 760 msec -Ended Job = job_201309101627_0200 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0201 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 21:00:10,622 Stage-2 map = 0%, reduce = 0% -2013-09-10 21:00:12,630 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-10 21:00:13,635 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-10 21:00:14,640 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-10 21:00:15,645 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-10 21:00:16,650 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-10 21:00:17,655 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-10 21:00:18,661 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-10 21:00:19,666 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.79 sec -2013-09-10 21:00:20,675 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.17 sec -2013-09-10 21:00:21,680 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.17 sec -MapReduce Total cumulative CPU time: 2 seconds 170 msec -Ended Job = job_201309101627_0201 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.76 sec HDFS Read: 136675723 HDFS Write: 5172 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.17 sec HDFS Read: 5941 HDFS Write: 984 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 930 msec -OK -Time taken: 44.122 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16145@mturlrep13_201309102100_902755262.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16578@mturlrep13_201309102100_20618285.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0202 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 21:00:35,897 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:00:42,923 Stage-1 map = 36%, reduce = 0% -2013-09-10 21:00:43,936 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 21:00:44,944 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 21:00:45,953 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 21:00:46,959 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 21:00:47,965 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 21:00:48,971 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 21:00:49,977 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.83 sec -2013-09-10 21:00:50,984 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.24 sec -2013-09-10 21:00:51,989 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.24 sec -2013-09-10 21:00:52,995 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.24 sec -2013-09-10 21:00:54,000 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-10 21:00:55,006 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-10 21:00:56,012 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-10 21:00:57,019 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.27 sec -2013-09-10 21:00:58,025 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.27 sec -2013-09-10 21:00:59,031 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.27 sec -MapReduce Total cumulative CPU time: 41 seconds 270 msec -Ended Job = job_201309101627_0202 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0203 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 21:01:01,551 Stage-2 map = 0%, reduce = 0% -2013-09-10 21:01:03,560 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:01:04,565 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:01:05,571 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:01:06,576 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:01:07,581 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:01:08,586 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:01:09,591 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:01:10,596 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.74 sec -2013-09-10 21:01:11,602 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -2013-09-10 21:01:12,607 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -2013-09-10 21:01:13,612 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -MapReduce Total cumulative CPU time: 2 seconds 130 msec -Ended Job = job_201309101627_0203 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.27 sec HDFS Read: 136675723 HDFS Write: 5172 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.13 sec HDFS Read: 5941 HDFS Write: 984 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 400 msec -OK -Time taken: 46.136 seconds, Fetched: 10 row(s) -hive> quit; --- вынимаем большие столбцы, фильтрация по строке.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18552@mturlrep13_201309102101_957503081.txt -hive> ; -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18999@mturlrep13_201309102101_604919900.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0204 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 21:01:34,309 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:01:41,339 Stage-1 map = 22%, reduce = 0% -2013-09-10 21:01:44,352 Stage-1 map = 43%, reduce = 0% -2013-09-10 21:01:45,365 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.13 sec -2013-09-10 21:01:46,373 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.13 sec -2013-09-10 21:01:47,381 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.13 sec -2013-09-10 21:01:48,388 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.13 sec -2013-09-10 21:01:49,395 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.13 sec -2013-09-10 21:01:50,402 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.13 sec -2013-09-10 21:01:51,408 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.13 sec -2013-09-10 21:01:52,414 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 24.13 sec -2013-09-10 21:01:53,420 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 24.13 sec -2013-09-10 21:01:54,426 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 43.07 sec -2013-09-10 21:01:55,433 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 46.23 sec -2013-09-10 21:01:56,439 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 50.07 sec -2013-09-10 21:01:57,446 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 50.07 sec -2013-09-10 21:01:58,454 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.11 sec -2013-09-10 21:01:59,460 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.11 sec -2013-09-10 21:02:00,466 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.11 sec -MapReduce Total cumulative CPU time: 53 seconds 110 msec -Ended Job = job_201309101627_0204 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0205 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 21:02:04,015 Stage-2 map = 0%, reduce = 0% -2013-09-10 21:02:05,020 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 21:02:06,026 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 21:02:07,032 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 21:02:08,036 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 21:02:09,041 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 21:02:10,046 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 21:02:11,051 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 21:02:12,056 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-10 21:02:13,077 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -2013-09-10 21:02:14,084 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -2013-09-10 21:02:15,089 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -MapReduce Total cumulative CPU time: 2 seconds 290 msec -Ended Job = job_201309101627_0205 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 53.11 sec HDFS Read: 298803179 HDFS Write: 12221 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.29 sec HDFS Read: 12990 HDFS Write: 2646 SUCCESS -Total MapReduce CPU Time Spent: 55 seconds 400 msec -OK -Time taken: 50.819 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21073@mturlrep13_201309102102_804313256.txt -hive> ; -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21509@mturlrep13_201309102102_1172673920.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0206 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 21:02:29,084 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:02:36,113 Stage-1 map = 29%, reduce = 0% -2013-09-10 21:02:39,134 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.74 sec -2013-09-10 21:02:40,142 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.74 sec -2013-09-10 21:02:41,150 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.74 sec -2013-09-10 21:02:42,156 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.74 sec -2013-09-10 21:02:43,163 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.74 sec -2013-09-10 21:02:44,170 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.74 sec -2013-09-10 21:02:45,176 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.74 sec -2013-09-10 21:02:46,181 Stage-1 map = 64%, reduce = 8%, Cumulative CPU 24.74 sec -2013-09-10 21:02:47,186 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 24.74 sec -2013-09-10 21:02:48,192 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 24.74 sec -2013-09-10 21:02:49,197 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.69 sec -2013-09-10 21:02:50,203 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.69 sec -2013-09-10 21:02:51,208 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.69 sec -2013-09-10 21:02:52,215 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.98 sec -2013-09-10 21:02:53,222 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.98 sec -2013-09-10 21:02:54,227 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.98 sec -MapReduce Total cumulative CPU time: 52 seconds 980 msec -Ended Job = job_201309101627_0206 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0207 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 21:02:58,034 Stage-2 map = 0%, reduce = 0% -2013-09-10 21:02:59,039 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:03:00,045 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:03:01,050 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:03:02,055 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:03:03,061 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:03:04,066 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:03:05,071 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:03:06,095 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-10 21:03:07,100 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -2013-09-10 21:03:08,106 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -2013-09-10 21:03:09,112 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -MapReduce Total cumulative CPU time: 2 seconds 160 msec -Ended Job = job_201309101627_0207 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 52.98 sec HDFS Read: 298803179 HDFS Write: 12221 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.16 sec HDFS Read: 12990 HDFS Write: 2646 SUCCESS -Total MapReduce CPU Time Spent: 55 seconds 140 msec -OK -Time taken: 48.425 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23586@mturlrep13_201309102103_1680692530.txt -hive> ; -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23990@mturlrep13_201309102103_1640399179.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0208 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 21:03:23,428 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:03:30,457 Stage-1 map = 29%, reduce = 0% -2013-09-10 21:03:33,485 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.39 sec -2013-09-10 21:03:34,492 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.39 sec -2013-09-10 21:03:35,500 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.39 sec -2013-09-10 21:03:36,507 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.39 sec -2013-09-10 21:03:37,513 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.39 sec -2013-09-10 21:03:38,521 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.39 sec -2013-09-10 21:03:39,526 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.39 sec -2013-09-10 21:03:40,532 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.39 sec -2013-09-10 21:03:41,538 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 24.39 sec -2013-09-10 21:03:42,544 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 24.39 sec -2013-09-10 21:03:43,550 Stage-1 map = 89%, reduce = 17%, Cumulative CPU 35.95 sec -2013-09-10 21:03:44,556 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.76 sec -2013-09-10 21:03:45,562 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.76 sec -2013-09-10 21:03:46,570 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 50.88 sec -2013-09-10 21:03:47,576 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.9 sec -2013-09-10 21:03:48,582 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.9 sec -MapReduce Total cumulative CPU time: 52 seconds 900 msec -Ended Job = job_201309101627_0208 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0209 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 21:03:52,133 Stage-2 map = 0%, reduce = 0% -2013-09-10 21:03:53,139 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.85 sec -2013-09-10 21:03:54,145 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.85 sec -2013-09-10 21:03:55,151 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.85 sec -2013-09-10 21:03:56,158 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.85 sec -2013-09-10 21:03:57,164 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.85 sec -2013-09-10 21:03:58,169 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.85 sec -2013-09-10 21:03:59,174 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.85 sec -2013-09-10 21:04:00,179 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.85 sec -2013-09-10 21:04:01,185 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -2013-09-10 21:04:02,190 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -2013-09-10 21:04:03,196 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -MapReduce Total cumulative CPU time: 2 seconds 230 msec -Ended Job = job_201309101627_0209 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 52.9 sec HDFS Read: 298803179 HDFS Write: 12221 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.23 sec HDFS Read: 12990 HDFS Write: 2646 SUCCESS -Total MapReduce CPU Time Spent: 55 seconds 130 msec -OK -Time taken: 48.259 seconds, Fetched: 10 row(s) -hive> quit; --- чуть больше столбцы.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26079@mturlrep13_201309102104_979944715.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26503@mturlrep13_201309102104_1111113065.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0210 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 21:04:24,063 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:04:34,103 Stage-1 map = 7%, reduce = 0% -2013-09-10 21:04:37,115 Stage-1 map = 14%, reduce = 0% -2013-09-10 21:04:40,129 Stage-1 map = 22%, reduce = 0% -2013-09-10 21:04:43,157 Stage-1 map = 29%, reduce = 0% -2013-09-10 21:04:49,179 Stage-1 map = 36%, reduce = 0% -2013-09-10 21:04:52,190 Stage-1 map = 43%, reduce = 0% -2013-09-10 21:04:54,206 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 83.32 sec -2013-09-10 21:04:55,213 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 87.85 sec -2013-09-10 21:04:56,219 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 87.85 sec -2013-09-10 21:04:57,225 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 87.85 sec -2013-09-10 21:04:58,230 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 87.85 sec -2013-09-10 21:04:59,241 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 87.85 sec -2013-09-10 21:05:00,250 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 87.85 sec -2013-09-10 21:05:01,256 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 87.85 sec -2013-09-10 21:05:02,262 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:03,268 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:04,274 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:05,302 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:06,307 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:07,312 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:08,317 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:09,322 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:10,328 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:11,333 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:12,338 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:13,343 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:14,348 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:15,354 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:16,367 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:17,373 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:18,377 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:19,382 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:20,387 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:21,392 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:22,397 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 87.85 sec -2013-09-10 21:05:23,401 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 129.86 sec -2013-09-10 21:05:24,406 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 129.86 sec -2013-09-10 21:05:25,411 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 167.87 sec -2013-09-10 21:05:26,416 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 167.87 sec -2013-09-10 21:05:27,422 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 171.01 sec -2013-09-10 21:05:28,427 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 171.01 sec -2013-09-10 21:05:29,433 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 171.01 sec -MapReduce Total cumulative CPU time: 2 minutes 51 seconds 10 msec -Ended Job = job_201309101627_0210 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 171.01 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 51 seconds 10 msec -OK -Time taken: 75.657 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28589@mturlrep13_201309102105_518152082.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28997@mturlrep13_201309102105_1789871872.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0211 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 21:05:43,799 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:05:50,827 Stage-1 map = 7%, reduce = 0% -2013-09-10 21:05:56,854 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:05:57,861 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:05:58,868 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:05:59,891 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:00,897 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:01,902 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:02,907 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:03,912 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:04,917 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:05,922 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:06,927 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:07,932 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:08,937 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:09,942 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:10,947 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 27.49 sec -2013-09-10 21:06:11,956 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.55 sec -2013-09-10 21:06:12,962 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.55 sec -2013-09-10 21:06:13,967 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.55 sec -2013-09-10 21:06:14,973 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.55 sec -2013-09-10 21:06:15,979 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.55 sec -2013-09-10 21:06:16,985 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.55 sec -2013-09-10 21:06:17,991 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.55 sec -2013-09-10 21:06:18,997 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.55 sec -2013-09-10 21:06:20,003 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:21,009 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:22,015 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:23,020 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:24,031 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:25,037 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:26,042 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:27,047 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:28,053 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:29,075 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:30,080 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:31,085 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:32,090 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:33,096 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:34,101 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:35,106 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:36,111 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:37,116 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:38,121 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 74.55 sec -2013-09-10 21:06:39,126 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 114.8 sec -2013-09-10 21:06:40,132 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 114.8 sec -2013-09-10 21:06:41,137 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.73 sec -2013-09-10 21:06:42,142 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.73 sec -2013-09-10 21:06:43,147 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.73 sec -2013-09-10 21:06:44,152 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.73 sec -2013-09-10 21:06:45,158 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.73 sec -2013-09-10 21:06:46,165 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 155.83 sec -2013-09-10 21:06:47,170 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 155.83 sec -MapReduce Total cumulative CPU time: 2 minutes 35 seconds 830 msec -Ended Job = job_201309101627_0211 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 155.83 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 35 seconds 830 msec -OK -Time taken: 72.161 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30422@mturlrep13_201309102106_1368463100.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30864@mturlrep13_201309102106_502369015.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0212 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 21:07:00,709 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:07:11,769 Stage-1 map = 7%, reduce = 0% -2013-09-10 21:07:14,794 Stage-1 map = 14%, reduce = 0% -2013-09-10 21:07:17,807 Stage-1 map = 22%, reduce = 0% -2013-09-10 21:07:20,819 Stage-1 map = 29%, reduce = 0% -2013-09-10 21:07:23,831 Stage-1 map = 32%, reduce = 0% -2013-09-10 21:07:26,842 Stage-1 map = 36%, reduce = 0% -2013-09-10 21:07:29,853 Stage-1 map = 43%, reduce = 0% -2013-09-10 21:07:30,865 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.26 sec -2013-09-10 21:07:31,872 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.26 sec -2013-09-10 21:07:32,877 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.26 sec -2013-09-10 21:07:33,883 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.26 sec -2013-09-10 21:07:34,888 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.26 sec -2013-09-10 21:07:35,893 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.26 sec -2013-09-10 21:07:36,898 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 74.26 sec -2013-09-10 21:07:37,903 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:38,908 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:39,913 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:40,918 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:41,923 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:42,928 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:43,959 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:44,982 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:46,012 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:47,017 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:48,022 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:49,050 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:50,062 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:51,068 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:52,073 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:53,091 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:54,096 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 74.26 sec -2013-09-10 21:07:55,102 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 140.78 sec -2013-09-10 21:07:56,123 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 140.78 sec -2013-09-10 21:07:57,128 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 140.78 sec -2013-09-10 21:07:58,133 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 149.05 sec -2013-09-10 21:07:59,138 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 149.05 sec -2013-09-10 21:08:00,143 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 159.54 sec -2013-09-10 21:08:01,148 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 159.54 sec -2013-09-10 21:08:02,153 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 159.54 sec -2013-09-10 21:08:03,158 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 159.54 sec -2013-09-10 21:08:04,165 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 161.75 sec -2013-09-10 21:08:05,171 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 161.75 sec -2013-09-10 21:08:06,177 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 161.75 sec -MapReduce Total cumulative CPU time: 2 minutes 41 seconds 750 msec -Ended Job = job_201309101627_0212 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 161.75 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 41 seconds 750 msec -OK -Time taken: 73.239 seconds, Fetched: 10 row(s) -hive> quit; --- плохой запрос - вынимаем все столбцы.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32256@mturlrep13_201309102108_1366898564.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32681@mturlrep13_201309102108_1470113641.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0213 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 21:08:27,012 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:08:34,045 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.79 sec -2013-09-10 21:08:35,053 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.79 sec -2013-09-10 21:08:36,060 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.79 sec -2013-09-10 21:08:37,067 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.79 sec -2013-09-10 21:08:38,073 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.79 sec -2013-09-10 21:08:39,078 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.79 sec -2013-09-10 21:08:40,084 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.79 sec -2013-09-10 21:08:41,089 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.79 sec -2013-09-10 21:08:42,096 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 28.06 sec -2013-09-10 21:08:43,100 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.48 sec -2013-09-10 21:08:44,104 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.48 sec -2013-09-10 21:08:45,108 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.48 sec -2013-09-10 21:08:46,113 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.48 sec -2013-09-10 21:08:47,118 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.48 sec -2013-09-10 21:08:48,124 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.48 sec -2013-09-10 21:08:49,130 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.58 sec -2013-09-10 21:08:50,136 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.58 sec -2013-09-10 21:08:51,141 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.58 sec -MapReduce Total cumulative CPU time: 43 seconds 580 msec -Ended Job = job_201309101627_0213 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 43.58 sec HDFS Read: 28228143 HDFS Write: 766 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 580 msec -OK -Time taken: 33.927 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1886@mturlrep13_201309102108_2113013029.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2309@mturlrep13_201309102108_179448674.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0214 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 21:09:04,342 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:09:12,378 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.06 sec -2013-09-10 21:09:13,392 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.06 sec -2013-09-10 21:09:14,406 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.06 sec -2013-09-10 21:09:15,411 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.06 sec -2013-09-10 21:09:16,417 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.06 sec -2013-09-10 21:09:17,422 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.06 sec -2013-09-10 21:09:18,428 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 27.59 sec -2013-09-10 21:09:19,434 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.52 sec -2013-09-10 21:09:20,439 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.52 sec -2013-09-10 21:09:21,444 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.52 sec -2013-09-10 21:09:22,450 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.52 sec -2013-09-10 21:09:23,455 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.52 sec -2013-09-10 21:09:24,460 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.52 sec -2013-09-10 21:09:25,465 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.52 sec -2013-09-10 21:09:26,470 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.52 sec -2013-09-10 21:09:27,476 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.58 sec -2013-09-10 21:09:28,481 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.58 sec -MapReduce Total cumulative CPU time: 42 seconds 580 msec -Ended Job = job_201309101627_0214 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 42.58 sec HDFS Read: 28228143 HDFS Write: 766 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 580 msec -OK -Time taken: 31.299 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3670@mturlrep13_201309102109_1995808280.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4075@mturlrep13_201309102109_1289626769.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0215 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 21:09:42,640 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:09:49,676 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.1 sec -2013-09-10 21:09:50,683 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.1 sec -2013-09-10 21:09:51,690 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.1 sec -2013-09-10 21:09:52,696 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.1 sec -2013-09-10 21:09:53,703 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.1 sec -2013-09-10 21:09:54,710 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.1 sec -2013-09-10 21:09:55,716 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.1 sec -2013-09-10 21:09:56,722 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-10 21:09:57,728 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-10 21:09:58,733 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-10 21:09:59,738 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-10 21:10:00,744 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-10 21:10:01,750 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-10 21:10:02,756 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-10 21:10:03,783 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-10 21:10:04,790 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.5 sec -2013-09-10 21:10:05,796 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.5 sec -MapReduce Total cumulative CPU time: 43 seconds 500 msec -Ended Job = job_201309101627_0215 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 43.5 sec HDFS Read: 28228143 HDFS Write: 766 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 500 msec -OK -Time taken: 31.328 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6065@mturlrep13_201309102110_1165648233.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6867@mturlrep13_201309102110_1247268932.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0216 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 21:10:27,797 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:10:34,833 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.67 sec -2013-09-10 21:10:35,841 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.67 sec -2013-09-10 21:10:36,848 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.67 sec -2013-09-10 21:10:37,854 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.67 sec -2013-09-10 21:10:38,860 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.67 sec -2013-09-10 21:10:39,866 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.67 sec -2013-09-10 21:10:40,872 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.67 sec -2013-09-10 21:10:41,878 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.67 sec -2013-09-10 21:10:42,884 Stage-1 map = 96%, reduce = 8%, Cumulative CPU 19.67 sec -2013-09-10 21:10:43,895 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 39.0 sec -2013-09-10 21:10:44,900 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 39.0 sec -2013-09-10 21:10:45,905 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.0 sec -2013-09-10 21:10:46,910 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.0 sec -2013-09-10 21:10:47,915 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.0 sec -2013-09-10 21:10:48,920 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.0 sec -2013-09-10 21:10:49,926 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.0 sec -2013-09-10 21:10:50,930 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.0 sec -2013-09-10 21:10:51,937 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.08 sec -2013-09-10 21:10:52,943 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.08 sec -MapReduce Total cumulative CPU time: 45 seconds 80 msec -Ended Job = job_201309101627_0216 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 45.08 sec HDFS Read: 27820105 HDFS Write: 666 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 80 msec -OK -ялта интурист -! как одеть трехнедельного ребенка при температуре 20 градусов -! отель rattana beach hotel 3* -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -!( центробежный скважинный калибр форумы) -!(!(storm master silmarils)) -!(!(storm master silmarils)) -!(!(title:(схема sputnik hi 4000))) -!(44-фз о контрактной системе) -Time taken: 35.466 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8346@mturlrep13_201309102110_1245725421.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8754@mturlrep13_201309102110_1375499441.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8981@mturlrep13_201309102111_72221677.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9402@mturlrep13_201309102111_58410451.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0217 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 21:11:13,005 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:11:21,043 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.54 sec -2013-09-10 21:11:22,051 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.54 sec -2013-09-10 21:11:23,058 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.54 sec -2013-09-10 21:11:24,064 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.54 sec -2013-09-10 21:11:25,071 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.54 sec -2013-09-10 21:11:26,077 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.54 sec -2013-09-10 21:11:27,084 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.54 sec -2013-09-10 21:11:28,090 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 28.93 sec -2013-09-10 21:11:29,096 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.94 sec -2013-09-10 21:11:30,101 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.94 sec -2013-09-10 21:11:31,106 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.94 sec -2013-09-10 21:11:32,111 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.94 sec -2013-09-10 21:11:33,116 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.94 sec -2013-09-10 21:11:34,121 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.94 sec -2013-09-10 21:11:35,127 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.94 sec -2013-09-10 21:11:36,142 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.86 sec -2013-09-10 21:11:37,148 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.86 sec -2013-09-10 21:11:38,154 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.86 sec -MapReduce Total cumulative CPU time: 44 seconds 860 msec -Ended Job = job_201309101627_0217 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 44.86 sec HDFS Read: 27820105 HDFS Write: 666 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 860 msec -OK -ялта интурист -! как одеть трехнедельного ребенка при температуре 20 градусов -! отель rattana beach hotel 3* -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -!( центробежный скважинный калибр форумы) -!(!(storm master silmarils)) -!(!(storm master silmarils)) -!(!(title:(схема sputnik hi 4000))) -!(44-фз о контрактной системе) -Time taken: 32.221 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка по строкам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10685@mturlrep13_201309102111_636633590.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11102@mturlrep13_201309102111_1946455694.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0218 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 21:11:58,014 Stage-1 map = 0%, reduce = 0% -2013-09-10 21:12:05,044 Stage-1 map = 43%, reduce = 0% -2013-09-10 21:12:06,056 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.29 sec -2013-09-10 21:12:07,063 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.29 sec -2013-09-10 21:12:08,070 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.29 sec -2013-09-10 21:12:09,076 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.29 sec -2013-09-10 21:12:10,081 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.29 sec -2013-09-10 21:12:11,087 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.29 sec -2013-09-10 21:12:12,093 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.29 sec -2013-09-10 21:12:13,100 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.29 sec -2013-09-10 21:12:14,106 Stage-1 map = 97%, reduce = 8%, Cumulative CPU 30.09 sec -2013-09-10 21:12:15,112 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 40.62 sec -2013-09-10 21:12:16,118 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 40.62 sec -2013-09-10 21:12:17,123 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.62 sec -2013-09-10 21:12:18,128 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.62 sec -2013-09-10 21:12:19,133 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.62 sec -2013-09-10 21:12:20,138 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.62 sec -2013-09-10 21:12:21,143 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.62 sec -2013-09-10 21:12:22,148 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.62 sec -2013-09-10 21:12:23,154 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 47.82 sec -status -spawn hive -restart server: /bin/echo restart -restart diff --git a/benchmark/hive/log/log_10m/log_10m_1 b/benchmark/hive/log/log_10m/log_10m_1 deleted file mode 100644 index 39516540778..00000000000 --- a/benchmark/hive/log/log_10m/log_10m_1 +++ /dev/null @@ -1,1720 +0,0 @@ -start time: Вт. сент. 10 16:45:24 MSK 2013 -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13509@mturlrep13_201309101645_260093116.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13935@mturlrep13_201309101645_1686838746.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0003 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 16:45:44,818 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:45:52,861 Stage-1 map = 7%, reduce = 0% -2013-09-10 16:45:58,920 Stage-1 map = 14%, reduce = 0% -2013-09-10 16:46:01,954 Stage-1 map = 22%, reduce = 0% -2013-09-10 16:46:04,968 Stage-1 map = 29%, reduce = 0% -2013-09-10 16:46:07,983 Stage-1 map = 36%, reduce = 0% -2013-09-10 16:46:10,997 Stage-1 map = 43%, reduce = 0% -2013-09-10 16:46:14,022 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 71.81 sec -2013-09-10 16:46:15,030 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 71.81 sec -2013-09-10 16:46:16,037 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 71.81 sec -2013-09-10 16:46:17,043 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 71.81 sec -2013-09-10 16:46:18,050 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 71.81 sec -2013-09-10 16:46:19,056 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 71.81 sec -2013-09-10 16:46:20,063 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 71.81 sec -2013-09-10 16:46:21,069 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:22,077 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:23,085 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:24,178 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:25,185 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:26,193 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:27,200 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:28,206 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:29,212 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:30,218 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:31,224 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:32,231 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:33,237 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:34,264 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:35,270 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:36,276 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:37,283 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:38,289 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:39,296 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 71.81 sec -2013-09-10 16:46:40,303 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 103.22 sec -2013-09-10 16:46:41,309 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 103.22 sec -2013-09-10 16:46:42,315 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 103.22 sec -2013-09-10 16:46:43,322 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 103.22 sec -2013-09-10 16:46:44,329 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 147.18 sec -2013-09-10 16:46:45,335 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 147.18 sec -2013-09-10 16:46:46,342 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 147.18 sec -2013-09-10 16:46:47,350 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 149.59 sec -2013-09-10 16:46:48,357 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 149.59 sec -MapReduce Total cumulative CPU time: 2 minutes 29 seconds 590 msec -Ended Job = job_201309101627_0003 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 149.59 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 29 seconds 590 msec -OK -10000000 -Time taken: 73.507 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15349@mturlrep13_201309101646_1271191044.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15772@mturlrep13_201309101646_1616941126.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0004 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 16:47:08,378 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:47:13,414 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.82 sec -2013-09-10 16:47:14,424 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.82 sec -2013-09-10 16:47:15,434 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.82 sec -2013-09-10 16:47:16,442 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.82 sec -2013-09-10 16:47:17,449 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.82 sec -2013-09-10 16:47:18,457 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.48 sec -2013-09-10 16:47:19,464 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.48 sec -2013-09-10 16:47:20,471 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 23.48 sec -2013-09-10 16:47:21,481 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.27 sec -2013-09-10 16:47:22,489 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.27 sec -2013-09-10 16:47:23,497 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.27 sec -MapReduce Total cumulative CPU time: 25 seconds 270 msec -Ended Job = job_201309101627_0004 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 25.27 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 25 seconds 270 msec -OK -171127 -Time taken: 25.174 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17086@mturlrep13_201309101647_1352171486.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17518@mturlrep13_201309101647_1464636237.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0005 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 16:47:44,849 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:47:51,893 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.27 sec -2013-09-10 16:47:52,903 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.27 sec -2013-09-10 16:47:53,911 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.27 sec -2013-09-10 16:47:54,919 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.27 sec -2013-09-10 16:47:55,926 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.27 sec -2013-09-10 16:47:56,933 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.27 sec -2013-09-10 16:47:57,941 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.27 sec -2013-09-10 16:47:58,949 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.27 sec -2013-09-10 16:47:59,958 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.8 sec -2013-09-10 16:48:00,965 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.8 sec -2013-09-10 16:48:01,972 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.8 sec -2013-09-10 16:48:02,979 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.8 sec -2013-09-10 16:48:03,986 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.8 sec -2013-09-10 16:48:04,993 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.8 sec -2013-09-10 16:48:06,002 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.82 sec -2013-09-10 16:48:07,009 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.82 sec -MapReduce Total cumulative CPU time: 34 seconds 820 msec -Ended Job = job_201309101627_0005 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 34.82 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 34 seconds 820 msec -OK -Time taken: 32.087 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18864@mturlrep13_201309101648_496452414.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19287@mturlrep13_201309101648_1766379833.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0006 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 16:48:27,840 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:48:34,885 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.61 sec -2013-09-10 16:48:35,895 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.61 sec -2013-09-10 16:48:36,903 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.61 sec -2013-09-10 16:48:37,910 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.61 sec -2013-09-10 16:48:38,917 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.61 sec -2013-09-10 16:48:39,924 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.61 sec -2013-09-10 16:48:40,932 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 22.28 sec -2013-09-10 16:48:41,940 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.04 sec -2013-09-10 16:48:42,947 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.04 sec -2013-09-10 16:48:43,954 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.04 sec -2013-09-10 16:48:44,960 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.04 sec -2013-09-10 16:48:45,967 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.04 sec -2013-09-10 16:48:46,974 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.04 sec -2013-09-10 16:48:47,983 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.88 sec -2013-09-10 16:48:48,990 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.88 sec -MapReduce Total cumulative CPU time: 31 seconds 880 msec -Ended Job = job_201309101627_0006 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 31.88 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 31 seconds 880 msec -OK --4662894107982093709 -Time taken: 31.273 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20537@mturlrep13_201309101648_1262292362.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20975@mturlrep13_201309101648_113339726.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0007 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 16:49:10,173 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:49:17,212 Stage-1 map = 43%, reduce = 0% -2013-09-10 16:49:20,236 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.2 sec -2013-09-10 16:49:21,245 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.2 sec -2013-09-10 16:49:22,252 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.2 sec -2013-09-10 16:49:23,260 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.2 sec -2013-09-10 16:49:24,268 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.2 sec -2013-09-10 16:49:25,276 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.2 sec -2013-09-10 16:49:26,283 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.2 sec -2013-09-10 16:49:27,290 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 26.2 sec -2013-09-10 16:49:28,296 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 26.2 sec -2013-09-10 16:49:29,303 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 26.2 sec -2013-09-10 16:49:30,311 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 39.68 sec -2013-09-10 16:49:31,317 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.63 sec -2013-09-10 16:49:32,324 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.63 sec -2013-09-10 16:49:33,331 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.63 sec -2013-09-10 16:49:34,337 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.63 sec -2013-09-10 16:49:35,344 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.63 sec -2013-09-10 16:49:36,351 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 54.63 sec -2013-09-10 16:49:37,360 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.37 sec -2013-09-10 16:49:38,367 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.37 sec -2013-09-10 16:49:39,373 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.37 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 370 msec -Ended Job = job_201309101627_0007 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 62.37 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 2 seconds 370 msec -OK -2037258 -Time taken: 39.398 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22271@mturlrep13_201309101649_29172802.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22687@mturlrep13_201309101649_1900606579.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0008 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 16:50:00,361 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:50:07,399 Stage-1 map = 43%, reduce = 0% -2013-09-10 16:50:09,418 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.65 sec -2013-09-10 16:50:10,426 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.65 sec -2013-09-10 16:50:11,433 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.65 sec -2013-09-10 16:50:12,439 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.65 sec -2013-09-10 16:50:13,446 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.65 sec -2013-09-10 16:50:14,454 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.65 sec -2013-09-10 16:50:15,462 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.65 sec -2013-09-10 16:50:16,468 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 22.65 sec -2013-09-10 16:50:17,475 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 33.11 sec -2013-09-10 16:50:18,482 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.48 sec -2013-09-10 16:50:19,488 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.48 sec -2013-09-10 16:50:20,494 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.48 sec -2013-09-10 16:50:21,500 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.48 sec -2013-09-10 16:50:22,506 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.48 sec -2013-09-10 16:50:23,512 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.48 sec -2013-09-10 16:50:24,518 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.48 sec -2013-09-10 16:50:25,524 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 45.48 sec -2013-09-10 16:50:26,533 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.03 sec -2013-09-10 16:50:27,539 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.03 sec -MapReduce Total cumulative CPU time: 53 seconds 30 msec -Ended Job = job_201309101627_0008 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 53.03 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 53 seconds 30 msec -OK -1110413 -Time taken: 37.022 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24668@mturlrep13_201309101650_422361635.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25092@mturlrep13_201309101650_1967761435.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0009 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 16:50:48,843 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:50:54,879 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.61 sec -2013-09-10 16:50:55,888 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.61 sec -2013-09-10 16:50:56,897 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.61 sec -2013-09-10 16:50:57,904 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.61 sec -2013-09-10 16:50:58,916 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.61 sec -2013-09-10 16:50:59,923 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.61 sec -2013-09-10 16:51:00,930 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 21.31 sec -2013-09-10 16:51:01,937 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 28.16 sec -2013-09-10 16:51:02,943 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.16 sec -2013-09-10 16:51:03,950 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.16 sec -2013-09-10 16:51:04,956 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.16 sec -2013-09-10 16:51:05,961 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.16 sec -2013-09-10 16:51:06,967 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 28.16 sec -2013-09-10 16:51:07,975 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.18 sec -2013-09-10 16:51:08,980 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.18 sec -2013-09-10 16:51:09,987 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.18 sec -MapReduce Total cumulative CPU time: 30 seconds 180 msec -Ended Job = job_201309101627_0009 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 30.18 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 30 seconds 180 msec -OK -Time taken: 30.662 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26365@mturlrep13_201309101651_1129316279.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26818@mturlrep13_201309101651_583414602.txt -hive> SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC;; -FAILED: SemanticException [Error 10128]: Line 1:96 Not yet supported place for UDAF 'count' -hive> quit; --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27044@mturlrep13_201309101651_1075126234.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27463@mturlrep13_201309101651_522831884.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0010 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 16:51:45,485 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:51:52,523 Stage-1 map = 39%, reduce = 0% -2013-09-10 16:51:55,538 Stage-1 map = 43%, reduce = 0% -2013-09-10 16:51:56,550 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.73 sec -2013-09-10 16:51:57,559 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.73 sec -2013-09-10 16:51:58,566 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.73 sec -2013-09-10 16:51:59,574 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.73 sec -2013-09-10 16:52:00,581 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.73 sec -2013-09-10 16:52:01,588 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.73 sec -2013-09-10 16:52:02,600 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.73 sec -2013-09-10 16:52:03,607 Stage-1 map = 72%, reduce = 8%, Cumulative CPU 28.73 sec -2013-09-10 16:52:04,613 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.73 sec -2013-09-10 16:52:05,619 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.73 sec -2013-09-10 16:52:06,626 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.73 sec -2013-09-10 16:52:07,633 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.58 sec -2013-09-10 16:52:08,640 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.58 sec -2013-09-10 16:52:09,646 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.58 sec -2013-09-10 16:52:10,653 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.58 sec -2013-09-10 16:52:11,660 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.58 sec -2013-09-10 16:52:12,669 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 63.3 sec -2013-09-10 16:52:13,676 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.55 sec -2013-09-10 16:52:14,683 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.55 sec -MapReduce Total cumulative CPU time: 1 minutes 10 seconds 550 msec -Ended Job = job_201309101627_0010 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0011 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 16:52:18,198 Stage-2 map = 0%, reduce = 0% -2013-09-10 16:52:20,209 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-10 16:52:21,215 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-10 16:52:22,220 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-10 16:52:23,225 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-10 16:52:24,231 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-10 16:52:25,237 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-10 16:52:26,243 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-10 16:52:27,249 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.02 sec -2013-09-10 16:52:28,256 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.02 sec -2013-09-10 16:52:29,261 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.02 sec -MapReduce Total cumulative CPU time: 3 seconds 20 msec -Ended Job = job_201309101627_0011 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 70.55 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.02 sec HDFS Read: 100909 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 13 seconds 570 msec -OK -Time taken: 53.605 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29381@mturlrep13_201309101652_1928148472.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29809@mturlrep13_201309101652_663585233.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10;; -FAILED: SemanticException [Error 10128]: Line 1:136 Not yet supported place for UDAF 'count' -hive> quit; --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30055@mturlrep13_201309101652_258714563.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30484@mturlrep13_201309101652_1382767114.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0012 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 16:53:04,576 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:53:10,611 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 16:53:11,619 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 16:53:12,628 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 16:53:13,635 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 16:53:14,643 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 16:53:15,651 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 16:53:16,658 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.18 sec -2013-09-10 16:53:17,666 Stage-1 map = 100%, reduce = 13%, Cumulative CPU 24.44 sec -2013-09-10 16:53:18,673 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.44 sec -2013-09-10 16:53:19,680 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.44 sec -2013-09-10 16:53:20,686 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.44 sec -2013-09-10 16:53:21,693 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.44 sec -2013-09-10 16:53:22,699 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.44 sec -2013-09-10 16:53:23,705 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 24.44 sec -2013-09-10 16:53:24,714 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.1 sec -2013-09-10 16:53:25,720 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.1 sec -2013-09-10 16:53:26,727 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.1 sec -MapReduce Total cumulative CPU time: 31 seconds 100 msec -Ended Job = job_201309101627_0012 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0013 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 16:53:29,230 Stage-2 map = 0%, reduce = 0% -2013-09-10 16:53:31,240 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 16:53:32,246 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 16:53:33,252 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 16:53:34,257 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 16:53:35,263 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 16:53:36,269 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 16:53:37,275 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-10 16:53:38,281 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.88 sec -2013-09-10 16:53:39,287 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.39 sec -2013-09-10 16:53:40,294 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.39 sec -2013-09-10 16:53:41,301 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.39 sec -MapReduce Total cumulative CPU time: 2 seconds 390 msec -Ended Job = job_201309101627_0013 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 31.1 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.39 sec HDFS Read: 21895 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 490 msec -OK -Time taken: 46.622 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по строкам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32427@mturlrep13_201309101653_1424309351.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_394@mturlrep13_201309101653_1101478749.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0014 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 16:54:01,350 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:54:07,385 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.6 sec -2013-09-10 16:54:08,394 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.6 sec -2013-09-10 16:54:09,401 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.6 sec -2013-09-10 16:54:10,409 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.6 sec -2013-09-10 16:54:11,417 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.6 sec -2013-09-10 16:54:12,424 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.6 sec -2013-09-10 16:54:13,431 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.6 sec -2013-09-10 16:54:14,438 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 17.6 sec -2013-09-10 16:54:15,445 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 24.9 sec -2013-09-10 16:54:16,452 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 24.9 sec -2013-09-10 16:54:17,459 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 24.9 sec -2013-09-10 16:54:18,466 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 24.9 sec -2013-09-10 16:54:19,474 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 24.9 sec -2013-09-10 16:54:20,481 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 24.9 sec -2013-09-10 16:54:21,489 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.02 sec -2013-09-10 16:54:22,497 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.02 sec -2013-09-10 16:54:23,504 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.02 sec -MapReduce Total cumulative CPU time: 32 seconds 20 msec -Ended Job = job_201309101627_0014 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0015 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 16:54:26,101 Stage-2 map = 0%, reduce = 0% -2013-09-10 16:54:28,111 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.99 sec -2013-09-10 16:54:29,118 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.99 sec -2013-09-10 16:54:30,124 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.99 sec -2013-09-10 16:54:31,131 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.99 sec -2013-09-10 16:54:32,137 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.99 sec -2013-09-10 16:54:33,142 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.99 sec -2013-09-10 16:54:34,148 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.99 sec -2013-09-10 16:54:35,154 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.99 sec -2013-09-10 16:54:36,161 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -2013-09-10 16:54:37,169 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -MapReduce Total cumulative CPU time: 2 seconds 450 msec -Ended Job = job_201309101627_0015 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 32.02 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.45 sec HDFS Read: 23477 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 34 seconds 470 msec -OK -Time taken: 45.572 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2646@mturlrep13_201309101654_1193376842.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3074@mturlrep13_201309101654_1095912087.txt -hive> SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10;; -FAILED: SemanticException [Error 10128]: Line 1:100 Not yet supported place for UDAF 'count' -hive> quit; --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3321@mturlrep13_201309101654_686760308.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3757@mturlrep13_201309101655_1578096171.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0016 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 16:55:13,612 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:55:20,646 Stage-1 map = 36%, reduce = 0% -2013-09-10 16:55:22,664 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 11.28 sec -2013-09-10 16:55:23,672 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.77 sec -2013-09-10 16:55:24,682 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.77 sec -2013-09-10 16:55:25,690 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.77 sec -2013-09-10 16:55:26,697 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.77 sec -2013-09-10 16:55:27,704 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.77 sec -2013-09-10 16:55:28,711 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.77 sec -2013-09-10 16:55:29,718 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.77 sec -2013-09-10 16:55:30,725 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 22.77 sec -2013-09-10 16:55:31,731 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 22.77 sec -2013-09-10 16:55:32,738 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.15 sec -2013-09-10 16:55:33,744 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.15 sec -2013-09-10 16:55:34,751 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.15 sec -2013-09-10 16:55:35,757 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.15 sec -2013-09-10 16:55:36,764 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.15 sec -2013-09-10 16:55:37,770 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.15 sec -2013-09-10 16:55:38,777 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.15 sec -2013-09-10 16:55:39,783 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 46.15 sec -2013-09-10 16:55:40,792 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.76 sec -2013-09-10 16:55:41,798 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.76 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 760 msec -Ended Job = job_201309101627_0016 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0017 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 16:55:44,327 Stage-2 map = 0%, reduce = 0% -2013-09-10 16:55:52,361 Stage-2 map = 50%, reduce = 0% -2013-09-10 16:55:54,370 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.94 sec -2013-09-10 16:55:55,376 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.94 sec -2013-09-10 16:55:56,381 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.94 sec -2013-09-10 16:55:57,386 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.94 sec -2013-09-10 16:55:58,391 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.94 sec -2013-09-10 16:55:59,396 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.94 sec -2013-09-10 16:56:00,403 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.94 sec -2013-09-10 16:56:01,408 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 12.94 sec -2013-09-10 16:56:02,413 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 12.94 sec -2013-09-10 16:56:03,419 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.72 sec -2013-09-10 16:56:04,431 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.72 sec -MapReduce Total cumulative CPU time: 17 seconds 720 msec -Ended Job = job_201309101627_0017 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.76 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.72 sec HDFS Read: 79727313 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 20 seconds 480 msec -OK -Time taken: 60.763 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация чуть сложнее.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6801@mturlrep13_201309101656_776043994.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7208@mturlrep13_201309101656_657090356.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10;; -FAILED: SemanticException [Error 10128]: Line 1:132 Not yet supported place for UDAF 'count' -hive> quit; --- агрегация по числу и строке, большое количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7458@mturlrep13_201309101656_1567054324.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7890@mturlrep13_201309101656_1969270039.txt -hive> SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10;; -FAILED: SemanticException [Error 10128]: Line 1:63 Not yet supported place for UDAF 'count' -hive> quit; --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8121@mturlrep13_201309101656_1108276275.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8542@mturlrep13_201309101656_1566254424.txt -hive> SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10;; -FAILED: SemanticException [Error 10128]: Line 1:91 Not yet supported place for UDAF 'count' -hive> quit; --- ещё более сложная агрегация.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8792@mturlrep13_201309101656_1642375381.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9207@mturlrep13_201309101657_1235758846.txt -hive> SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0018 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 16:57:11,290 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:57:18,357 Stage-1 map = 36%, reduce = 0% -2013-09-10 16:57:21,373 Stage-1 map = 43%, reduce = 0% -2013-09-10 16:57:23,389 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 17.23 sec -2013-09-10 16:57:24,396 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.41 sec -2013-09-10 16:57:25,405 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.41 sec -2013-09-10 16:57:26,413 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.41 sec -2013-09-10 16:57:27,420 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.41 sec -2013-09-10 16:57:28,426 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.41 sec -2013-09-10 16:57:29,433 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.41 sec -2013-09-10 16:57:30,440 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.41 sec -2013-09-10 16:57:31,453 Stage-1 map = 85%, reduce = 17%, Cumulative CPU 34.41 sec -2013-09-10 16:57:32,460 Stage-1 map = 85%, reduce = 17%, Cumulative CPU 34.41 sec -2013-09-10 16:57:33,466 Stage-1 map = 85%, reduce = 17%, Cumulative CPU 34.41 sec -2013-09-10 16:57:34,472 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.41 sec -2013-09-10 16:57:35,479 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.41 sec -2013-09-10 16:57:36,486 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 50.94 sec -2013-09-10 16:57:37,491 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.84 sec -2013-09-10 16:57:38,498 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.84 sec -2013-09-10 16:57:39,504 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.84 sec -2013-09-10 16:57:40,511 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.84 sec -2013-09-10 16:57:41,516 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.84 sec -2013-09-10 16:57:42,523 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.84 sec -2013-09-10 16:57:43,531 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 73.66 sec -2013-09-10 16:57:44,538 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 78.48 sec -2013-09-10 16:57:45,544 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 78.48 sec -MapReduce Total cumulative CPU time: 1 minutes 18 seconds 480 msec -Ended Job = job_201309101627_0018 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 78.48 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 18 seconds 480 msec -OK -Time taken: 43.997 seconds, Fetched: 10 row(s) -hive> quit; --- то же самое, но без сортировки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10713@mturlrep13_201309101657_1877182176.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11137@mturlrep13_201309101657_663767903.txt -hive> SELECT UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10;; -hive> quit; --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11382@mturlrep13_201309101658_145683406.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11815@mturlrep13_201309101658_1403531499.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0019 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-10 16:58:19,917 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:58:24,947 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.88 sec -2013-09-10 16:58:25,955 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.88 sec -2013-09-10 16:58:26,964 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.88 sec -2013-09-10 16:58:27,971 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.88 sec -2013-09-10 16:58:28,977 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.88 sec -2013-09-10 16:58:29,984 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 11.45 sec -2013-09-10 16:58:30,990 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.59 sec -2013-09-10 16:58:31,996 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 15.59 sec -MapReduce Total cumulative CPU time: 15 seconds 590 msec -Ended Job = job_201309101627_0019 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 15.59 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 15 seconds 590 msec -OK -Time taken: 21.833 seconds -hive> quit; --- мощная фильтрация по столбцу типа UInt64.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12917@mturlrep13_201309101658_31678322.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13347@mturlrep13_201309101658_1130637656.txt -hive> SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0020 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 16:58:52,391 Stage-1 map = 0%, reduce = 0% -2013-09-10 16:58:59,424 Stage-1 map = 43%, reduce = 0% -2013-09-10 16:59:00,438 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.87 sec -2013-09-10 16:59:01,446 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.87 sec -2013-09-10 16:59:02,454 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.87 sec -2013-09-10 16:59:03,461 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.87 sec -2013-09-10 16:59:04,468 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.87 sec -2013-09-10 16:59:05,475 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.87 sec -2013-09-10 16:59:06,483 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.87 sec -2013-09-10 16:59:07,490 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 26.09 sec -2013-09-10 16:59:08,497 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.91 sec -2013-09-10 16:59:09,503 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.91 sec -2013-09-10 16:59:10,508 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.91 sec -2013-09-10 16:59:11,515 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.91 sec -2013-09-10 16:59:12,522 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.91 sec -2013-09-10 16:59:13,530 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.04 sec -2013-09-10 16:59:14,538 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.04 sec -MapReduce Total cumulative CPU time: 37 seconds 40 msec -Ended Job = job_201309101627_0020 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 37.04 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 37 seconds 40 msec -OK -8428 -Time taken: 32.099 seconds, Fetched: 1 row(s) -hive> quit; --- фильтрация по поиску подстроки в строке.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14584@mturlrep13_201309101659_2137143541.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15034@mturlrep13_201309101659_646733649.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10;; -FAILED: SemanticException [Error 10128]: Line 1:135 Not yet supported place for UDAF 'count' -hive> quit; --- вынимаем большие столбцы, фильтрация по строке.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15279@mturlrep13_201309101659_658953966.txt -hive> ; -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15691@mturlrep13_201309101659_532790270.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10;; -FAILED: SemanticException [Error 10128]: Line 1:207 Not yet supported place for UDAF 'count' -hive> quit; --- чуть больше столбцы.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15936@mturlrep13_201309101659_671465679.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16390@mturlrep13_201309101659_896460481.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0021 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 17:00:05,665 Stage-1 map = 0%, reduce = 0% -2013-09-10 17:00:16,730 Stage-1 map = 7%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:17,737 Stage-1 map = 7%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:18,745 Stage-1 map = 7%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:19,751 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:20,758 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:21,765 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:22,771 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:23,777 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:24,783 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:25,788 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:26,794 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:27,800 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:28,806 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:29,811 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:30,817 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:31,823 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:32,829 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:33,835 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:34,841 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 20.36 sec -2013-09-10 17:00:35,850 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 72.6 sec -2013-09-10 17:00:36,857 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 72.6 sec -2013-09-10 17:00:37,865 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 72.6 sec -2013-09-10 17:00:38,872 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 72.6 sec -2013-09-10 17:00:39,878 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 72.6 sec -2013-09-10 17:00:40,884 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 72.6 sec -2013-09-10 17:00:41,890 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 72.6 sec -2013-09-10 17:00:42,897 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:43,904 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:44,909 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:45,916 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:46,922 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:47,928 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:48,944 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:49,950 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:50,956 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:51,963 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:52,968 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:53,975 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:54,981 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:55,987 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:56,992 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:58,002 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:00:59,008 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:01:00,014 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:01:01,020 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:01:02,026 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:01:03,041 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 72.6 sec -2013-09-10 17:01:04,047 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 106.42 sec -2013-09-10 17:01:05,052 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 106.42 sec -2013-09-10 17:01:06,058 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 106.42 sec -2013-09-10 17:01:07,063 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 146.86 sec -2013-09-10 17:01:08,069 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 146.86 sec -2013-09-10 17:01:09,078 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 149.89 sec -2013-09-10 17:01:10,084 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 149.89 sec -2013-09-10 17:01:11,090 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 149.89 sec -MapReduce Total cumulative CPU time: 2 minutes 29 seconds 890 msec -Ended Job = job_201309101627_0021 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 149.89 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 29 seconds 890 msec -OK -Time taken: 75.953 seconds, Fetched: 10 row(s) -hive> quit; --- плохой запрос - вынимаем все столбцы.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18541@mturlrep13_201309101701_742167591.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18989@mturlrep13_201309101701_291925169.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -FAILED: SemanticException [Error 10004]: Line 1:68 Invalid table alias or column reference 'EventTime': (possible column names are: searchphrase) -hive> quit; --- большая сортировка.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19233@mturlrep13_201309101701_2095092578.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19634@mturlrep13_201309101701_1908098506.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0022 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 17:01:47,013 Stage-1 map = 0%, reduce = 0% -2013-09-10 17:01:54,048 Stage-1 map = 43%, reduce = 0% -2013-09-10 17:01:55,063 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.59 sec -2013-09-10 17:01:56,071 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.59 sec -2013-09-10 17:01:57,082 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.59 sec -2013-09-10 17:01:58,088 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.59 sec -2013-09-10 17:01:59,095 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.59 sec -2013-09-10 17:02:00,102 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.59 sec -2013-09-10 17:02:01,108 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.59 sec -2013-09-10 17:02:02,116 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 19.59 sec -2013-09-10 17:02:03,122 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.26 sec -2013-09-10 17:02:04,128 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.26 sec -2013-09-10 17:02:05,134 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.26 sec -2013-09-10 17:02:06,140 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.26 sec -2013-09-10 17:02:07,145 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.26 sec -2013-09-10 17:02:08,151 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.26 sec -2013-09-10 17:02:09,204 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.26 sec -2013-09-10 17:02:10,209 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.26 sec -2013-09-10 17:02:11,218 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 46.32 sec -2013-09-10 17:02:12,225 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 46.32 sec -2013-09-10 17:02:13,230 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 46.32 sec -MapReduce Total cumulative CPU time: 46 seconds 320 msec -Ended Job = job_201309101627_0022 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 46.32 sec HDFS Read: 27820105 HDFS Write: 666 SUCCESS -Total MapReduce CPU Time Spent: 46 seconds 320 msec -OK -ялта интурист -! как одеть трехнедельного ребенка при температуре 20 градусов -! отель rattana beach hotel 3* -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -!( центробежный скважинный калибр форумы) -!(!(storm master silmarils)) -!(!(storm master silmarils)) -!(!(title:(схема sputnik hi 4000))) -!(44-фз о контрактной системе) -Time taken: 35.815 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка по строкам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20928@mturlrep13_201309101702_1035938831.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21350@mturlrep13_201309101702_1140115628.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -FAILED: SemanticException [Error 10004]: Line 1:68 Invalid table alias or column reference 'EventTime': (possible column names are: searchphrase) -hive> quit; --- большая сортировка по кортежу.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21597@mturlrep13_201309101702_910063178.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22017@mturlrep13_201309101702_1828172994.txt -hive> 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;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0023 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 17:02:48,816 Stage-1 map = 0%, reduce = 0% -2013-09-10 17:02:55,850 Stage-1 map = 14%, reduce = 0% -2013-09-10 17:02:58,883 Stage-1 map = 22%, reduce = 0% -2013-09-10 17:03:01,897 Stage-1 map = 36%, reduce = 0% -2013-09-10 17:03:04,913 Stage-1 map = 43%, reduce = 0% -2013-09-10 17:03:06,931 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 43.83 sec -2013-09-10 17:03:07,939 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 43.83 sec -2013-09-10 17:03:08,949 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 43.83 sec -2013-09-10 17:03:09,956 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 43.83 sec -2013-09-10 17:03:10,964 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 43.83 sec -2013-09-10 17:03:11,970 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 43.83 sec -2013-09-10 17:03:12,976 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 43.83 sec -2013-09-10 17:03:13,983 Stage-1 map = 57%, reduce = 4%, Cumulative CPU 43.83 sec -2013-09-10 17:03:14,991 Stage-1 map = 61%, reduce = 13%, Cumulative CPU 43.83 sec -2013-09-10 17:03:15,999 Stage-1 map = 61%, reduce = 13%, Cumulative CPU 66.49 sec -2013-09-10 17:03:17,005 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 66.49 sec -2013-09-10 17:03:18,013 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 66.49 sec -2013-09-10 17:03:19,019 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 66.49 sec -2013-09-10 17:03:20,026 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 66.49 sec -2013-09-10 17:03:21,032 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 66.49 sec -2013-09-10 17:03:22,039 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 66.49 sec -2013-09-10 17:03:23,047 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 66.49 sec -2013-09-10 17:03:24,054 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 77.28 sec -2013-09-10 17:03:25,061 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 89.49 sec -2013-09-10 17:03:26,067 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 89.49 sec -2013-09-10 17:03:27,074 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 89.49 sec -2013-09-10 17:03:28,081 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 89.49 sec -2013-09-10 17:03:29,088 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 89.49 sec -2013-09-10 17:03:30,095 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 89.49 sec -2013-09-10 17:03:31,103 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 89.49 sec -2013-09-10 17:03:32,110 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 89.49 sec -2013-09-10 17:03:33,116 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 89.49 sec -2013-09-10 17:03:34,122 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 89.49 sec -2013-09-10 17:03:35,129 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 89.49 sec -2013-09-10 17:03:36,141 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 89.49 sec -2013-09-10 17:03:37,339 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 98.61 sec -2013-09-10 17:03:38,347 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 98.61 sec -2013-09-10 17:03:39,354 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 98.61 sec -2013-09-10 17:03:40,360 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 98.61 sec -2013-09-10 17:03:41,367 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 98.61 sec -2013-09-10 17:03:42,374 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 112.26 sec -2013-09-10 17:03:43,392 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 112.26 sec -MapReduce Total cumulative CPU time: 1 minutes 52 seconds 260 msec -Ended Job = job_201309101627_0023 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0024 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 17:03:46,871 Stage-2 map = 0%, reduce = 0% -2013-09-10 17:03:47,877 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-10 17:03:48,883 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-10 17:03:49,889 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-10 17:03:50,895 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-10 17:03:51,900 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-10 17:03:52,905 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-10 17:03:53,911 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-10 17:03:54,916 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-10 17:03:55,922 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.22 sec -2013-09-10 17:03:56,928 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.22 sec -2013-09-10 17:03:57,934 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.22 sec -MapReduce Total cumulative CPU time: 2 seconds 220 msec -Ended Job = job_201309101627_0024 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 112.26 sec HDFS Read: 117363067 HDFS Write: 794 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.22 sec HDFS Read: 1563 HDFS Write: 571 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 54 seconds 480 msec -OK -Time taken: 79.029 seconds, Fetched: 19 row(s) -hive> quit; --- считаем средние длины URL для крупных счётчиков.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24060@mturlrep13_201309101704_697176297.txt -hive> ; -hive> quit; - -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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24498@mturlrep13_201309101704_1925741849.txt -hive> 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -NoViableAltException(26@[()* loopback of 364:5: ( ( KW_NOT precedenceEqualNegatableOperator notExpr= precedenceBitwiseOrExpression ) -> ^( KW_NOT ^( precedenceEqualNegatableOperator $precedenceEqualExpression $notExpr) ) | ( precedenceEqualOperator equalExpr= precedenceBitwiseOrExpression ) -> ^( precedenceEqualOperator $precedenceEqualExpression $equalExpr) | ( KW_NOT KW_IN expressions ) -> ^( KW_NOT ^( TOK_FUNCTION KW_IN $precedenceEqualExpression expressions ) ) | ( KW_IN expressions ) -> ^( TOK_FUNCTION KW_IN $precedenceEqualExpression expressions ) | ( KW_NOT KW_BETWEEN (min= precedenceBitwiseOrExpression ) KW_AND (max= precedenceBitwiseOrExpression ) ) -> ^( TOK_FUNCTION Identifier["between"] KW_TRUE $left $min $max) | ( KW_BETWEEN (min= precedenceBitwiseOrExpression ) KW_AND (max= precedenceBitwiseOrExpression ) ) -> ^( TOK_FUNCTION Identifier["between"] KW_FALSE $left $min $max) )*]) -FAILED: ParseException line 1:53 cannot recognize input near 'IN' 'Referer' ')' in expression specification - -hive> quit; --- то же самое, но с разбивкой по доменам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24706@mturlrep13_201309101704_325408643.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25137@mturlrep13_201309101704_464421422.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0025 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 17:04:28,745 Stage-1 map = 0%, reduce = 0% -2013-09-10 17:04:41,802 Stage-1 map = 7%, reduce = 0% -2013-09-10 17:04:47,828 Stage-1 map = 14%, reduce = 0% -2013-09-10 17:04:53,852 Stage-1 map = 22%, reduce = 0% -2013-09-10 17:05:03,892 Stage-1 map = 29%, reduce = 0% -2013-09-10 17:05:09,916 Stage-1 map = 36%, reduce = 0% -2013-09-10 17:05:15,942 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 99.49 sec -2013-09-10 17:05:16,947 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 99.49 sec -2013-09-10 17:05:17,953 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 99.49 sec -2013-09-10 17:05:18,958 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 99.49 sec -2013-09-10 17:05:19,962 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 99.49 sec -2013-09-10 17:05:20,968 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 99.49 sec -2013-09-10 17:05:21,975 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 106.03 sec -2013-09-10 17:05:22,980 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.63 sec -2013-09-10 17:05:23,986 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.63 sec -2013-09-10 17:05:24,991 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.63 sec -2013-09-10 17:05:25,997 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.63 sec -2013-09-10 17:05:27,003 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.63 sec -2013-09-10 17:05:28,009 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.63 sec -2013-09-10 17:05:29,014 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:30,020 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:31,026 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:32,031 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:33,037 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:34,042 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:35,048 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:36,054 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:37,059 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:38,065 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:39,070 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:40,076 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:41,082 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:42,087 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:43,093 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:44,098 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:45,104 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:46,109 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:47,115 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:48,125 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:49,131 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:50,136 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:51,141 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:52,146 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:53,152 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:54,157 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:55,165 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:56,170 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:57,176 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:58,181 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:05:59,186 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:00,193 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:01,198 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:02,204 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:03,210 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:04,215 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:05,222 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:06,245 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:07,251 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:08,256 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:09,261 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:10,267 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:11,273 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 113.63 sec -2013-09-10 17:06:12,278 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 166.1 sec -2013-09-10 17:06:13,283 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 166.1 sec -2013-09-10 17:06:14,289 Stage-1 map = 97%, reduce = 25%, Cumulative CPU 166.1 sec -2013-09-10 17:06:15,294 Stage-1 map = 97%, reduce = 25%, Cumulative CPU 166.1 sec -2013-09-10 17:06:16,299 Stage-1 map = 97%, reduce = 25%, Cumulative CPU 223.78 sec -2013-09-10 17:06:17,305 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 226.31 sec -2013-09-10 17:06:18,310 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 226.31 sec -2013-09-10 17:06:19,315 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 226.31 sec -2013-09-10 17:06:20,323 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 228.69 sec -2013-09-10 17:06:21,329 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 228.69 sec -2013-09-10 17:06:22,335 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 228.69 sec -MapReduce Total cumulative CPU time: 3 minutes 48 seconds 690 msec -Ended Job = job_201309101627_0025 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 228.69 sec HDFS Read: 7797536 HDFS Write: 1080 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 48 seconds 690 msec -OK -Time taken: 124.625 seconds, Fetched: 1 row(s) -hive> quit; --- много тупых агрегатных функций.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27420@mturlrep13_201309101706_2102923774.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27827@mturlrep13_201309101706_1115249262.txt -hive> 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;; -FAILED: SemanticException [Error 10128]: Line 1:165 Not yet supported place for UDAF 'count' -hive> quit; --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28077@mturlrep13_201309101706_2124378603.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28492@mturlrep13_201309101706_64062784.txt -hive> 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;; -FAILED: SemanticException [Error 10128]: Line 1:151 Not yet supported place for UDAF 'count' -hive> quit; --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28748@mturlrep13_201309101706_115048628.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29182@mturlrep13_201309101707_799532651.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10;; -FAILED: SemanticException [Error 10128]: Line 1:126 Not yet supported place for UDAF 'count' -hive> quit; --- то же самое, но ещё и без фильтрации.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29398@mturlrep13_201309101707_1250937879.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29832@mturlrep13_201309101707_220900109.txt -hive> SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10;; -FAILED: SemanticException [Error 10128]: Line 1:57 Not yet supported place for UDAF 'count' -hive> quit; --- агрегация по URL.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30076@mturlrep13_201309101707_1998522483.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30483@mturlrep13_201309101707_601921974.txt -hive> SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10;; -FAILED: SemanticException [Error 10128]: Line 1:63 Not yet supported place for UDAF 'count' -hive> quit; --- агрегация по URL и числу.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30739@mturlrep13_201309101707_969907764.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31158@mturlrep13_201309101707_1022790326.txt -hive> 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;; -FAILED: SemanticException [Error 10128]: Line 1:151 Not yet supported place for UDAF 'count' -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31402@mturlrep13_201309101707_604885932.txt -hive> ; -hive> quit; - -times: 1 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31838@mturlrep13_201309101708_1455783747.txt -hive> ; -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32045@mturlrep13_201309101708_1506845175.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32470@mturlrep13_201309101708_252581742.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -FAILED: ClassCastException org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableShortObjectInspector cannot be cast to org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32697@mturlrep13_201309101708_453394488.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_691@mturlrep13_201309101708_1243233989.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -FAILED: ClassCastException org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableShortObjectInspector cannot be cast to org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_937@mturlrep13_201309101708_2043875791.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1630@mturlrep13_201309101708_1723964633.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -FAILED: ClassCastException org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableByteObjectInspector cannot be cast to org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1875@mturlrep13_201309101708_280630488.txt -hive> ; -hive> quit; - -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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2302@mturlrep13_201309101708_1494219422.txt -hive> 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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; ; -FAILED: ClassCastException org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableByteObjectInspector cannot be cast to org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2546@mturlrep13_201309101709_1003990940.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2963@mturlrep13_201309101709_1534270046.txt -hive> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; ; -FAILED: ClassCastException org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableByteObjectInspector cannot be cast to org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3205@mturlrep13_201309101709_665683333.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3636@mturlrep13_201309101709_67803216.txt -hive> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; ; -FAILED: ClassCastException org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableByteObjectInspector cannot be cast to org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3862@mturlrep13_201309101709_1829457435.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4309@mturlrep13_201309101709_1514553234.txt -hive> SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; ; -NoViableAltException(26@[]) -FAILED: ParseException line 1:34 missing FROM at '(' near '(' in subquery source -line 1:35 cannot recognize input near 'EventTime' ')' 'SECOND' in subquery source - -hive> quit; -stop time: Вт. сент. 10 17:09:45 MSK 2013 diff --git a/benchmark/hive/log/log_10m/log_10m_2 b/benchmark/hive/log/log_10m/log_10m_2 deleted file mode 100644 index 6728a58aaf1..00000000000 --- a/benchmark/hive/log/log_10m/log_10m_2 +++ /dev/null @@ -1,624 +0,0 @@ -start time: Вт. сент. 10 18:46:00 MSK 2013 -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15579@mturlrep13_201309101846_67163557.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16038@mturlrep13_201309101846_623079473.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0036 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 18:46:20,061 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:46:27,089 Stage-1 map = 7%, reduce = 0% -2013-09-10 18:46:33,113 Stage-1 map = 14%, reduce = 0% -2013-09-10 18:46:36,127 Stage-1 map = 22%, reduce = 0% -2013-09-10 18:46:39,143 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 46.41 sec -2013-09-10 18:46:40,149 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 46.41 sec -2013-09-10 18:46:41,156 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 46.41 sec -2013-09-10 18:46:42,162 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 46.41 sec -2013-09-10 18:46:43,168 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 46.41 sec -2013-09-10 18:46:44,174 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 46.41 sec -2013-09-10 18:46:45,179 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 46.41 sec -2013-09-10 18:46:46,185 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 46.41 sec -2013-09-10 18:46:47,191 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 46.41 sec -2013-09-10 18:46:48,197 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 46.41 sec -2013-09-10 18:46:49,205 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 62.51 sec -2013-09-10 18:46:50,211 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 83.95 sec -2013-09-10 18:46:51,217 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 83.95 sec -2013-09-10 18:46:52,222 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 83.95 sec -2013-09-10 18:46:53,227 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 83.95 sec -2013-09-10 18:46:54,233 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 83.95 sec -2013-09-10 18:46:55,238 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 83.95 sec -2013-09-10 18:46:56,244 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 83.95 sec -2013-09-10 18:46:57,250 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:46:58,255 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:46:59,261 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:00,266 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:01,272 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:02,277 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:03,282 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:04,287 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:05,305 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:06,310 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:07,316 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:08,321 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:09,326 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:10,331 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:11,336 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:12,341 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:13,346 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:14,351 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 83.95 sec -2013-09-10 18:47:15,356 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 118.21 sec -2013-09-10 18:47:16,372 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 118.21 sec -2013-09-10 18:47:17,379 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 118.21 sec -2013-09-10 18:47:18,384 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 118.21 sec -2013-09-10 18:47:19,388 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 162.76 sec -2013-09-10 18:47:20,393 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 162.76 sec -2013-09-10 18:47:21,397 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 162.76 sec -2013-09-10 18:47:22,404 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 165.27 sec -2013-09-10 18:47:23,410 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 165.27 sec -2013-09-10 18:47:24,415 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 165.27 sec -MapReduce Total cumulative CPU time: 2 minutes 45 seconds 270 msec -Ended Job = job_201309101627_0036 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 165.27 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 45 seconds 270 msec -OK -10000000 -Time taken: 74.228 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17475@mturlrep13_201309101847_1783698271.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17882@mturlrep13_201309101847_1295809350.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0037 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 18:47:44,058 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:47:49,086 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 18:47:50,093 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 18:47:51,101 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 18:47:52,107 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 18:47:53,113 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.21 sec -2013-09-10 18:47:54,119 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 18.18 sec -2013-09-10 18:47:55,125 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.81 sec -2013-09-10 18:47:56,130 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.81 sec -2013-09-10 18:47:57,138 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.64 sec -2013-09-10 18:47:58,144 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.64 sec -2013-09-10 18:47:59,150 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.64 sec -MapReduce Total cumulative CPU time: 25 seconds 640 msec -Ended Job = job_201309101627_0037 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 25.64 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 25 seconds 640 msec -OK -171127 -Time taken: 25.153 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19147@mturlrep13_201309101848_1891179156.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19567@mturlrep13_201309101848_690102300.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0038 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 18:48:18,837 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:48:25,865 Stage-1 map = 39%, reduce = 0% -2013-09-10 18:48:26,875 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.45 sec -2013-09-10 18:48:27,882 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.45 sec -2013-09-10 18:48:28,889 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.45 sec -2013-09-10 18:48:29,895 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.45 sec -2013-09-10 18:48:30,901 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.45 sec -2013-09-10 18:48:31,907 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.45 sec -2013-09-10 18:48:32,914 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.45 sec -2013-09-10 18:48:33,920 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.59 sec -2013-09-10 18:48:34,925 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.59 sec -2013-09-10 18:48:35,930 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.59 sec -2013-09-10 18:48:36,935 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.59 sec -2013-09-10 18:48:37,940 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 34.59 sec -2013-09-10 18:48:38,945 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.24 sec -2013-09-10 18:48:39,952 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 36.63 sec -2013-09-10 18:48:40,958 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 36.63 sec -MapReduce Total cumulative CPU time: 36 seconds 630 msec -Ended Job = job_201309101627_0038 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 36.63 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 36 seconds 630 msec -OK -Time taken: 31.961 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20898@mturlrep13_201309101848_327652001.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21336@mturlrep13_201309101848_1975614127.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0039 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 18:49:00,561 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:49:07,617 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.12 sec -2013-09-10 18:49:08,626 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.12 sec -2013-09-10 18:49:09,634 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.12 sec -2013-09-10 18:49:10,639 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.12 sec -2013-09-10 18:49:11,646 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.12 sec -2013-09-10 18:49:12,652 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.12 sec -2013-09-10 18:49:13,658 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 21.86 sec -2013-09-10 18:49:14,664 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.08 sec -2013-09-10 18:49:15,670 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.08 sec -2013-09-10 18:49:16,675 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.08 sec -2013-09-10 18:49:17,680 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.08 sec -2013-09-10 18:49:18,685 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.08 sec -2013-09-10 18:49:19,690 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.08 sec -2013-09-10 18:49:20,697 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.07 sec -2013-09-10 18:49:21,703 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.07 sec -MapReduce Total cumulative CPU time: 32 seconds 70 msec -Ended Job = job_201309101627_0039 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 32.07 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 32 seconds 70 msec -OK --4662894107982093709 -Time taken: 30.94 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22560@mturlrep13_201309101849_2023198520.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22993@mturlrep13_201309101849_961728603.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0040 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 18:49:41,232 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:49:48,264 Stage-1 map = 43%, reduce = 0% -2013-09-10 18:49:51,283 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.01 sec -2013-09-10 18:49:52,291 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.01 sec -2013-09-10 18:49:53,298 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.01 sec -2013-09-10 18:49:54,304 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.01 sec -2013-09-10 18:49:55,310 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.01 sec -2013-09-10 18:49:56,317 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.01 sec -2013-09-10 18:49:57,332 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.01 sec -2013-09-10 18:49:58,337 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.01 sec -2013-09-10 18:49:59,342 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.01 sec -2013-09-10 18:50:00,348 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.01 sec -2013-09-10 18:50:01,353 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-10 18:50:02,360 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-10 18:50:03,365 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-10 18:50:04,369 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-10 18:50:05,375 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-10 18:50:06,379 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-10 18:50:07,385 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 55.01 sec -2013-09-10 18:50:08,391 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.95 sec -2013-09-10 18:50:09,397 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.95 sec -2013-09-10 18:50:10,402 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.95 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 950 msec -Ended Job = job_201309101627_0040 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 62.95 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 2 seconds 950 msec -OK -2037258 -Time taken: 38.84 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24634@mturlrep13_201309101850_840502487.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25401@mturlrep13_201309101850_84750246.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0041 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 18:50:31,472 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:50:38,501 Stage-1 map = 43%, reduce = 0% -2013-09-10 18:50:40,517 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.42 sec -2013-09-10 18:50:41,523 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.42 sec -2013-09-10 18:50:42,531 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.42 sec -2013-09-10 18:50:43,536 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.42 sec -2013-09-10 18:50:44,542 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.42 sec -2013-09-10 18:50:45,548 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.42 sec -2013-09-10 18:50:46,555 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.42 sec -2013-09-10 18:50:47,561 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 21.42 sec -2013-09-10 18:50:48,566 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 31.8 sec -2013-09-10 18:50:49,571 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.95 sec -2013-09-10 18:50:50,576 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.95 sec -2013-09-10 18:50:51,581 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.95 sec -2013-09-10 18:50:52,587 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.95 sec -2013-09-10 18:50:53,592 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.95 sec -2013-09-10 18:50:54,597 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.95 sec -2013-09-10 18:50:55,602 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.95 sec -2013-09-10 18:50:56,607 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 42.95 sec -2013-09-10 18:50:57,615 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.6 sec -2013-09-10 18:50:58,642 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.6 sec -MapReduce Total cumulative CPU time: 50 seconds 600 msec -Ended Job = job_201309101627_0041 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 50.6 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 50 seconds 600 msec -OK -1110413 -Time taken: 37.04 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26718@mturlrep13_201309101851_285967686.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27149@mturlrep13_201309101851_2135309314.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0042 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 18:51:19,077 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:51:25,106 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-10 18:51:26,114 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-10 18:51:27,123 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-10 18:51:28,129 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-10 18:51:29,135 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-10 18:51:30,141 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-10 18:51:31,147 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 20.4 sec -2013-09-10 18:51:32,152 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.44 sec -2013-09-10 18:51:33,158 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 27.44 sec -2013-09-10 18:51:34,163 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 27.44 sec -2013-09-10 18:51:35,168 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 27.44 sec -2013-09-10 18:51:36,173 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 27.44 sec -2013-09-10 18:51:37,179 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 27.44 sec -2013-09-10 18:51:38,184 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 27.44 sec -2013-09-10 18:51:39,192 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.39 sec -2013-09-10 18:51:40,198 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.39 sec -MapReduce Total cumulative CPU time: 29 seconds 390 msec -Ended Job = job_201309101627_0042 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 29.39 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 29 seconds 390 msec -OK -Time taken: 30.908 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28401@mturlrep13_201309101851_891001725.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28836@mturlrep13_201309101851_1054092389.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0043 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 18:51:59,809 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:52:04,838 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.48 sec -2013-09-10 18:52:05,847 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.48 sec -2013-09-10 18:52:06,855 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.48 sec -2013-09-10 18:52:07,861 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.48 sec -2013-09-10 18:52:08,868 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.48 sec -2013-09-10 18:52:09,875 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 18.07 sec -2013-09-10 18:52:10,881 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.92 sec -2013-09-10 18:52:11,887 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.92 sec -2013-09-10 18:52:12,894 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 25.68 sec -2013-09-10 18:52:13,901 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.53 sec -2013-09-10 18:52:14,908 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.53 sec -MapReduce Total cumulative CPU time: 27 seconds 530 msec -Ended Job = job_201309101627_0043 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0044 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 18:52:17,388 Stage-2 map = 0%, reduce = 0% -2013-09-10 18:52:19,396 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 18:52:20,401 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 18:52:21,406 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 18:52:22,411 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 18:52:23,415 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 18:52:24,420 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 18:52:25,425 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 18:52:26,430 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.75 sec -2013-09-10 18:52:27,436 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -2013-09-10 18:52:28,442 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -2013-09-10 18:52:29,448 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -MapReduce Total cumulative CPU time: 2 seconds 140 msec -Ended Job = job_201309101627_0044 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 27.53 sec HDFS Read: 907716 HDFS Write: 384 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.14 sec HDFS Read: 1153 HDFS Write: 60 SUCCESS -Total MapReduce CPU Time Spent: 29 seconds 670 msec -OK -Time taken: 39.506 seconds, Fetched: 9 row(s) -hive> quit; --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30667@mturlrep13_201309101852_966681525.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31123@mturlrep13_201309101852_1252745596.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0045 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 18:52:49,457 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:52:56,485 Stage-1 map = 43%, reduce = 0% -2013-09-10 18:52:59,503 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 14.56 sec -2013-09-10 18:53:00,511 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.73 sec -2013-09-10 18:53:01,519 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.73 sec -2013-09-10 18:53:02,526 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.73 sec -2013-09-10 18:53:03,533 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.73 sec -2013-09-10 18:53:04,539 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.73 sec -2013-09-10 18:53:05,545 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.73 sec -2013-09-10 18:53:06,550 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.73 sec -2013-09-10 18:53:07,557 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 29.73 sec -2013-09-10 18:53:08,563 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 29.73 sec -2013-09-10 18:53:09,569 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 29.73 sec -2013-09-10 18:53:10,575 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 44.01 sec -2013-09-10 18:53:11,598 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 58.47 sec -2013-09-10 18:53:12,604 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 58.47 sec -2013-09-10 18:53:13,609 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 58.47 sec -2013-09-10 18:53:14,615 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 58.47 sec -2013-09-10 18:53:15,620 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 58.47 sec -2013-09-10 18:53:16,627 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 65.64 sec -2013-09-10 18:53:17,634 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 65.64 sec -2013-09-10 18:53:18,640 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 65.64 sec -2013-09-10 18:53:19,646 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 65.64 sec -2013-09-10 18:53:20,653 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 71.27 sec -2013-09-10 18:53:21,659 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 71.27 sec -MapReduce Total cumulative CPU time: 1 minutes 11 seconds 270 msec -Ended Job = job_201309101627_0045 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0046 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 18:53:25,187 Stage-2 map = 0%, reduce = 0% -2013-09-10 18:53:27,196 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.42 sec -2013-09-10 18:53:28,202 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.42 sec -2013-09-10 18:53:29,207 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.42 sec -2013-09-10 18:53:30,211 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.42 sec -2013-09-10 18:53:31,216 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.42 sec -2013-09-10 18:53:32,220 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.42 sec -2013-09-10 18:53:33,226 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.42 sec -2013-09-10 18:53:34,231 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.42 sec -2013-09-10 18:53:35,237 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.16 sec -2013-09-10 18:53:36,243 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.16 sec -MapReduce Total cumulative CPU time: 3 seconds 160 msec -Ended Job = job_201309101627_0046 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 71.27 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.16 sec HDFS Read: 100911 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 14 seconds 430 msec -OK -Time taken: 56.439 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_609@mturlrep13_201309101853_355533849.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1183@mturlrep13_201309101853_289725544.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0047 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 18:53:55,838 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:54:02,865 Stage-1 map = 29%, reduce = 0% -2013-09-10 18:54:05,876 Stage-1 map = 43%, reduce = 0% -2013-09-10 18:54:08,894 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 16.8 sec -2013-09-10 18:54:09,901 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.85 sec -2013-09-10 18:54:10,909 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.85 sec -2013-09-10 18:54:11,915 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.85 sec -2013-09-10 18:54:12,921 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.85 sec -2013-09-10 18:54:13,927 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.85 sec -2013-09-10 18:54:14,932 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.85 sec -2013-09-10 18:54:15,938 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.85 sec -2013-09-10 18:54:16,943 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 34.85 sec -2013-09-10 18:54:17,949 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 34.85 sec -2013-09-10 18:54:18,954 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 34.85 sec -2013-09-10 18:54:19,959 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.85 sec -2013-09-10 18:54:20,964 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.85 sec -2013-09-10 18:54:21,970 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.35 sec -2013-09-10 18:54:22,975 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.35 sec -2013-09-10 18:54:23,980 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.35 sec -2013-09-10 18:54:24,986 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.35 sec -2013-09-10 18:54:25,991 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.35 sec -2013-09-10 18:54:26,997 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.35 sec -2013-09-10 18:54:28,002 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.35 sec -2013-09-10 18:54:29,008 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.35 sec -2013-09-10 18:54:30,014 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.35 sec -2013-09-10 18:54:31,021 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 74.39 sec -2013-09-10 18:54:32,027 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 74.39 sec -2013-09-10 18:54:33,033 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 84.05 sec -2013-09-10 18:54:34,038 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 84.05 sec -2013-09-10 18:54:35,044 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 84.05 sec -MapReduce Total cumulative CPU time: 1 minutes 24 seconds 50 msec -Ended Job = job_201309101627_0047 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0048 diff --git a/benchmark/hive/log/log_10m/log_10m_3 b/benchmark/hive/log/log_10m/log_10m_3 deleted file mode 100644 index 113f1e82f32..00000000000 --- a/benchmark/hive/log/log_10m/log_10m_3 +++ /dev/null @@ -1,2627 +0,0 @@ -start time: Вт. сент. 10 18:59:07 MSK 2013 -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6050@mturlrep13_201309101859_631689796.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6464@mturlrep13_201309101859_985586033.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0049 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 18:59:26,237 Stage-1 map = 0%, reduce = 0% -2013-09-10 18:59:33,264 Stage-1 map = 29%, reduce = 0% -2013-09-10 18:59:36,276 Stage-1 map = 43%, reduce = 0% -2013-09-10 18:59:39,296 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 36.62 sec -2013-09-10 18:59:40,304 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 36.62 sec -2013-09-10 18:59:41,312 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 36.62 sec -2013-09-10 18:59:42,319 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 36.62 sec -2013-09-10 18:59:43,325 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 36.62 sec -2013-09-10 18:59:44,331 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 36.62 sec -2013-09-10 18:59:45,337 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 36.62 sec -2013-09-10 18:59:46,343 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 36.62 sec -2013-09-10 18:59:47,349 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 36.62 sec -2013-09-10 18:59:48,355 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 36.62 sec -2013-09-10 18:59:49,362 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 36.62 sec -2013-09-10 18:59:50,368 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 36.62 sec -2013-09-10 18:59:51,374 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 73.0 sec -2013-09-10 18:59:52,379 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 73.0 sec -2013-09-10 18:59:53,385 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 73.0 sec -2013-09-10 18:59:54,390 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 73.0 sec -2013-09-10 18:59:55,396 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 73.0 sec -2013-09-10 18:59:56,401 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 73.0 sec -2013-09-10 18:59:57,407 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 73.0 sec -2013-09-10 18:59:58,412 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 73.0 sec -2013-09-10 18:59:59,418 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 73.0 sec -2013-09-10 19:00:00,425 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 80.12 sec -2013-09-10 19:00:01,431 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 80.12 sec -2013-09-10 19:00:02,437 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 80.12 sec -2013-09-10 19:00:03,443 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.63 sec -2013-09-10 19:00:04,449 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.63 sec -MapReduce Total cumulative CPU time: 1 minutes 29 seconds 630 msec -Ended Job = job_201309101627_0049 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0050 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:00:07,945 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:00:09,952 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-10 19:00:10,957 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-10 19:00:11,962 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-10 19:00:12,968 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-10 19:00:13,973 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-10 19:00:14,978 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-10 19:00:15,983 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.6 sec -2013-09-10 19:00:16,988 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.6 sec -2013-09-10 19:00:17,993 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.15 sec -2013-09-10 19:00:18,999 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.15 sec -MapReduce Total cumulative CPU time: 3 seconds 150 msec -Ended Job = job_201309101627_0050 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 89.63 sec HDFS Read: 74853201 HDFS Write: 148871 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.15 sec HDFS Read: 149640 HDFS Write: 414 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 32 seconds 780 msec -OK -Time taken: 62.773 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9300@mturlrep13_201309101900_1506085852.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9711@mturlrep13_201309101900_924615795.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0051 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:00:39,435 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:00:45,467 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.94 sec -2013-09-10 19:00:46,474 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.94 sec -2013-09-10 19:00:47,482 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.94 sec -2013-09-10 19:00:48,488 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.94 sec -2013-09-10 19:00:49,495 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.94 sec -2013-09-10 19:00:50,500 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.94 sec -2013-09-10 19:00:51,506 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.94 sec -2013-09-10 19:00:52,512 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 19.9 sec -2013-09-10 19:00:53,517 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 26.33 sec -2013-09-10 19:00:54,523 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 26.33 sec -2013-09-10 19:00:55,528 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 26.33 sec -2013-09-10 19:00:56,533 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 26.33 sec -2013-09-10 19:00:57,539 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 26.33 sec -2013-09-10 19:00:58,545 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 26.33 sec -2013-09-10 19:00:59,553 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.07 sec -2013-09-10 19:01:00,560 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.07 sec -2013-09-10 19:01:01,567 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.07 sec -MapReduce Total cumulative CPU time: 33 seconds 70 msec -Ended Job = job_201309101627_0051 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0052 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:01:04,042 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:01:06,051 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 19:01:07,057 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 19:01:08,063 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 19:01:09,068 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 19:01:10,073 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 19:01:11,079 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 19:01:12,084 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.87 sec -2013-09-10 19:01:13,090 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.87 sec -2013-09-10 19:01:14,095 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.22 sec -2013-09-10 19:01:15,102 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.22 sec -2013-09-10 19:01:16,107 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.22 sec -MapReduce Total cumulative CPU time: 2 seconds 220 msec -Ended Job = job_201309101627_0052 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 33.07 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.22 sec HDFS Read: 21897 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 35 seconds 290 msec -OK -Time taken: 46.54 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по строкам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11706@mturlrep13_201309101901_1717979914.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12124@mturlrep13_201309101901_499676225.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0053 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:01:37,263 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:01:44,301 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.64 sec -2013-09-10 19:01:45,310 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.64 sec -2013-09-10 19:01:46,319 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.64 sec -2013-09-10 19:01:47,326 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.64 sec -2013-09-10 19:01:48,333 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.64 sec -2013-09-10 19:01:49,340 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.64 sec -2013-09-10 19:01:50,346 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.68 sec -2013-09-10 19:01:51,353 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.68 sec -2013-09-10 19:01:52,360 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.68 sec -2013-09-10 19:01:53,367 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.68 sec -2013-09-10 19:01:54,373 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.68 sec -2013-09-10 19:01:55,379 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.68 sec -2013-09-10 19:01:56,386 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.68 sec -2013-09-10 19:01:57,392 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 23.68 sec -2013-09-10 19:01:58,401 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.08 sec -2013-09-10 19:01:59,407 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.08 sec -MapReduce Total cumulative CPU time: 31 seconds 80 msec -Ended Job = job_201309101627_0053 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0054 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:02:02,871 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:02:03,876 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-10 19:02:04,882 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-10 19:02:05,889 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-10 19:02:06,895 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-10 19:02:07,900 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-10 19:02:08,906 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-10 19:02:09,912 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-10 19:02:10,927 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-10 19:02:11,934 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.47 sec -2013-09-10 19:02:12,939 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.47 sec -2013-09-10 19:02:13,945 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.47 sec -MapReduce Total cumulative CPU time: 2 seconds 470 msec -Ended Job = job_201309101627_0054 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 31.08 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.47 sec HDFS Read: 23479 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 550 msec -OK -Time taken: 46.823 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14060@mturlrep13_201309101902_1323690190.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14501@mturlrep13_201309101902_2124415926.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0055 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:02:34,187 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:02:41,216 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:02:42,230 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.35 sec -2013-09-10 19:02:43,238 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.35 sec -2013-09-10 19:02:44,246 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.35 sec -2013-09-10 19:02:45,253 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.35 sec -2013-09-10 19:02:46,260 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.35 sec -2013-09-10 19:02:47,266 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.35 sec -2013-09-10 19:02:48,273 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.35 sec -2013-09-10 19:02:49,280 Stage-1 map = 72%, reduce = 8%, Cumulative CPU 20.35 sec -2013-09-10 19:02:50,286 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.1 sec -2013-09-10 19:02:51,292 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.1 sec -2013-09-10 19:02:52,297 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.1 sec -2013-09-10 19:02:53,302 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.1 sec -2013-09-10 19:02:54,309 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.1 sec -2013-09-10 19:02:55,315 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.1 sec -2013-09-10 19:02:56,321 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.1 sec -2013-09-10 19:02:57,327 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.1 sec -2013-09-10 19:02:58,333 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 40.1 sec -2013-09-10 19:02:59,340 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 54.51 sec -2013-09-10 19:03:00,346 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 54.51 sec -2013-09-10 19:03:01,352 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 54.51 sec -MapReduce Total cumulative CPU time: 54 seconds 510 msec -Ended Job = job_201309101627_0055 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0056 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:03:03,804 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:03:11,833 Stage-2 map = 50%, reduce = 0% -2013-09-10 19:03:13,842 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.34 sec -2013-09-10 19:03:14,847 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.34 sec -2013-09-10 19:03:15,851 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.34 sec -2013-09-10 19:03:16,856 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.34 sec -2013-09-10 19:03:17,860 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.34 sec -2013-09-10 19:03:18,864 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.34 sec -2013-09-10 19:03:19,868 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.34 sec -2013-09-10 19:03:20,873 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.34 sec -2013-09-10 19:03:21,878 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.34 sec -2013-09-10 19:03:22,883 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.49 sec -2013-09-10 19:03:23,887 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.49 sec -2013-09-10 19:03:24,892 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.49 sec -MapReduce Total cumulative CPU time: 18 seconds 490 msec -Ended Job = job_201309101627_0056 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 54.51 sec HDFS Read: 27820105 HDFS Write: 79726641 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.49 sec HDFS Read: 79727410 HDFS Write: 275 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 13 seconds 0 msec -OK -Time taken: 60.622 seconds, Fetched: 10 row(s) -hive> quit; --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16498@mturlrep13_201309101903_1119256282.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16962@mturlrep13_201309101903_443904173.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0057 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:03:45,087 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:03:52,116 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:03:54,132 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.1 sec -2013-09-10 19:03:55,140 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.1 sec -2013-09-10 19:03:56,148 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.1 sec -2013-09-10 19:03:57,155 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.1 sec -2013-09-10 19:03:58,161 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.1 sec -2013-09-10 19:03:59,167 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.1 sec -2013-09-10 19:04:00,174 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.1 sec -2013-09-10 19:04:01,181 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.1 sec -2013-09-10 19:04:02,187 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 22.1 sec -2013-09-10 19:04:03,193 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 33.77 sec -2013-09-10 19:04:04,199 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.59 sec -2013-09-10 19:04:05,205 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.59 sec -2013-09-10 19:04:06,212 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.59 sec -2013-09-10 19:04:07,218 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.59 sec -2013-09-10 19:04:08,225 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.59 sec -2013-09-10 19:04:09,232 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.59 sec -2013-09-10 19:04:10,238 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.59 sec -2013-09-10 19:04:11,244 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 45.59 sec -2013-09-10 19:04:12,252 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 61.87 sec -2013-09-10 19:04:13,259 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 61.87 sec -MapReduce Total cumulative CPU time: 1 minutes 1 seconds 870 msec -Ended Job = job_201309101627_0057 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0058 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:04:15,702 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:04:23,732 Stage-2 map = 50%, reduce = 0% -2013-09-10 19:04:25,740 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.18 sec -2013-09-10 19:04:26,746 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.18 sec -2013-09-10 19:04:27,751 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.18 sec -2013-09-10 19:04:28,757 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.18 sec -2013-09-10 19:04:29,762 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.18 sec -2013-09-10 19:04:30,767 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.18 sec -2013-09-10 19:04:31,782 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.18 sec -2013-09-10 19:04:32,787 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.18 sec -2013-09-10 19:04:33,793 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.18 sec -2013-09-10 19:04:34,799 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.89 sec -2013-09-10 19:04:35,810 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.89 sec -MapReduce Total cumulative CPU time: 17 seconds 890 msec -Ended Job = job_201309101627_0058 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 61.87 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.89 sec HDFS Read: 79727313 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 19 seconds 760 msec -OK -Time taken: 60.639 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация чуть сложнее.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18982@mturlrep13_201309101904_1735457209.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19424@mturlrep13_201309101904_1653965293.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0059 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:04:57,169 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:05:04,197 Stage-1 map = 39%, reduce = 0% -2013-09-10 19:05:05,209 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.06 sec -2013-09-10 19:05:06,217 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.06 sec -2013-09-10 19:05:07,227 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.06 sec -2013-09-10 19:05:08,235 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.06 sec -2013-09-10 19:05:09,241 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.06 sec -2013-09-10 19:05:10,247 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.06 sec -2013-09-10 19:05:11,254 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.06 sec -2013-09-10 19:05:12,260 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.06 sec -2013-09-10 19:05:13,267 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 24.06 sec -2013-09-10 19:05:14,273 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 34.72 sec -2013-09-10 19:05:15,279 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.1 sec -2013-09-10 19:05:16,285 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.1 sec -2013-09-10 19:05:17,292 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.1 sec -2013-09-10 19:05:18,298 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.1 sec -2013-09-10 19:05:19,303 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.1 sec -2013-09-10 19:05:20,309 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.1 sec -2013-09-10 19:05:21,315 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.1 sec -2013-09-10 19:05:22,322 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 46.1 sec -2013-09-10 19:05:23,330 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 61.52 sec -2013-09-10 19:05:24,337 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 61.52 sec -MapReduce Total cumulative CPU time: 1 minutes 1 seconds 520 msec -Ended Job = job_201309101627_0059 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0060 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:05:27,785 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:05:34,809 Stage-2 map = 50%, reduce = 0% -2013-09-10 19:05:37,822 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.13 sec -2013-09-10 19:05:38,827 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.13 sec -2013-09-10 19:05:39,832 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.13 sec -2013-09-10 19:05:40,838 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.13 sec -2013-09-10 19:05:41,843 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.13 sec -2013-09-10 19:05:42,848 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.13 sec -2013-09-10 19:05:43,853 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.13 sec -2013-09-10 19:05:44,858 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.13 sec -2013-09-10 19:05:45,864 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.13 sec -2013-09-10 19:05:46,870 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.06 sec -2013-09-10 19:05:47,875 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.06 sec -2013-09-10 19:05:48,882 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.06 sec -MapReduce Total cumulative CPU time: 20 seconds 60 msec -Ended Job = job_201309101627_0060 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 61.52 sec HDFS Read: 30310112 HDFS Write: 84160093 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 20.06 sec HDFS Read: 84160862 HDFS Write: 297 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 21 seconds 580 msec -OK -Time taken: 61.589 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по числу и строке, большое количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22049@mturlrep13_201309101905_1061955650.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22475@mturlrep13_201309101906_1586052270.txt -hive> SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0061 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:06:10,398 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:06:17,425 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:06:20,444 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.97 sec -2013-09-10 19:06:21,451 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.97 sec -2013-09-10 19:06:22,460 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.97 sec -2013-09-10 19:06:23,466 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.97 sec -2013-09-10 19:06:24,472 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.97 sec -2013-09-10 19:06:25,479 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.97 sec -2013-09-10 19:06:26,484 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.97 sec -2013-09-10 19:06:27,496 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.97 sec -2013-09-10 19:06:28,502 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 26.97 sec -2013-09-10 19:06:29,508 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 26.97 sec -2013-09-10 19:06:30,514 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 53.98 sec -2013-09-10 19:06:31,520 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 53.98 sec -2013-09-10 19:06:32,526 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 53.98 sec -2013-09-10 19:06:33,532 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 53.98 sec -2013-09-10 19:06:34,538 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 53.98 sec -2013-09-10 19:06:35,544 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 53.98 sec -2013-09-10 19:06:36,550 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 53.98 sec -2013-09-10 19:06:37,559 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 61.23 sec -2013-09-10 19:06:38,565 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.59 sec -2013-09-10 19:06:39,571 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.59 sec -MapReduce Total cumulative CPU time: 1 minutes 8 seconds 590 msec -Ended Job = job_201309101627_0061 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0062 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:06:43,004 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:06:53,038 Stage-2 map = 50%, reduce = 0% -2013-09-10 19:06:57,053 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.42 sec -2013-09-10 19:06:58,059 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.42 sec -2013-09-10 19:06:59,064 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.42 sec -2013-09-10 19:07:00,068 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.42 sec -2013-09-10 19:07:01,073 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.42 sec -2013-09-10 19:07:02,078 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.42 sec -2013-09-10 19:07:03,083 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.42 sec -2013-09-10 19:07:04,088 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.42 sec -2013-09-10 19:07:05,093 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.42 sec -2013-09-10 19:07:06,099 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.68 sec -2013-09-10 19:07:07,105 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.68 sec -2013-09-10 19:07:08,110 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.68 sec -MapReduce Total cumulative CPU time: 22 seconds 680 msec -Ended Job = job_201309101627_0062 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 68.59 sec HDFS Read: 57312623 HDFS Write: 55475412 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 22.68 sec HDFS Read: 55476181 HDFS Write: 246 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 31 seconds 270 msec -OK -Time taken: 67.438 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24441@mturlrep13_201309101907_2142712344.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24859@mturlrep13_201309101907_1554318689.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0063 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:07:28,202 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:07:35,236 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:07:38,249 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:07:40,264 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 17.49 sec -2013-09-10 19:07:41,271 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.27 sec -2013-09-10 19:07:42,285 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.27 sec -2013-09-10 19:07:43,293 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.27 sec -2013-09-10 19:07:44,300 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.27 sec -2013-09-10 19:07:45,306 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.27 sec -2013-09-10 19:07:46,312 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.27 sec -2013-09-10 19:07:47,319 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.27 sec -2013-09-10 19:07:48,325 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.27 sec -2013-09-10 19:07:49,332 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.27 sec -2013-09-10 19:07:50,339 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.27 sec -2013-09-10 19:07:51,346 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.27 sec -2013-09-10 19:07:52,352 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.27 sec -2013-09-10 19:07:53,359 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.38 sec -2013-09-10 19:07:54,368 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.38 sec -2013-09-10 19:07:55,374 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.38 sec -2013-09-10 19:07:56,380 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.38 sec -2013-09-10 19:07:57,386 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.38 sec -2013-09-10 19:07:58,393 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.38 sec -2013-09-10 19:07:59,398 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.38 sec -2013-09-10 19:08:00,404 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.38 sec -2013-09-10 19:08:01,410 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.38 sec -2013-09-10 19:08:02,416 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.38 sec -2013-09-10 19:08:03,422 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 68.38 sec -2013-09-10 19:08:04,428 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 68.38 sec -2013-09-10 19:08:05,436 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 90.47 sec -2013-09-10 19:08:06,443 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 90.47 sec -2013-09-10 19:08:07,449 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 90.47 sec -MapReduce Total cumulative CPU time: 1 minutes 30 seconds 470 msec -Ended Job = job_201309101627_0063 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0064 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:08:10,060 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:08:21,113 Stage-2 map = 46%, reduce = 0% -2013-09-10 19:08:24,124 Stage-2 map = 50%, reduce = 0% -2013-09-10 19:08:27,134 Stage-2 map = 96%, reduce = 0% -2013-09-10 19:08:29,142 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.42 sec -2013-09-10 19:08:30,148 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.42 sec -2013-09-10 19:08:31,153 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.42 sec -2013-09-10 19:08:32,157 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.42 sec -2013-09-10 19:08:33,162 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.42 sec -2013-09-10 19:08:34,166 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.42 sec -2013-09-10 19:08:35,170 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.42 sec -2013-09-10 19:08:36,175 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.42 sec -2013-09-10 19:08:37,180 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.42 sec -2013-09-10 19:08:38,185 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.42 sec -2013-09-10 19:08:39,191 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.83 sec -2013-09-10 19:08:40,196 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.83 sec -2013-09-10 19:08:41,202 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.83 sec -MapReduce Total cumulative CPU time: 28 seconds 830 msec -Ended Job = job_201309101627_0064 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 90.47 sec HDFS Read: 84536695 HDFS Write: 146202868 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 28.83 sec HDFS Read: 146210123 HDFS Write: 256 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 59 seconds 300 msec -OK -Time taken: 82.888 seconds, Fetched: 10 row(s) -hive> quit; --- ещё более сложная агрегация.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26996@mturlrep13_201309101908_242539464.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27403@mturlrep13_201309101908_1527602828.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0065 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:09:00,979 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:09:08,005 Stage-1 map = 39%, reduce = 0% -2013-09-10 19:09:11,017 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:09:13,032 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.58 sec -2013-09-10 19:09:14,039 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.58 sec -2013-09-10 19:09:15,046 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.58 sec -2013-09-10 19:09:16,052 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.58 sec -2013-09-10 19:09:17,058 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.58 sec -2013-09-10 19:09:18,064 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.58 sec -2013-09-10 19:09:19,069 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.58 sec -2013-09-10 19:09:20,074 Stage-1 map = 68%, reduce = 8%, Cumulative CPU 33.58 sec -2013-09-10 19:09:21,083 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 33.58 sec -2013-09-10 19:09:22,088 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 33.58 sec -2013-09-10 19:09:23,093 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 33.58 sec -2013-09-10 19:09:24,098 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.58 sec -2013-09-10 19:09:25,103 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.58 sec -2013-09-10 19:09:26,109 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-10 19:09:27,114 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-10 19:09:28,119 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-10 19:09:29,124 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-10 19:09:30,130 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-10 19:09:31,135 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-10 19:09:32,140 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-10 19:09:33,148 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.77 sec -2013-09-10 19:09:34,154 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.77 sec -2013-09-10 19:09:35,160 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.77 sec -MapReduce Total cumulative CPU time: 1 minutes 17 seconds 770 msec -Ended Job = job_201309101627_0065 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 77.77 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 770 msec -OK -Time taken: 44.016 seconds, Fetched: 10 row(s) -hive> quit; --- то же самое, но без сортировки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28914@mturlrep13_201309101909_482341198.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29362@mturlrep13_201309101909_120722838.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0066 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:09:54,681 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:10:01,708 Stage-1 map = 7%, reduce = 0% -2013-09-10 19:10:04,721 Stage-1 map = 22%, reduce = 0% -2013-09-10 19:10:07,733 Stage-1 map = 29%, reduce = 0% -2013-09-10 19:10:10,746 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:10:13,758 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:10:15,772 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.31 sec -2013-09-10 19:10:16,779 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.31 sec -2013-09-10 19:10:17,787 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.31 sec -2013-09-10 19:10:18,793 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.31 sec -2013-09-10 19:10:19,799 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.31 sec -2013-09-10 19:10:20,804 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.31 sec -2013-09-10 19:10:21,810 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.31 sec -2013-09-10 19:10:22,816 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 55.31 sec -2013-09-10 19:10:23,822 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 55.31 sec -2013-09-10 19:10:24,828 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 55.31 sec -2013-09-10 19:10:25,834 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.31 sec -2013-09-10 19:10:26,840 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.31 sec -2013-09-10 19:10:27,846 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.31 sec -2013-09-10 19:10:28,852 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.31 sec -2013-09-10 19:10:29,858 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.31 sec -2013-09-10 19:10:30,863 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.31 sec -2013-09-10 19:10:31,891 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 55.31 sec -2013-09-10 19:10:32,897 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 55.31 sec -2013-09-10 19:10:33,903 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 79.15 sec -2013-09-10 19:10:34,909 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 79.15 sec -2013-09-10 19:10:35,915 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 79.15 sec -2013-09-10 19:10:36,921 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.79 sec -2013-09-10 19:10:37,926 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 106.79 sec -2013-09-10 19:10:38,931 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 106.79 sec -2013-09-10 19:10:39,936 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 106.79 sec -2013-09-10 19:10:40,941 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 106.79 sec -2013-09-10 19:10:41,949 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 117.74 sec -2013-09-10 19:10:42,955 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 117.74 sec -2013-09-10 19:10:43,961 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 117.74 sec -2013-09-10 19:10:44,967 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 117.74 sec -2013-09-10 19:10:45,973 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 117.74 sec -2013-09-10 19:10:47,436 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 117.74 sec -2013-09-10 19:10:48,441 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 117.74 sec -2013-09-10 19:10:49,447 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 117.74 sec -2013-09-10 19:10:50,453 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 117.74 sec -2013-09-10 19:10:51,459 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 117.74 sec -2013-09-10 19:10:52,464 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 117.74 sec -2013-09-10 19:10:53,470 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 117.74 sec -2013-09-10 19:10:54,477 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 117.74 sec -2013-09-10 19:10:55,483 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 132.78 sec -2013-09-10 19:10:56,489 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 149.34 sec -2013-09-10 19:10:57,495 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 149.34 sec -MapReduce Total cumulative CPU time: 2 minutes 29 seconds 340 msec -Ended Job = job_201309101627_0066 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0067 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:11:00,980 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:11:14,017 Stage-2 map = 28%, reduce = 0% -2013-09-10 19:11:20,035 Stage-2 map = 50%, reduce = 0% -2013-09-10 19:11:26,052 Stage-2 map = 78%, reduce = 0% -2013-09-10 19:11:37,085 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:38,090 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:39,095 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:40,100 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:41,105 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:42,110 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:43,115 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:44,120 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:45,125 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:46,130 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:47,135 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 44.56 sec -2013-09-10 19:11:48,141 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 44.56 sec -2013-09-10 19:11:49,146 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 51.16 sec -2013-09-10 19:11:50,151 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 51.16 sec -2013-09-10 19:11:51,156 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 51.16 sec -MapReduce Total cumulative CPU time: 51 seconds 160 msec -Ended Job = job_201309101627_0067 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 149.34 sec HDFS Read: 84944733 HDFS Write: 241346048 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 51.16 sec HDFS Read: 241349358 HDFS Write: 268 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 20 seconds 500 msec -OK -Time taken: 126.4 seconds, Fetched: 10 row(s) -hive> quit; --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32415@mturlrep13_201309101911_1857306718.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_382@mturlrep13_201309101912_1225260718.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0068 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-10 19:12:10,928 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:12:15,958 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.87 sec -2013-09-10 19:12:16,966 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.87 sec -2013-09-10 19:12:17,974 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.87 sec -2013-09-10 19:12:18,980 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.87 sec -2013-09-10 19:12:19,986 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.87 sec -2013-09-10 19:12:20,991 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.7 sec -2013-09-10 19:12:21,997 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.7 sec -2013-09-10 19:12:23,003 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 15.7 sec -MapReduce Total cumulative CPU time: 15 seconds 700 msec -Ended Job = job_201309101627_0068 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 15.7 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 15 seconds 700 msec -OK -Time taken: 21.644 seconds -hive> quit; --- мощная фильтрация по столбцу типа UInt64.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1792@mturlrep13_201309101912_2025297481.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2232@mturlrep13_201309101912_482585319.txt -hive> SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0069 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 19:12:42,684 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:12:49,731 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 8.94 sec -2013-09-10 19:12:50,740 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.06 sec -2013-09-10 19:12:51,747 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.06 sec -2013-09-10 19:12:52,753 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.06 sec -2013-09-10 19:12:53,760 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.06 sec -2013-09-10 19:12:54,765 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.06 sec -2013-09-10 19:12:55,771 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.06 sec -2013-09-10 19:12:56,777 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.06 sec -2013-09-10 19:12:57,783 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.42 sec -2013-09-10 19:12:58,788 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.42 sec -2013-09-10 19:12:59,793 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.42 sec -2013-09-10 19:13:00,798 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.42 sec -2013-09-10 19:13:01,802 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.42 sec -2013-09-10 19:13:02,809 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.55 sec -2013-09-10 19:13:03,815 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.55 sec -2013-09-10 19:13:04,821 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.55 sec -MapReduce Total cumulative CPU time: 37 seconds 550 msec -Ended Job = job_201309101627_0069 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 37.55 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 37 seconds 550 msec -OK -8428 -Time taken: 31.899 seconds, Fetched: 1 row(s) -hive> quit; --- фильтрация по поиску подстроки в строке.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3508@mturlrep13_201309101913_1663904191.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3940@mturlrep13_201309101913_52737902.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0070 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:13:26,058 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:13:33,084 Stage-1 map = 29%, reduce = 0% -2013-09-10 19:13:35,102 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.32 sec -2013-09-10 19:13:36,110 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.32 sec -2013-09-10 19:13:37,119 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.32 sec -2013-09-10 19:13:38,125 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.32 sec -2013-09-10 19:13:39,132 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.32 sec -2013-09-10 19:13:40,139 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.32 sec -2013-09-10 19:13:41,145 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.32 sec -2013-09-10 19:13:42,152 Stage-1 map = 80%, reduce = 8%, Cumulative CPU 19.32 sec -2013-09-10 19:13:43,158 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 19.32 sec -2013-09-10 19:13:44,164 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.92 sec -2013-09-10 19:13:45,170 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.92 sec -2013-09-10 19:13:46,175 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.92 sec -2013-09-10 19:13:47,181 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.92 sec -2013-09-10 19:13:48,189 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.12 sec -2013-09-10 19:13:49,195 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.12 sec -2013-09-10 19:13:50,202 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.12 sec -MapReduce Total cumulative CPU time: 42 seconds 120 msec -Ended Job = job_201309101627_0070 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0071 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:13:52,707 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:13:54,716 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-10 19:13:55,722 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-10 19:13:56,727 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-10 19:13:57,732 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-10 19:13:58,737 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-10 19:13:59,742 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-10 19:14:00,747 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-10 19:14:01,752 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-10 19:14:02,757 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -2013-09-10 19:14:03,763 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -2013-09-10 19:14:04,770 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -MapReduce Total cumulative CPU time: 2 seconds 350 msec -Ended Job = job_201309101627_0071 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.12 sec HDFS Read: 136675723 HDFS Write: 5172 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.35 sec HDFS Read: 5941 HDFS Write: 984 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 470 msec -OK -Time taken: 48.699 seconds, Fetched: 10 row(s) -hive> quit; --- вынимаем большие столбцы, фильтрация по строке.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6191@mturlrep13_201309101914_772958419.txt -hive> ; -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6615@mturlrep13_201309101914_691833901.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0072 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:14:25,492 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:14:32,521 Stage-1 map = 22%, reduce = 0% -2013-09-10 19:14:35,534 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:14:36,547 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.89 sec -2013-09-10 19:14:37,555 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.89 sec -2013-09-10 19:14:38,563 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.89 sec -2013-09-10 19:14:39,570 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.89 sec -2013-09-10 19:14:40,577 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.89 sec -2013-09-10 19:14:41,584 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.89 sec -2013-09-10 19:14:42,592 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.89 sec -2013-09-10 19:14:43,598 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 23.89 sec -2013-09-10 19:14:44,604 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 23.89 sec -2013-09-10 19:14:45,611 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 23.89 sec -2013-09-10 19:14:46,617 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 35.11 sec -2013-09-10 19:14:47,623 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.67 sec -2013-09-10 19:14:48,629 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.67 sec -2013-09-10 19:14:49,638 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.04 sec -2013-09-10 19:14:50,644 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.04 sec -2013-09-10 19:14:51,650 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.04 sec -MapReduce Total cumulative CPU time: 52 seconds 40 msec -Ended Job = job_201309101627_0072 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0073 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:14:55,180 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:14:56,185 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 19:14:57,191 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 19:14:58,196 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 19:14:59,201 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 19:15:00,206 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 19:15:01,211 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 19:15:02,217 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 19:15:03,222 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-10 19:15:04,228 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -2013-09-10 19:15:05,235 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -2013-09-10 19:15:06,241 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -MapReduce Total cumulative CPU time: 2 seconds 190 msec -Ended Job = job_201309101627_0073 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 52.04 sec HDFS Read: 298803179 HDFS Write: 12221 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.19 sec HDFS Read: 12988 HDFS Write: 2646 SUCCESS -Total MapReduce CPU Time Spent: 54 seconds 230 msec -OK -Time taken: 51.01 seconds, Fetched: 10 row(s) -hive> quit; --- чуть больше столбцы.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8983@mturlrep13_201309101915_314385462.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9580@mturlrep13_201309101915_1458362516.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0074 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 19:15:27,409 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:15:37,449 Stage-1 map = 7%, reduce = 0% -2013-09-10 19:15:40,463 Stage-1 map = 14%, reduce = 0% -2013-09-10 19:15:43,482 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:44,488 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:45,495 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:46,501 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:47,507 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:48,512 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:49,517 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:50,522 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:51,527 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:52,532 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:53,537 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:54,543 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:55,548 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:56,553 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:57,558 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 38.02 sec -2013-09-10 19:15:58,567 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 61.53 sec -2013-09-10 19:15:59,573 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 88.84 sec -2013-09-10 19:16:00,579 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 88.84 sec -2013-09-10 19:16:01,584 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 88.84 sec -2013-09-10 19:16:02,590 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 88.84 sec -2013-09-10 19:16:03,595 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 88.84 sec -2013-09-10 19:16:04,600 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 88.84 sec -2013-09-10 19:16:05,606 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:06,611 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:07,616 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:08,621 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:09,626 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:10,631 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:11,641 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:12,707 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:13,712 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:14,717 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:15,723 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:16,728 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:17,732 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:18,737 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:19,741 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:20,746 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:21,750 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:22,755 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:23,759 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:24,764 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:25,782 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 88.84 sec -2013-09-10 19:16:26,787 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 131.41 sec -2013-09-10 19:16:27,792 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 131.41 sec -2013-09-10 19:16:28,796 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 170.49 sec -2013-09-10 19:16:29,801 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 170.49 sec -2013-09-10 19:16:30,805 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 170.49 sec -2013-09-10 19:16:31,810 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 170.49 sec -2013-09-10 19:16:32,820 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 173.54 sec -2013-09-10 19:16:33,825 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 173.54 sec -MapReduce Total cumulative CPU time: 2 minutes 53 seconds 540 msec -Ended Job = job_201309101627_0074 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 173.54 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 53 seconds 540 msec -OK -Time taken: 76.991 seconds, Fetched: 10 row(s) -hive> quit; --- плохой запрос - вынимаем все столбцы.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11265@mturlrep13_201309101916_250616078.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11680@mturlrep13_201309101916_986198421.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -FAILED: SemanticException [Error 10004]: Line 1:68 Invalid table alias or column reference 'EventTime': (possible column names are: searchphrase) -hive> quit; --- большая сортировка.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11926@mturlrep13_201309101916_887631966.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12346@mturlrep13_201309101916_544482405.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0075 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 19:17:08,453 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:17:15,483 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:17:16,496 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.34 sec -2013-09-10 19:17:17,503 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.34 sec -2013-09-10 19:17:18,511 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.34 sec -2013-09-10 19:17:19,517 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.34 sec -2013-09-10 19:17:20,523 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.34 sec -2013-09-10 19:17:21,529 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.34 sec -2013-09-10 19:17:22,536 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.34 sec -2013-09-10 19:17:23,542 Stage-1 map = 97%, reduce = 8%, Cumulative CPU 27.55 sec -2013-09-10 19:17:24,548 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 37.45 sec -2013-09-10 19:17:25,553 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 37.45 sec -2013-09-10 19:17:26,558 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.45 sec -2013-09-10 19:17:27,563 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.45 sec -2013-09-10 19:17:28,568 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.45 sec -2013-09-10 19:17:29,573 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.45 sec -2013-09-10 19:17:30,578 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.45 sec -2013-09-10 19:17:31,582 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.45 sec -2013-09-10 19:17:32,589 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.38 sec -2013-09-10 19:17:33,596 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.38 sec -2013-09-10 19:17:34,601 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.38 sec -MapReduce Total cumulative CPU time: 44 seconds 380 msec -Ended Job = job_201309101627_0075 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 44.38 sec HDFS Read: 27820105 HDFS Write: 666 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 380 msec -OK -ялта интурист -! как одеть трехнедельного ребенка при температуре 20 градусов -! отель rattana beach hotel 3* -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -!( центробежный скважинный калибр форумы) -!(!(storm master silmarils)) -!(!(storm master silmarils)) -!(!(title:(схема sputnik hi 4000))) -!(44-фз о контрактной системе) -Time taken: 35.966 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка по строкам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13633@mturlrep13_201309101917_1660065208.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14049@mturlrep13_201309101917_1265821928.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -FAILED: SemanticException [Error 10004]: Line 1:68 Invalid table alias or column reference 'EventTime': (possible column names are: searchphrase) -hive> quit; --- большая сортировка по кортежу.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14295@mturlrep13_201309101917_1504626918.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14730@mturlrep13_201309101918_1497798883.txt -hive> SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0076 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:18:11,561 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:18:18,592 Stage-1 map = 14%, reduce = 0% -2013-09-10 19:18:21,605 Stage-1 map = 22%, reduce = 0% -2013-09-10 19:18:24,620 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:18:27,635 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:18:29,651 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.35 sec -2013-09-10 19:18:30,659 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.35 sec -2013-09-10 19:18:31,666 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.35 sec -2013-09-10 19:18:32,673 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.35 sec -2013-09-10 19:18:33,685 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.35 sec -2013-09-10 19:18:34,690 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.35 sec -2013-09-10 19:18:35,696 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.35 sec -2013-09-10 19:18:36,702 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 44.35 sec -2013-09-10 19:18:37,708 Stage-1 map = 65%, reduce = 13%, Cumulative CPU 44.35 sec -2013-09-10 19:18:38,714 Stage-1 map = 65%, reduce = 13%, Cumulative CPU 44.35 sec -2013-09-10 19:18:39,719 Stage-1 map = 65%, reduce = 13%, Cumulative CPU 44.35 sec -2013-09-10 19:18:40,725 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 44.35 sec -2013-09-10 19:18:41,731 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 44.35 sec -2013-09-10 19:18:42,738 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 74.28 sec -2013-09-10 19:18:43,744 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 74.28 sec -2013-09-10 19:18:44,750 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 74.28 sec -2013-09-10 19:18:45,756 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 74.28 sec -2013-09-10 19:18:46,761 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 81.57 sec -2013-09-10 19:18:47,767 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 81.57 sec -2013-09-10 19:18:48,773 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 90.73 sec -2013-09-10 19:18:49,778 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 90.73 sec -2013-09-10 19:18:50,783 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 90.73 sec -2013-09-10 19:18:51,789 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 90.73 sec -2013-09-10 19:18:52,794 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 90.73 sec -2013-09-10 19:18:53,800 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 90.73 sec -2013-09-10 19:18:54,806 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 90.73 sec -2013-09-10 19:18:55,811 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 90.73 sec -2013-09-10 19:18:56,817 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 90.73 sec -2013-09-10 19:18:57,822 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 90.73 sec -2013-09-10 19:18:58,828 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 90.73 sec -2013-09-10 19:18:59,836 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 99.84 sec -2013-09-10 19:19:00,841 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 99.84 sec -2013-09-10 19:19:01,847 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 99.84 sec -2013-09-10 19:19:02,852 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 99.84 sec -2013-09-10 19:19:03,858 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 99.84 sec -2013-09-10 19:19:04,880 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 113.54 sec -2013-09-10 19:19:05,886 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 113.54 sec -2013-09-10 19:19:06,891 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 113.54 sec -MapReduce Total cumulative CPU time: 1 minutes 53 seconds 540 msec -Ended Job = job_201309101627_0076 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0077 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:19:09,375 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:19:11,384 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.89 sec -2013-09-10 19:19:12,388 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.89 sec -2013-09-10 19:19:13,393 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.89 sec -2013-09-10 19:19:14,398 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.89 sec -2013-09-10 19:19:15,403 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.89 sec -2013-09-10 19:19:16,407 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.89 sec -2013-09-10 19:19:17,412 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.89 sec -2013-09-10 19:19:18,417 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.89 sec -2013-09-10 19:19:19,422 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.31 sec -2013-09-10 19:19:20,428 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.31 sec -2013-09-10 19:19:21,433 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.31 sec -MapReduce Total cumulative CPU time: 2 seconds 310 msec -Ended Job = job_201309101627_0077 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 113.54 sec HDFS Read: 117363067 HDFS Write: 794 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.31 sec HDFS Read: 1563 HDFS Write: 571 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 55 seconds 850 msec -OK -Time taken: 79.996 seconds, Fetched: 19 row(s) -hive> quit; --- считаем средние длины URL для крупных счётчиков.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16867@mturlrep13_201309101919_377804928.txt -hive> ; -hive> quit; - -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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17304@mturlrep13_201309101919_1014637585.txt -hive> 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -NoViableAltException(26@[()* loopback of 364:5: ( ( KW_NOT precedenceEqualNegatableOperator notExpr= precedenceBitwiseOrExpression ) -> ^( KW_NOT ^( precedenceEqualNegatableOperator $precedenceEqualExpression $notExpr) ) | ( precedenceEqualOperator equalExpr= precedenceBitwiseOrExpression ) -> ^( precedenceEqualOperator $precedenceEqualExpression $equalExpr) | ( KW_NOT KW_IN expressions ) -> ^( KW_NOT ^( TOK_FUNCTION KW_IN $precedenceEqualExpression expressions ) ) | ( KW_IN expressions ) -> ^( TOK_FUNCTION KW_IN $precedenceEqualExpression expressions ) | ( KW_NOT KW_BETWEEN (min= precedenceBitwiseOrExpression ) KW_AND (max= precedenceBitwiseOrExpression ) ) -> ^( TOK_FUNCTION Identifier["between"] KW_TRUE $left $min $max) | ( KW_BETWEEN (min= precedenceBitwiseOrExpression ) KW_AND (max= precedenceBitwiseOrExpression ) ) -> ^( TOK_FUNCTION Identifier["between"] KW_FALSE $left $min $max) )*]) -FAILED: ParseException line 1:53 cannot recognize input near 'IN' 'Referer' ')' in expression specification - -hive> quit; --- то же самое, но с разбивкой по доменам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17527@mturlrep13_201309101919_1522610318.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17943@mturlrep13_201309101919_574580111.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0078 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-10 19:19:51,666 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:20:04,716 Stage-1 map = 7%, reduce = 0% -2013-09-10 19:20:10,740 Stage-1 map = 14%, reduce = 0% -2013-09-10 19:20:16,762 Stage-1 map = 22%, reduce = 0% -2013-09-10 19:20:25,794 Stage-1 map = 29%, reduce = 0% -2013-09-10 19:20:31,815 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:20:37,834 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:20:44,864 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.2 sec -2013-09-10 19:20:45,870 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.2 sec -2013-09-10 19:20:46,875 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.2 sec -2013-09-10 19:20:47,879 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.2 sec -2013-09-10 19:20:48,884 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.2 sec -2013-09-10 19:20:49,889 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.2 sec -2013-09-10 19:20:50,894 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 113.2 sec -2013-09-10 19:20:51,898 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:20:52,903 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:20:53,908 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:20:54,912 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:20:55,917 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:20:56,922 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:20:57,970 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:20:58,975 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:20:59,980 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:00,984 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:01,988 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:02,993 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:03,997 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:05,002 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:06,006 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:07,011 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:08,021 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:09,025 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:10,030 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:11,035 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:12,039 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:13,044 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:14,048 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:15,052 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:16,057 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:17,061 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:18,078 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:19,083 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:20,088 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:21,093 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:22,097 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:23,102 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:24,106 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:25,111 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:26,116 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:27,120 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:28,125 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:29,130 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:30,134 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:31,139 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:32,143 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 113.2 sec -2013-09-10 19:21:33,149 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 164.14 sec -2013-09-10 19:21:34,153 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 164.14 sec -2013-09-10 19:21:35,157 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 164.14 sec -2013-09-10 19:21:36,166 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 164.14 sec -2013-09-10 19:21:37,170 Stage-1 map = 97%, reduce = 25%, Cumulative CPU 164.14 sec -2013-09-10 19:21:38,174 Stage-1 map = 97%, reduce = 25%, Cumulative CPU 164.14 sec -2013-09-10 19:21:39,179 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 222.36 sec -2013-09-10 19:21:40,183 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 222.36 sec -2013-09-10 19:21:41,188 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 222.36 sec -2013-09-10 19:21:42,193 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 223.78 sec -2013-09-10 19:21:43,200 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 226.08 sec -2013-09-10 19:21:44,205 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 226.08 sec -2013-09-10 19:21:45,210 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 226.08 sec -MapReduce Total cumulative CPU time: 3 minutes 46 seconds 80 msec -Ended Job = job_201309101627_0078 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 226.08 sec HDFS Read: 7797536 HDFS Write: 1080 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 46 seconds 80 msec -OK -Time taken: 124.725 seconds, Fetched: 1 row(s) -hive> quit; --- много тупых агрегатных функций.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20232@mturlrep13_201309101921_1369480322.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20659@mturlrep13_201309101921_854127867.txt -hive> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0079 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:22:05,246 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:22:12,275 Stage-1 map = 29%, reduce = 0% -2013-09-10 19:22:15,294 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.49 sec -2013-09-10 19:22:16,301 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.49 sec -2013-09-10 19:22:17,308 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.49 sec -2013-09-10 19:22:18,314 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.49 sec -2013-09-10 19:22:19,321 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.49 sec -2013-09-10 19:22:20,328 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.49 sec -2013-09-10 19:22:21,334 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.49 sec -2013-09-10 19:22:22,339 Stage-1 map = 80%, reduce = 8%, Cumulative CPU 24.49 sec -2013-09-10 19:22:23,344 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 24.49 sec -2013-09-10 19:22:24,350 Stage-1 map = 89%, reduce = 17%, Cumulative CPU 35.91 sec -2013-09-10 19:22:25,355 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.26 sec -2013-09-10 19:22:26,360 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.26 sec -2013-09-10 19:22:27,366 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.26 sec -2013-09-10 19:22:28,372 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.26 sec -2013-09-10 19:22:29,378 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.26 sec -2013-09-10 19:22:30,383 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.26 sec -2013-09-10 19:22:31,388 Stage-1 map = 100%, reduce = 54%, Cumulative CPU 48.26 sec -2013-09-10 19:22:32,396 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.73 sec -2013-09-10 19:22:33,401 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.73 sec -2013-09-10 19:22:34,408 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.73 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 730 msec -Ended Job = job_201309101627_0079 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0080 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:22:36,988 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:22:45,015 Stage-2 map = 52%, reduce = 0% -2013-09-10 19:22:46,020 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.51 sec -2013-09-10 19:22:47,024 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.51 sec -2013-09-10 19:22:48,029 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.51 sec -2013-09-10 19:22:49,034 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.51 sec -2013-09-10 19:22:50,039 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.51 sec -2013-09-10 19:22:51,043 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.51 sec -2013-09-10 19:22:52,048 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.51 sec -2013-09-10 19:22:53,053 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.51 sec -2013-09-10 19:22:54,058 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 11.51 sec -2013-09-10 19:22:55,063 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.28 sec -2013-09-10 19:22:56,069 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.28 sec -2013-09-10 19:22:57,074 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.28 sec -MapReduce Total cumulative CPU time: 14 seconds 280 msec -Ended Job = job_201309101627_0080 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.73 sec HDFS Read: 69312553 HDFS Write: 31841963 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 14.28 sec HDFS Read: 31842730 HDFS Write: 372 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 10 msec -OK -Time taken: 61.939 seconds, Fetched: 10 row(s) -hive> quit; --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22886@mturlrep13_201309101923_1655441357.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23308@mturlrep13_201309101923_1184621924.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0081 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:23:22,523 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:23:29,551 Stage-1 map = 29%, reduce = 0% -2013-09-10 19:23:32,570 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.6 sec -2013-09-10 19:23:33,577 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.6 sec -2013-09-10 19:23:34,585 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.6 sec -2013-09-10 19:23:35,590 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.6 sec -2013-09-10 19:23:36,597 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.6 sec -2013-09-10 19:23:37,604 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.6 sec -2013-09-10 19:23:38,609 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.6 sec -2013-09-10 19:23:39,614 Stage-1 map = 64%, reduce = 8%, Cumulative CPU 23.6 sec -2013-09-10 19:23:40,619 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 23.6 sec -2013-09-10 19:23:41,624 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 23.6 sec -2013-09-10 19:23:42,630 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.29 sec -2013-09-10 19:23:43,636 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.29 sec -2013-09-10 19:23:44,642 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.29 sec -2013-09-10 19:23:45,648 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.29 sec -2013-09-10 19:23:46,654 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.29 sec -2013-09-10 19:23:47,660 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.29 sec -2013-09-10 19:23:48,666 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 49.29 sec -2013-09-10 19:23:49,671 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 49.29 sec -2013-09-10 19:23:50,679 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 57.04 sec -2013-09-10 19:23:51,685 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 65.11 sec -2013-09-10 19:23:52,692 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 65.11 sec -MapReduce Total cumulative CPU time: 1 minutes 5 seconds 110 msec -Ended Job = job_201309101627_0081 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0082 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:23:56,222 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:24:06,254 Stage-2 map = 50%, reduce = 0% -2013-09-10 19:24:09,264 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.25 sec -2013-09-10 19:24:10,269 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.25 sec -2013-09-10 19:24:11,274 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.25 sec -2013-09-10 19:24:12,279 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.25 sec -2013-09-10 19:24:13,284 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.25 sec -2013-09-10 19:24:14,289 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.25 sec -2013-09-10 19:24:15,294 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.25 sec -2013-09-10 19:24:16,300 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.25 sec -2013-09-10 19:24:17,305 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.25 sec -2013-09-10 19:24:18,310 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.0 sec -2013-09-10 19:24:19,316 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.0 sec -2013-09-10 19:24:20,321 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.0 sec -MapReduce Total cumulative CPU time: 22 seconds 0 msec -Ended Job = job_201309101627_0082 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 65.11 sec HDFS Read: 112931901 HDFS Write: 72725701 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 22.0 sec HDFS Read: 72726470 HDFS Write: 417 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 27 seconds 110 msec -OK -Time taken: 69.096 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25616@mturlrep13_201309101924_1244442553.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26032@mturlrep13_201309101924_522232031.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0083 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:24:41,696 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:24:48,727 Stage-1 map = 7%, reduce = 0% -2013-09-10 19:24:51,739 Stage-1 map = 22%, reduce = 0% -2013-09-10 19:24:54,752 Stage-1 map = 29%, reduce = 0% -2013-09-10 19:24:57,766 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:25:00,780 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:25:01,790 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 51.85 sec -2013-09-10 19:25:02,797 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 51.85 sec -2013-09-10 19:25:03,803 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 51.85 sec -2013-09-10 19:25:04,808 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 51.85 sec -2013-09-10 19:25:05,813 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 51.85 sec -2013-09-10 19:25:06,819 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 51.85 sec -2013-09-10 19:25:07,825 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 51.85 sec -2013-09-10 19:25:08,831 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 51.85 sec -2013-09-10 19:25:09,911 Stage-1 map = 57%, reduce = 13%, Cumulative CPU 51.85 sec -2013-09-10 19:25:10,916 Stage-1 map = 57%, reduce = 13%, Cumulative CPU 51.85 sec -2013-09-10 19:25:11,922 Stage-1 map = 57%, reduce = 13%, Cumulative CPU 51.85 sec -2013-09-10 19:25:12,928 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 51.85 sec -2013-09-10 19:25:13,933 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 51.85 sec -2013-09-10 19:25:14,939 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 51.85 sec -2013-09-10 19:25:15,945 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 51.85 sec -2013-09-10 19:25:16,950 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 51.85 sec -2013-09-10 19:25:17,956 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 51.85 sec -2013-09-10 19:25:18,962 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 51.85 sec -2013-09-10 19:25:20,009 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 51.85 sec -2013-09-10 19:25:21,015 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 74.74 sec -2013-09-10 19:25:22,020 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 74.74 sec -2013-09-10 19:25:23,025 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 74.74 sec -2013-09-10 19:25:24,030 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 101.13 sec -2013-09-10 19:25:25,035 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 101.13 sec -2013-09-10 19:25:26,041 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 101.13 sec -2013-09-10 19:25:27,046 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 101.13 sec -2013-09-10 19:25:28,052 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 101.13 sec -2013-09-10 19:25:29,057 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 101.13 sec -2013-09-10 19:25:30,062 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 101.13 sec -2013-09-10 19:25:31,068 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 101.13 sec -2013-09-10 19:25:32,073 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 101.13 sec -2013-09-10 19:25:34,159 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 101.13 sec -2013-09-10 19:25:35,164 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 101.13 sec -2013-09-10 19:25:36,170 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 101.13 sec -2013-09-10 19:25:37,176 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 101.13 sec -2013-09-10 19:25:38,182 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 101.13 sec -2013-09-10 19:25:39,187 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 101.13 sec -2013-09-10 19:25:40,193 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 101.13 sec -2013-09-10 19:25:41,198 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 101.13 sec -2013-09-10 19:25:42,204 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 101.13 sec -2013-09-10 19:25:43,211 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 142.09 sec -2013-09-10 19:25:44,217 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 142.09 sec -2013-09-10 19:25:45,227 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 142.09 sec -2013-09-10 19:25:46,233 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 142.09 sec -2013-09-10 19:25:47,239 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 142.09 sec -2013-09-10 19:25:48,244 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 142.09 sec -2013-09-10 19:25:49,250 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 142.09 sec -2013-09-10 19:25:50,255 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 142.09 sec -2013-09-10 19:25:51,261 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 142.09 sec -2013-09-10 19:25:52,267 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 154.04 sec -2013-09-10 19:25:53,272 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 166.92 sec -2013-09-10 19:25:54,278 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 166.92 sec -2013-09-10 19:25:55,284 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 166.92 sec -MapReduce Total cumulative CPU time: 2 minutes 46 seconds 920 msec -Ended Job = job_201309101627_0083 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0084 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-10 19:25:58,773 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:26:11,814 Stage-2 map = 36%, reduce = 0% -2013-09-10 19:26:17,832 Stage-2 map = 60%, reduce = 0% -2013-09-10 19:26:20,844 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 28.8 sec -2013-09-10 19:26:21,848 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 28.8 sec -2013-09-10 19:26:22,853 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 28.8 sec -2013-09-10 19:26:23,856 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 28.8 sec -2013-09-10 19:26:24,860 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 28.8 sec -2013-09-10 19:26:25,864 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 28.8 sec -2013-09-10 19:26:26,882 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 28.8 sec -2013-09-10 19:26:27,886 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 28.8 sec -2013-09-10 19:26:28,890 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.8 sec -2013-09-10 19:26:29,893 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.8 sec -2013-09-10 19:26:30,897 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.8 sec -2013-09-10 19:26:31,901 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.8 sec -2013-09-10 19:26:32,905 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.8 sec -2013-09-10 19:26:33,910 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.8 sec -2013-09-10 19:26:34,913 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.8 sec -2013-09-10 19:26:35,917 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.8 sec -2013-09-10 19:26:36,921 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.8 sec -2013-09-10 19:26:37,924 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.8 sec -2013-09-10 19:26:38,929 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 76.5 sec -2013-09-10 19:26:39,933 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 76.5 sec -2013-09-10 19:26:40,937 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 76.5 sec -2013-09-10 19:26:41,941 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 76.5 sec -2013-09-10 19:26:42,946 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.76 sec -2013-09-10 19:26:43,951 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.76 sec -2013-09-10 19:26:44,955 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.76 sec -2013-09-10 19:26:45,959 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.76 sec -2013-09-10 19:26:46,966 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.76 sec -2013-09-10 19:26:47,971 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.76 sec -2013-09-10 19:26:48,975 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.76 sec -2013-09-10 19:26:49,979 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 77.76 sec -2013-09-10 19:26:50,991 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 77.76 sec -2013-09-10 19:26:51,995 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 77.76 sec -2013-09-10 19:26:53,000 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 86.76 sec -2013-09-10 19:26:54,005 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 86.76 sec -MapReduce Total cumulative CPU time: 1 minutes 26 seconds 760 msec -Ended Job = job_201309101627_0084 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 166.92 sec HDFS Read: 85707829 HDFS Write: 413932232 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 86.76 sec HDFS Read: 413942944 HDFS Write: 420 SUCCESS -Total MapReduce CPU Time Spent: 4 minutes 13 seconds 680 msec -OK -Time taken: 142.131 seconds, Fetched: 10 row(s) -hive> quit; --- то же самое, но ещё и без фильтрации.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29353@mturlrep13_201309101927_1354960960.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29777@mturlrep13_201309101927_1772629937.txt -hive> SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0085 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:27:16,160 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:27:23,191 Stage-1 map = 7%, reduce = 0% -2013-09-10 19:27:26,204 Stage-1 map = 22%, reduce = 0% -2013-09-10 19:27:29,217 Stage-1 map = 29%, reduce = 0% -2013-09-10 19:27:32,231 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:27:35,244 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:27:38,262 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.06 sec -2013-09-10 19:27:39,268 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.06 sec -2013-09-10 19:27:40,275 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.06 sec -2013-09-10 19:27:41,281 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.06 sec -2013-09-10 19:27:42,287 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.06 sec -2013-09-10 19:27:43,293 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.06 sec -2013-09-10 19:27:44,298 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.06 sec -2013-09-10 19:27:45,304 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 55.06 sec -2013-09-10 19:27:46,310 Stage-1 map = 61%, reduce = 4%, Cumulative CPU 55.06 sec -2013-09-10 19:27:47,316 Stage-1 map = 61%, reduce = 4%, Cumulative CPU 55.06 sec -2013-09-10 19:27:48,322 Stage-1 map = 65%, reduce = 13%, Cumulative CPU 55.06 sec -2013-09-10 19:27:49,327 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.06 sec -2013-09-10 19:27:50,333 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.06 sec -2013-09-10 19:27:51,339 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 55.06 sec -2013-09-10 19:27:52,344 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.06 sec -2013-09-10 19:27:53,350 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.06 sec -2013-09-10 19:27:54,356 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 55.06 sec -2013-09-10 19:27:55,361 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 55.06 sec -2013-09-10 19:27:56,366 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 55.06 sec -2013-09-10 19:27:57,372 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 55.06 sec -2013-09-10 19:27:58,377 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 79.16 sec -2013-09-10 19:27:59,383 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 79.16 sec -2013-09-10 19:28:00,389 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.09 sec -2013-09-10 19:28:01,394 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.09 sec -2013-09-10 19:28:02,400 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.09 sec -2013-09-10 19:28:03,405 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.09 sec -2013-09-10 19:28:04,410 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 106.09 sec -2013-09-10 19:28:05,416 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 106.09 sec -2013-09-10 19:28:06,421 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 106.09 sec -2013-09-10 19:28:07,426 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 106.09 sec -2013-09-10 19:28:09,260 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 106.09 sec -2013-09-10 19:28:10,265 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 106.09 sec -2013-09-10 19:28:11,271 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 106.09 sec -2013-09-10 19:28:12,276 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 106.09 sec -2013-09-10 19:28:13,282 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 106.09 sec -2013-09-10 19:28:14,287 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 106.09 sec -2013-09-10 19:28:15,293 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 106.09 sec -2013-09-10 19:28:16,298 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 106.09 sec -2013-09-10 19:28:17,304 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 106.09 sec -2013-09-10 19:28:18,312 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 133.23 sec -2013-09-10 19:28:19,318 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 133.23 sec -2013-09-10 19:28:20,324 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 157.37 sec -2013-09-10 19:28:21,330 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 157.37 sec -2013-09-10 19:28:22,335 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 157.37 sec -MapReduce Total cumulative CPU time: 2 minutes 37 seconds 370 msec -Ended Job = job_201309101627_0085 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0086 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-10 19:28:25,827 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:28:32,848 Stage-2 map = 25%, reduce = 0% -2013-09-10 19:28:34,856 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 13.41 sec -2013-09-10 19:28:35,861 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.41 sec -2013-09-10 19:28:36,866 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.41 sec -2013-09-10 19:28:37,871 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.41 sec -2013-09-10 19:28:38,875 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.41 sec -2013-09-10 19:28:39,880 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.41 sec -2013-09-10 19:28:40,885 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.41 sec -2013-09-10 19:28:41,890 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 13.41 sec -2013-09-10 19:28:42,896 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 36.49 sec -2013-09-10 19:28:43,900 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.1 sec -2013-09-10 19:28:44,905 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.1 sec -2013-09-10 19:28:45,909 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.1 sec -2013-09-10 19:28:46,913 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.1 sec -2013-09-10 19:28:47,917 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.1 sec -2013-09-10 19:28:48,921 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.1 sec -2013-09-10 19:28:49,925 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.1 sec -2013-09-10 19:28:50,930 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.1 sec -2013-09-10 19:28:51,934 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 38.1 sec -2013-09-10 19:28:52,939 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 44.49 sec -2013-09-10 19:28:53,944 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 44.49 sec -2013-09-10 19:28:54,961 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 44.49 sec -MapReduce Total cumulative CPU time: 44 seconds 490 msec -Ended Job = job_201309101627_0086 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 157.37 sec HDFS Read: 109451651 HDFS Write: 399298510 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 44.49 sec HDFS Read: 399308173 HDFS Write: 445 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 21 seconds 860 msec -OK -Time taken: 108.452 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по URL.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32048@mturlrep13_201309101929_256321783.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32462@mturlrep13_201309101929_599784390.txt -hive> SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0087 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:29:17,670 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:29:24,699 Stage-1 map = 7%, reduce = 0% -2013-09-10 19:29:27,711 Stage-1 map = 22%, reduce = 0% -2013-09-10 19:29:30,724 Stage-1 map = 29%, reduce = 0% -2013-09-10 19:29:33,738 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:29:36,750 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:29:40,770 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 60.13 sec -2013-09-10 19:29:41,777 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 60.13 sec -2013-09-10 19:29:42,784 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 60.13 sec -2013-09-10 19:29:43,791 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 60.13 sec -2013-09-10 19:29:44,797 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 60.13 sec -2013-09-10 19:29:45,802 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 60.13 sec -2013-09-10 19:29:46,808 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 60.13 sec -2013-09-10 19:29:47,814 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 60.13 sec -2013-09-10 19:29:48,820 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 60.13 sec -2013-09-10 19:29:49,825 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 60.13 sec -2013-09-10 19:29:50,831 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 60.13 sec -2013-09-10 19:29:51,837 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 60.13 sec -2013-09-10 19:29:52,843 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 60.13 sec -2013-09-10 19:29:53,849 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 60.13 sec -2013-09-10 19:29:54,854 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 60.13 sec -2013-09-10 19:29:55,860 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 60.13 sec -2013-09-10 19:29:56,866 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 60.13 sec -2013-09-10 19:29:57,871 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 60.13 sec -2013-09-10 19:29:58,877 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 60.13 sec -2013-09-10 19:29:59,882 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 60.13 sec -2013-09-10 19:30:00,888 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 84.82 sec -2013-09-10 19:30:01,893 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 84.82 sec -2013-09-10 19:30:02,898 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 84.82 sec -2013-09-10 19:30:03,904 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 113.03 sec -2013-09-10 19:30:04,909 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 113.03 sec -2013-09-10 19:30:05,914 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 113.03 sec -2013-09-10 19:30:06,919 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 113.03 sec -2013-09-10 19:30:07,924 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 113.03 sec -2013-09-10 19:30:08,930 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 113.03 sec -2013-09-10 19:30:11,147 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 113.03 sec -2013-09-10 19:30:12,152 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 113.03 sec -2013-09-10 19:30:13,158 Stage-1 map = 100%, reduce = 53%, Cumulative CPU 113.03 sec -2013-09-10 19:30:14,163 Stage-1 map = 100%, reduce = 53%, Cumulative CPU 113.03 sec -2013-09-10 19:30:15,169 Stage-1 map = 100%, reduce = 53%, Cumulative CPU 113.03 sec -2013-09-10 19:30:16,175 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 113.03 sec -2013-09-10 19:30:17,180 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 113.03 sec -2013-09-10 19:30:18,185 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 113.03 sec -2013-09-10 19:30:19,191 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 113.03 sec -2013-09-10 19:30:20,196 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 113.03 sec -2013-09-10 19:30:21,202 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 113.03 sec -2013-09-10 19:30:22,208 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 113.03 sec -2013-09-10 19:30:23,215 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 141.84 sec -2013-09-10 19:30:24,221 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 141.84 sec -2013-09-10 19:30:25,227 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 141.84 sec -2013-09-10 19:30:26,232 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 141.84 sec -2013-09-10 19:30:27,238 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.57 sec -2013-09-10 19:30:28,256 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.57 sec -MapReduce Total cumulative CPU time: 2 minutes 49 seconds 570 msec -Ended Job = job_201309101627_0087 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0088 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-10 19:30:30,840 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:30:38,866 Stage-2 map = 25%, reduce = 0% -2013-09-10 19:30:41,877 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 14.25 sec -2013-09-10 19:30:42,883 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 28.71 sec -2013-09-10 19:30:43,888 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 28.71 sec -2013-09-10 19:30:44,893 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 28.71 sec -2013-09-10 19:30:45,898 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 28.71 sec -2013-09-10 19:30:46,903 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 28.71 sec -2013-09-10 19:30:47,908 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 28.71 sec -2013-09-10 19:30:48,914 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.71 sec -2013-09-10 19:30:49,919 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.71 sec -2013-09-10 19:30:50,923 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.99 sec -2013-09-10 19:30:51,928 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.99 sec -2013-09-10 19:30:52,933 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.99 sec -2013-09-10 19:30:53,937 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.99 sec -2013-09-10 19:30:54,941 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.99 sec -2013-09-10 19:30:55,945 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.99 sec -2013-09-10 19:30:56,949 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.99 sec -2013-09-10 19:30:57,954 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 38.99 sec -2013-09-10 19:30:58,959 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 38.99 sec -2013-09-10 19:30:59,963 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.82 sec -2013-09-10 19:31:00,968 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.82 sec -MapReduce Total cumulative CPU time: 46 seconds 820 msec -Ended Job = job_201309101627_0088 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 169.57 sec HDFS Read: 109451651 HDFS Write: 402873759 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 46.82 sec HDFS Read: 402889658 HDFS Write: 465 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 36 seconds 390 msec -OK -Time taken: 112.977 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по URL и числу.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3367@mturlrep13_201309101931_394587693.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3777@mturlrep13_201309101931_1148397867.txt -hive> SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0089 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:31:21,578 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:31:28,603 Stage-1 map = 29%, reduce = 0% -2013-09-10 19:31:31,614 Stage-1 map = 43%, reduce = 0% -2013-09-10 19:31:33,628 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.78 sec -2013-09-10 19:31:34,635 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.78 sec -2013-09-10 19:31:35,643 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.78 sec -2013-09-10 19:31:36,650 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.78 sec -2013-09-10 19:31:37,657 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.78 sec -2013-09-10 19:31:38,663 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.78 sec -2013-09-10 19:31:39,669 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.78 sec -2013-09-10 19:31:40,675 Stage-1 map = 80%, reduce = 8%, Cumulative CPU 30.78 sec -2013-09-10 19:31:41,681 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 30.78 sec -2013-09-10 19:31:42,688 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 30.78 sec -2013-09-10 19:31:43,694 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 48.88 sec -2013-09-10 19:31:44,700 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 55.21 sec -2013-09-10 19:31:45,707 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 62.78 sec -2013-09-10 19:31:46,712 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 62.78 sec -2013-09-10 19:31:47,717 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 62.78 sec -2013-09-10 19:31:48,723 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 62.78 sec -2013-09-10 19:31:49,729 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 62.78 sec -2013-09-10 19:31:50,734 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 62.78 sec -2013-09-10 19:31:51,742 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 64.41 sec -2013-09-10 19:31:52,748 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 64.41 sec -2013-09-10 19:31:53,753 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 64.41 sec -2013-09-10 19:31:54,759 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 64.41 sec -2013-09-10 19:31:55,765 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 64.41 sec -2013-09-10 19:31:56,771 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 64.41 sec -2013-09-10 19:31:57,777 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 64.41 sec -2013-09-10 19:31:58,784 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 74.38 sec -2013-09-10 19:31:59,790 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 74.38 sec -MapReduce Total cumulative CPU time: 1 minutes 14 seconds 380 msec -Ended Job = job_201309101627_0089 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0090 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:32:02,272 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:32:14,305 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.3 sec -2013-09-10 19:32:15,310 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.3 sec -2013-09-10 19:32:16,315 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.3 sec -2013-09-10 19:32:17,319 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.3 sec -2013-09-10 19:32:18,324 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.3 sec -2013-09-10 19:32:19,328 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.3 sec -2013-09-10 19:32:20,333 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.3 sec -2013-09-10 19:32:21,338 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.3 sec -2013-09-10 19:32:22,344 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 16.3 sec -2013-09-10 19:32:23,351 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.67 sec -2013-09-10 19:32:24,356 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.67 sec -2013-09-10 19:32:25,362 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.67 sec -MapReduce Total cumulative CPU time: 20 seconds 670 msec -Ended Job = job_201309101627_0090 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 74.38 sec HDFS Read: 31344843 HDFS Write: 51717050 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 20.67 sec HDFS Read: 51717819 HDFS Write: 490 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 35 seconds 50 msec -OK -Time taken: 73.956 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6068@mturlrep13_201309101932_879967620.txt -hive> ; -hive> quit; - -times: 1 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6494@mturlrep13_201309101932_510743290.txt -hive> ; -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6720@mturlrep13_201309101932_1476479155.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7143@mturlrep13_201309101932_831179276.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0091 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:32:55,725 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:33:02,754 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:33:04,773 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.76 sec -2013-09-10 19:33:05,781 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.76 sec -2013-09-10 19:33:06,788 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.76 sec -2013-09-10 19:33:07,795 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.76 sec -2013-09-10 19:33:08,802 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.76 sec -2013-09-10 19:33:09,809 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.76 sec -2013-09-10 19:33:10,816 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.76 sec -2013-09-10 19:33:11,822 Stage-1 map = 84%, reduce = 8%, Cumulative CPU 19.76 sec -2013-09-10 19:33:12,828 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 19.76 sec -2013-09-10 19:33:13,835 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.12 sec -2013-09-10 19:33:14,841 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.12 sec -2013-09-10 19:33:15,846 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.12 sec -2013-09-10 19:33:16,853 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.12 sec -2013-09-10 19:33:17,861 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.24 sec -2013-09-10 19:33:18,867 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.24 sec -2013-09-10 19:33:19,874 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.24 sec -MapReduce Total cumulative CPU time: 42 seconds 240 msec -Ended Job = job_201309101627_0091 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0092 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:33:22,368 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:33:24,377 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:33:25,383 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:33:26,388 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:33:27,393 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:33:28,399 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:33:29,405 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:33:30,411 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:33:31,417 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.72 sec -2013-09-10 19:33:32,423 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -2013-09-10 19:33:33,429 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -2013-09-10 19:33:34,436 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -MapReduce Total cumulative CPU time: 2 seconds 140 msec -Ended Job = job_201309101627_0092 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.24 sec HDFS Read: 118784021 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.14 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 380 msec -OK -Time taken: 48.75 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9263@mturlrep13_201309101933_336058899.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9686@mturlrep13_201309101933_1339458822.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0093 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:33:55,210 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:34:02,239 Stage-1 map = 32%, reduce = 0% -2013-09-10 19:34:04,259 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.6 sec -2013-09-10 19:34:05,267 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.6 sec -2013-09-10 19:34:06,275 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.6 sec -2013-09-10 19:34:07,282 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.6 sec -2013-09-10 19:34:08,288 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.6 sec -2013-09-10 19:34:09,296 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.6 sec -2013-09-10 19:34:10,303 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.6 sec -2013-09-10 19:34:11,309 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 18.6 sec -2013-09-10 19:34:12,316 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.28 sec -2013-09-10 19:34:13,322 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.28 sec -2013-09-10 19:34:14,328 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.28 sec -2013-09-10 19:34:15,334 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.28 sec -2013-09-10 19:34:16,341 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.28 sec -2013-09-10 19:34:17,349 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.65 sec -2013-09-10 19:34:18,355 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.65 sec -2013-09-10 19:34:19,361 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.65 sec -MapReduce Total cumulative CPU time: 40 seconds 650 msec -Ended Job = job_201309101627_0093 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0094 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:34:21,901 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:34:23,911 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:34:24,916 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:34:25,920 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:34:26,924 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:34:27,929 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:34:28,933 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:34:29,938 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-10 19:34:30,943 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.72 sec -2013-09-10 19:34:31,949 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -2013-09-10 19:34:32,955 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -2013-09-10 19:34:33,960 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -MapReduce Total cumulative CPU time: 2 seconds 80 msec -Ended Job = job_201309101627_0094 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.65 sec HDFS Read: 115339269 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.08 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 730 msec -OK -Time taken: 49.009 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11721@mturlrep13_201309101934_1061275067.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12142@mturlrep13_201309101934_727094050.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0095 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:34:54,406 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:35:01,434 Stage-1 map = 29%, reduce = 0% -2013-09-10 19:35:03,451 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 19:35:04,459 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 19:35:05,466 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 19:35:06,472 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 19:35:07,479 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 19:35:08,485 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 19:35:09,492 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 19:35:10,498 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.12 sec -2013-09-10 19:35:11,504 Stage-1 map = 89%, reduce = 17%, Cumulative CPU 28.23 sec -2013-09-10 19:35:12,510 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.0 sec -2013-09-10 19:35:13,527 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.0 sec -2013-09-10 19:35:14,532 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.0 sec -2013-09-10 19:35:15,537 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.0 sec -2013-09-10 19:35:16,546 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 40.08 sec -2013-09-10 19:35:17,552 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.17 sec -2013-09-10 19:35:18,558 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.17 sec -MapReduce Total cumulative CPU time: 42 seconds 170 msec -Ended Job = job_201309101627_0095 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0096 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:35:22,063 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:35:23,069 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-10 19:35:24,074 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-10 19:35:25,079 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-10 19:35:26,083 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-10 19:35:27,088 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-10 19:35:28,093 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-10 19:35:29,098 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-10 19:35:30,103 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.73 sec -2013-09-10 19:35:31,108 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -2013-09-10 19:35:32,113 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -2013-09-10 19:35:33,119 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -MapReduce Total cumulative CPU time: 2 seconds 80 msec -Ended Job = job_201309101627_0096 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.17 sec HDFS Read: 118662691 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.08 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 250 msec -OK -Time taken: 48.833 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14932@mturlrep13_201309101935_1105454818.txt -hive> ; -hive> quit; - -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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15362@mturlrep13_201309101935_1538330096.txt -hive> 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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; ; -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15619@mturlrep13_201309101935_1070373557.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16082@mturlrep13_201309101935_708045225.txt -hive> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0097 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:36:08,381 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:36:15,410 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:36:16,424 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 9.96 sec -2013-09-10 19:36:17,432 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.48 sec -2013-09-10 19:36:18,440 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.48 sec -2013-09-10 19:36:19,449 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.48 sec -2013-09-10 19:36:20,456 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.48 sec -2013-09-10 19:36:21,462 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.48 sec -2013-09-10 19:36:22,468 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.48 sec -2013-09-10 19:36:23,475 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.48 sec -2013-09-10 19:36:24,481 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 20.48 sec -2013-09-10 19:36:25,487 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.4 sec -2013-09-10 19:36:26,492 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.4 sec -2013-09-10 19:36:27,497 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.4 sec -2013-09-10 19:36:28,503 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.4 sec -2013-09-10 19:36:29,508 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.4 sec -2013-09-10 19:36:30,527 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.46 sec -2013-09-10 19:36:31,533 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.46 sec -MapReduce Total cumulative CPU time: 42 seconds 460 msec -Ended Job = job_201309101627_0097 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0098 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:36:35,142 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:36:36,148 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-10 19:36:37,152 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-10 19:36:38,157 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-10 19:36:39,162 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-10 19:36:40,166 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-10 19:36:41,171 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-10 19:36:42,176 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-10 19:36:43,181 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-10 19:36:44,186 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.03 sec -2013-09-10 19:36:45,192 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.03 sec -2013-09-10 19:36:46,196 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.03 sec -MapReduce Total cumulative CPU time: 2 seconds 30 msec -Ended Job = job_201309101627_0098 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.46 sec HDFS Read: 148406904 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.03 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 490 msec -OK -Time taken: 47.801 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18252@mturlrep13_201309101936_1758564620.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18670@mturlrep13_201309101936_1898149333.txt -hive> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0099 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-10 19:37:07,178 Stage-1 map = 0%, reduce = 0% -2013-09-10 19:37:14,205 Stage-1 map = 36%, reduce = 0% -2013-09-10 19:37:16,221 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.41 sec -2013-09-10 19:37:17,228 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.41 sec -2013-09-10 19:37:18,236 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.41 sec -2013-09-10 19:37:19,243 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.41 sec -2013-09-10 19:37:20,249 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.41 sec -2013-09-10 19:37:21,255 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.41 sec -2013-09-10 19:37:22,262 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.41 sec -2013-09-10 19:37:23,268 Stage-1 map = 80%, reduce = 8%, Cumulative CPU 18.41 sec -2013-09-10 19:37:24,273 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 18.41 sec -2013-09-10 19:37:25,279 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.99 sec -2013-09-10 19:37:26,284 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.99 sec -2013-09-10 19:37:27,290 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.99 sec -2013-09-10 19:37:28,296 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.99 sec -2013-09-10 19:37:29,304 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.19 sec -2013-09-10 19:37:30,310 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.19 sec -2013-09-10 19:37:31,316 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.19 sec -MapReduce Total cumulative CPU time: 40 seconds 190 msec -Ended Job = job_201309101627_0099 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0100 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-10 19:37:33,844 Stage-2 map = 0%, reduce = 0% -2013-09-10 19:37:35,853 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 19:37:36,859 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 19:37:37,865 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 19:37:38,870 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 19:37:39,875 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 19:37:40,880 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 19:37:41,885 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-10 19:37:42,890 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.75 sec -2013-09-10 19:37:43,896 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.05 sec -2013-09-10 19:37:44,902 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.05 sec -2013-09-10 19:37:45,908 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.05 sec -MapReduce Total cumulative CPU time: 2 seconds 50 msec -Ended Job = job_201309101627_0100 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.19 sec HDFS Read: 105631340 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.05 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 240 msec -OK -Time taken: 48.865 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20877@mturlrep13_201309101937_323156088.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS m, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21297@mturlrep13_201309101937_1619127801.txt -hive> SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS m, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; ; -NoViableAltException(26@[]) -FAILED: ParseException line 1:34 missing FROM at '(' near '(' in subquery source -line 1:35 cannot recognize input near 'EventTime' ')' 'SECOND' in subquery source - -hive> quit; -stop time: Вт. сент. 10 19:37:56 MSK 2013 diff --git a/benchmark/hive/log/log_10m/log_10m_tuned b/benchmark/hive/log/log_10m/log_10m_tuned deleted file mode 100644 index 3589288d0b5..00000000000 --- a/benchmark/hive/log/log_10m/log_10m_tuned +++ /dev/null @@ -1,7608 +0,0 @@ -Not found: /etc/init.d/service with path= -start time: Ср. сент. 18 02:11:52 MSK 2013 - -times: 1 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8923@mturlrep13_201309180211_341387203.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0199 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:12:08,040 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:12:16,074 Stage-1 map = 4%, reduce = 0% -2013-09-18 02:12:19,086 Stage-1 map = 15%, reduce = 0% -2013-09-18 02:12:22,098 Stage-1 map = 29%, reduce = 0% -2013-09-18 02:12:25,110 Stage-1 map = 44%, reduce = 0% -2013-09-18 02:12:28,121 Stage-1 map = 59%, reduce = 0% -2013-09-18 02:12:34,140 Stage-1 map = 74%, reduce = 0% -2013-09-18 02:12:35,150 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 31.42 sec -2013-09-18 02:12:36,156 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 31.42 sec -2013-09-18 02:12:37,162 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 31.42 sec -2013-09-18 02:12:38,167 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 68.28 sec -2013-09-18 02:12:39,173 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 143.22 sec -2013-09-18 02:12:40,178 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 143.22 sec -2013-09-18 02:12:41,182 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 143.22 sec -2013-09-18 02:12:42,188 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 143.22 sec -2013-09-18 02:12:43,195 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 145.37 sec -2013-09-18 02:12:44,201 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 145.37 sec -2013-09-18 02:12:45,206 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 145.37 sec -MapReduce Total cumulative CPU time: 2 minutes 25 seconds 370 msec -Ended Job = job_201309172235_0199 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 145.37 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 25 seconds 370 msec -OK -10000000 -Time taken: 47.388 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10060@mturlrep13_201309180212_360076553.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0200 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:12:55,466 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:13:02,498 Stage-1 map = 4%, reduce = 0% -2013-09-18 02:13:05,516 Stage-1 map = 18%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:06,523 Stage-1 map = 18%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:07,530 Stage-1 map = 18%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:08,535 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:09,542 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:10,549 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:11,555 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:12,561 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:13,567 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:14,572 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:15,577 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:16,582 Stage-1 map = 56%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:17,588 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:18,594 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:19,600 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:20,605 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:21,611 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 41.56 sec -2013-09-18 02:13:22,618 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 65.53 sec -2013-09-18 02:13:23,623 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 95.51 sec -2013-09-18 02:13:24,629 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 95.51 sec -2013-09-18 02:13:25,634 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 151.45 sec -2013-09-18 02:13:26,639 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 151.45 sec -2013-09-18 02:13:27,643 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 151.45 sec -2013-09-18 02:13:28,648 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 151.45 sec -2013-09-18 02:13:29,653 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 151.45 sec -2013-09-18 02:13:30,661 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 153.61 sec -2013-09-18 02:13:31,667 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 153.61 sec -MapReduce Total cumulative CPU time: 2 minutes 33 seconds 610 msec -Ended Job = job_201309172235_0200 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 153.61 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 33 seconds 610 msec -OK -10000000 -Time taken: 44.55 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11198@mturlrep13_201309180213_1431795111.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0201 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:13:41,815 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:13:48,846 Stage-1 map = 15%, reduce = 0% -2013-09-18 02:13:51,858 Stage-1 map = 18%, reduce = 0% -2013-09-18 02:13:54,871 Stage-1 map = 29%, reduce = 0% -2013-09-18 02:13:57,884 Stage-1 map = 44%, reduce = 0% -2013-09-18 02:14:00,897 Stage-1 map = 59%, reduce = 0% -2013-09-18 02:14:03,913 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 105.42 sec -2013-09-18 02:14:04,919 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 105.42 sec -2013-09-18 02:14:05,925 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 105.42 sec -2013-09-18 02:14:06,931 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 105.42 sec -2013-09-18 02:14:07,937 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 113.25 sec -2013-09-18 02:14:08,942 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 121.6 sec -2013-09-18 02:14:09,947 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 128.96 sec -2013-09-18 02:14:10,952 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 140.92 sec -2013-09-18 02:14:11,957 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 140.92 sec -2013-09-18 02:14:12,967 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 140.92 sec -2013-09-18 02:14:13,971 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 140.92 sec -2013-09-18 02:14:14,976 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 140.92 sec -2013-09-18 02:14:15,983 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 142.87 sec -2013-09-18 02:14:16,988 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 142.87 sec -MapReduce Total cumulative CPU time: 2 minutes 22 seconds 870 msec -Ended Job = job_201309172235_0201 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 142.87 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 22 seconds 870 msec -OK -10000000 -Time taken: 43.513 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12353@mturlrep13_201309180214_1687900933.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0202 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:14:33,532 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:14:38,563 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 5.8 sec -2013-09-18 02:14:39,570 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.27 sec -2013-09-18 02:14:40,577 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.27 sec -2013-09-18 02:14:41,582 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.27 sec -2013-09-18 02:14:42,588 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.27 sec -2013-09-18 02:14:43,593 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.27 sec -2013-09-18 02:14:44,598 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.27 sec -2013-09-18 02:14:45,603 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.27 sec -2013-09-18 02:14:46,611 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.34 sec -2013-09-18 02:14:47,617 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.34 sec -2013-09-18 02:14:48,624 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.34 sec -MapReduce Total cumulative CPU time: 27 seconds 340 msec -Ended Job = job_201309172235_0202 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 27.34 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 27 seconds 340 msec -OK -171127 -Time taken: 25.332 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13496@mturlrep13_201309180214_186944360.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0203 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:14:58,925 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:15:03,954 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.94 sec -2013-09-18 02:15:04,962 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.94 sec -2013-09-18 02:15:05,970 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.94 sec -2013-09-18 02:15:06,975 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.94 sec -2013-09-18 02:15:07,982 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.94 sec -2013-09-18 02:15:08,988 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.94 sec -2013-09-18 02:15:09,994 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.94 sec -2013-09-18 02:15:11,001 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 23.94 sec -2013-09-18 02:15:12,008 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.75 sec -2013-09-18 02:15:13,015 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.75 sec -MapReduce Total cumulative CPU time: 25 seconds 750 msec -Ended Job = job_201309172235_0203 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 25.75 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 25 seconds 750 msec -OK -171127 -Time taken: 22.592 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15074@mturlrep13_201309180215_1069318863.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0204 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:15:22,365 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:15:27,393 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 17.92 sec -2013-09-18 02:15:28,401 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.51 sec -2013-09-18 02:15:29,409 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.51 sec -2013-09-18 02:15:30,415 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.51 sec -2013-09-18 02:15:31,421 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.51 sec -2013-09-18 02:15:32,427 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.51 sec -2013-09-18 02:15:33,433 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.51 sec -2013-09-18 02:15:34,439 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.51 sec -2013-09-18 02:15:35,447 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 26.63 sec -2013-09-18 02:15:36,454 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 26.63 sec -2013-09-18 02:15:37,461 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 26.63 sec -MapReduce Total cumulative CPU time: 26 seconds 630 msec -Ended Job = job_201309172235_0204 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 26.63 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 26 seconds 630 msec -OK -171127 -Time taken: 22.629 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16492@mturlrep13_201309180215_2118658323.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0205 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:15:54,647 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:16:02,687 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.0 sec -2013-09-18 02:16:03,694 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.0 sec -2013-09-18 02:16:04,700 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.0 sec -2013-09-18 02:16:05,706 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.0 sec -2013-09-18 02:16:06,712 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.0 sec -2013-09-18 02:16:07,717 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.0 sec -2013-09-18 02:16:08,724 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.0 sec -2013-09-18 02:16:09,731 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 34.0 sec -2013-09-18 02:16:10,739 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 36.22 sec -2013-09-18 02:16:11,745 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 36.22 sec -MapReduce Total cumulative CPU time: 36 seconds 220 msec -Ended Job = job_201309172235_0205 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 36.22 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 36 seconds 220 msec -OK -Time taken: 27.558 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17739@mturlrep13_201309180216_1507251512.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0206 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:16:21,573 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:16:28,610 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.85 sec -2013-09-18 02:16:29,617 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.85 sec -2013-09-18 02:16:30,623 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.85 sec -2013-09-18 02:16:31,628 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.85 sec -2013-09-18 02:16:32,634 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.85 sec -2013-09-18 02:16:33,639 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.85 sec -2013-09-18 02:16:34,645 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.85 sec -2013-09-18 02:16:35,651 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 31.85 sec -2013-09-18 02:16:36,661 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.81 sec -2013-09-18 02:16:37,667 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.81 sec -MapReduce Total cumulative CPU time: 33 seconds 810 msec -Ended Job = job_201309172235_0206 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 33.81 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 810 msec -OK -Time taken: 23.861 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18970@mturlrep13_201309180216_474859328.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0207 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:16:48,424 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:16:54,457 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 7.2 sec -2013-09-18 02:16:55,465 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.03 sec -2013-09-18 02:16:56,471 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.03 sec -2013-09-18 02:16:57,477 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.03 sec -2013-09-18 02:16:58,482 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.03 sec -2013-09-18 02:16:59,487 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.03 sec -2013-09-18 02:17:00,493 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.03 sec -2013-09-18 02:17:01,499 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.03 sec -2013-09-18 02:17:02,505 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 32.03 sec -2013-09-18 02:17:03,513 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.98 sec -2013-09-18 02:17:04,519 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.98 sec -MapReduce Total cumulative CPU time: 33 seconds 980 msec -Ended Job = job_201309172235_0207 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 33.98 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 980 msec -OK -Time taken: 24.986 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20233@mturlrep13_201309180217_1020705203.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0208 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:17:21,071 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:17:27,108 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 6.28 sec -2013-09-18 02:17:28,115 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.34 sec -2013-09-18 02:17:29,122 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.34 sec -2013-09-18 02:17:30,127 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.34 sec -2013-09-18 02:17:31,132 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.34 sec -2013-09-18 02:17:32,137 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.34 sec -2013-09-18 02:17:33,143 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.34 sec -2013-09-18 02:17:34,149 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.34 sec -2013-09-18 02:17:35,155 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 27.34 sec -2013-09-18 02:17:36,163 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.22 sec -2013-09-18 02:17:37,169 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.22 sec -MapReduce Total cumulative CPU time: 29 seconds 220 msec -Ended Job = job_201309172235_0208 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 29.22 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 29 seconds 220 msec -OK --4662894107982093709 -Time taken: 26.148 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21379@mturlrep13_201309180217_1577746245.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0209 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:17:46,496 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:17:52,527 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.37 sec -2013-09-18 02:17:53,534 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.37 sec -2013-09-18 02:17:54,540 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.37 sec -2013-09-18 02:17:55,545 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.37 sec -2013-09-18 02:17:56,550 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.37 sec -2013-09-18 02:17:57,555 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.37 sec -2013-09-18 02:17:58,561 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.37 sec -2013-09-18 02:17:59,565 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.37 sec -2013-09-18 02:18:00,574 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.46 sec -2013-09-18 02:18:01,581 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.46 sec -2013-09-18 02:18:02,587 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.46 sec -MapReduce Total cumulative CPU time: 29 seconds 460 msec -Ended Job = job_201309172235_0209 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 29.46 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 29 seconds 460 msec -OK --4662894107982093709 -Time taken: 23.564 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22513@mturlrep13_201309180218_799027004.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0210 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:18:12,872 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:18:17,906 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-18 02:18:18,915 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-18 02:18:19,922 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-18 02:18:20,928 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-18 02:18:21,933 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-18 02:18:22,938 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-18 02:18:23,943 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-18 02:18:24,947 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.76 sec -2013-09-18 02:18:25,953 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 27.76 sec -2013-09-18 02:18:26,960 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.92 sec -2013-09-18 02:18:27,966 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.92 sec -MapReduce Total cumulative CPU time: 29 seconds 920 msec -Ended Job = job_201309172235_0210 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 29.92 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 29 seconds 920 msec -OK --4662894107982093709 -Time taken: 23.508 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23657@mturlrep13_201309180218_452234815.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0211 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:18:45,243 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:18:53,314 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:18:56,332 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.1 sec -2013-09-18 02:18:57,339 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.1 sec -2013-09-18 02:18:58,346 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.1 sec -2013-09-18 02:18:59,352 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.1 sec -2013-09-18 02:19:00,359 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.1 sec -2013-09-18 02:19:01,364 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.1 sec -2013-09-18 02:19:02,370 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.1 sec -2013-09-18 02:19:03,375 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 50.1 sec -2013-09-18 02:19:04,380 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 50.1 sec -2013-09-18 02:19:05,386 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 50.9 sec -2013-09-18 02:19:06,391 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 50.9 sec -2013-09-18 02:19:07,396 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 50.9 sec -2013-09-18 02:19:08,404 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.82 sec -2013-09-18 02:19:09,409 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.82 sec -2013-09-18 02:19:10,416 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.82 sec -MapReduce Total cumulative CPU time: 57 seconds 820 msec -Ended Job = job_201309172235_0211 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 57.82 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 57 seconds 820 msec -OK -2037258 -Time taken: 35.237 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24818@mturlrep13_201309180219_76164215.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0212 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:19:20,706 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:19:27,739 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:19:30,759 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.88 sec -2013-09-18 02:19:31,766 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.88 sec -2013-09-18 02:19:32,772 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.88 sec -2013-09-18 02:19:33,777 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.88 sec -2013-09-18 02:19:34,784 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.88 sec -2013-09-18 02:19:35,790 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.88 sec -2013-09-18 02:19:36,796 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.88 sec -2013-09-18 02:19:37,801 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 50.88 sec -2013-09-18 02:19:38,807 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 50.88 sec -2013-09-18 02:19:39,812 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 50.88 sec -2013-09-18 02:19:40,818 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 50.88 sec -2013-09-18 02:19:41,823 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 50.88 sec -2013-09-18 02:19:42,830 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 58.39 sec -2013-09-18 02:19:43,835 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 58.39 sec -MapReduce Total cumulative CPU time: 58 seconds 390 msec -Ended Job = job_201309172235_0212 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 58.39 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 58 seconds 390 msec -OK -2037258 -Time taken: 31.445 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25977@mturlrep13_201309180219_1809129811.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0213 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:19:53,965 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:20:00,997 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:20:03,011 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 25.89 sec -2013-09-18 02:20:04,018 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.47 sec -2013-09-18 02:20:05,026 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.47 sec -2013-09-18 02:20:06,031 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.47 sec -2013-09-18 02:20:07,036 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.47 sec -2013-09-18 02:20:08,042 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.47 sec -2013-09-18 02:20:09,049 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.47 sec -2013-09-18 02:20:10,055 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.47 sec -2013-09-18 02:20:11,061 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 52.47 sec -2013-09-18 02:20:12,066 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 52.47 sec -2013-09-18 02:20:13,072 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 52.47 sec -2013-09-18 02:20:14,078 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 52.47 sec -2013-09-18 02:20:15,084 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 52.47 sec -2013-09-18 02:20:16,091 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 61.2 sec -2013-09-18 02:20:17,096 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 61.2 sec -2013-09-18 02:20:18,102 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 61.2 sec -MapReduce Total cumulative CPU time: 1 minutes 1 seconds 200 msec -Ended Job = job_201309172235_0213 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 61.2 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 1 seconds 200 msec -OK -2037258 -Time taken: 32.552 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27834@mturlrep13_201309180220_1270552368.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0214 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:20:35,012 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:20:43,045 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:20:45,062 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:46,068 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:47,074 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:48,079 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:49,085 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:50,091 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:51,097 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:52,103 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:53,109 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:54,115 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:55,121 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:56,126 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.7 sec -2013-09-18 02:20:57,133 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.24 sec -2013-09-18 02:20:58,139 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.24 sec -2013-09-18 02:20:59,144 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.24 sec -MapReduce Total cumulative CPU time: 50 seconds 240 msec -Ended Job = job_201309172235_0214 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 50.24 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 50 seconds 240 msec -OK -1110413 -Time taken: 34.063 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29013@mturlrep13_201309180221_818576677.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0215 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:21:08,404 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:21:16,449 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 10.79 sec -2013-09-18 02:21:17,457 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.2 sec -2013-09-18 02:21:18,464 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.2 sec -2013-09-18 02:21:19,469 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.2 sec -2013-09-18 02:21:20,475 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.2 sec -2013-09-18 02:21:21,481 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.2 sec -2013-09-18 02:21:22,488 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.2 sec -2013-09-18 02:21:23,494 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.2 sec -2013-09-18 02:21:24,500 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.2 sec -2013-09-18 02:21:25,506 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.2 sec -2013-09-18 02:21:26,511 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.2 sec -2013-09-18 02:21:27,516 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 43.2 sec -2013-09-18 02:21:28,523 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.7 sec -2013-09-18 02:21:29,528 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.7 sec -2013-09-18 02:21:30,533 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.7 sec -MapReduce Total cumulative CPU time: 50 seconds 700 msec -Ended Job = job_201309172235_0215 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 50.7 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 50 seconds 700 msec -OK -1110413 -Time taken: 29.607 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30188@mturlrep13_201309180221_356829267.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0216 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:21:40,608 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:21:47,636 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:21:48,646 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 31.99 sec -2013-09-18 02:21:49,653 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:21:50,659 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:21:51,666 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:21:52,671 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:21:53,677 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:21:54,683 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:21:55,690 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.43 sec -2013-09-18 02:21:56,695 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.43 sec -2013-09-18 02:21:57,700 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.43 sec -2013-09-18 02:21:58,704 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 43.43 sec -2013-09-18 02:21:59,710 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 43.43 sec -2013-09-18 02:22:00,717 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 51.3 sec -2013-09-18 02:22:01,727 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 51.3 sec -MapReduce Total cumulative CPU time: 51 seconds 300 msec -Ended Job = job_201309172235_0216 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 51.3 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 51 seconds 300 msec -OK -1110413 -Time taken: 29.268 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31349@mturlrep13_201309180222_1184363660.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0217 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:22:19,289 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:22:25,321 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 6.4 sec -2013-09-18 02:22:26,329 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.67 sec -2013-09-18 02:22:27,336 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.67 sec -2013-09-18 02:22:28,342 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.67 sec -2013-09-18 02:22:29,347 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.67 sec -2013-09-18 02:22:30,352 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.67 sec -2013-09-18 02:22:31,358 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.67 sec -2013-09-18 02:22:32,364 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.67 sec -2013-09-18 02:22:33,372 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.78 sec -2013-09-18 02:22:34,379 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.78 sec -2013-09-18 02:22:35,384 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.78 sec -MapReduce Total cumulative CPU time: 28 seconds 780 msec -Ended Job = job_201309172235_0217 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 28.78 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 28 seconds 780 msec -OK -Time taken: 25.999 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32505@mturlrep13_201309180222_1320628760.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0218 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:22:44,642 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:22:50,675 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.81 sec -2013-09-18 02:22:51,682 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.81 sec -2013-09-18 02:22:52,688 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.81 sec -2013-09-18 02:22:53,694 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.81 sec -2013-09-18 02:22:54,699 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.81 sec -2013-09-18 02:22:55,704 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.81 sec -2013-09-18 02:22:56,710 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.81 sec -2013-09-18 02:22:57,717 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 26.81 sec -2013-09-18 02:22:58,725 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.69 sec -2013-09-18 02:22:59,731 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.69 sec -MapReduce Total cumulative CPU time: 28 seconds 690 msec -Ended Job = job_201309172235_0218 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 28.69 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 28 seconds 690 msec -OK -Time taken: 22.443 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1222@mturlrep13_201309180223_1487045078.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0219 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:23:09,025 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:23:15,057 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.89 sec -2013-09-18 02:23:16,064 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.89 sec -2013-09-18 02:23:17,072 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.89 sec -2013-09-18 02:23:18,077 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.89 sec -2013-09-18 02:23:19,083 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.89 sec -2013-09-18 02:23:20,088 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.89 sec -2013-09-18 02:23:21,094 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.89 sec -2013-09-18 02:23:22,100 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 25.89 sec -2013-09-18 02:23:23,108 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.99 sec -2013-09-18 02:23:24,115 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.99 sec -MapReduce Total cumulative CPU time: 27 seconds 990 msec -Ended Job = job_201309172235_0219 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 27.99 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 27 seconds 990 msec -OK -Time taken: 22.559 seconds, Fetched: 1 row(s) -hive> quit; - -times: 1 -query: SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2436@mturlrep13_201309180223_855141249.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0220 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:23:40,593 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:23:45,623 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 6.06 sec -2013-09-18 02:23:46,632 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.31 sec -2013-09-18 02:23:47,640 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.31 sec -2013-09-18 02:23:48,647 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.31 sec -2013-09-18 02:23:49,652 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.31 sec -2013-09-18 02:23:50,659 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.31 sec -2013-09-18 02:23:51,665 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.31 sec -2013-09-18 02:23:52,672 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 25.31 sec -2013-09-18 02:23:53,677 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 25.31 sec -2013-09-18 02:23:54,686 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.84 sec -2013-09-18 02:23:55,693 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.84 sec -MapReduce Total cumulative CPU time: 29 seconds 840 msec -Ended Job = job_201309172235_0220 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0221 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:23:58,461 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:24:00,483 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 02:24:01,490 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 02:24:02,496 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 02:24:03,501 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 02:24:04,517 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 02:24:05,523 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 02:24:06,528 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 02:24:07,534 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.72 sec -2013-09-18 02:24:08,540 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.3 sec -2013-09-18 02:24:09,547 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.3 sec -MapReduce Total cumulative CPU time: 2 seconds 300 msec -Ended Job = job_201309172235_0221 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 29.84 sec HDFS Read: 907716 HDFS Write: 384 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.3 sec HDFS Read: 1153 HDFS Write: 60 SUCCESS -Total MapReduce CPU Time Spent: 32 seconds 140 msec -OK -Time taken: 38.784 seconds, Fetched: 9 row(s) -hive> quit; - -times: 2 -query: SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4138@mturlrep13_201309180224_1265964624.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0222 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:24:19,801 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:24:24,833 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.53 sec -2013-09-18 02:24:25,840 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.53 sec -2013-09-18 02:24:26,847 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.53 sec -2013-09-18 02:24:27,852 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.53 sec -2013-09-18 02:24:28,858 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.53 sec -2013-09-18 02:24:29,863 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.53 sec -2013-09-18 02:24:30,870 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.53 sec -2013-09-18 02:24:31,876 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 26.53 sec -2013-09-18 02:24:32,884 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.38 sec -2013-09-18 02:24:33,891 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.38 sec -2013-09-18 02:24:34,898 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 30.38 sec -MapReduce Total cumulative CPU time: 30 seconds 380 msec -Ended Job = job_201309172235_0222 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0223 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:24:37,418 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:24:39,427 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 02:24:40,433 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 02:24:41,438 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 02:24:42,447 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 02:24:43,452 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 02:24:44,457 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 02:24:45,462 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 02:24:46,468 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.69 sec -2013-09-18 02:24:47,474 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.26 sec -2013-09-18 02:24:48,480 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.26 sec -MapReduce Total cumulative CPU time: 2 seconds 260 msec -Ended Job = job_201309172235_0223 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 30.38 sec HDFS Read: 907716 HDFS Write: 384 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.26 sec HDFS Read: 1153 HDFS Write: 60 SUCCESS -Total MapReduce CPU Time Spent: 32 seconds 640 msec -OK -Time taken: 37.082 seconds, Fetched: 9 row(s) -hive> quit; - -times: 3 -query: SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5920@mturlrep13_201309180224_277886677.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0224 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:24:57,648 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:25:02,677 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.93 sec -2013-09-18 02:25:03,685 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.93 sec -2013-09-18 02:25:04,692 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.93 sec -2013-09-18 02:25:05,699 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.93 sec -2013-09-18 02:25:06,705 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.93 sec -2013-09-18 02:25:07,711 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.93 sec -2013-09-18 02:25:08,717 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.93 sec -2013-09-18 02:25:09,723 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.93 sec -2013-09-18 02:25:10,731 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 26.14 sec -2013-09-18 02:25:11,738 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.35 sec -2013-09-18 02:25:12,745 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.35 sec -MapReduce Total cumulative CPU time: 28 seconds 350 msec -Ended Job = job_201309172235_0224 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0225 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:25:15,963 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:25:17,971 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 02:25:18,976 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 02:25:19,986 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 02:25:20,991 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 02:25:21,995 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 02:25:23,000 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 02:25:24,004 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 02:25:25,009 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.67 sec -2013-09-18 02:25:26,014 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -2013-09-18 02:25:27,020 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -2013-09-18 02:25:28,025 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -MapReduce Total cumulative CPU time: 2 seconds 100 msec -Ended Job = job_201309172235_0225 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 28.35 sec HDFS Read: 907716 HDFS Write: 384 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.1 sec HDFS Read: 1153 HDFS Write: 60 SUCCESS -Total MapReduce CPU Time Spent: 30 seconds 450 msec -OK -Time taken: 37.652 seconds, Fetched: 9 row(s) -hive> quit; --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8653@mturlrep13_201309180225_1991831138.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0226 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:25:45,679 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:25:52,712 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:25:56,736 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.02 sec -2013-09-18 02:25:57,743 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.02 sec -2013-09-18 02:25:58,751 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.02 sec -2013-09-18 02:25:59,757 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.02 sec -2013-09-18 02:26:00,764 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.02 sec -2013-09-18 02:26:01,770 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.02 sec -2013-09-18 02:26:02,776 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.02 sec -2013-09-18 02:26:03,782 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.02 sec -2013-09-18 02:26:04,788 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 56.02 sec -2013-09-18 02:26:05,793 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 57.54 sec -2013-09-18 02:26:06,802 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 62.25 sec -2013-09-18 02:26:07,808 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 62.25 sec -2013-09-18 02:26:08,813 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.6 sec -2013-09-18 02:26:09,820 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.6 sec -MapReduce Total cumulative CPU time: 1 minutes 8 seconds 600 msec -Ended Job = job_201309172235_0226 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0227 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:26:13,294 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:26:15,303 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.41 sec -2013-09-18 02:26:16,320 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.41 sec -2013-09-18 02:26:17,325 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.41 sec -2013-09-18 02:26:18,330 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.41 sec -2013-09-18 02:26:19,335 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.41 sec -2013-09-18 02:26:20,341 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.41 sec -2013-09-18 02:26:21,346 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.41 sec -2013-09-18 02:26:22,352 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.41 sec -2013-09-18 02:26:23,358 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.16 sec -2013-09-18 02:26:24,365 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.16 sec -2013-09-18 02:26:25,371 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.16 sec -MapReduce Total cumulative CPU time: 3 seconds 160 msec -Ended Job = job_201309172235_0227 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 68.6 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.16 sec HDFS Read: 100911 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 11 seconds 760 msec -OK -Time taken: 49.973 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10416@mturlrep13_201309180226_2116920110.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0228 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:26:34,811 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:26:42,870 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:26:44,885 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 27.09 sec -2013-09-18 02:26:45,892 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.71 sec -2013-09-18 02:26:46,901 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.71 sec -2013-09-18 02:26:47,907 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.71 sec -2013-09-18 02:26:48,913 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.71 sec -2013-09-18 02:26:49,920 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.71 sec -2013-09-18 02:26:50,926 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.71 sec -2013-09-18 02:26:51,932 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.71 sec -2013-09-18 02:26:52,939 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 56.71 sec -2013-09-18 02:26:53,945 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 56.71 sec -2013-09-18 02:26:54,951 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 56.71 sec -2013-09-18 02:26:55,959 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 62.1 sec -2013-09-18 02:26:56,965 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 62.1 sec -2013-09-18 02:26:57,971 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.76 sec -2013-09-18 02:26:58,977 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.76 sec -MapReduce Total cumulative CPU time: 1 minutes 8 seconds 760 msec -Ended Job = job_201309172235_0228 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0229 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:27:02,462 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:27:04,472 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.37 sec -2013-09-18 02:27:05,478 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.37 sec -2013-09-18 02:27:06,483 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.37 sec -2013-09-18 02:27:07,488 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.37 sec -2013-09-18 02:27:08,493 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.37 sec -2013-09-18 02:27:09,498 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.37 sec -2013-09-18 02:27:10,504 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.37 sec -2013-09-18 02:27:11,511 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.37 sec -2013-09-18 02:27:12,517 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.12 sec -2013-09-18 02:27:13,523 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.12 sec -2013-09-18 02:27:14,530 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.12 sec -MapReduce Total cumulative CPU time: 3 seconds 120 msec -Ended Job = job_201309172235_0229 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 68.76 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.12 sec HDFS Read: 100911 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 11 seconds 880 msec -OK -Time taken: 47.282 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12148@mturlrep13_201309180227_553335692.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0230 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:27:23,737 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:27:31,772 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:27:34,792 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.34 sec -2013-09-18 02:27:35,800 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.34 sec -2013-09-18 02:27:36,806 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.34 sec -2013-09-18 02:27:37,812 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.34 sec -2013-09-18 02:27:38,819 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.34 sec -2013-09-18 02:27:39,825 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.34 sec -2013-09-18 02:27:40,831 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 52.34 sec -2013-09-18 02:27:41,837 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 52.34 sec -2013-09-18 02:27:42,842 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 52.34 sec -2013-09-18 02:27:43,847 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 52.34 sec -2013-09-18 02:27:44,853 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 57.59 sec -2013-09-18 02:27:45,858 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 64.28 sec -2013-09-18 02:27:46,864 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 64.28 sec -2013-09-18 02:27:47,870 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 64.28 sec -MapReduce Total cumulative CPU time: 1 minutes 4 seconds 280 msec -Ended Job = job_201309172235_0230 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0231 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:27:50,366 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:27:52,375 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.35 sec -2013-09-18 02:27:53,380 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.35 sec -2013-09-18 02:27:54,385 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.35 sec -2013-09-18 02:27:55,389 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.35 sec -2013-09-18 02:27:56,393 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.35 sec -2013-09-18 02:27:57,398 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.35 sec -2013-09-18 02:27:58,403 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.35 sec -2013-09-18 02:27:59,408 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.35 sec -2013-09-18 02:28:00,413 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.35 sec -2013-09-18 02:28:01,419 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.99 sec -2013-09-18 02:28:02,425 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.99 sec -MapReduce Total cumulative CPU time: 2 seconds 990 msec -Ended Job = job_201309172235_0231 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 64.28 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.99 sec HDFS Read: 100911 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 7 seconds 270 msec -OK -Time taken: 46.027 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей.; - - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13875@mturlrep13_201309180228_1421953116.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0232 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:28:19,616 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:28:26,647 Stage-1 map = 43%, reduce = 0% -2013-09-18 02:28:27,653 Stage-1 map = 59%, reduce = 0% -2013-09-18 02:28:29,662 Stage-1 map = 73%, reduce = 0% -2013-09-18 02:28:30,668 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:28:33,688 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 65.54 sec -2013-09-18 02:28:34,696 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 65.54 sec -2013-09-18 02:28:35,703 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 65.54 sec -2013-09-18 02:28:36,710 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 65.54 sec -2013-09-18 02:28:37,716 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 65.54 sec -2013-09-18 02:28:38,721 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 65.54 sec -2013-09-18 02:28:39,727 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 65.54 sec -2013-09-18 02:28:40,733 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 65.54 sec -2013-09-18 02:28:41,739 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 65.54 sec -2013-09-18 02:28:42,745 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 65.54 sec -2013-09-18 02:28:43,755 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 65.54 sec -2013-09-18 02:28:44,763 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 72.69 sec -2013-09-18 02:28:45,768 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 72.69 sec -2013-09-18 02:28:46,774 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 81.91 sec -2013-09-18 02:28:47,780 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 81.91 sec -2013-09-18 02:28:48,786 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 81.91 sec -MapReduce Total cumulative CPU time: 1 minutes 21 seconds 910 msec -Ended Job = job_201309172235_0232 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0233 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:28:51,883 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:28:54,897 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.55 sec -2013-09-18 02:28:55,903 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.55 sec -2013-09-18 02:28:56,908 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.55 sec -2013-09-18 02:28:57,913 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.55 sec -2013-09-18 02:28:58,917 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.55 sec -2013-09-18 02:28:59,923 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.55 sec -2013-09-18 02:29:00,929 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.55 sec -2013-09-18 02:29:01,934 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.55 sec -2013-09-18 02:29:02,941 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.3 sec -2013-09-18 02:29:03,947 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.3 sec -MapReduce Total cumulative CPU time: 3 seconds 300 msec -Ended Job = job_201309172235_0233 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 81.91 sec HDFS Read: 74853201 HDFS Write: 148871 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.3 sec HDFS Read: 149640 HDFS Write: 414 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 25 seconds 210 msec -OK -Time taken: 54.759 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15771@mturlrep13_201309180229_811845410.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0234 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:29:13,428 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:29:21,464 Stage-1 map = 74%, reduce = 0% -2013-09-18 02:29:24,478 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:29:26,494 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 49.2 sec -2013-09-18 02:29:27,501 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 67.11 sec -2013-09-18 02:29:28,508 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 67.11 sec -2013-09-18 02:29:29,514 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 67.11 sec -2013-09-18 02:29:30,519 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 67.11 sec -2013-09-18 02:29:31,525 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 67.11 sec -2013-09-18 02:29:32,531 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 67.11 sec -2013-09-18 02:29:33,536 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 67.11 sec -2013-09-18 02:29:34,542 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 67.11 sec -2013-09-18 02:29:35,547 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 67.11 sec -2013-09-18 02:29:36,552 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 67.11 sec -2013-09-18 02:29:37,559 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 73.92 sec -2013-09-18 02:29:38,566 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 73.92 sec -2013-09-18 02:29:39,572 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 73.92 sec -2013-09-18 02:29:40,578 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 83.02 sec -2013-09-18 02:29:41,585 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 83.02 sec -MapReduce Total cumulative CPU time: 1 minutes 23 seconds 20 msec -Ended Job = job_201309172235_0234 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0235 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:29:45,075 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:29:47,084 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-18 02:29:48,089 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-18 02:29:49,094 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-18 02:29:50,099 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-18 02:29:51,104 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-18 02:29:52,109 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-18 02:29:53,114 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.47 sec -2013-09-18 02:29:54,120 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.47 sec -2013-09-18 02:29:55,125 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.2 sec -2013-09-18 02:29:56,130 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.2 sec -MapReduce Total cumulative CPU time: 3 seconds 200 msec -Ended Job = job_201309172235_0235 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 83.02 sec HDFS Read: 74853201 HDFS Write: 148871 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.2 sec HDFS Read: 149640 HDFS Write: 414 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 26 seconds 220 msec -OK -Time taken: 50.301 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17673@mturlrep13_201309180229_1772334372.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0236 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:30:06,838 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:30:13,869 Stage-1 map = 74%, reduce = 0% -2013-09-18 02:30:16,883 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:30:18,898 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 32.43 sec -2013-09-18 02:30:19,906 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 66.34 sec -2013-09-18 02:30:20,942 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 66.34 sec -2013-09-18 02:30:21,949 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 66.34 sec -2013-09-18 02:30:22,955 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 66.34 sec -2013-09-18 02:30:23,960 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 66.34 sec -2013-09-18 02:30:24,965 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 66.34 sec -2013-09-18 02:30:25,971 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 66.34 sec -2013-09-18 02:30:26,976 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 66.34 sec -2013-09-18 02:30:27,982 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 66.34 sec -2013-09-18 02:30:28,989 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 66.34 sec -2013-09-18 02:30:29,995 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 66.34 sec -2013-09-18 02:30:31,003 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 73.15 sec -2013-09-18 02:30:32,010 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 73.15 sec -2013-09-18 02:30:33,016 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 82.57 sec -2013-09-18 02:30:34,022 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 82.57 sec -2013-09-18 02:30:35,108 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 82.57 sec -MapReduce Total cumulative CPU time: 1 minutes 22 seconds 570 msec -Ended Job = job_201309172235_0236 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0237 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:30:37,758 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:30:39,767 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.45 sec -2013-09-18 02:30:40,773 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.45 sec -2013-09-18 02:30:41,779 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.45 sec -2013-09-18 02:30:42,783 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.45 sec -2013-09-18 02:30:43,788 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.45 sec -2013-09-18 02:30:44,792 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.45 sec -2013-09-18 02:30:45,796 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.45 sec -2013-09-18 02:30:46,801 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.45 sec -2013-09-18 02:30:47,806 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.45 sec -2013-09-18 02:30:48,812 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.2 sec -2013-09-18 02:30:49,817 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.2 sec -MapReduce Total cumulative CPU time: 3 seconds 200 msec -Ended Job = job_201309172235_0237 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 82.57 sec HDFS Read: 74853201 HDFS Write: 148871 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.2 sec HDFS Read: 149640 HDFS Write: 414 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 25 seconds 770 msec -OK -Time taken: 51.858 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей, несколько агрегатных функций.; - - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20314@mturlrep13_201309180230_1304388954.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0238 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:31:06,469 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:31:13,508 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 8.59 sec -2013-09-18 02:31:14,520 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.31 sec -2013-09-18 02:31:15,529 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.31 sec -2013-09-18 02:31:16,536 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.31 sec -2013-09-18 02:31:17,543 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.31 sec -2013-09-18 02:31:18,549 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.31 sec -2013-09-18 02:31:19,555 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.31 sec -2013-09-18 02:31:20,562 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 34.31 sec -2013-09-18 02:31:21,569 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 34.31 sec -2013-09-18 02:31:22,576 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 34.31 sec -2013-09-18 02:31:23,584 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.23 sec -2013-09-18 02:31:24,591 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.23 sec -MapReduce Total cumulative CPU time: 41 seconds 230 msec -Ended Job = job_201309172235_0238 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0239 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:31:28,037 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:31:29,043 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:31:30,049 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:31:31,055 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:31:32,060 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:31:33,065 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:31:34,070 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:31:35,076 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:31:36,082 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.79 sec -2013-09-18 02:31:37,088 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -2013-09-18 02:31:38,095 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -2013-09-18 02:31:39,102 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.35 sec -MapReduce Total cumulative CPU time: 2 seconds 350 msec -Ended Job = job_201309172235_0239 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.23 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.35 sec HDFS Read: 21897 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 580 msec -OK -Time taken: 42.793 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22123@mturlrep13_201309180231_845723400.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0240 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:31:49,498 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:31:55,532 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.77 sec -2013-09-18 02:31:56,540 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.77 sec -2013-09-18 02:31:57,547 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.77 sec -2013-09-18 02:31:58,552 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.77 sec -2013-09-18 02:31:59,557 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.77 sec -2013-09-18 02:32:00,563 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.77 sec -2013-09-18 02:32:01,570 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.77 sec -2013-09-18 02:32:02,576 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 31.77 sec -2013-09-18 02:32:03,582 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 31.77 sec -2013-09-18 02:32:04,590 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 38.18 sec -2013-09-18 02:32:05,597 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 38.18 sec -MapReduce Total cumulative CPU time: 38 seconds 180 msec -Ended Job = job_201309172235_0240 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0241 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:32:09,026 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:32:10,031 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-18 02:32:11,037 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-18 02:32:12,043 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-18 02:32:13,048 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-18 02:32:14,053 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-18 02:32:15,058 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-18 02:32:16,063 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-18 02:32:17,068 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-18 02:32:18,074 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -2013-09-18 02:32:19,079 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -2013-09-18 02:32:20,085 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -MapReduce Total cumulative CPU time: 2 seconds 440 msec -Ended Job = job_201309172235_0241 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 38.18 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.44 sec HDFS Read: 21897 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 40 seconds 620 msec -OK -Time taken: 39.001 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23905@mturlrep13_201309180232_33315104.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0242 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:32:30,268 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:32:36,302 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.2 sec -2013-09-18 02:32:37,310 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.2 sec -2013-09-18 02:32:38,317 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.2 sec -2013-09-18 02:32:39,323 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.2 sec -2013-09-18 02:32:40,329 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.2 sec -2013-09-18 02:32:41,336 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.2 sec -2013-09-18 02:32:42,342 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.2 sec -2013-09-18 02:32:43,348 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.2 sec -2013-09-18 02:32:44,355 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 32.2 sec -2013-09-18 02:32:45,363 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.05 sec -2013-09-18 02:32:46,370 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.05 sec -MapReduce Total cumulative CPU time: 39 seconds 50 msec -Ended Job = job_201309172235_0242 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0243 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:32:49,830 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:32:50,836 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:32:51,842 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:32:52,847 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:32:53,852 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:32:54,860 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:32:55,865 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:32:56,871 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:32:57,876 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-18 02:32:58,881 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.37 sec -2013-09-18 02:32:59,886 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.37 sec -2013-09-18 02:33:00,892 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.37 sec -MapReduce Total cumulative CPU time: 2 seconds 370 msec -Ended Job = job_201309172235_0243 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 39.05 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.37 sec HDFS Read: 21897 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 41 seconds 420 msec -OK -Time taken: 38.998 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по строкам.; - - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25692@mturlrep13_201309180233_546222477.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0244 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:33:17,631 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:33:24,670 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 23.16 sec -2013-09-18 02:33:25,678 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.61 sec -2013-09-18 02:33:26,685 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.61 sec -2013-09-18 02:33:27,693 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.61 sec -2013-09-18 02:33:28,699 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.61 sec -2013-09-18 02:33:29,705 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.61 sec -2013-09-18 02:33:30,711 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.61 sec -2013-09-18 02:33:31,718 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.61 sec -2013-09-18 02:33:32,725 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 31.61 sec -2013-09-18 02:33:33,733 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.08 sec -2013-09-18 02:33:34,740 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.08 sec -2013-09-18 02:33:35,747 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.08 sec -MapReduce Total cumulative CPU time: 39 seconds 80 msec -Ended Job = job_201309172235_0244 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0245 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:33:39,294 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:33:40,299 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:33:41,304 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:33:42,310 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:33:43,316 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:33:44,321 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:33:45,326 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:33:46,332 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:33:47,337 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:33:48,344 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.33 sec -2013-09-18 02:33:49,350 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.33 sec -2013-09-18 02:33:50,356 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.33 sec -MapReduce Total cumulative CPU time: 2 seconds 330 msec -Ended Job = job_201309172235_0245 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 39.08 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.33 sec HDFS Read: 23477 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 41 seconds 410 msec -OK -Time taken: 42.858 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27484@mturlrep13_201309180233_1733545584.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0246 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:34:00,545 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:34:06,577 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.0 sec -2013-09-18 02:34:07,584 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.0 sec -2013-09-18 02:34:08,591 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.0 sec -2013-09-18 02:34:09,598 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.0 sec -2013-09-18 02:34:10,604 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.0 sec -2013-09-18 02:34:11,610 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.0 sec -2013-09-18 02:34:12,615 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.0 sec -2013-09-18 02:34:13,621 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 32.0 sec -2013-09-18 02:34:14,628 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 32.0 sec -2013-09-18 02:34:15,636 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.52 sec -2013-09-18 02:34:16,642 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.52 sec -MapReduce Total cumulative CPU time: 39 seconds 520 msec -Ended Job = job_201309172235_0246 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0247 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:34:20,082 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:34:21,088 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:34:22,093 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:34:23,098 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:34:24,102 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:34:25,107 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:34:26,111 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:34:27,116 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:34:28,121 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-18 02:34:29,126 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.36 sec -2013-09-18 02:34:30,132 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.36 sec -2013-09-18 02:34:31,138 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.36 sec -MapReduce Total cumulative CPU time: 2 seconds 360 msec -Ended Job = job_201309172235_0247 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 39.52 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.36 sec HDFS Read: 23479 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 41 seconds 880 msec -OK -Time taken: 38.928 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29266@mturlrep13_201309180234_1724427023.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0248 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:34:41,249 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:34:47,281 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.23 sec -2013-09-18 02:34:48,289 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.23 sec -2013-09-18 02:34:49,295 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.23 sec -2013-09-18 02:34:50,301 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.23 sec -2013-09-18 02:34:51,307 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.23 sec -2013-09-18 02:34:52,313 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.23 sec -2013-09-18 02:34:53,319 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.23 sec -2013-09-18 02:34:54,325 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 33.23 sec -2013-09-18 02:34:55,331 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 33.23 sec -2013-09-18 02:34:56,340 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.78 sec -2013-09-18 02:34:57,346 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.78 sec -2013-09-18 02:34:58,351 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.78 sec -MapReduce Total cumulative CPU time: 40 seconds 780 msec -Ended Job = job_201309172235_0248 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0249 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:35:00,867 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:35:03,879 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:35:04,884 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:35:05,889 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:35:06,894 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:35:07,899 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:35:08,904 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:35:09,909 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-18 02:35:10,915 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -2013-09-18 02:35:11,921 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -2013-09-18 02:35:12,927 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -MapReduce Total cumulative CPU time: 2 seconds 450 msec -Ended Job = job_201309172235_0249 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.78 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.45 sec HDFS Read: 23479 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 230 msec -OK -Time taken: 40.035 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - - -times: 1 -query: SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31746@mturlrep13_201309180235_1665432259.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0250 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:35:30,446 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:35:37,477 Stage-1 map = 74%, reduce = 0% -2013-09-18 02:35:38,490 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 11.18 sec -2013-09-18 02:35:39,498 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.48 sec -2013-09-18 02:35:40,505 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.48 sec -2013-09-18 02:35:41,511 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.48 sec -2013-09-18 02:35:42,516 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.48 sec -2013-09-18 02:35:43,521 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.48 sec -2013-09-18 02:35:44,527 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.48 sec -2013-09-18 02:35:45,534 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.48 sec -2013-09-18 02:35:46,540 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.48 sec -2013-09-18 02:35:47,547 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.48 sec -2013-09-18 02:35:48,554 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.48 sec -2013-09-18 02:35:49,560 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 42.48 sec -2013-09-18 02:35:50,569 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 49.79 sec -2013-09-18 02:35:51,575 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.98 sec -2013-09-18 02:35:52,582 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.98 sec -MapReduce Total cumulative CPU time: 56 seconds 980 msec -Ended Job = job_201309172235_0250 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0251 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:35:56,085 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:36:03,111 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:36:05,121 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.4 sec -2013-09-18 02:36:06,127 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.4 sec -2013-09-18 02:36:07,132 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.4 sec -2013-09-18 02:36:08,137 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.4 sec -2013-09-18 02:36:09,142 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.4 sec -2013-09-18 02:36:10,147 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.4 sec -2013-09-18 02:36:11,153 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.4 sec -2013-09-18 02:36:12,158 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.4 sec -2013-09-18 02:36:13,163 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.4 sec -2013-09-18 02:36:14,171 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.23 sec -2013-09-18 02:36:15,177 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.23 sec -2013-09-18 02:36:16,183 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.23 sec -MapReduce Total cumulative CPU time: 18 seconds 230 msec -Ended Job = job_201309172235_0251 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 56.98 sec HDFS Read: 27820105 HDFS Write: 79726641 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.23 sec HDFS Read: 79727410 HDFS Write: 275 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 15 seconds 210 msec -OK -Time taken: 55.967 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1103@mturlrep13_201309180236_553015116.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0252 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:36:25,454 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:36:33,496 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 32.69 sec -2013-09-18 02:36:34,504 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 44.11 sec -2013-09-18 02:36:35,511 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 44.11 sec -2013-09-18 02:36:36,518 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 44.11 sec -2013-09-18 02:36:37,524 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 44.11 sec -2013-09-18 02:36:38,530 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 44.11 sec -2013-09-18 02:36:39,537 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 44.11 sec -2013-09-18 02:36:40,544 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.11 sec -2013-09-18 02:36:41,551 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 44.11 sec -2013-09-18 02:36:42,557 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 44.11 sec -2013-09-18 02:36:43,563 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 44.11 sec -2013-09-18 02:36:44,570 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 44.11 sec -2013-09-18 02:36:45,577 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 58.39 sec -2013-09-18 02:36:46,583 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 58.39 sec -2013-09-18 02:36:47,590 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 58.39 sec -MapReduce Total cumulative CPU time: 58 seconds 390 msec -Ended Job = job_201309172235_0252 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0253 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:36:50,133 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:36:58,161 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:37:00,169 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.07 sec -2013-09-18 02:37:01,175 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.07 sec -2013-09-18 02:37:02,180 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.07 sec -2013-09-18 02:37:03,184 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.07 sec -2013-09-18 02:37:04,189 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.07 sec -2013-09-18 02:37:05,195 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.07 sec -2013-09-18 02:37:06,200 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.07 sec -2013-09-18 02:37:07,205 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 14.97 sec -2013-09-18 02:37:08,211 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 14.97 sec -2013-09-18 02:37:09,216 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.86 sec -2013-09-18 02:37:10,221 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.86 sec -2013-09-18 02:37:11,226 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.86 sec -MapReduce Total cumulative CPU time: 18 seconds 860 msec -Ended Job = job_201309172235_0253 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 58.39 sec HDFS Read: 27820105 HDFS Write: 79726641 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.86 sec HDFS Read: 79727410 HDFS Write: 275 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 250 msec -OK -Time taken: 53.253 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2921@mturlrep13_201309180237_1642997595.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0254 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:37:21,370 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:37:28,400 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:37:29,413 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.06 sec -2013-09-18 02:37:30,420 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.06 sec -2013-09-18 02:37:31,427 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.06 sec -2013-09-18 02:37:32,433 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.06 sec -2013-09-18 02:37:33,439 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.06 sec -2013-09-18 02:37:34,445 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.06 sec -2013-09-18 02:37:35,451 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.06 sec -2013-09-18 02:37:36,457 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.06 sec -2013-09-18 02:37:37,464 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.06 sec -2013-09-18 02:37:38,470 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.06 sec -2013-09-18 02:37:39,476 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 43.06 sec -2013-09-18 02:37:40,483 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.99 sec -2013-09-18 02:37:41,490 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.99 sec -2013-09-18 02:37:42,495 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.99 sec -MapReduce Total cumulative CPU time: 56 seconds 990 msec -Ended Job = job_201309172235_0254 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0255 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:37:46,017 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:37:53,043 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:37:55,051 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.12 sec -2013-09-18 02:37:56,056 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.12 sec -2013-09-18 02:37:57,061 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.12 sec -2013-09-18 02:37:58,065 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.12 sec -2013-09-18 02:37:59,070 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.12 sec -2013-09-18 02:38:00,074 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.12 sec -2013-09-18 02:38:01,079 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.12 sec -2013-09-18 02:38:02,085 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.12 sec -2013-09-18 02:38:03,090 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.12 sec -2013-09-18 02:38:04,096 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.65 sec -2013-09-18 02:38:05,101 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.65 sec -2013-09-18 02:38:06,106 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.65 sec -MapReduce Total cumulative CPU time: 17 seconds 650 msec -Ended Job = job_201309172235_0255 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 56.99 sec HDFS Read: 27820105 HDFS Write: 79726641 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.65 sec HDFS Read: 79727410 HDFS Write: 275 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 14 seconds 640 msec -OK -Time taken: 53.053 seconds, Fetched: 10 row(s) -hive> quit; --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4654@mturlrep13_201309180238_1576859277.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0256 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:38:23,923 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:38:30,955 Stage-1 map = 59%, reduce = 0% -2013-09-18 02:38:33,979 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.79 sec -2013-09-18 02:38:34,986 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.79 sec -2013-09-18 02:38:35,992 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.79 sec -2013-09-18 02:38:36,999 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.79 sec -2013-09-18 02:38:38,005 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.79 sec -2013-09-18 02:38:39,011 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.79 sec -2013-09-18 02:38:40,018 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.79 sec -2013-09-18 02:38:41,024 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.79 sec -2013-09-18 02:38:42,030 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.79 sec -2013-09-18 02:38:43,036 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.79 sec -2013-09-18 02:38:44,042 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 48.79 sec -2013-09-18 02:38:45,049 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 48.79 sec -2013-09-18 02:38:46,057 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 66.21 sec -2013-09-18 02:38:47,063 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 66.21 sec -2013-09-18 02:38:48,070 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 66.21 sec -MapReduce Total cumulative CPU time: 1 minutes 6 seconds 210 msec -Ended Job = job_201309172235_0256 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0257 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:38:50,633 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:38:58,663 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:39:00,672 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.14 sec -2013-09-18 02:39:01,679 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.14 sec -2013-09-18 02:39:02,684 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.14 sec -2013-09-18 02:39:03,690 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.14 sec -2013-09-18 02:39:04,695 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.14 sec -2013-09-18 02:39:05,700 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.14 sec -2013-09-18 02:39:06,705 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.14 sec -2013-09-18 02:39:07,710 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.14 sec -2013-09-18 02:39:08,715 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 14.0 sec -2013-09-18 02:39:09,720 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.9 sec -2013-09-18 02:39:10,726 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.9 sec -2013-09-18 02:39:11,731 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.9 sec -MapReduce Total cumulative CPU time: 17 seconds 900 msec -Ended Job = job_201309172235_0257 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 66.21 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.9 sec HDFS Read: 79727313 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 24 seconds 110 msec -OK -Time taken: 58.068 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6870@mturlrep13_201309180239_5166481.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0258 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:39:22,278 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:39:29,309 Stage-1 map = 74%, reduce = 0% -2013-09-18 02:39:30,322 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 11.14 sec -2013-09-18 02:39:31,329 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.0 sec -2013-09-18 02:39:32,337 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.0 sec -2013-09-18 02:39:33,342 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.0 sec -2013-09-18 02:39:34,347 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.0 sec -2013-09-18 02:39:35,353 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.0 sec -2013-09-18 02:39:36,359 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.0 sec -2013-09-18 02:39:37,365 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.0 sec -2013-09-18 02:39:38,374 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.0 sec -2013-09-18 02:39:39,381 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.0 sec -2013-09-18 02:39:40,387 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.0 sec -2013-09-18 02:39:41,394 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 48.0 sec -2013-09-18 02:39:42,399 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 48.0 sec -2013-09-18 02:39:43,407 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 65.26 sec -2013-09-18 02:39:44,413 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 65.26 sec -MapReduce Total cumulative CPU time: 1 minutes 5 seconds 260 msec -Ended Job = job_201309172235_0258 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0259 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:39:47,918 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:39:54,941 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:39:56,950 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.07 sec -2013-09-18 02:39:57,956 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.07 sec -2013-09-18 02:39:58,961 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.07 sec -2013-09-18 02:39:59,965 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.07 sec -2013-09-18 02:40:00,970 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.07 sec -2013-09-18 02:40:01,975 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.07 sec -2013-09-18 02:40:02,980 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.07 sec -2013-09-18 02:40:03,984 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.07 sec -2013-09-18 02:40:04,989 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.07 sec -2013-09-18 02:40:05,995 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.73 sec -2013-09-18 02:40:07,001 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.73 sec -2013-09-18 02:40:08,006 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.73 sec -MapReduce Total cumulative CPU time: 17 seconds 730 msec -Ended Job = job_201309172235_0259 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 65.26 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.73 sec HDFS Read: 79727313 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 22 seconds 990 msec -OK -Time taken: 54.393 seconds, Fetched: 10 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8928@mturlrep13_201309180240_2088574054.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0260 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:40:18,597 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:40:25,630 Stage-1 map = 74%, reduce = 0% -2013-09-18 02:40:26,645 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 11.55 sec -2013-09-18 02:40:27,653 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 47.89 sec -2013-09-18 02:40:28,661 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 47.89 sec -2013-09-18 02:40:29,668 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 47.89 sec -2013-09-18 02:40:30,674 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 47.89 sec -2013-09-18 02:40:31,680 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 47.89 sec -2013-09-18 02:40:32,687 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 47.89 sec -2013-09-18 02:40:33,694 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.89 sec -2013-09-18 02:40:34,701 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 47.89 sec -2013-09-18 02:40:35,707 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 47.89 sec -2013-09-18 02:40:36,714 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 47.89 sec -2013-09-18 02:40:37,720 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 47.89 sec -2013-09-18 02:40:38,726 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 47.89 sec -2013-09-18 02:40:39,734 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 64.72 sec -2013-09-18 02:40:40,740 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 64.72 sec -MapReduce Total cumulative CPU time: 1 minutes 4 seconds 720 msec -Ended Job = job_201309172235_0260 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0261 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:40:43,177 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:40:51,206 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:40:53,216 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.81 sec -2013-09-18 02:40:54,221 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.81 sec -2013-09-18 02:40:55,226 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.81 sec -2013-09-18 02:40:56,231 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.81 sec -2013-09-18 02:40:57,236 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.81 sec -2013-09-18 02:40:58,242 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.81 sec -2013-09-18 02:40:59,247 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.81 sec -2013-09-18 02:41:00,253 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 12.81 sec -2013-09-18 02:41:01,259 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 12.81 sec -2013-09-18 02:41:02,264 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.59 sec -2013-09-18 02:41:03,271 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.59 sec -MapReduce Total cumulative CPU time: 17 seconds 590 msec -Ended Job = job_201309172235_0261 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 64.72 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.59 sec HDFS Read: 79727311 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 22 seconds 310 msec -OK -Time taken: 53.189 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация чуть сложнее.; - - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11232@mturlrep13_201309180241_2050644034.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0262 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:41:21,239 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:41:28,272 Stage-1 map = 18%, reduce = 0% -2013-09-18 02:41:29,278 Stage-1 map = 74%, reduce = 0% -2013-09-18 02:41:30,291 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 22.42 sec -2013-09-18 02:41:31,298 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 45.45 sec -2013-09-18 02:41:32,306 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 45.45 sec -2013-09-18 02:41:33,312 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 45.45 sec -2013-09-18 02:41:34,319 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 45.45 sec -2013-09-18 02:41:35,326 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 45.45 sec -2013-09-18 02:41:36,333 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 45.45 sec -2013-09-18 02:41:37,340 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.45 sec -2013-09-18 02:41:38,347 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 45.45 sec -2013-09-18 02:41:39,353 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 45.45 sec -2013-09-18 02:41:40,360 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 45.45 sec -2013-09-18 02:41:41,366 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 45.45 sec -2013-09-18 02:41:42,375 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 60.58 sec -2013-09-18 02:41:43,382 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 60.58 sec -2013-09-18 02:41:44,388 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 60.58 sec -MapReduce Total cumulative CPU time: 1 minutes 0 seconds 580 msec -Ended Job = job_201309172235_0262 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0263 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:41:48,080 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:41:55,103 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:41:58,114 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.06 sec -2013-09-18 02:41:59,120 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.06 sec -2013-09-18 02:42:00,125 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.06 sec -2013-09-18 02:42:01,131 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.06 sec -2013-09-18 02:42:02,136 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.06 sec -2013-09-18 02:42:03,142 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.06 sec -2013-09-18 02:42:04,147 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.06 sec -2013-09-18 02:42:05,153 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.06 sec -2013-09-18 02:42:06,159 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 14.06 sec -2013-09-18 02:42:07,165 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.95 sec -2013-09-18 02:42:08,172 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.95 sec -2013-09-18 02:42:09,178 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.95 sec -MapReduce Total cumulative CPU time: 18 seconds 950 msec -Ended Job = job_201309172235_0263 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 60.58 sec HDFS Read: 30310112 HDFS Write: 84160093 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.95 sec HDFS Read: 84160862 HDFS Write: 297 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 19 seconds 530 msec -OK -Time taken: 58.359 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12984@mturlrep13_201309180242_2082311040.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0264 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:42:19,501 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:42:26,537 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 10.64 sec -2013-09-18 02:42:27,545 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:42:28,553 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:42:29,559 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:42:30,565 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:42:31,571 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:42:32,576 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:42:33,583 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.43 sec -2013-09-18 02:42:34,590 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.43 sec -2013-09-18 02:42:35,596 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.43 sec -2013-09-18 02:42:36,602 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.43 sec -2013-09-18 02:42:37,608 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 43.43 sec -2013-09-18 02:42:38,614 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 43.43 sec -2013-09-18 02:42:39,621 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 59.06 sec -2013-09-18 02:42:40,627 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 59.06 sec -MapReduce Total cumulative CPU time: 59 seconds 60 msec -Ended Job = job_201309172235_0264 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0265 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:42:43,197 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:42:51,226 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:42:54,237 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.46 sec -2013-09-18 02:42:55,243 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.46 sec -2013-09-18 02:42:56,247 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.46 sec -2013-09-18 02:42:57,254 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.46 sec -2013-09-18 02:42:58,260 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.46 sec -2013-09-18 02:42:59,265 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.46 sec -2013-09-18 02:43:00,275 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 14.46 sec -2013-09-18 02:43:01,281 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 14.46 sec -2013-09-18 02:43:02,286 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 14.46 sec -2013-09-18 02:43:03,291 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.26 sec -2013-09-18 02:43:04,296 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.26 sec -MapReduce Total cumulative CPU time: 19 seconds 260 msec -Ended Job = job_201309172235_0265 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 59.06 sec HDFS Read: 30310112 HDFS Write: 84160093 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 19.26 sec HDFS Read: 84160862 HDFS Write: 297 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 18 seconds 320 msec -OK -Time taken: 53.181 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14722@mturlrep13_201309180243_131124739.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0266 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:43:13,609 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:43:21,644 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:43:22,656 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.26 sec -2013-09-18 02:43:23,663 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.26 sec -2013-09-18 02:43:24,669 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.26 sec -2013-09-18 02:43:25,675 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.26 sec -2013-09-18 02:43:26,680 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.26 sec -2013-09-18 02:43:27,686 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.26 sec -2013-09-18 02:43:28,693 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 43.26 sec -2013-09-18 02:43:29,698 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.26 sec -2013-09-18 02:43:30,704 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.26 sec -2013-09-18 02:43:31,711 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 43.26 sec -2013-09-18 02:43:32,717 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 43.26 sec -2013-09-18 02:43:33,724 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 43.26 sec -2013-09-18 02:43:34,732 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 58.63 sec -2013-09-18 02:43:35,739 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 58.63 sec -MapReduce Total cumulative CPU time: 58 seconds 630 msec -Ended Job = job_201309172235_0266 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0267 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:43:39,244 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:43:46,270 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:43:49,283 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-18 02:43:50,289 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-18 02:43:51,294 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-18 02:43:52,300 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-18 02:43:53,305 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-18 02:43:54,313 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-18 02:43:55,318 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-18 02:43:56,323 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-18 02:43:57,329 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.92 sec -2013-09-18 02:43:58,334 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.06 sec -2013-09-18 02:43:59,340 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.06 sec -2013-09-18 02:44:00,345 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.06 sec -MapReduce Total cumulative CPU time: 19 seconds 60 msec -Ended Job = job_201309172235_0267 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 58.63 sec HDFS Read: 30310112 HDFS Write: 84160093 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 19.06 sec HDFS Read: 84160862 HDFS Write: 297 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 690 msec -OK -Time taken: 54.164 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по числу и строке, большое количество ключей.; - - -times: 1 -query: SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16467@mturlrep13_201309180244_1529367412.txt -hive> SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0268 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:44:18,084 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:44:25,121 Stage-1 map = 81%, reduce = 0% -2013-09-18 02:44:28,140 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 23.45 sec -2013-09-18 02:44:29,148 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.87 sec -2013-09-18 02:44:30,155 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.87 sec -2013-09-18 02:44:31,161 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.87 sec -2013-09-18 02:44:32,167 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.87 sec -2013-09-18 02:44:33,174 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.87 sec -2013-09-18 02:44:34,180 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.87 sec -2013-09-18 02:44:35,186 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.87 sec -2013-09-18 02:44:36,192 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.87 sec -2013-09-18 02:44:37,198 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.87 sec -2013-09-18 02:44:38,204 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.87 sec -2013-09-18 02:44:39,209 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 48.87 sec -2013-09-18 02:44:40,217 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.8 sec -2013-09-18 02:44:41,222 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.8 sec -2013-09-18 02:44:42,229 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.8 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 800 msec -Ended Job = job_201309172235_0268 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0269 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:44:45,729 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:44:55,761 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:44:58,774 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.29 sec -2013-09-18 02:44:59,779 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.29 sec -2013-09-18 02:45:00,790 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.29 sec -2013-09-18 02:45:01,795 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.29 sec -2013-09-18 02:45:02,800 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.29 sec -2013-09-18 02:45:03,805 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.29 sec -2013-09-18 02:45:04,809 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.29 sec -2013-09-18 02:45:05,814 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.29 sec -2013-09-18 02:45:06,819 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.29 sec -2013-09-18 02:45:07,824 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.29 sec -2013-09-18 02:45:08,830 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.02 sec -2013-09-18 02:45:09,835 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.02 sec -2013-09-18 02:45:10,840 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.02 sec -MapReduce Total cumulative CPU time: 23 seconds 20 msec -Ended Job = job_201309172235_0269 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.8 sec HDFS Read: 57312623 HDFS Write: 55475412 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 23.02 sec HDFS Read: 55476181 HDFS Write: 246 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 25 seconds 820 msec -OK -Time taken: 63.096 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18524@mturlrep13_201309180245_141259504.txt -hive> SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0270 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:45:21,420 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:45:28,450 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:45:30,465 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 35.49 sec -2013-09-18 02:45:31,472 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.62 sec -2013-09-18 02:45:32,478 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.62 sec -2013-09-18 02:45:33,483 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.62 sec -2013-09-18 02:45:34,488 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.62 sec -2013-09-18 02:45:35,494 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.62 sec -2013-09-18 02:45:36,501 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.62 sec -2013-09-18 02:45:37,507 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.62 sec -2013-09-18 02:45:38,513 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.62 sec -2013-09-18 02:45:39,518 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.62 sec -2013-09-18 02:45:40,523 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 48.62 sec -2013-09-18 02:45:41,529 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 48.62 sec -2013-09-18 02:45:42,537 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.57 sec -2013-09-18 02:45:43,543 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.57 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 570 msec -Ended Job = job_201309172235_0270 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0271 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:45:47,340 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:45:57,375 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:46:01,391 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.09 sec -2013-09-18 02:46:02,397 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.09 sec -2013-09-18 02:46:03,402 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.09 sec -2013-09-18 02:46:04,420 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.09 sec -2013-09-18 02:46:05,425 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.09 sec -2013-09-18 02:46:06,429 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.09 sec -2013-09-18 02:46:07,435 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.09 sec -2013-09-18 02:46:08,440 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.09 sec -2013-09-18 02:46:09,446 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.88 sec -2013-09-18 02:46:10,452 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.96 sec -2013-09-18 02:46:11,457 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.96 sec -MapReduce Total cumulative CPU time: 22 seconds 960 msec -Ended Job = job_201309172235_0271 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.57 sec HDFS Read: 57312623 HDFS Write: 55475412 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 22.96 sec HDFS Read: 55476181 HDFS Write: 246 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 25 seconds 530 msec -OK -Time taken: 58.614 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20702@mturlrep13_201309180246_772055237.txt -hive> SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0272 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:46:21,609 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:46:28,640 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:46:30,656 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 23.58 sec -2013-09-18 02:46:31,664 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.28 sec -2013-09-18 02:46:32,671 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.28 sec -2013-09-18 02:46:33,676 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.28 sec -2013-09-18 02:46:34,682 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.28 sec -2013-09-18 02:46:35,688 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.28 sec -2013-09-18 02:46:36,698 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.28 sec -2013-09-18 02:46:37,706 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.28 sec -2013-09-18 02:46:38,712 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 49.28 sec -2013-09-18 02:46:39,717 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 49.28 sec -2013-09-18 02:46:40,723 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 49.28 sec -2013-09-18 02:46:41,728 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 49.28 sec -2013-09-18 02:46:42,735 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.18 sec -2013-09-18 02:46:43,741 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.18 sec -2013-09-18 02:46:44,747 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.18 sec -MapReduce Total cumulative CPU time: 1 minutes 3 seconds 180 msec -Ended Job = job_201309172235_0272 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0273 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:46:48,292 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:46:58,320 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:47:02,333 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.58 sec -2013-09-18 02:47:03,338 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.58 sec -2013-09-18 02:47:04,343 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.58 sec -2013-09-18 02:47:05,347 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.58 sec -2013-09-18 02:47:06,351 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.58 sec -2013-09-18 02:47:07,355 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.58 sec -2013-09-18 02:47:08,360 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.58 sec -2013-09-18 02:47:09,365 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 19.39 sec -2013-09-18 02:47:10,369 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 19.39 sec -2013-09-18 02:47:11,375 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.3 sec -2013-09-18 02:47:12,380 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.3 sec -2013-09-18 02:47:13,386 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.3 sec -MapReduce Total cumulative CPU time: 23 seconds 300 msec -Ended Job = job_201309172235_0273 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 63.18 sec HDFS Read: 57312623 HDFS Write: 55475412 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 23.3 sec HDFS Read: 55476181 HDFS Write: 246 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 26 seconds 480 msec -OK -Time taken: 60.153 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22448@mturlrep13_201309180247_332946336.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0274 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:47:30,239 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:47:37,271 Stage-1 map = 63%, reduce = 0% -2013-09-18 02:47:40,286 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:47:43,305 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 51.04 sec -2013-09-18 02:47:44,312 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 70.28 sec -2013-09-18 02:47:45,319 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 70.28 sec -2013-09-18 02:47:46,324 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 70.28 sec -2013-09-18 02:47:47,329 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 70.28 sec -2013-09-18 02:47:48,334 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 70.28 sec -2013-09-18 02:47:49,340 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 70.28 sec -2013-09-18 02:47:50,346 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 70.28 sec -2013-09-18 02:47:51,352 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 70.28 sec -2013-09-18 02:47:52,359 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 70.28 sec -2013-09-18 02:47:53,365 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 70.28 sec -2013-09-18 02:47:54,371 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 70.28 sec -2013-09-18 02:47:55,378 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 70.28 sec -2013-09-18 02:47:56,384 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 70.28 sec -2013-09-18 02:47:57,392 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 92.27 sec -2013-09-18 02:47:58,399 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 92.27 sec -2013-09-18 02:47:59,404 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 92.27 sec -MapReduce Total cumulative CPU time: 1 minutes 32 seconds 270 msec -Ended Job = job_201309172235_0274 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0275 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:48:02,066 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:48:13,104 Stage-2 map = 46%, reduce = 0% -2013-09-18 02:48:16,114 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:48:19,125 Stage-2 map = 96%, reduce = 0% -2013-09-18 02:48:21,132 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.92 sec -2013-09-18 02:48:22,138 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.92 sec -2013-09-18 02:48:23,142 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.92 sec -2013-09-18 02:48:24,146 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.92 sec -2013-09-18 02:48:25,150 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.92 sec -2013-09-18 02:48:26,154 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.92 sec -2013-09-18 02:48:27,159 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.92 sec -2013-09-18 02:48:28,165 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.92 sec -2013-09-18 02:48:29,170 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.92 sec -2013-09-18 02:48:30,175 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.92 sec -2013-09-18 02:48:31,182 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.57 sec -2013-09-18 02:48:32,187 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.57 sec -2013-09-18 02:48:33,193 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.57 sec -MapReduce Total cumulative CPU time: 28 seconds 570 msec -Ended Job = job_201309172235_0275 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 92.27 sec HDFS Read: 84536695 HDFS Write: 146202868 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 28.57 sec HDFS Read: 146210123 HDFS Write: 256 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 0 seconds 840 msec -OK -Time taken: 73.175 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24287@mturlrep13_201309180248_957091100.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0276 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:48:43,644 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:48:50,677 Stage-1 map = 81%, reduce = 0% -2013-09-18 02:48:53,691 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:48:55,754 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.03 sec -2013-09-18 02:48:56,761 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.03 sec -2013-09-18 02:48:57,769 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.03 sec -2013-09-18 02:48:58,775 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.03 sec -2013-09-18 02:48:59,781 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.03 sec -2013-09-18 02:49:00,786 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.03 sec -2013-09-18 02:49:01,792 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.03 sec -2013-09-18 02:49:02,797 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.03 sec -2013-09-18 02:49:03,804 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 69.03 sec -2013-09-18 02:49:04,810 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 69.03 sec -2013-09-18 02:49:05,816 Stage-1 map = 100%, reduce = 53%, Cumulative CPU 69.03 sec -2013-09-18 02:49:06,822 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 69.03 sec -2013-09-18 02:49:07,828 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 69.03 sec -2013-09-18 02:49:08,834 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 69.03 sec -2013-09-18 02:49:09,843 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 87.96 sec -2013-09-18 02:49:10,849 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 90.76 sec -2013-09-18 02:49:11,855 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 90.76 sec -MapReduce Total cumulative CPU time: 1 minutes 30 seconds 760 msec -Ended Job = job_201309172235_0276 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0277 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:49:15,354 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:49:25,387 Stage-2 map = 46%, reduce = 0% -2013-09-18 02:49:28,396 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:49:31,406 Stage-2 map = 96%, reduce = 0% -2013-09-18 02:49:34,416 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.84 sec -2013-09-18 02:49:35,421 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.84 sec -2013-09-18 02:49:36,510 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.84 sec -2013-09-18 02:49:37,521 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.84 sec -2013-09-18 02:49:38,526 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.84 sec -2013-09-18 02:49:39,530 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.84 sec -2013-09-18 02:49:40,536 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.84 sec -2013-09-18 02:49:41,541 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.84 sec -2013-09-18 02:49:42,546 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.84 sec -2013-09-18 02:49:43,551 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.84 sec -2013-09-18 02:49:44,557 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.37 sec -2013-09-18 02:49:45,562 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.37 sec -MapReduce Total cumulative CPU time: 28 seconds 370 msec -Ended Job = job_201309172235_0277 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 90.76 sec HDFS Read: 84536695 HDFS Write: 146202868 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 28.37 sec HDFS Read: 146210119 HDFS Write: 256 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 59 seconds 130 msec -OK -Time taken: 70.386 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26119@mturlrep13_201309180249_746088235.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0278 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:49:54,756 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:50:02,790 Stage-1 map = 78%, reduce = 0% -2013-09-18 02:50:05,803 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:50:07,818 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 72.7 sec -2013-09-18 02:50:08,826 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 72.7 sec -2013-09-18 02:50:09,834 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 72.7 sec -2013-09-18 02:50:10,840 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 72.7 sec -2013-09-18 02:50:11,846 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 72.7 sec -2013-09-18 02:50:12,852 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 72.7 sec -2013-09-18 02:50:13,858 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 72.7 sec -2013-09-18 02:50:14,864 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 72.7 sec -2013-09-18 02:50:15,870 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 72.7 sec -2013-09-18 02:50:16,877 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 72.7 sec -2013-09-18 02:50:17,883 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 72.7 sec -2013-09-18 02:50:18,889 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 72.7 sec -2013-09-18 02:50:19,895 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 72.7 sec -2013-09-18 02:50:20,901 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 72.7 sec -2013-09-18 02:50:21,909 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 83.46 sec -2013-09-18 02:50:22,915 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 94.56 sec -2013-09-18 02:50:23,921 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 94.56 sec -MapReduce Total cumulative CPU time: 1 minutes 34 seconds 560 msec -Ended Job = job_201309172235_0278 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0279 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:50:27,487 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:50:37,520 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:50:43,538 Stage-2 map = 96%, reduce = 0% -2013-09-18 02:50:45,545 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.15 sec -2013-09-18 02:50:46,551 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.15 sec -2013-09-18 02:50:47,555 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.15 sec -2013-09-18 02:50:48,560 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.15 sec -2013-09-18 02:50:49,564 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.15 sec -2013-09-18 02:50:50,568 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.15 sec -2013-09-18 02:50:51,572 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.15 sec -2013-09-18 02:50:52,577 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.15 sec -2013-09-18 02:50:53,583 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.15 sec -2013-09-18 02:50:54,588 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.15 sec -2013-09-18 02:50:55,594 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 27.69 sec -2013-09-18 02:50:56,599 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 27.69 sec -MapReduce Total cumulative CPU time: 27 seconds 690 msec -Ended Job = job_201309172235_0279 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 94.56 sec HDFS Read: 84536695 HDFS Write: 146202868 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 27.69 sec HDFS Read: 146210123 HDFS Write: 256 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 2 seconds 250 msec -OK -Time taken: 69.204 seconds, Fetched: 10 row(s) -hive> quit; --- ещё более сложная агрегация.; - - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28670@mturlrep13_201309180251_345585890.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0280 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:51:14,036 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:51:21,066 Stage-1 map = 59%, reduce = 0% -2013-09-18 02:51:24,080 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:51:27,098 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 34.73 sec -2013-09-18 02:51:28,105 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.99 sec -2013-09-18 02:51:29,113 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.99 sec -2013-09-18 02:51:30,119 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.99 sec -2013-09-18 02:51:31,124 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.99 sec -2013-09-18 02:51:32,130 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.99 sec -2013-09-18 02:51:33,135 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.99 sec -2013-09-18 02:51:34,140 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 69.99 sec -2013-09-18 02:51:35,146 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 69.99 sec -2013-09-18 02:51:36,151 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 69.99 sec -2013-09-18 02:51:37,158 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 79.66 sec -2013-09-18 02:51:38,164 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 79.66 sec -2013-09-18 02:51:39,170 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 79.66 sec -MapReduce Total cumulative CPU time: 1 minutes 19 seconds 660 msec -Ended Job = job_201309172235_0280 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 79.66 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 19 seconds 660 msec -OK -Time taken: 35.511 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30045@mturlrep13_201309180251_893832227.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0281 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:51:48,380 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:51:56,430 Stage-1 map = 78%, reduce = 0% -2013-09-18 02:51:59,444 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:52:00,455 Stage-1 map = 92%, reduce = 0%, Cumulative CPU 17.14 sec -2013-09-18 02:52:01,463 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.67 sec -2013-09-18 02:52:02,470 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.67 sec -2013-09-18 02:52:03,476 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.67 sec -2013-09-18 02:52:04,482 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.67 sec -2013-09-18 02:52:05,487 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.67 sec -2013-09-18 02:52:06,491 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.67 sec -2013-09-18 02:52:07,496 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.67 sec -2013-09-18 02:52:08,502 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 68.67 sec -2013-09-18 02:52:09,508 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 70.32 sec -2013-09-18 02:52:10,516 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.77 sec -2013-09-18 02:52:11,522 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.77 sec -2013-09-18 02:52:12,528 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.77 sec -MapReduce Total cumulative CPU time: 1 minutes 17 seconds 770 msec -Ended Job = job_201309172235_0281 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 77.77 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 770 msec -OK -Time taken: 31.512 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31399@mturlrep13_201309180252_43587713.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0282 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:52:22,770 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:52:29,799 Stage-1 map = 78%, reduce = 0% -2013-09-18 02:52:32,813 Stage-1 map = 89%, reduce = 0% -2013-09-18 02:52:34,829 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.3 sec -2013-09-18 02:52:35,836 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.3 sec -2013-09-18 02:52:36,843 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.3 sec -2013-09-18 02:52:37,849 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.3 sec -2013-09-18 02:52:38,855 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.3 sec -2013-09-18 02:52:39,861 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.3 sec -2013-09-18 02:52:40,866 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 68.3 sec -2013-09-18 02:52:41,872 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 68.3 sec -2013-09-18 02:52:42,877 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 68.3 sec -2013-09-18 02:52:43,885 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.39 sec -2013-09-18 02:52:44,891 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.39 sec -2013-09-18 02:52:45,897 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.39 sec -MapReduce Total cumulative CPU time: 1 minutes 17 seconds 390 msec -Ended Job = job_201309172235_0282 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 77.39 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 390 msec -OK -Time taken: 31.482 seconds, Fetched: 10 row(s) -hive> quit; --- то же самое, но без сортировки.; - - -times: 1 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_300@mturlrep13_201309180252_1115673732.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0283 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:53:02,084 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:53:09,112 Stage-1 map = 15%, reduce = 0% -2013-09-18 02:53:12,137 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 40.15 sec -2013-09-18 02:53:13,143 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 40.15 sec -2013-09-18 02:53:14,151 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 40.15 sec -2013-09-18 02:53:15,163 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 40.15 sec -2013-09-18 02:53:16,169 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 40.15 sec -2013-09-18 02:53:17,177 Stage-1 map = 48%, reduce = 0%, Cumulative CPU 40.15 sec -2013-09-18 02:53:18,184 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 40.15 sec -2013-09-18 02:53:19,189 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 40.15 sec -2013-09-18 02:53:20,216 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 40.15 sec -2013-09-18 02:53:21,223 Stage-1 map = 85%, reduce = 0%, Cumulative CPU 40.15 sec -2013-09-18 02:53:22,231 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 56.66 sec -2013-09-18 02:53:23,237 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 56.66 sec -2013-09-18 02:53:24,244 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 95.24 sec -2013-09-18 02:53:25,249 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 115.4 sec -2013-09-18 02:53:26,255 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 115.4 sec -2013-09-18 02:53:27,260 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 115.4 sec -2013-09-18 02:53:28,266 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 115.4 sec -2013-09-18 02:53:29,272 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 115.4 sec -2013-09-18 02:53:30,278 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 115.4 sec -2013-09-18 02:53:31,283 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 115.4 sec -2013-09-18 02:53:32,288 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 115.4 sec -2013-09-18 02:53:33,293 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 115.4 sec -2013-09-18 02:53:34,298 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 115.4 sec -2013-09-18 02:53:35,303 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 115.4 sec -2013-09-18 02:53:36,308 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 115.4 sec -2013-09-18 02:53:37,314 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 115.4 sec -2013-09-18 02:53:38,320 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 115.4 sec -2013-09-18 02:53:39,325 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 115.4 sec -2013-09-18 02:53:40,330 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 115.4 sec -2013-09-18 02:53:41,336 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 115.4 sec -2013-09-18 02:53:42,341 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 115.4 sec -2013-09-18 02:53:43,345 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 115.4 sec -2013-09-18 02:53:44,351 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 115.4 sec -2013-09-18 02:53:45,356 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 115.4 sec -2013-09-18 02:53:46,361 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 115.4 sec -2013-09-18 02:53:47,367 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 159.4 sec -2013-09-18 02:53:48,372 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 159.4 sec -2013-09-18 02:53:49,378 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 159.4 sec -MapReduce Total cumulative CPU time: 2 minutes 39 seconds 400 msec -Ended Job = job_201309172235_0283 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0284 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:53:51,875 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:54:05,919 Stage-2 map = 28%, reduce = 0% -2013-09-18 02:54:11,975 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:12,981 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:13,986 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:14,991 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:15,996 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:17,001 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:18,006 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:19,011 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:20,016 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:21,021 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:22,026 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:23,032 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:24,037 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:25,042 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:26,047 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:27,052 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 20.31 sec -2013-09-18 02:54:28,057 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.15 sec -2013-09-18 02:54:29,062 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.15 sec -2013-09-18 02:54:30,066 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.15 sec -2013-09-18 02:54:31,071 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.15 sec -2013-09-18 02:54:32,075 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.15 sec -2013-09-18 02:54:33,080 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.15 sec -2013-09-18 02:54:34,085 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.15 sec -2013-09-18 02:54:35,090 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.15 sec -2013-09-18 02:54:36,095 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.15 sec -2013-09-18 02:54:37,100 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.15 sec -2013-09-18 02:54:38,105 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 42.15 sec -2013-09-18 02:54:39,110 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 42.15 sec -2013-09-18 02:54:40,115 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 48.87 sec -2013-09-18 02:54:41,121 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 48.87 sec -MapReduce Total cumulative CPU time: 48 seconds 870 msec -Ended Job = job_201309172235_0284 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 159.4 sec HDFS Read: 84944733 HDFS Write: 241346048 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 48.87 sec HDFS Read: 241349358 HDFS Write: 268 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 28 seconds 270 msec -OK -Time taken: 109.132 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2351@mturlrep13_201309180254_1547940573.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0285 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:54:50,345 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:54:58,379 Stage-1 map = 15%, reduce = 0% -2013-09-18 02:55:01,391 Stage-1 map = 40%, reduce = 0% -2013-09-18 02:55:04,405 Stage-1 map = 59%, reduce = 0% -2013-09-18 02:55:07,418 Stage-1 map = 74%, reduce = 0% -2013-09-18 02:55:10,437 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 93.25 sec -2013-09-18 02:55:11,444 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 93.25 sec -2013-09-18 02:55:12,452 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.72 sec -2013-09-18 02:55:13,458 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.72 sec -2013-09-18 02:55:14,464 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.72 sec -2013-09-18 02:55:15,469 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.72 sec -2013-09-18 02:55:16,474 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.72 sec -2013-09-18 02:55:17,481 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.72 sec -2013-09-18 02:55:18,487 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 112.72 sec -2013-09-18 02:55:19,492 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 112.72 sec -2013-09-18 02:55:20,497 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 112.72 sec -2013-09-18 02:55:21,503 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 112.72 sec -2013-09-18 02:55:22,509 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 112.72 sec -2013-09-18 02:55:23,515 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 112.72 sec -2013-09-18 02:55:24,524 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 112.72 sec -2013-09-18 02:55:25,529 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 112.72 sec -2013-09-18 02:55:26,535 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 112.72 sec -2013-09-18 02:55:27,540 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 112.72 sec -2013-09-18 02:55:28,545 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 112.72 sec -2013-09-18 02:55:29,551 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 112.72 sec -2013-09-18 02:55:30,557 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 112.72 sec -2013-09-18 02:55:31,562 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 112.72 sec -2013-09-18 02:55:32,568 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 112.72 sec -2013-09-18 02:55:33,573 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 112.72 sec -2013-09-18 02:55:34,580 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 156.37 sec -2013-09-18 02:55:35,585 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 156.37 sec -2013-09-18 02:55:36,591 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 156.37 sec -MapReduce Total cumulative CPU time: 2 minutes 36 seconds 370 msec -Ended Job = job_201309172235_0285 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0286 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:55:40,097 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:55:53,132 Stage-2 map = 28%, reduce = 0% -2013-09-18 02:55:59,148 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:56:08,170 Stage-2 map = 78%, reduce = 0% -2013-09-18 02:56:17,199 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.74 sec -2013-09-18 02:56:18,204 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.74 sec -2013-09-18 02:56:19,209 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.74 sec -2013-09-18 02:56:20,214 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.74 sec -2013-09-18 02:56:21,218 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.74 sec -2013-09-18 02:56:22,222 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.74 sec -2013-09-18 02:56:23,227 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.74 sec -2013-09-18 02:56:24,232 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.74 sec -2013-09-18 02:56:25,238 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.74 sec -2013-09-18 02:56:26,243 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.74 sec -2013-09-18 02:56:27,249 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 43.74 sec -2013-09-18 02:56:28,254 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.54 sec -2013-09-18 02:56:29,260 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.54 sec -2013-09-18 02:56:30,265 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.54 sec -MapReduce Total cumulative CPU time: 50 seconds 540 msec -Ended Job = job_201309172235_0286 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 156.37 sec HDFS Read: 84944733 HDFS Write: 241346048 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 50.54 sec HDFS Read: 241349358 HDFS Write: 268 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 26 seconds 910 msec -OK -Time taken: 107.333 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5002@mturlrep13_201309180256_1377188535.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0287 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 02:56:40,498 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:56:47,527 Stage-1 map = 15%, reduce = 0% -2013-09-18 02:56:50,540 Stage-1 map = 44%, reduce = 0% -2013-09-18 02:56:53,554 Stage-1 map = 59%, reduce = 0% -2013-09-18 02:56:56,569 Stage-1 map = 78%, reduce = 0% -2013-09-18 02:56:59,590 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 26.21 sec -2013-09-18 02:57:00,596 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 53.06 sec -2013-09-18 02:57:01,604 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 109.54 sec -2013-09-18 02:57:02,609 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 109.54 sec -2013-09-18 02:57:03,614 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 109.54 sec -2013-09-18 02:57:04,619 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 109.54 sec -2013-09-18 02:57:05,625 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 109.54 sec -2013-09-18 02:57:06,631 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 109.54 sec -2013-09-18 02:57:07,637 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 109.54 sec -2013-09-18 02:57:08,643 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 109.54 sec -2013-09-18 02:57:09,649 Stage-1 map = 100%, reduce = 46%, Cumulative CPU 109.54 sec -2013-09-18 02:57:10,657 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 121.89 sec -2013-09-18 02:57:11,664 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 121.89 sec -2013-09-18 02:57:12,670 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 121.89 sec -2013-09-18 02:57:13,676 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 121.89 sec -2013-09-18 02:57:14,682 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 121.89 sec -2013-09-18 02:57:15,689 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 121.89 sec -2013-09-18 02:57:16,695 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 121.89 sec -2013-09-18 02:57:17,702 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 121.89 sec -2013-09-18 02:57:18,708 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 121.89 sec -2013-09-18 02:57:19,715 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 121.89 sec -2013-09-18 02:57:20,722 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 121.89 sec -2013-09-18 02:57:21,728 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 121.89 sec -2013-09-18 02:57:22,733 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 121.89 sec -2013-09-18 02:57:23,740 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 152.17 sec -2013-09-18 02:57:24,747 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 152.17 sec -MapReduce Total cumulative CPU time: 2 minutes 32 seconds 170 msec -Ended Job = job_201309172235_0287 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0288 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 02:57:28,223 Stage-2 map = 0%, reduce = 0% -2013-09-18 02:57:41,263 Stage-2 map = 28%, reduce = 0% -2013-09-18 02:57:47,345 Stage-2 map = 50%, reduce = 0% -2013-09-18 02:57:53,362 Stage-2 map = 78%, reduce = 0% -2013-09-18 02:58:04,395 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 02:58:05,401 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 02:58:06,406 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 02:58:07,411 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 02:58:08,416 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 02:58:09,420 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 02:58:10,425 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 02:58:11,431 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 02:58:12,436 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 02:58:13,442 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.57 sec -2013-09-18 02:58:14,446 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 43.57 sec -2013-09-18 02:58:15,451 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 43.57 sec -2013-09-18 02:58:16,455 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.29 sec -2013-09-18 02:58:17,460 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.29 sec -2013-09-18 02:58:18,465 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.29 sec -MapReduce Total cumulative CPU time: 50 seconds 290 msec -Ended Job = job_201309172235_0288 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 152.17 sec HDFS Read: 84944733 HDFS Write: 241346048 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 50.29 sec HDFS Read: 241349358 HDFS Write: 268 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 22 seconds 460 msec -OK -Time taken: 106.376 seconds, Fetched: 10 row(s) -hive> quit; --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - - -times: 1 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7338@mturlrep13_201309180258_545165403.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0289 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-18 02:58:36,403 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:58:42,436 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 16.49 sec -2013-09-18 02:58:43,443 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 16.49 sec -2013-09-18 02:58:44,451 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 16.49 sec -MapReduce Total cumulative CPU time: 16 seconds 490 msec -Ended Job = job_201309172235_0289 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 16.49 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 16 seconds 490 msec -OK -Time taken: 17.948 seconds -hive> quit; - -times: 2 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8366@mturlrep13_201309180258_1808547962.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0290 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-18 02:58:54,728 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:58:58,753 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 16.35 sec -2013-09-18 02:58:59,761 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 16.35 sec -2013-09-18 02:59:00,768 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 16.35 sec -MapReduce Total cumulative CPU time: 16 seconds 350 msec -Ended Job = job_201309172235_0290 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 16.35 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 16 seconds 350 msec -OK -Time taken: 14.47 seconds -hive> quit; - -times: 3 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9390@mturlrep13_201309180259_978485030.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0291 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-18 02:59:10,689 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:59:14,716 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 4.14 sec -2013-09-18 02:59:15,723 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 17.49 sec -2013-09-18 02:59:16,732 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 17.49 sec -MapReduce Total cumulative CPU time: 17 seconds 490 msec -Ended Job = job_201309172235_0291 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 17.49 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 17 seconds 490 msec -OK -Time taken: 14.154 seconds -hive> quit; --- мощная фильтрация по столбцу типа UInt64.; - - -times: 1 -query: SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10422@mturlrep13_201309180259_1489695405.txt -hive> SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0292 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:59:33,176 Stage-1 map = 0%, reduce = 0% -2013-09-18 02:59:40,206 Stage-1 map = 74%, reduce = 0% -2013-09-18 02:59:41,218 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.3 sec -2013-09-18 02:59:42,225 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.3 sec -2013-09-18 02:59:43,232 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.3 sec -2013-09-18 02:59:44,236 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.3 sec -2013-09-18 02:59:45,242 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.3 sec -2013-09-18 02:59:46,247 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.3 sec -2013-09-18 02:59:47,254 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.3 sec -2013-09-18 02:59:48,261 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 36.3 sec -2013-09-18 02:59:49,269 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 38.08 sec -2013-09-18 02:59:50,275 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 38.08 sec -MapReduce Total cumulative CPU time: 38 seconds 80 msec -Ended Job = job_201309172235_0292 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 38.08 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 38 seconds 80 msec -OK -8428 -Time taken: 27.452 seconds, Fetched: 1 row(s) -hive> quit; - -times: 2 -query: SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11577@mturlrep13_201309180259_208606502.txt -hive> SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0293 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 02:59:59,752 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:00:07,795 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:00:08,802 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:00:09,809 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:00:10,815 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:00:11,821 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:00:12,827 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:00:13,834 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:00:14,841 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 36.09 sec -2013-09-18 03:00:15,848 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.96 sec -2013-09-18 03:00:16,855 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.96 sec -MapReduce Total cumulative CPU time: 37 seconds 960 msec -Ended Job = job_201309172235_0293 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 37.96 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 37 seconds 960 msec -OK -8428 -Time taken: 24.527 seconds, Fetched: 1 row(s) -hive> quit; - -times: 3 -query: SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13398@mturlrep13_201309180300_29036781.txt -hive> SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0294 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:00:27,278 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:00:33,314 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 8.78 sec -2013-09-18 03:00:34,322 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.08 sec -2013-09-18 03:00:35,329 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.08 sec -2013-09-18 03:00:36,334 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.08 sec -2013-09-18 03:00:37,340 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.08 sec -2013-09-18 03:00:38,346 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.08 sec -2013-09-18 03:00:39,351 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.08 sec -2013-09-18 03:00:40,356 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.08 sec -2013-09-18 03:00:41,364 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.24 sec -2013-09-18 03:00:42,372 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.24 sec -2013-09-18 03:00:43,377 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.24 sec -MapReduce Total cumulative CPU time: 39 seconds 240 msec -Ended Job = job_201309172235_0294 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 39.24 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 39 seconds 240 msec -OK -8428 -Time taken: 24.674 seconds, Fetched: 1 row(s) -hive> quit; --- фильтрация по поиску подстроки в строке.; - - -times: 1 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14574@mturlrep13_201309180300_1950004320.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0295 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:00:59,984 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:01:08,021 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:01:09,034 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 28.57 sec -2013-09-18 03:01:10,042 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.55 sec -2013-09-18 03:01:11,051 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.55 sec -2013-09-18 03:01:12,057 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.55 sec -2013-09-18 03:01:13,064 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.55 sec -2013-09-18 03:01:14,071 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.55 sec -2013-09-18 03:01:15,078 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.55 sec -2013-09-18 03:01:16,084 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.55 sec -2013-09-18 03:01:17,103 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.91 sec -2013-09-18 03:01:18,109 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.91 sec -MapReduce Total cumulative CPU time: 42 seconds 910 msec -Ended Job = job_201309172235_0295 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0296 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:01:20,611 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:01:22,620 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:01:23,626 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:01:24,631 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:01:25,636 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:01:26,641 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:01:27,647 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:01:28,652 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:01:29,657 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.66 sec -2013-09-18 03:01:30,663 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.26 sec -2013-09-18 03:01:31,669 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.26 sec -MapReduce Total cumulative CPU time: 2 seconds 260 msec -Ended Job = job_201309172235_0296 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.91 sec HDFS Read: 136675723 HDFS Write: 5172 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.26 sec HDFS Read: 5941 HDFS Write: 984 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 170 msec -OK -Time taken: 41.792 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16383@mturlrep13_201309180301_278669941.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0297 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:01:41,964 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:01:49,008 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 9.23 sec -2013-09-18 03:01:50,016 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.34 sec -2013-09-18 03:01:51,024 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.34 sec -2013-09-18 03:01:52,031 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.34 sec -2013-09-18 03:01:53,036 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.34 sec -2013-09-18 03:01:54,042 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.34 sec -2013-09-18 03:01:55,049 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.34 sec -2013-09-18 03:01:56,056 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.34 sec -2013-09-18 03:01:57,065 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.44 sec -2013-09-18 03:01:58,071 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.44 sec -2013-09-18 03:01:59,078 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.44 sec -MapReduce Total cumulative CPU time: 43 seconds 440 msec -Ended Job = job_201309172235_0297 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0298 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:02:01,570 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:02:03,580 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:04,586 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:05,591 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:06,596 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:07,601 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:08,606 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:09,611 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:10,617 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.21 sec -2013-09-18 03:02:11,623 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-18 03:02:12,629 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-18 03:02:13,635 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -MapReduce Total cumulative CPU time: 2 seconds 210 msec -Ended Job = job_201309172235_0298 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 43.44 sec HDFS Read: 136675723 HDFS Write: 5172 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.21 sec HDFS Read: 5941 HDFS Write: 984 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 650 msec -OK -Time taken: 40.17 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18173@mturlrep13_201309180302_1009071937.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0299 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:02:23,943 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:02:30,983 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 9.23 sec -2013-09-18 03:02:31,990 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.61 sec -2013-09-18 03:02:32,998 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.61 sec -2013-09-18 03:02:34,004 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.61 sec -2013-09-18 03:02:35,010 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.61 sec -2013-09-18 03:02:36,015 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.61 sec -2013-09-18 03:02:37,021 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.61 sec -2013-09-18 03:02:38,027 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.61 sec -2013-09-18 03:02:39,036 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.69 sec -2013-09-18 03:02:40,042 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.69 sec -2013-09-18 03:02:41,048 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.69 sec -MapReduce Total cumulative CPU time: 42 seconds 690 msec -Ended Job = job_201309172235_0299 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0300 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:02:43,536 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:02:45,545 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:46,551 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:47,556 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:48,560 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:49,565 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:50,570 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:51,575 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:02:52,580 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.64 sec -2013-09-18 03:02:53,585 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-18 03:02:54,591 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-18 03:02:55,597 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -MapReduce Total cumulative CPU time: 2 seconds 210 msec -Ended Job = job_201309172235_0300 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.69 sec HDFS Read: 136675723 HDFS Write: 5172 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.21 sec HDFS Read: 5941 HDFS Write: 984 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 900 msec -OK -Time taken: 40.052 seconds, Fetched: 10 row(s) -hive> quit; --- вынимаем большие столбцы, фильтрация по строке.; - - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19998@mturlrep13_201309180303_483326023.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0301 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:03:12,709 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:03:19,743 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:03:22,757 Stage-1 map = 81%, reduce = 0% -2013-09-18 03:03:23,769 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.01 sec -2013-09-18 03:03:24,777 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.01 sec -2013-09-18 03:03:25,783 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.01 sec -2013-09-18 03:03:26,790 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.01 sec -2013-09-18 03:03:27,797 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.01 sec -2013-09-18 03:03:28,802 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.01 sec -2013-09-18 03:03:29,808 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.01 sec -2013-09-18 03:03:30,814 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 51.01 sec -2013-09-18 03:03:31,821 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 55.69 sec -2013-09-18 03:03:32,828 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 55.69 sec -MapReduce Total cumulative CPU time: 55 seconds 690 msec -Ended Job = job_201309172235_0301 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0302 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:03:36,496 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:03:38,506 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:03:39,511 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:03:40,516 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:03:41,521 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:03:42,526 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:03:43,531 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:03:44,536 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:03:45,542 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-18 03:03:46,549 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -2013-09-18 03:03:47,555 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.21 sec -MapReduce Total cumulative CPU time: 2 seconds 210 msec -Ended Job = job_201309172235_0302 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 55.69 sec HDFS Read: 298803179 HDFS Write: 12221 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.21 sec HDFS Read: 12990 HDFS Write: 2646 SUCCESS -Total MapReduce CPU Time Spent: 57 seconds 900 msec -OK -Time taken: 45.079 seconds, Fetched: 10 row(s) -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21928@mturlrep13_201309180303_564237676.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0303 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:03:56,744 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:04:04,788 Stage-1 map = 48%, reduce = 0% -2013-09-18 03:04:06,804 Stage-1 map = 57%, reduce = 0%, Cumulative CPU 11.57 sec -2013-09-18 03:04:07,820 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.6 sec -2013-09-18 03:04:08,827 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.6 sec -2013-09-18 03:04:09,833 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.6 sec -2013-09-18 03:04:10,839 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.6 sec -2013-09-18 03:04:11,845 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.6 sec -2013-09-18 03:04:12,850 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.6 sec -2013-09-18 03:04:13,856 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.6 sec -2013-09-18 03:04:14,864 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.66 sec -2013-09-18 03:04:15,870 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.66 sec -2013-09-18 03:04:16,877 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.66 sec -MapReduce Total cumulative CPU time: 53 seconds 660 msec -Ended Job = job_201309172235_0303 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0304 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:04:20,500 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:04:21,506 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:04:22,512 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:04:23,518 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:04:24,523 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:04:25,527 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:04:26,532 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:04:27,537 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:04:28,542 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:04:29,548 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.66 sec -2013-09-18 03:04:30,554 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -2013-09-18 03:04:31,560 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -MapReduce Total cumulative CPU time: 2 seconds 250 msec -Ended Job = job_201309172235_0304 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 53.66 sec HDFS Read: 298803179 HDFS Write: 12221 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.25 sec HDFS Read: 12990 HDFS Write: 2646 SUCCESS -Total MapReduce CPU Time Spent: 55 seconds 910 msec -OK -Time taken: 42.12 seconds, Fetched: 10 row(s) -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23843@mturlrep13_201309180304_1844665833.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0305 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:04:42,274 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:04:49,306 Stage-1 map = 48%, reduce = 0% -2013-09-18 03:04:52,325 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 37.24 sec -2013-09-18 03:04:53,332 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.15 sec -2013-09-18 03:04:54,339 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.15 sec -2013-09-18 03:04:55,345 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.15 sec -2013-09-18 03:04:56,351 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.15 sec -2013-09-18 03:04:57,357 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.15 sec -2013-09-18 03:04:58,363 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 50.15 sec -2013-09-18 03:04:59,369 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 50.15 sec -2013-09-18 03:05:00,379 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 54.25 sec -2013-09-18 03:05:01,386 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 54.25 sec -MapReduce Total cumulative CPU time: 54 seconds 250 msec -Ended Job = job_201309172235_0305 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0306 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:05:05,825 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:05:07,833 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:05:08,839 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:05:09,844 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:05:10,848 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:05:11,853 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:05:12,858 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:05:13,863 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:05:14,868 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.66 sec -2013-09-18 03:05:15,874 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.32 sec -2013-09-18 03:05:16,880 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.32 sec -MapReduce Total cumulative CPU time: 2 seconds 320 msec -Ended Job = job_201309172235_0306 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 54.25 sec HDFS Read: 298803179 HDFS Write: 12221 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.32 sec HDFS Read: 12990 HDFS Write: 2646 SUCCESS -Total MapReduce CPU Time Spent: 56 seconds 570 msec -OK -Time taken: 43.438 seconds, Fetched: 10 row(s) -hive> quit; --- чуть больше столбцы.; - - -times: 1 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26447@mturlrep13_201309180305_468727938.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0307 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:05:34,147 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:05:44,189 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:05:47,201 Stage-1 map = 29%, reduce = 0% -2013-09-18 03:05:51,218 Stage-1 map = 33%, reduce = 0% -2013-09-18 03:05:53,227 Stage-1 map = 37%, reduce = 0% -2013-09-18 03:05:54,247 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:05:56,278 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:05:59,290 Stage-1 map = 67%, reduce = 0% -2013-09-18 03:06:03,314 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 39.6 sec -2013-09-18 03:06:04,322 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 39.6 sec -2013-09-18 03:06:05,327 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 77.13 sec -2013-09-18 03:06:06,333 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 77.13 sec -2013-09-18 03:06:07,338 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 166.73 sec -2013-09-18 03:06:08,344 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 166.73 sec -2013-09-18 03:06:09,350 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 166.73 sec -2013-09-18 03:06:10,355 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 166.73 sec -2013-09-18 03:06:11,361 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 167.32 sec -2013-09-18 03:06:12,368 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.65 sec -2013-09-18 03:06:13,374 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.65 sec -2013-09-18 03:06:14,380 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.65 sec -MapReduce Total cumulative CPU time: 2 minutes 49 seconds 650 msec -Ended Job = job_201309172235_0307 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 169.65 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 49 seconds 650 msec -OK -Time taken: 50.847 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27643@mturlrep13_201309180306_526482942.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0308 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:06:24,046 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:06:35,093 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:06:38,107 Stage-1 map = 29%, reduce = 0% -2013-09-18 03:06:41,122 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:06:44,158 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:06:47,171 Stage-1 map = 63%, reduce = 0% -2013-09-18 03:06:50,185 Stage-1 map = 78%, reduce = 0% -2013-09-18 03:06:52,213 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 33.78 sec -2013-09-18 03:06:53,220 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 33.78 sec -2013-09-18 03:06:54,225 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 72.08 sec -2013-09-18 03:06:55,230 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 148.07 sec -2013-09-18 03:06:56,235 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 148.07 sec -2013-09-18 03:06:57,241 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 148.07 sec -2013-09-18 03:06:58,246 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 148.07 sec -2013-09-18 03:06:59,252 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 148.07 sec -2013-09-18 03:07:00,260 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 150.7 sec -2013-09-18 03:07:01,266 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 150.7 sec -2013-09-18 03:07:02,273 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 150.7 sec -MapReduce Total cumulative CPU time: 2 minutes 30 seconds 700 msec -Ended Job = job_201309172235_0308 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 150.7 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 30 seconds 700 msec -OK -Time taken: 46.004 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28785@mturlrep13_201309180307_1226276178.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0309 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:07:12,867 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:07:22,931 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:07:25,943 Stage-1 map = 29%, reduce = 0% -2013-09-18 03:07:28,956 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:07:31,968 Stage-1 map = 48%, reduce = 0% -2013-09-18 03:07:34,979 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:07:37,991 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:07:40,007 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 37.4 sec -2013-09-18 03:07:41,014 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 37.4 sec -2013-09-18 03:07:42,019 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 37.4 sec -2013-09-18 03:07:43,024 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 37.4 sec -2013-09-18 03:07:44,030 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 160.55 sec -2013-09-18 03:07:45,035 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 160.55 sec -2013-09-18 03:07:46,040 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 160.55 sec -2013-09-18 03:07:47,045 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 160.55 sec -2013-09-18 03:07:48,052 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 163.46 sec -2013-09-18 03:07:49,059 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 163.46 sec -2013-09-18 03:07:50,065 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 163.46 sec -MapReduce Total cumulative CPU time: 2 minutes 43 seconds 460 msec -Ended Job = job_201309172235_0309 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 163.46 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 43 seconds 460 msec -OK -Time taken: 45.95 seconds, Fetched: 10 row(s) -hive> quit; --- плохой запрос - вынимаем все столбцы.; - - -times: 1 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29936@mturlrep13_201309180307_1379307985.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0310 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:08:06,541 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:08:14,581 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 19.92 sec -2013-09-18 03:08:15,589 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.4 sec -2013-09-18 03:08:16,597 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.4 sec -2013-09-18 03:08:17,602 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.4 sec -2013-09-18 03:08:18,607 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.4 sec -2013-09-18 03:08:19,613 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.4 sec -2013-09-18 03:08:20,619 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.4 sec -2013-09-18 03:08:21,626 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.4 sec -2013-09-18 03:08:22,632 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.4 sec -2013-09-18 03:08:23,637 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.4 sec -2013-09-18 03:08:24,643 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.4 sec -2013-09-18 03:08:25,651 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.56 sec -2013-09-18 03:08:26,656 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.56 sec -2013-09-18 03:08:27,662 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.56 sec -MapReduce Total cumulative CPU time: 45 seconds 560 msec -Ended Job = job_201309172235_0310 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 45.56 sec HDFS Read: 28228143 HDFS Write: 766 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 560 msec -OK -Time taken: 31.007 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31173@mturlrep13_201309180308_1952201032.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0311 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:08:37,914 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:08:44,951 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.48 sec -2013-09-18 03:08:45,959 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.48 sec -2013-09-18 03:08:46,969 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.48 sec -2013-09-18 03:08:47,974 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.48 sec -2013-09-18 03:08:48,980 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.48 sec -2013-09-18 03:08:49,986 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.48 sec -2013-09-18 03:08:50,992 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.48 sec -2013-09-18 03:08:51,998 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 39.48 sec -2013-09-18 03:08:53,004 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 39.48 sec -2013-09-18 03:08:54,009 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 39.48 sec -2013-09-18 03:08:55,017 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.59 sec -2013-09-18 03:08:56,023 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.59 sec -MapReduce Total cumulative CPU time: 44 seconds 590 msec -Ended Job = job_201309172235_0311 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 44.59 sec HDFS Read: 28228143 HDFS Write: 766 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 590 msec -OK -Time taken: 26.473 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32425@mturlrep13_201309180308_2035672137.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0312 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:09:05,993 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:09:13,029 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 29.62 sec -2013-09-18 03:09:14,037 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.74 sec -2013-09-18 03:09:15,043 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.74 sec -2013-09-18 03:09:16,049 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.74 sec -2013-09-18 03:09:17,054 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.74 sec -2013-09-18 03:09:18,060 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.74 sec -2013-09-18 03:09:19,066 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.74 sec -2013-09-18 03:09:20,072 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.74 sec -2013-09-18 03:09:21,079 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 39.74 sec -2013-09-18 03:09:22,085 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 39.74 sec -2013-09-18 03:09:23,092 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.03 sec -2013-09-18 03:09:24,098 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.03 sec -MapReduce Total cumulative CPU time: 45 seconds 30 msec -Ended Job = job_201309172235_0312 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 45.03 sec HDFS Read: 28228143 HDFS Write: 766 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 30 msec -OK -Time taken: 26.277 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка.; - - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1250@mturlrep13_201309180309_358088813.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0313 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:09:40,971 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:09:49,018 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 10.24 sec -2013-09-18 03:09:50,025 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.07 sec -2013-09-18 03:09:51,032 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.07 sec -2013-09-18 03:09:52,037 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.07 sec -2013-09-18 03:09:53,042 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.07 sec -2013-09-18 03:09:54,048 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.07 sec -2013-09-18 03:09:55,053 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.07 sec -2013-09-18 03:09:56,059 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.07 sec -2013-09-18 03:09:57,065 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.07 sec -2013-09-18 03:09:58,070 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.07 sec -2013-09-18 03:09:59,075 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.07 sec -2013-09-18 03:10:00,080 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 42.07 sec -2013-09-18 03:10:01,087 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 48.23 sec -2013-09-18 03:10:02,093 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 48.23 sec -MapReduce Total cumulative CPU time: 48 seconds 230 msec -Ended Job = job_201309172235_0313 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 48.23 sec HDFS Read: 27820105 HDFS Write: 666 SUCCESS -Total MapReduce CPU Time Spent: 48 seconds 230 msec -OK -ялта интурист -! как одеть трехнедельного ребенка при температуре 20 градусов -! отель rattana beach hotel 3* -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -!( центробежный скважинный калибр форумы) -!(!(storm master silmarils)) -!(!(storm master silmarils)) -!(!(title:(схема sputnik hi 4000))) -!(44-фз о контрактной системе) -Time taken: 30.985 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2534@mturlrep13_201309180310_1314367270.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0314 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:10:12,384 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:10:19,428 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 20.28 sec -2013-09-18 03:10:20,435 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.07 sec -2013-09-18 03:10:21,442 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.07 sec -2013-09-18 03:10:22,447 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.07 sec -2013-09-18 03:10:23,452 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.07 sec -2013-09-18 03:10:24,457 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.07 sec -2013-09-18 03:10:25,463 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.07 sec -2013-09-18 03:10:26,469 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 41.07 sec -2013-09-18 03:10:27,475 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 41.07 sec -2013-09-18 03:10:28,481 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 41.07 sec -2013-09-18 03:10:29,486 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 41.07 sec -2013-09-18 03:10:30,494 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 47.39 sec -2013-09-18 03:10:31,500 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 47.39 sec -MapReduce Total cumulative CPU time: 47 seconds 390 msec -Ended Job = job_201309172235_0314 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 47.39 sec HDFS Read: 27820105 HDFS Write: 666 SUCCESS -Total MapReduce CPU Time Spent: 47 seconds 390 msec -OK -ялта интурист -! как одеть трехнедельного ребенка при температуре 20 градусов -! отель rattana beach hotel 3* -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -!( центробежный скважинный калибр форумы) -!(!(storm master silmarils)) -!(!(storm master silmarils)) -!(!(title:(схема sputnik hi 4000))) -!(44-фз о контрактной системе) -Time taken: 27.724 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4327@mturlrep13_201309180310_2107081385.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0315 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:10:40,801 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:10:48,841 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.7 sec -2013-09-18 03:10:49,849 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.7 sec -2013-09-18 03:10:50,856 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.7 sec -2013-09-18 03:10:51,862 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.7 sec -2013-09-18 03:10:52,867 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.7 sec -2013-09-18 03:10:53,872 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.7 sec -2013-09-18 03:10:54,877 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.7 sec -2013-09-18 03:10:55,884 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.7 sec -2013-09-18 03:10:56,889 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 40.7 sec -2013-09-18 03:10:57,894 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 40.7 sec -2013-09-18 03:10:58,899 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 40.7 sec -2013-09-18 03:10:59,906 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 47.23 sec -2013-09-18 03:11:00,915 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 47.23 sec -MapReduce Total cumulative CPU time: 47 seconds 230 msec -Ended Job = job_201309172235_0315 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 47.23 sec HDFS Read: 27820105 HDFS Write: 666 SUCCESS -Total MapReduce CPU Time Spent: 47 seconds 230 msec -OK -ялта интурист -! как одеть трехнедельного ребенка при температуре 20 градусов -! отель rattana beach hotel 3* -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -!( центробежный скважинный калибр форумы) -!(!(storm master silmarils)) -!(!(storm master silmarils)) -!(!(title:(схема sputnik hi 4000))) -!(44-фз о контрактной системе) -Time taken: 27.357 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка по строкам.; - - -times: 1 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5527@mturlrep13_201309180311_656071928.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0316 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:11:19,126 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:11:26,155 Stage-1 map = 85%, reduce = 0% -2013-09-18 03:11:27,169 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 10.31 sec -2013-09-18 03:11:28,177 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.22 sec -2013-09-18 03:11:29,184 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.22 sec -2013-09-18 03:11:30,190 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.22 sec -2013-09-18 03:11:31,196 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.22 sec -2013-09-18 03:11:32,201 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.22 sec -2013-09-18 03:11:33,206 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.22 sec -2013-09-18 03:11:34,213 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.22 sec -2013-09-18 03:11:35,219 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 41.22 sec -2013-09-18 03:11:36,225 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 41.22 sec -2013-09-18 03:11:37,255 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 41.22 sec -2013-09-18 03:11:38,260 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 41.22 sec -2013-09-18 03:11:39,268 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 48.23 sec -2013-09-18 03:11:40,274 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 48.23 sec -2013-09-18 03:11:41,280 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 48.23 sec -MapReduce Total cumulative CPU time: 48 seconds 230 msec -Ended Job = job_201309172235_0316 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 48.23 sec HDFS Read: 28228143 HDFS Write: 762 SUCCESS -Total MapReduce CPU Time Spent: 48 seconds 230 msec -OK -Time taken: 32.747 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7168@mturlrep13_201309180311_1915819780.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0317 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:11:50,402 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:11:58,444 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 10.7 sec -2013-09-18 03:11:59,451 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.62 sec -2013-09-18 03:12:00,458 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.62 sec -2013-09-18 03:12:01,463 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.62 sec -2013-09-18 03:12:02,468 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.62 sec -2013-09-18 03:12:03,473 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.62 sec -2013-09-18 03:12:04,479 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 41.62 sec -2013-09-18 03:12:05,485 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 41.62 sec -2013-09-18 03:12:06,490 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 41.62 sec -2013-09-18 03:12:07,496 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 41.62 sec -2013-09-18 03:12:08,501 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 41.62 sec -2013-09-18 03:12:09,508 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 48.08 sec -2013-09-18 03:12:10,515 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 48.08 sec -2013-09-18 03:12:11,520 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 48.08 sec -MapReduce Total cumulative CPU time: 48 seconds 80 msec -Ended Job = job_201309172235_0317 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 48.08 sec HDFS Read: 28228143 HDFS Write: 762 SUCCESS -Total MapReduce CPU Time Spent: 48 seconds 80 msec -OK -Time taken: 28.329 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8422@mturlrep13_201309180312_364801550.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0318 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:12:21,723 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:12:28,768 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 9.8 sec -2013-09-18 03:12:29,775 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.77 sec -2013-09-18 03:12:30,780 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.77 sec -2013-09-18 03:12:31,785 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.77 sec -2013-09-18 03:12:32,790 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.77 sec -2013-09-18 03:12:33,796 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.77 sec -2013-09-18 03:12:34,801 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 42.77 sec -2013-09-18 03:12:35,807 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 42.77 sec -2013-09-18 03:12:36,814 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 42.77 sec -2013-09-18 03:12:37,820 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 42.77 sec -2013-09-18 03:12:38,828 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 49.41 sec -2013-09-18 03:12:39,833 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 49.41 sec -2013-09-18 03:12:40,839 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 49.41 sec -MapReduce Total cumulative CPU time: 49 seconds 410 msec -Ended Job = job_201309172235_0318 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 49.41 sec HDFS Read: 28228143 HDFS Write: 762 SUCCESS -Total MapReduce CPU Time Spent: 49 seconds 410 msec -OK -Time taken: 27.439 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка по кортежу.; - - -times: 1 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9670@mturlrep13_201309180312_462358371.txt -hive> SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0319 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:12:57,315 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:13:04,346 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:13:07,360 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:13:10,373 Stage-1 map = 63%, reduce = 0% -2013-09-18 03:13:13,393 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 68.42 sec -2013-09-18 03:13:14,400 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 68.42 sec -2013-09-18 03:13:15,412 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 73.5 sec -2013-09-18 03:13:16,420 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 79.72 sec -2013-09-18 03:13:17,427 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 93.63 sec -2013-09-18 03:13:18,434 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 93.63 sec -2013-09-18 03:13:19,439 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 93.63 sec -2013-09-18 03:13:20,444 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 93.63 sec -2013-09-18 03:13:21,449 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 93.63 sec -2013-09-18 03:13:22,456 Stage-1 map = 100%, reduce = 4%, Cumulative CPU 93.63 sec -2013-09-18 03:13:23,462 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 93.63 sec -2013-09-18 03:13:24,468 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 93.63 sec -2013-09-18 03:13:25,474 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 93.63 sec -2013-09-18 03:13:26,481 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 93.63 sec -2013-09-18 03:13:27,487 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 93.63 sec -2013-09-18 03:13:28,493 Stage-1 map = 100%, reduce = 71%, Cumulative CPU 93.63 sec -2013-09-18 03:13:29,498 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 93.63 sec -2013-09-18 03:13:30,505 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 103.43 sec -2013-09-18 03:13:31,511 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 103.43 sec -2013-09-18 03:13:32,516 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 103.43 sec -2013-09-18 03:13:33,521 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 103.43 sec -2013-09-18 03:13:34,527 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 117.54 sec -2013-09-18 03:13:35,533 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 117.54 sec -MapReduce Total cumulative CPU time: 1 minutes 57 seconds 540 msec -Ended Job = job_201309172235_0319 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0320 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:13:39,017 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:13:40,022 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:13:41,027 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:13:42,033 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:13:43,039 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:13:44,044 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:13:45,049 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:13:46,053 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:13:47,058 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:13:48,064 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -2013-09-18 03:13:49,069 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -2013-09-18 03:13:50,074 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -MapReduce Total cumulative CPU time: 2 seconds 230 msec -Ended Job = job_201309172235_0320 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 117.54 sec HDFS Read: 117363067 HDFS Write: 794 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.23 sec HDFS Read: 1563 HDFS Write: 571 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 59 seconds 770 msec -OK -Time taken: 62.932 seconds, Fetched: 19 row(s) -hive> quit; - -times: 2 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11483@mturlrep13_201309180313_1683269213.txt -hive> SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0321 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:13:59,358 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:14:07,402 Stage-1 map = 29%, reduce = 0% -2013-09-18 03:14:10,416 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:14:13,434 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 56.74 sec -2013-09-18 03:14:14,442 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 56.74 sec -2013-09-18 03:14:15,450 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 56.74 sec -2013-09-18 03:14:16,460 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 63.68 sec -2013-09-18 03:14:17,467 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 72.77 sec -2013-09-18 03:14:18,473 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 92.25 sec -2013-09-18 03:14:19,479 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 92.25 sec -2013-09-18 03:14:20,484 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 92.25 sec -2013-09-18 03:14:21,490 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 92.25 sec -2013-09-18 03:14:22,496 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 92.25 sec -2013-09-18 03:14:23,503 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 92.25 sec -2013-09-18 03:14:24,509 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 92.25 sec -2013-09-18 03:14:25,515 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 92.25 sec -2013-09-18 03:14:26,521 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 92.25 sec -2013-09-18 03:14:27,528 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 92.25 sec -2013-09-18 03:14:28,534 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 92.25 sec -2013-09-18 03:14:29,540 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 92.25 sec -2013-09-18 03:14:30,548 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 102.34 sec -2013-09-18 03:14:31,554 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 102.34 sec -2013-09-18 03:14:32,561 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 102.34 sec -2013-09-18 03:14:33,567 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 102.34 sec -2013-09-18 03:14:34,573 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 116.97 sec -2013-09-18 03:14:35,579 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 116.97 sec -MapReduce Total cumulative CPU time: 1 minutes 56 seconds 970 msec -Ended Job = job_201309172235_0321 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0322 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:14:38,052 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:14:40,061 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:14:41,066 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:14:42,071 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:14:43,075 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:14:44,081 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:14:45,086 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:14:46,090 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:14:47,096 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.66 sec -2013-09-18 03:14:48,102 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.07 sec -2013-09-18 03:14:49,107 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.07 sec -MapReduce Total cumulative CPU time: 2 seconds 70 msec -Ended Job = job_201309172235_0322 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 116.97 sec HDFS Read: 117363067 HDFS Write: 794 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.07 sec HDFS Read: 1563 HDFS Write: 571 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 59 seconds 40 msec -OK -Time taken: 57.159 seconds, Fetched: 19 row(s) -hive> quit; - -times: 3 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13299@mturlrep13_201309180314_854084991.txt -hive> SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0323 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:14:59,355 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:15:06,385 Stage-1 map = 25%, reduce = 0% -2013-09-18 03:15:09,399 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:15:12,413 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:15:15,516 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 74.75 sec -2013-09-18 03:15:16,523 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 80.15 sec -2013-09-18 03:15:17,530 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 91.8 sec -2013-09-18 03:15:18,536 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 91.8 sec -2013-09-18 03:15:19,542 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 91.8 sec -2013-09-18 03:15:20,547 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 91.8 sec -2013-09-18 03:15:21,552 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 91.8 sec -2013-09-18 03:15:22,558 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 91.8 sec -2013-09-18 03:15:23,563 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 91.8 sec -2013-09-18 03:15:24,568 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 91.8 sec -2013-09-18 03:15:25,574 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 91.8 sec -2013-09-18 03:15:26,579 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 91.8 sec -2013-09-18 03:15:27,585 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 91.8 sec -2013-09-18 03:15:28,590 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 91.8 sec -2013-09-18 03:15:29,595 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 91.8 sec -2013-09-18 03:15:30,603 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 101.93 sec -2013-09-18 03:15:31,608 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 101.93 sec -2013-09-18 03:15:32,613 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 101.93 sec -2013-09-18 03:15:33,626 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 116.67 sec -2013-09-18 03:15:34,633 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 116.67 sec -2013-09-18 03:15:35,639 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 116.67 sec -MapReduce Total cumulative CPU time: 1 minutes 56 seconds 670 msec -Ended Job = job_201309172235_0323 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0324 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:15:39,125 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:15:40,130 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:15:41,136 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:15:42,141 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:15:43,145 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:15:44,150 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:15:45,155 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:15:46,159 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:15:47,165 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:15:48,170 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -2013-09-18 03:15:49,176 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -2013-09-18 03:15:50,182 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -MapReduce Total cumulative CPU time: 2 seconds 190 msec -Ended Job = job_201309172235_0324 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 116.67 sec HDFS Read: 117363067 HDFS Write: 794 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.19 sec HDFS Read: 1563 HDFS Write: 571 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 58 seconds 860 msec -OK -Time taken: 59.233 seconds, Fetched: 19 row(s) -hive> quit; --- считаем средние длины URL для крупных счётчиков.; - - -times: 1 -query: SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15829@mturlrep13_201309180315_1054632059.txt -hive> SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -FAILED: SemanticException [Error 10011]: Line 1:336 Invalid function 'GREATEST' -hive> quit; - -times: 2 -query: SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16050@mturlrep13_201309180316_1145304248.txt -hive> SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -FAILED: SemanticException [Error 10011]: Line 1:336 Invalid function 'GREATEST' -hive> quit; - -times: 3 -query: SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16266@mturlrep13_201309180316_80926883.txt -hive> SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -FAILED: SemanticException [Error 10011]: Line 1:336 Invalid function 'GREATEST' -hive> quit; --- то же самое, но с разбивкой по доменам.; - - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16487@mturlrep13_201309180316_664462210.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0325 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:16:31,560 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:16:45,621 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:16:51,650 Stage-1 map = 29%, reduce = 0% -2013-09-18 03:16:57,674 Stage-1 map = 33%, reduce = 0% -2013-09-18 03:17:00,686 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:17:06,709 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:17:12,731 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:17:18,761 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 173.32 sec -2013-09-18 03:17:19,768 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 181.18 sec -2013-09-18 03:17:20,774 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 181.18 sec -2013-09-18 03:17:21,778 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 181.18 sec -2013-09-18 03:17:22,783 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 181.18 sec -2013-09-18 03:17:23,787 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 181.18 sec -2013-09-18 03:17:24,792 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 181.18 sec -2013-09-18 03:17:25,797 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 193.89 sec -2013-09-18 03:17:26,802 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 222.98 sec -2013-09-18 03:17:27,806 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 222.98 sec -2013-09-18 03:17:28,811 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 222.98 sec -2013-09-18 03:17:29,815 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 222.98 sec -2013-09-18 03:17:30,819 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 222.98 sec -2013-09-18 03:17:31,824 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 222.98 sec -2013-09-18 03:17:32,828 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 222.98 sec -2013-09-18 03:17:33,835 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 226.19 sec -2013-09-18 03:17:34,846 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 226.19 sec -MapReduce Total cumulative CPU time: 3 minutes 46 seconds 190 msec -Ended Job = job_201309172235_0325 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 226.19 sec HDFS Read: 7797536 HDFS Write: 1080 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 46 seconds 190 msec -OK -Time taken: 74.663 seconds, Fetched: 1 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17636@mturlrep13_201309180317_1915588567.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0326 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:17:45,771 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:17:56,817 Stage-1 map = 7%, reduce = 0% -2013-09-18 03:17:59,831 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:18:05,856 Stage-1 map = 29%, reduce = 0% -2013-09-18 03:18:11,880 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:18:17,911 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:18,916 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:19,924 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:20,929 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:21,934 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:22,939 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:23,945 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:24,951 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:25,956 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:26,961 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:27,966 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:28,971 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:29,976 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:30,982 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:31,986 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 115.96 sec -2013-09-18 03:18:32,993 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 138.53 sec -2013-09-18 03:18:33,999 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 138.53 sec -2013-09-18 03:18:35,004 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 138.53 sec -2013-09-18 03:18:36,010 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 138.53 sec -2013-09-18 03:18:37,015 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 138.53 sec -2013-09-18 03:18:38,020 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 192.67 sec -2013-09-18 03:18:39,024 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 220.35 sec -2013-09-18 03:18:40,029 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 220.35 sec -2013-09-18 03:18:41,034 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 220.35 sec -2013-09-18 03:18:42,040 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 223.54 sec -2013-09-18 03:18:43,046 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 223.54 sec -2013-09-18 03:18:44,052 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 223.54 sec -MapReduce Total cumulative CPU time: 3 minutes 43 seconds 540 msec -Ended Job = job_201309172235_0326 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 223.54 sec HDFS Read: 7797536 HDFS Write: 1080 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 43 seconds 540 msec -OK -Time taken: 67.206 seconds, Fetched: 1 row(s) -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18790@mturlrep13_201309180318_731576789.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0327 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-18 03:18:55,297 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:19:05,340 Stage-1 map = 4%, reduce = 0% -2013-09-18 03:19:08,352 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:19:14,381 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:15,388 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:16,395 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:17,401 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:18,407 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:19,412 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:20,417 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:21,422 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:22,427 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:23,432 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:24,437 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:25,443 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:26,448 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:27,452 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:28,458 Stage-1 map = 44%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:29,463 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:30,468 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:31,473 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:32,478 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:33,484 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:34,489 Stage-1 map = 59%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:35,493 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:36,498 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:37,502 Stage-1 map = 70%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:38,507 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:39,511 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:40,516 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:41,520 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:42,525 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:43,530 Stage-1 map = 78%, reduce = 0%, Cumulative CPU 77.56 sec -2013-09-18 03:19:44,536 Stage-1 map = 82%, reduce = 0%, Cumulative CPU 111.44 sec -2013-09-18 03:19:45,541 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 111.44 sec -2013-09-18 03:19:46,546 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 111.44 sec -2013-09-18 03:19:47,551 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 111.44 sec -2013-09-18 03:19:48,556 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 149.94 sec -2013-09-18 03:19:49,561 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 229.05 sec -2013-09-18 03:19:50,565 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 229.05 sec -2013-09-18 03:19:51,569 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 229.05 sec -2013-09-18 03:19:52,574 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 229.05 sec -2013-09-18 03:19:53,581 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 232.1 sec -2013-09-18 03:19:54,586 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 232.1 sec -MapReduce Total cumulative CPU time: 3 minutes 52 seconds 100 msec -Ended Job = job_201309172235_0327 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 232.1 sec HDFS Read: 7797536 HDFS Write: 1080 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 52 seconds 100 msec -OK -Time taken: 68.586 seconds, Fetched: 1 row(s) -hive> quit; --- много тупых агрегатных функций.; - - -times: 1 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19971@mturlrep13_201309180319_1200397175.txt -hive> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0328 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:20:11,947 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:20:18,978 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:20:22,002 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:20:23,010 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:20:24,018 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:20:25,023 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:20:26,029 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:20:27,036 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:20:28,043 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:20:29,050 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 49.3 sec -2013-09-18 03:20:30,056 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 49.3 sec -2013-09-18 03:20:31,061 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 49.3 sec -2013-09-18 03:20:32,067 Stage-1 map = 100%, reduce = 57%, Cumulative CPU 49.3 sec -2013-09-18 03:20:33,072 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 49.3 sec -2013-09-18 03:20:34,080 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.8 sec -2013-09-18 03:20:35,087 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.8 sec -2013-09-18 03:20:36,093 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.8 sec -MapReduce Total cumulative CPU time: 1 minutes 3 seconds 800 msec -Ended Job = job_201309172235_0328 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0329 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:20:39,567 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:20:46,589 Stage-2 map = 52%, reduce = 0% -2013-09-18 03:20:48,598 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.27 sec -2013-09-18 03:20:49,603 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.27 sec -2013-09-18 03:20:50,607 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.27 sec -2013-09-18 03:20:51,612 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.27 sec -2013-09-18 03:20:52,616 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.27 sec -2013-09-18 03:20:53,621 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.27 sec -2013-09-18 03:20:54,626 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.27 sec -2013-09-18 03:20:55,632 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 12.27 sec -2013-09-18 03:20:56,637 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 12.27 sec -2013-09-18 03:20:57,642 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 15.17 sec -2013-09-18 03:20:58,647 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 15.17 sec -MapReduce Total cumulative CPU time: 15 seconds 170 msec -Ended Job = job_201309172235_0329 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 63.8 sec HDFS Read: 69312553 HDFS Write: 31841963 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 15.17 sec HDFS Read: 31842732 HDFS Write: 372 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 18 seconds 970 msec -OK -Time taken: 58.017 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22672@mturlrep13_201309180321_532082554.txt -hive> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0330 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:21:09,059 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:21:16,090 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:21:17,103 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 11.29 sec -2013-09-18 03:21:18,111 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.51 sec -2013-09-18 03:21:19,120 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.51 sec -2013-09-18 03:21:20,126 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.51 sec -2013-09-18 03:21:21,132 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.51 sec -2013-09-18 03:21:22,138 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.51 sec -2013-09-18 03:21:23,145 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.51 sec -2013-09-18 03:21:24,152 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.51 sec -2013-09-18 03:21:25,159 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.51 sec -2013-09-18 03:21:26,165 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.51 sec -2013-09-18 03:21:27,172 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.51 sec -2013-09-18 03:21:28,178 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 48.51 sec -2013-09-18 03:21:29,184 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 48.51 sec -2013-09-18 03:21:30,191 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.9 sec -2013-09-18 03:21:31,198 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.9 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 900 msec -Ended Job = job_201309172235_0330 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0331 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:21:34,779 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:21:41,804 Stage-2 map = 52%, reduce = 0% -2013-09-18 03:21:42,810 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.4 sec -2013-09-18 03:21:43,816 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.4 sec -2013-09-18 03:21:44,822 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.4 sec -2013-09-18 03:21:45,827 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.4 sec -2013-09-18 03:21:46,832 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.4 sec -2013-09-18 03:21:47,838 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.4 sec -2013-09-18 03:21:48,843 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.4 sec -2013-09-18 03:21:49,849 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 11.4 sec -2013-09-18 03:21:50,855 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 11.4 sec -2013-09-18 03:21:51,860 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.23 sec -2013-09-18 03:21:52,866 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.23 sec -MapReduce Total cumulative CPU time: 14 seconds 230 msec -Ended Job = job_201309172235_0331 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.9 sec HDFS Read: 69312553 HDFS Write: 31841963 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 14.23 sec HDFS Read: 31842732 HDFS Write: 372 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 130 msec -OK -Time taken: 52.364 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24667@mturlrep13_201309180321_1625792841.txt -hive> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0332 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:22:03,145 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:22:10,175 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:22:11,188 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 11.62 sec -2013-09-18 03:22:12,196 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.16 sec -2013-09-18 03:22:13,204 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.16 sec -2013-09-18 03:22:14,211 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.16 sec -2013-09-18 03:22:15,217 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.16 sec -2013-09-18 03:22:16,223 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.16 sec -2013-09-18 03:22:17,229 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 48.16 sec -2013-09-18 03:22:18,236 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.16 sec -2013-09-18 03:22:19,243 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.16 sec -2013-09-18 03:22:20,249 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 48.16 sec -2013-09-18 03:22:21,255 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 48.16 sec -2013-09-18 03:22:22,261 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 48.16 sec -2013-09-18 03:22:23,269 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.71 sec -2013-09-18 03:22:24,275 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.71 sec -2013-09-18 03:22:25,281 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.71 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 710 msec -Ended Job = job_201309172235_0332 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0333 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:22:28,772 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:22:35,794 Stage-2 map = 52%, reduce = 0% -2013-09-18 03:22:37,803 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.58 sec -2013-09-18 03:22:38,808 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.58 sec -2013-09-18 03:22:39,813 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.58 sec -2013-09-18 03:22:40,817 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.58 sec -2013-09-18 03:22:41,822 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.58 sec -2013-09-18 03:22:42,826 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.58 sec -2013-09-18 03:22:43,831 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.58 sec -2013-09-18 03:22:44,837 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 11.58 sec -2013-09-18 03:22:45,843 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 11.58 sec -2013-09-18 03:22:46,848 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.44 sec -2013-09-18 03:22:47,853 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.44 sec -MapReduce Total cumulative CPU time: 14 seconds 440 msec -Ended Job = job_201309172235_0333 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.71 sec HDFS Read: 69312553 HDFS Write: 31841963 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 14.44 sec HDFS Read: 31842732 HDFS Write: 372 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 150 msec -OK -Time taken: 53.155 seconds, Fetched: 10 row(s) -hive> quit; --- сложная агрегация, для больших таблиц может не хватить оперативки.; - - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26648@mturlrep13_201309180322_1394561217.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0334 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:23:04,981 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:23:12,010 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:23:15,034 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:23:16,041 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:23:17,049 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:23:18,055 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:23:19,061 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:23:20,068 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:23:21,073 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:23:22,078 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 49.3 sec -2013-09-18 03:23:23,084 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 49.3 sec -2013-09-18 03:23:24,089 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 49.3 sec -2013-09-18 03:23:25,095 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 49.3 sec -2013-09-18 03:23:26,101 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 49.3 sec -2013-09-18 03:23:27,106 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 49.3 sec -2013-09-18 03:23:28,114 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 57.83 sec -2013-09-18 03:23:29,122 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 66.46 sec -2013-09-18 03:23:30,129 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 66.46 sec -MapReduce Total cumulative CPU time: 1 minutes 6 seconds 460 msec -Ended Job = job_201309172235_0334 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0335 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:23:33,600 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:23:43,633 Stage-2 map = 50%, reduce = 0% -2013-09-18 03:23:46,644 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.06 sec -2013-09-18 03:23:47,649 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.06 sec -2013-09-18 03:23:48,654 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.06 sec -2013-09-18 03:23:49,659 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.06 sec -2013-09-18 03:23:50,664 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.06 sec -2013-09-18 03:23:51,668 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.06 sec -2013-09-18 03:23:52,674 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.06 sec -2013-09-18 03:23:53,679 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.06 sec -2013-09-18 03:23:54,685 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.06 sec -2013-09-18 03:23:55,690 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.06 sec -2013-09-18 03:23:56,696 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 21.88 sec -2013-09-18 03:23:57,701 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 21.88 sec -MapReduce Total cumulative CPU time: 21 seconds 880 msec -Ended Job = job_201309172235_0335 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 66.46 sec HDFS Read: 112931901 HDFS Write: 72725701 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 21.88 sec HDFS Read: 72726470 HDFS Write: 417 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 28 seconds 340 msec -OK -Time taken: 62.907 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28712@mturlrep13_201309180323_447001239.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0336 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:24:06,957 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:24:14,997 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 41.58 sec -2013-09-18 03:24:16,004 Stage-1 map = 63%, reduce = 0%, Cumulative CPU 41.58 sec -2013-09-18 03:24:17,015 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.71 sec -2013-09-18 03:24:18,023 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.71 sec -2013-09-18 03:24:19,029 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.71 sec -2013-09-18 03:24:20,034 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.71 sec -2013-09-18 03:24:21,040 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.71 sec -2013-09-18 03:24:22,047 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.71 sec -2013-09-18 03:24:23,052 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 51.71 sec -2013-09-18 03:24:24,058 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 51.71 sec -2013-09-18 03:24:25,064 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 51.71 sec -2013-09-18 03:24:26,070 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 51.71 sec -2013-09-18 03:24:27,075 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 51.71 sec -2013-09-18 03:24:28,081 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 51.71 sec -2013-09-18 03:24:29,087 Stage-1 map = 100%, reduce = 76%, Cumulative CPU 51.71 sec -2013-09-18 03:24:30,095 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.32 sec -2013-09-18 03:24:31,104 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.32 sec -2013-09-18 03:24:32,110 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.32 sec -MapReduce Total cumulative CPU time: 1 minutes 9 seconds 320 msec -Ended Job = job_201309172235_0336 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0337 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:24:34,678 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:24:45,718 Stage-2 map = 50%, reduce = 0% -2013-09-18 03:24:48,730 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.64 sec -2013-09-18 03:24:49,735 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.64 sec -2013-09-18 03:24:50,739 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.64 sec -2013-09-18 03:24:51,743 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.64 sec -2013-09-18 03:24:52,748 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.64 sec -2013-09-18 03:24:53,752 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.64 sec -2013-09-18 03:24:54,758 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.64 sec -2013-09-18 03:24:55,763 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.64 sec -2013-09-18 03:24:56,769 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.64 sec -2013-09-18 03:24:57,774 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.51 sec -2013-09-18 03:24:58,778 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.51 sec -2013-09-18 03:24:59,784 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.51 sec -MapReduce Total cumulative CPU time: 22 seconds 510 msec -Ended Job = job_201309172235_0337 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 69.32 sec HDFS Read: 112931901 HDFS Write: 72725701 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 22.51 sec HDFS Read: 72726466 HDFS Write: 417 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 31 seconds 830 msec -OK -Time taken: 60.202 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30775@mturlrep13_201309180325_1495569206.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0338 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:25:10,560 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:25:17,591 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:25:19,608 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 53.02 sec -2013-09-18 03:25:20,616 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 53.02 sec -2013-09-18 03:25:21,624 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 53.02 sec -2013-09-18 03:25:22,630 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 53.02 sec -2013-09-18 03:25:23,635 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 53.02 sec -2013-09-18 03:25:24,641 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 53.02 sec -2013-09-18 03:25:25,647 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 53.02 sec -2013-09-18 03:25:26,653 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 53.02 sec -2013-09-18 03:25:27,659 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 53.02 sec -2013-09-18 03:25:28,664 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 53.02 sec -2013-09-18 03:25:29,670 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 53.02 sec -2013-09-18 03:25:30,676 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 53.02 sec -2013-09-18 03:25:31,681 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 53.02 sec -2013-09-18 03:25:32,689 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 61.27 sec -2013-09-18 03:25:33,713 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.91 sec -2013-09-18 03:25:34,719 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.91 sec -MapReduce Total cumulative CPU time: 1 minutes 9 seconds 910 msec -Ended Job = job_201309172235_0338 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0339 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:25:37,201 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:25:48,240 Stage-2 map = 50%, reduce = 0% -2013-09-18 03:25:51,252 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.68 sec -2013-09-18 03:25:52,258 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.68 sec -2013-09-18 03:25:53,262 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.68 sec -2013-09-18 03:25:54,267 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.68 sec -2013-09-18 03:25:55,272 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.68 sec -2013-09-18 03:25:56,277 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.68 sec -2013-09-18 03:25:57,283 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 16.68 sec -2013-09-18 03:25:58,288 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 16.68 sec -2013-09-18 03:25:59,293 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 16.68 sec -2013-09-18 03:26:00,299 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 21.77 sec -2013-09-18 03:26:01,304 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 21.77 sec -MapReduce Total cumulative CPU time: 21 seconds 770 msec -Ended Job = job_201309172235_0339 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 69.91 sec HDFS Read: 112931901 HDFS Write: 72725701 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 21.77 sec HDFS Read: 72726470 HDFS Write: 417 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 31 seconds 680 msec -OK -Time taken: 59.653 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1150@mturlrep13_201309180326_1951977116.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0340 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:26:18,728 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:26:26,762 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:26:29,776 Stage-1 map = 29%, reduce = 0% -2013-09-18 03:26:32,790 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:26:35,804 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:26:38,824 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 24.87 sec -2013-09-18 03:26:39,832 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 24.87 sec -2013-09-18 03:26:40,840 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 80.51 sec -2013-09-18 03:26:41,846 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 108.69 sec -2013-09-18 03:26:42,852 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 108.69 sec -2013-09-18 03:26:43,857 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 108.69 sec -2013-09-18 03:26:44,862 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 108.69 sec -2013-09-18 03:26:45,868 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 108.69 sec -2013-09-18 03:26:46,873 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 108.69 sec -2013-09-18 03:26:47,879 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 108.69 sec -2013-09-18 03:26:48,884 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 108.69 sec -2013-09-18 03:26:49,890 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 108.69 sec -2013-09-18 03:26:50,895 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 108.69 sec -2013-09-18 03:26:51,901 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 108.69 sec -2013-09-18 03:26:52,907 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 108.69 sec -2013-09-18 03:26:53,913 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 108.69 sec -2013-09-18 03:26:54,918 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 108.69 sec -2013-09-18 03:26:55,924 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 108.69 sec -2013-09-18 03:26:56,930 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 108.69 sec -2013-09-18 03:26:57,936 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 108.69 sec -2013-09-18 03:26:58,941 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 108.69 sec -2013-09-18 03:26:59,947 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 108.69 sec -2013-09-18 03:27:00,953 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 108.69 sec -2013-09-18 03:27:01,958 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 108.69 sec -2013-09-18 03:27:02,963 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 108.69 sec -2013-09-18 03:27:03,968 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 108.69 sec -2013-09-18 03:27:04,974 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 108.69 sec -2013-09-18 03:27:05,980 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 108.69 sec -2013-09-18 03:27:06,986 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 108.69 sec -2013-09-18 03:27:07,992 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 108.69 sec -2013-09-18 03:27:08,998 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 108.69 sec -2013-09-18 03:27:11,596 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 108.69 sec -2013-09-18 03:27:12,610 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 108.69 sec -2013-09-18 03:27:13,618 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 170.18 sec -2013-09-18 03:27:14,642 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 170.18 sec -MapReduce Total cumulative CPU time: 2 minutes 50 seconds 180 msec -Ended Job = job_201309172235_0340 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0341 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 03:27:18,241 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:27:31,283 Stage-2 map = 36%, reduce = 0% -2013-09-18 03:27:40,311 Stage-2 map = 72%, reduce = 0% -2013-09-18 03:27:42,320 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 30.17 sec -2013-09-18 03:27:43,325 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 30.17 sec -2013-09-18 03:27:44,330 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 30.17 sec -2013-09-18 03:27:45,335 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 30.17 sec -2013-09-18 03:27:46,341 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 30.17 sec -2013-09-18 03:27:47,346 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 30.17 sec -2013-09-18 03:27:48,350 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 30.17 sec -2013-09-18 03:27:49,355 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 30.17 sec -2013-09-18 03:27:50,360 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 30.17 sec -2013-09-18 03:27:51,365 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 30.17 sec -2013-09-18 03:27:52,370 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 30.17 sec -2013-09-18 03:27:53,375 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 30.17 sec -2013-09-18 03:27:54,380 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 30.17 sec -2013-09-18 03:27:55,385 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 30.17 sec -2013-09-18 03:27:56,390 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 30.17 sec -2013-09-18 03:27:57,395 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 30.17 sec -2013-09-18 03:27:58,400 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 30.17 sec -2013-09-18 03:27:59,405 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.9 sec -2013-09-18 03:28:00,409 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.9 sec -2013-09-18 03:28:01,414 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.9 sec -2013-09-18 03:28:02,418 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.9 sec -2013-09-18 03:28:03,423 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.9 sec -2013-09-18 03:28:04,427 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.9 sec -2013-09-18 03:28:05,432 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.9 sec -2013-09-18 03:28:06,437 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.9 sec -2013-09-18 03:28:07,441 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 78.9 sec -2013-09-18 03:28:08,446 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 78.9 sec -2013-09-18 03:28:09,450 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 78.9 sec -2013-09-18 03:28:10,455 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 78.9 sec -2013-09-18 03:28:11,464 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 78.9 sec -2013-09-18 03:28:12,469 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 78.9 sec -2013-09-18 03:28:13,473 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 89.45 sec -2013-09-18 03:28:14,478 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 89.45 sec -2013-09-18 03:28:15,483 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 89.45 sec -MapReduce Total cumulative CPU time: 1 minutes 29 seconds 450 msec -Ended Job = job_201309172235_0341 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 170.18 sec HDFS Read: 85707829 HDFS Write: 413932232 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 89.45 sec HDFS Read: 413942944 HDFS Write: 420 SUCCESS -Total MapReduce CPU Time Spent: 4 minutes 19 seconds 630 msec -OK -Time taken: 127.206 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3393@mturlrep13_201309180328_1674206959.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0342 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:28:25,773 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:28:32,804 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:28:35,821 Stage-1 map = 41%, reduce = 0% -2013-09-18 03:28:38,834 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:28:41,849 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:28:44,868 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 25.88 sec -2013-09-18 03:28:45,875 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 53.48 sec -2013-09-18 03:28:46,883 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 53.48 sec -2013-09-18 03:28:47,888 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.03 sec -2013-09-18 03:28:48,894 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.03 sec -2013-09-18 03:28:49,908 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.03 sec -2013-09-18 03:28:50,913 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.03 sec -2013-09-18 03:28:51,919 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.03 sec -2013-09-18 03:28:52,925 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 112.03 sec -2013-09-18 03:28:53,931 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 112.03 sec -2013-09-18 03:28:54,936 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 112.03 sec -2013-09-18 03:28:55,942 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 112.03 sec -2013-09-18 03:28:56,948 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 112.03 sec -2013-09-18 03:28:57,954 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 112.03 sec -2013-09-18 03:28:58,960 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 112.03 sec -2013-09-18 03:28:59,966 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 112.03 sec -2013-09-18 03:29:00,971 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 112.03 sec -2013-09-18 03:29:01,977 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 112.03 sec -2013-09-18 03:29:02,983 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 112.03 sec -2013-09-18 03:29:03,989 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 112.03 sec -2013-09-18 03:29:04,995 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 112.03 sec -2013-09-18 03:29:06,000 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 112.03 sec -2013-09-18 03:29:07,005 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 112.03 sec -2013-09-18 03:29:08,010 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 112.03 sec -2013-09-18 03:29:09,016 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 112.03 sec -2013-09-18 03:29:10,021 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 112.03 sec -2013-09-18 03:29:11,026 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 112.03 sec -2013-09-18 03:29:12,032 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 112.03 sec -2013-09-18 03:29:13,037 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 112.03 sec -2013-09-18 03:29:14,043 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 112.03 sec -2013-09-18 03:29:15,048 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 112.03 sec -2013-09-18 03:29:16,055 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 165.0 sec -2013-09-18 03:29:19,092 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 168.62 sec -2013-09-18 03:29:20,097 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 168.62 sec -2013-09-18 03:29:21,103 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 173.79 sec -2013-09-18 03:29:22,108 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 173.79 sec -MapReduce Total cumulative CPU time: 2 minutes 53 seconds 790 msec -Ended Job = job_201309172235_0342 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0343 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 03:29:25,610 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:29:38,653 Stage-2 map = 36%, reduce = 0% -2013-09-18 03:29:47,683 Stage-2 map = 72%, reduce = 0% -2013-09-18 03:29:49,691 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-18 03:29:50,696 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-18 03:29:51,701 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-18 03:29:52,706 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-18 03:29:53,711 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-18 03:29:54,715 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-18 03:29:55,720 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 28.46 sec -2013-09-18 03:29:56,725 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:29:57,730 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:29:58,735 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:29:59,739 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:30:00,743 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:30:01,748 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:30:02,753 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:30:03,758 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:30:04,762 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:30:05,767 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:30:06,771 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 28.46 sec -2013-09-18 03:30:07,775 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.77 sec -2013-09-18 03:30:08,780 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.77 sec -2013-09-18 03:30:09,784 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.77 sec -2013-09-18 03:30:10,788 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.77 sec -2013-09-18 03:30:11,792 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.77 sec -2013-09-18 03:30:12,797 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.77 sec -2013-09-18 03:30:13,802 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.77 sec -2013-09-18 03:30:14,806 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.77 sec -2013-09-18 03:30:15,831 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 83.0 sec -2013-09-18 03:30:16,836 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 83.0 sec -2013-09-18 03:30:18,119 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 83.0 sec -2013-09-18 03:30:19,123 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 83.0 sec -2013-09-18 03:30:20,128 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 83.0 sec -2013-09-18 03:30:21,133 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 88.85 sec -2013-09-18 03:30:22,138 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 88.85 sec -MapReduce Total cumulative CPU time: 1 minutes 28 seconds 850 msec -Ended Job = job_201309172235_0343 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 173.79 sec HDFS Read: 85707829 HDFS Write: 413932232 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 88.85 sec HDFS Read: 413942944 HDFS Write: 420 SUCCESS -Total MapReduce CPU Time Spent: 4 minutes 22 seconds 640 msec -OK -Time taken: 124.701 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6613@mturlrep13_201309180330_1704971326.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0344 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:30:32,334 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:30:39,366 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:30:42,380 Stage-1 map = 41%, reduce = 0% -2013-09-18 03:30:45,392 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:30:48,407 Stage-1 map = 78%, reduce = 0% -2013-09-18 03:30:51,428 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 25.55 sec -2013-09-18 03:30:52,435 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 52.78 sec -2013-09-18 03:30:53,442 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 81.04 sec -2013-09-18 03:30:54,447 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 110.99 sec -2013-09-18 03:30:55,453 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 110.99 sec -2013-09-18 03:30:56,458 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 110.99 sec -2013-09-18 03:30:57,463 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 110.99 sec -2013-09-18 03:30:58,469 Stage-1 map = 100%, reduce = 13%, Cumulative CPU 110.99 sec -2013-09-18 03:30:59,474 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 110.99 sec -2013-09-18 03:31:00,480 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 110.99 sec -2013-09-18 03:31:01,485 Stage-1 map = 100%, reduce = 42%, Cumulative CPU 110.99 sec -2013-09-18 03:31:02,491 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 110.99 sec -2013-09-18 03:31:03,497 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 110.99 sec -2013-09-18 03:31:04,503 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 110.99 sec -2013-09-18 03:31:05,509 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 110.99 sec -2013-09-18 03:31:06,514 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 110.99 sec -2013-09-18 03:31:07,520 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 110.99 sec -2013-09-18 03:31:08,525 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 110.99 sec -2013-09-18 03:31:09,531 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 110.99 sec -2013-09-18 03:31:10,536 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 110.99 sec -2013-09-18 03:31:11,541 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 110.99 sec -2013-09-18 03:31:12,547 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 110.99 sec -2013-09-18 03:31:13,552 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 110.99 sec -2013-09-18 03:31:14,557 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 110.99 sec -2013-09-18 03:31:15,564 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 151.68 sec -2013-09-18 03:31:16,570 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 151.68 sec -2013-09-18 03:31:17,575 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 151.68 sec -2013-09-18 03:31:18,580 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 151.68 sec -2013-09-18 03:31:19,585 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 151.68 sec -2013-09-18 03:31:20,590 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 151.68 sec -2013-09-18 03:31:21,595 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 151.68 sec -2013-09-18 03:31:22,600 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 151.68 sec -2013-09-18 03:31:25,469 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 161.77 sec -2013-09-18 03:31:26,474 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 173.12 sec -2013-09-18 03:31:27,479 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 173.12 sec -2013-09-18 03:31:28,485 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 173.12 sec -MapReduce Total cumulative CPU time: 2 minutes 53 seconds 120 msec -Ended Job = job_201309172235_0344 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0345 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 03:31:30,990 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:31:45,038 Stage-2 map = 36%, reduce = 0% -2013-09-18 03:31:51,058 Stage-2 map = 48%, reduce = 0% -2013-09-18 03:31:54,070 Stage-2 map = 72%, reduce = 0% -2013-09-18 03:31:55,075 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 27.96 sec -2013-09-18 03:31:56,080 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 27.96 sec -2013-09-18 03:31:57,086 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 27.96 sec -2013-09-18 03:31:58,090 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 27.96 sec -2013-09-18 03:31:59,096 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 27.96 sec -2013-09-18 03:32:00,101 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 27.96 sec -2013-09-18 03:32:01,106 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 27.96 sec -2013-09-18 03:32:02,112 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 27.96 sec -2013-09-18 03:32:03,117 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 27.96 sec -2013-09-18 03:32:04,122 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 27.96 sec -2013-09-18 03:32:05,127 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 27.96 sec -2013-09-18 03:32:06,132 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 27.96 sec -2013-09-18 03:32:07,137 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 27.96 sec -2013-09-18 03:32:08,141 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 27.96 sec -2013-09-18 03:32:09,147 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 27.96 sec -2013-09-18 03:32:10,151 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 27.96 sec -2013-09-18 03:32:11,156 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 74.69 sec -2013-09-18 03:32:12,161 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 74.69 sec -2013-09-18 03:32:13,165 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 74.69 sec -2013-09-18 03:32:14,170 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 74.69 sec -2013-09-18 03:32:15,174 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 74.69 sec -2013-09-18 03:32:16,179 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 75.98 sec -2013-09-18 03:32:17,183 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 75.98 sec -2013-09-18 03:32:18,188 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 75.98 sec -2013-09-18 03:32:19,192 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 75.98 sec -2013-09-18 03:32:20,202 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 75.98 sec -2013-09-18 03:32:21,207 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 75.98 sec -2013-09-18 03:32:22,211 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 75.98 sec -2013-09-18 03:32:23,215 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 75.98 sec -2013-09-18 03:32:24,220 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 75.98 sec -2013-09-18 03:32:25,225 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 85.63 sec -2013-09-18 03:32:26,229 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 85.63 sec -2013-09-18 03:32:27,234 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 85.63 sec -MapReduce Total cumulative CPU time: 1 minutes 25 seconds 630 msec -Ended Job = job_201309172235_0345 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 173.12 sec HDFS Read: 85707829 HDFS Write: 413932232 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 85.63 sec HDFS Read: 413942944 HDFS Write: 420 SUCCESS -Total MapReduce CPU Time Spent: 4 minutes 18 seconds 750 msec -OK -Time taken: 123.291 seconds, Fetched: 10 row(s) -hive> quit; --- то же самое, но ещё и без фильтрации.; - - -times: 1 -query: SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8833@mturlrep13_201309180332_1808208086.txt -hive> SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0346 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:32:43,874 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:32:50,903 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:32:53,915 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:32:57,932 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:33:00,946 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:33:03,965 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 25.99 sec -2013-09-18 03:33:04,972 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 25.99 sec -2013-09-18 03:33:05,979 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 54.38 sec -2013-09-18 03:33:06,986 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 83.01 sec -2013-09-18 03:33:07,992 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.86 sec -2013-09-18 03:33:08,998 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.86 sec -2013-09-18 03:33:10,003 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.86 sec -2013-09-18 03:33:11,010 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 112.86 sec -2013-09-18 03:33:12,016 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 112.86 sec -2013-09-18 03:33:13,023 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 112.86 sec -2013-09-18 03:33:14,029 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 112.86 sec -2013-09-18 03:33:15,036 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 112.86 sec -2013-09-18 03:33:16,042 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 136.61 sec -2013-09-18 03:33:17,049 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 136.61 sec -2013-09-18 03:33:18,055 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 136.61 sec -2013-09-18 03:33:19,061 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 136.61 sec -2013-09-18 03:33:20,068 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 136.61 sec -2013-09-18 03:33:21,075 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 136.61 sec -2013-09-18 03:33:22,081 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 136.61 sec -2013-09-18 03:33:23,087 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 136.61 sec -2013-09-18 03:33:24,094 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 136.61 sec -2013-09-18 03:33:25,099 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 136.61 sec -2013-09-18 03:33:26,106 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 136.61 sec -2013-09-18 03:33:27,112 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 136.61 sec -2013-09-18 03:33:28,118 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 136.61 sec -2013-09-18 03:33:29,126 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 170.16 sec -2013-09-18 03:33:30,132 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 170.16 sec -2013-09-18 03:33:31,138 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 170.16 sec -MapReduce Total cumulative CPU time: 2 minutes 50 seconds 160 msec -Ended Job = job_201309172235_0346 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0347 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 03:33:34,663 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:33:41,687 Stage-2 map = 25%, reduce = 0% -2013-09-18 03:33:43,695 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-18 03:33:44,701 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-18 03:33:45,705 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-18 03:33:46,711 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-18 03:33:47,716 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-18 03:33:48,720 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-18 03:33:49,726 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-18 03:33:50,731 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 13.37 sec -2013-09-18 03:33:51,736 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 13.37 sec -2013-09-18 03:33:52,741 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 36.76 sec -2013-09-18 03:33:53,746 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 36.76 sec -2013-09-18 03:33:54,753 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 36.76 sec -2013-09-18 03:33:55,759 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 36.76 sec -2013-09-18 03:33:56,765 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 36.76 sec -2013-09-18 03:33:57,771 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 36.76 sec -2013-09-18 03:33:58,776 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 36.76 sec -2013-09-18 03:33:59,782 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 36.76 sec -2013-09-18 03:34:00,787 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 36.76 sec -2013-09-18 03:34:01,793 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 44.31 sec -2013-09-18 03:34:02,799 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 44.31 sec -2013-09-18 03:34:03,804 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 44.31 sec -MapReduce Total cumulative CPU time: 44 seconds 310 msec -Ended Job = job_201309172235_0347 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 170.16 sec HDFS Read: 109451651 HDFS Write: 399298510 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 44.31 sec HDFS Read: 399308167 HDFS Write: 445 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 34 seconds 470 msec -OK -Time taken: 89.931 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10716@mturlrep13_201309180334_544608212.txt -hive> SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0348 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:34:13,365 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:34:21,405 Stage-1 map = 18%, reduce = 0% -2013-09-18 03:34:24,418 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:34:27,431 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:34:30,444 Stage-1 map = 78%, reduce = 0% -2013-09-18 03:34:33,463 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 25.62 sec -2013-09-18 03:34:34,469 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 25.62 sec -2013-09-18 03:34:35,477 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 25.62 sec -2013-09-18 03:34:36,485 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 114.36 sec -2013-09-18 03:34:37,490 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 114.36 sec -2013-09-18 03:34:38,495 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 114.36 sec -2013-09-18 03:34:39,500 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 114.36 sec -2013-09-18 03:34:40,506 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 114.36 sec -2013-09-18 03:34:41,512 Stage-1 map = 100%, reduce = 13%, Cumulative CPU 114.36 sec -2013-09-18 03:34:42,518 Stage-1 map = 100%, reduce = 13%, Cumulative CPU 114.36 sec -2013-09-18 03:34:43,523 Stage-1 map = 100%, reduce = 13%, Cumulative CPU 114.36 sec -2013-09-18 03:34:44,529 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 114.36 sec -2013-09-18 03:34:45,534 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 114.36 sec -2013-09-18 03:34:46,540 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 114.36 sec -2013-09-18 03:34:47,546 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 114.36 sec -2013-09-18 03:34:48,552 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 114.36 sec -2013-09-18 03:34:49,559 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 114.36 sec -2013-09-18 03:34:50,565 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 114.36 sec -2013-09-18 03:34:51,570 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 114.36 sec -2013-09-18 03:34:52,577 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 114.36 sec -2013-09-18 03:34:53,583 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 114.36 sec -2013-09-18 03:34:54,588 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 114.36 sec -2013-09-18 03:34:55,593 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 114.36 sec -2013-09-18 03:34:56,599 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 114.36 sec -2013-09-18 03:34:57,605 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 114.36 sec -2013-09-18 03:34:58,612 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 168.31 sec -2013-09-18 03:34:59,618 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 168.31 sec -2013-09-18 03:35:00,625 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 168.31 sec -MapReduce Total cumulative CPU time: 2 minutes 48 seconds 310 msec -Ended Job = job_201309172235_0348 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0349 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 03:35:04,093 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:35:11,115 Stage-2 map = 25%, reduce = 0% -2013-09-18 03:35:14,127 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 14.31 sec -2013-09-18 03:35:15,135 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 14.31 sec -2013-09-18 03:35:16,141 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 28.98 sec -2013-09-18 03:35:17,146 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 28.98 sec -2013-09-18 03:35:18,152 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 28.98 sec -2013-09-18 03:35:19,158 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 28.98 sec -2013-09-18 03:35:20,163 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 28.98 sec -2013-09-18 03:35:21,169 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 28.98 sec -2013-09-18 03:35:22,174 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-18 03:35:23,178 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-18 03:35:24,183 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-18 03:35:25,187 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-18 03:35:26,192 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-18 03:35:27,197 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-18 03:35:28,202 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-18 03:35:29,207 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.51 sec -2013-09-18 03:35:30,212 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 38.51 sec -2013-09-18 03:35:31,217 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 38.51 sec -2013-09-18 03:35:32,222 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.15 sec -2013-09-18 03:35:33,228 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.15 sec -MapReduce Total cumulative CPU time: 46 seconds 150 msec -Ended Job = job_201309172235_0349 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 168.31 sec HDFS Read: 109451651 HDFS Write: 399298510 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 46.15 sec HDFS Read: 399308173 HDFS Write: 445 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 34 seconds 460 msec -OK -Time taken: 87.6 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13289@mturlrep13_201309180335_999822129.txt -hive> SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0350 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:35:43,626 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:35:50,655 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:35:53,668 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:35:56,682 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:35:59,697 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:36:02,709 Stage-1 map = 89%, reduce = 0% -2013-09-18 03:36:06,735 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 116.11 sec -2013-09-18 03:36:07,740 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 116.11 sec -2013-09-18 03:36:08,747 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 116.11 sec -2013-09-18 03:36:09,752 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 116.11 sec -2013-09-18 03:36:10,758 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 116.11 sec -2013-09-18 03:36:11,764 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 116.11 sec -2013-09-18 03:36:12,769 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 116.11 sec -2013-09-18 03:36:13,779 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 116.11 sec -2013-09-18 03:36:14,784 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 116.11 sec -2013-09-18 03:36:15,790 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 116.11 sec -2013-09-18 03:36:16,795 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 130.52 sec -2013-09-18 03:36:17,801 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 130.52 sec -2013-09-18 03:36:18,808 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 130.52 sec -2013-09-18 03:36:19,814 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 130.52 sec -2013-09-18 03:36:20,820 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 130.52 sec -2013-09-18 03:36:21,826 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 130.52 sec -2013-09-18 03:36:22,832 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 130.52 sec -2013-09-18 03:36:23,838 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 130.52 sec -2013-09-18 03:36:24,843 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 130.52 sec -2013-09-18 03:36:25,849 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 130.52 sec -2013-09-18 03:36:26,854 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 130.52 sec -2013-09-18 03:36:27,860 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 130.52 sec -2013-09-18 03:36:28,867 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 172.7 sec -2013-09-18 03:36:29,873 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 172.7 sec -MapReduce Total cumulative CPU time: 2 minutes 52 seconds 700 msec -Ended Job = job_201309172235_0350 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0351 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 03:36:33,293 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:36:40,314 Stage-2 map = 25%, reduce = 0% -2013-09-18 03:36:43,325 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.65 sec -2013-09-18 03:36:44,330 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.65 sec -2013-09-18 03:36:45,335 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.65 sec -2013-09-18 03:36:46,340 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.65 sec -2013-09-18 03:36:47,344 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.65 sec -2013-09-18 03:36:48,349 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.65 sec -2013-09-18 03:36:49,354 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.65 sec -2013-09-18 03:36:50,359 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 13.65 sec -2013-09-18 03:36:51,364 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.93 sec -2013-09-18 03:36:52,368 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.93 sec -2013-09-18 03:36:53,373 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.93 sec -2013-09-18 03:36:54,377 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.93 sec -2013-09-18 03:36:55,383 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.93 sec -2013-09-18 03:36:56,388 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.93 sec -2013-09-18 03:36:57,392 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.93 sec -2013-09-18 03:36:58,397 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.93 sec -2013-09-18 03:36:59,402 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 37.93 sec -2013-09-18 03:37:00,407 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 45.36 sec -2013-09-18 03:37:01,413 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 45.36 sec -2013-09-18 03:37:02,418 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 45.36 sec -MapReduce Total cumulative CPU time: 45 seconds 360 msec -Ended Job = job_201309172235_0351 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 172.7 sec HDFS Read: 109451651 HDFS Write: 399298510 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 45.36 sec HDFS Read: 399308173 HDFS Write: 445 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 38 seconds 60 msec -OK -Time taken: 87.325 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по URL.; - - -times: 1 -query: SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15219@mturlrep13_201309180337_1137207213.txt -hive> SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0352 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:37:21,097 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:37:29,132 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:37:32,145 Stage-1 map = 40%, reduce = 0% -2013-09-18 03:37:35,160 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:37:38,174 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:37:41,187 Stage-1 map = 89%, reduce = 0% -2013-09-18 03:37:45,212 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 116.3 sec -2013-09-18 03:37:46,218 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 116.3 sec -2013-09-18 03:37:47,225 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 116.3 sec -2013-09-18 03:37:48,230 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 116.3 sec -2013-09-18 03:37:49,235 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 116.3 sec -2013-09-18 03:37:50,241 Stage-1 map = 100%, reduce = 4%, Cumulative CPU 116.3 sec -2013-09-18 03:37:51,247 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 116.3 sec -2013-09-18 03:37:52,252 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 116.3 sec -2013-09-18 03:37:53,259 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 116.3 sec -2013-09-18 03:37:54,265 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 116.3 sec -2013-09-18 03:37:55,271 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 116.3 sec -2013-09-18 03:37:56,278 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 116.3 sec -2013-09-18 03:37:57,283 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 116.3 sec -2013-09-18 03:37:58,289 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 116.3 sec -2013-09-18 03:37:59,295 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 116.3 sec -2013-09-18 03:38:00,301 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 116.3 sec -2013-09-18 03:38:01,307 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 116.3 sec -2013-09-18 03:38:02,312 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 116.3 sec -2013-09-18 03:38:03,319 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 116.3 sec -2013-09-18 03:38:04,324 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 116.3 sec -2013-09-18 03:38:05,331 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 116.3 sec -2013-09-18 03:38:06,337 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 116.3 sec -2013-09-18 03:38:07,343 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 116.3 sec -2013-09-18 03:38:08,348 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 116.3 sec -2013-09-18 03:38:09,356 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 146.05 sec -2013-09-18 03:38:10,363 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 177.94 sec -2013-09-18 03:38:11,369 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 177.94 sec -MapReduce Total cumulative CPU time: 2 minutes 57 seconds 940 msec -Ended Job = job_201309172235_0352 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0353 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 03:38:19,184 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:38:27,211 Stage-2 map = 25%, reduce = 0% -2013-09-18 03:38:30,222 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-18 03:38:31,227 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-18 03:38:32,232 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-18 03:38:33,237 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-18 03:38:34,242 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-18 03:38:35,247 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-18 03:38:36,252 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.05 sec -2013-09-18 03:38:37,256 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 14.05 sec -2013-09-18 03:38:38,260 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.6 sec -2013-09-18 03:38:39,264 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.6 sec -2013-09-18 03:38:40,268 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.6 sec -2013-09-18 03:38:41,272 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.6 sec -2013-09-18 03:38:42,277 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.6 sec -2013-09-18 03:38:43,282 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.6 sec -2013-09-18 03:38:44,287 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.6 sec -2013-09-18 03:38:45,293 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.6 sec -2013-09-18 03:38:46,297 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 38.6 sec -2013-09-18 03:38:47,303 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 38.6 sec -2013-09-18 03:38:48,308 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.2 sec -2013-09-18 03:38:49,314 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.2 sec -MapReduce Total cumulative CPU time: 46 seconds 200 msec -Ended Job = job_201309172235_0353 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 177.94 sec HDFS Read: 109451651 HDFS Write: 402873759 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 46.2 sec HDFS Read: 402889658 HDFS Write: 465 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 44 seconds 140 msec -OK -Time taken: 98.879 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17127@mturlrep13_201309180338_1157991305.txt -hive> SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0354 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:38:59,272 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:39:07,305 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:39:10,318 Stage-1 map = 37%, reduce = 0% -2013-09-18 03:39:13,332 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:39:16,349 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 86.63 sec -2013-09-18 03:39:17,356 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 86.63 sec -2013-09-18 03:39:18,364 Stage-1 map = 74%, reduce = 0%, Cumulative CPU 86.63 sec -2013-09-18 03:39:19,371 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 90.6 sec -2013-09-18 03:39:20,378 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 90.6 sec -2013-09-18 03:39:21,383 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 90.6 sec -2013-09-18 03:39:22,389 Stage-1 map = 89%, reduce = 0%, Cumulative CPU 90.6 sec -2013-09-18 03:39:23,395 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 99.43 sec -2013-09-18 03:39:24,400 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 118.35 sec -2013-09-18 03:39:25,406 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 118.35 sec -2013-09-18 03:39:26,412 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 118.35 sec -2013-09-18 03:39:27,417 Stage-1 map = 100%, reduce = 13%, Cumulative CPU 118.35 sec -2013-09-18 03:39:28,423 Stage-1 map = 100%, reduce = 13%, Cumulative CPU 118.35 sec -2013-09-18 03:39:29,429 Stage-1 map = 100%, reduce = 13%, Cumulative CPU 118.35 sec -2013-09-18 03:39:30,434 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 118.35 sec -2013-09-18 03:39:31,439 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 118.35 sec -2013-09-18 03:39:32,445 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 118.35 sec -2013-09-18 03:39:33,450 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 118.35 sec -2013-09-18 03:39:34,456 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 118.35 sec -2013-09-18 03:39:35,462 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 118.35 sec -2013-09-18 03:39:36,468 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 118.35 sec -2013-09-18 03:39:37,474 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 118.35 sec -2013-09-18 03:39:38,480 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 118.35 sec -2013-09-18 03:39:39,486 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 118.35 sec -2013-09-18 03:39:40,491 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 118.35 sec -2013-09-18 03:39:41,496 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 118.35 sec -2013-09-18 03:39:42,502 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 118.35 sec -2013-09-18 03:39:43,507 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 118.35 sec -2013-09-18 03:39:44,513 Stage-1 map = 100%, reduce = 91%, Cumulative CPU 118.35 sec -2013-09-18 03:39:45,519 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 146.1 sec -2013-09-18 03:39:46,524 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 174.21 sec -2013-09-18 03:39:47,529 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 174.21 sec -MapReduce Total cumulative CPU time: 2 minutes 54 seconds 210 msec -Ended Job = job_201309172235_0354 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0355 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 03:39:50,983 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:39:58,006 Stage-2 map = 25%, reduce = 0% -2013-09-18 03:40:01,016 Stage-2 map = 62%, reduce = 0% -2013-09-18 03:40:04,029 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.78 sec -2013-09-18 03:40:05,035 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.78 sec -2013-09-18 03:40:06,039 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.78 sec -2013-09-18 03:40:07,044 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.78 sec -2013-09-18 03:40:08,049 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.78 sec -2013-09-18 03:40:09,053 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 14.78 sec -2013-09-18 03:40:10,058 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 40.36 sec -2013-09-18 03:40:11,063 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 40.36 sec -2013-09-18 03:40:12,067 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 40.36 sec -2013-09-18 03:40:13,073 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 40.36 sec -2013-09-18 03:40:14,079 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 40.36 sec -2013-09-18 03:40:15,084 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 40.36 sec -2013-09-18 03:40:16,089 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 40.36 sec -2013-09-18 03:40:17,094 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 41.49 sec -2013-09-18 03:40:18,099 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 41.49 sec -2013-09-18 03:40:19,104 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 41.49 sec -2013-09-18 03:40:20,109 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 48.21 sec -2013-09-18 03:40:21,114 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 48.21 sec -MapReduce Total cumulative CPU time: 48 seconds 210 msec -Ended Job = job_201309172235_0355 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 174.21 sec HDFS Read: 109451651 HDFS Write: 402873759 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 48.21 sec HDFS Read: 402889658 HDFS Write: 465 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 42 seconds 420 msec -OK -Time taken: 89.299 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19706@mturlrep13_201309180340_906204677.txt -hive> SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0356 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:40:31,369 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:40:38,401 Stage-1 map = 15%, reduce = 0% -2013-09-18 03:40:41,415 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:40:44,429 Stage-1 map = 59%, reduce = 0% -2013-09-18 03:40:47,444 Stage-1 map = 78%, reduce = 0% -2013-09-18 03:40:50,457 Stage-1 map = 89%, reduce = 0% -2013-09-18 03:40:53,477 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 57.18 sec -2013-09-18 03:40:54,483 Stage-1 map = 96%, reduce = 0%, Cumulative CPU 88.61 sec -2013-09-18 03:40:55,490 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 120.2 sec -2013-09-18 03:40:56,494 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 120.2 sec -2013-09-18 03:40:57,499 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 120.2 sec -2013-09-18 03:40:58,503 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 120.2 sec -2013-09-18 03:40:59,509 Stage-1 map = 100%, reduce = 4%, Cumulative CPU 120.2 sec -2013-09-18 03:41:00,514 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 120.2 sec -2013-09-18 03:41:01,520 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 120.2 sec -2013-09-18 03:41:02,526 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 120.2 sec -2013-09-18 03:41:03,532 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 120.2 sec -2013-09-18 03:41:04,538 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 120.2 sec -2013-09-18 03:41:05,544 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 120.2 sec -2013-09-18 03:41:06,550 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 120.2 sec -2013-09-18 03:41:07,556 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 120.2 sec -2013-09-18 03:41:08,562 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 120.2 sec -2013-09-18 03:41:09,568 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 120.2 sec -2013-09-18 03:41:10,573 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 120.2 sec -2013-09-18 03:41:11,580 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 120.2 sec -2013-09-18 03:41:12,586 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 120.2 sec -2013-09-18 03:41:13,591 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 120.2 sec -2013-09-18 03:41:14,597 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 120.2 sec -2013-09-18 03:41:15,602 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 120.2 sec -2013-09-18 03:41:16,610 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 173.32 sec -2013-09-18 03:41:17,616 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 173.32 sec -2013-09-18 03:41:18,622 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 180.14 sec -2013-09-18 03:41:19,628 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 180.14 sec -MapReduce Total cumulative CPU time: 3 minutes 0 seconds 140 msec -Ended Job = job_201309172235_0356 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0357 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-18 03:41:23,145 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:41:30,169 Stage-2 map = 25%, reduce = 0% -2013-09-18 03:41:33,182 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 14.51 sec -2013-09-18 03:41:34,187 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 14.51 sec -2013-09-18 03:41:35,193 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 14.51 sec -2013-09-18 03:41:36,198 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.51 sec -2013-09-18 03:41:37,203 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.51 sec -2013-09-18 03:41:38,208 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.51 sec -2013-09-18 03:41:39,214 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.51 sec -2013-09-18 03:41:40,219 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 14.51 sec -2013-09-18 03:41:41,224 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.04 sec -2013-09-18 03:41:42,229 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.04 sec -2013-09-18 03:41:43,234 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.04 sec -2013-09-18 03:41:44,238 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.04 sec -2013-09-18 03:41:45,243 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.04 sec -2013-09-18 03:41:46,248 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.04 sec -2013-09-18 03:41:47,252 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.04 sec -2013-09-18 03:41:48,257 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.04 sec -2013-09-18 03:41:49,262 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.04 sec -2013-09-18 03:41:50,266 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 39.04 sec -2013-09-18 03:41:51,271 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.66 sec -2013-09-18 03:41:52,277 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.66 sec -2013-09-18 03:41:53,282 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.66 sec -MapReduce Total cumulative CPU time: 46 seconds 660 msec -Ended Job = job_201309172235_0357 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 180.14 sec HDFS Read: 109451651 HDFS Write: 402873759 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 46.66 sec HDFS Read: 402889658 HDFS Write: 465 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 46 seconds 800 msec -OK -Time taken: 90.377 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по URL и числу.; - - -times: 1 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21698@mturlrep13_201309180342_83883651.txt -hive> SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0358 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:42:11,758 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:42:18,790 Stage-1 map = 63%, reduce = 0% -2013-09-18 03:42:21,804 Stage-1 map = 89%, reduce = 0% -2013-09-18 03:42:22,816 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 26.17 sec -2013-09-18 03:42:23,825 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 54.18 sec -2013-09-18 03:42:24,832 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 54.18 sec -2013-09-18 03:42:25,838 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 54.18 sec -2013-09-18 03:42:26,845 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 54.18 sec -2013-09-18 03:42:27,851 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 54.18 sec -2013-09-18 03:42:28,856 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 54.18 sec -2013-09-18 03:42:29,862 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 54.18 sec -2013-09-18 03:42:30,869 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 54.18 sec -2013-09-18 03:42:31,876 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 56.28 sec -2013-09-18 03:42:32,882 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 56.28 sec -2013-09-18 03:42:33,889 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 56.28 sec -2013-09-18 03:42:34,895 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 56.28 sec -2013-09-18 03:42:35,902 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 56.28 sec -2013-09-18 03:42:36,909 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 56.28 sec -2013-09-18 03:42:37,915 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 67.05 sec -2013-09-18 03:42:38,921 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 67.05 sec -2013-09-18 03:42:39,927 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 67.05 sec -MapReduce Total cumulative CPU time: 1 minutes 7 seconds 50 msec -Ended Job = job_201309172235_0358 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0359 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:42:42,428 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:42:54,471 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.53 sec -2013-09-18 03:42:55,477 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.53 sec -2013-09-18 03:42:56,482 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.53 sec -2013-09-18 03:42:57,487 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.53 sec -2013-09-18 03:42:58,491 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.53 sec -2013-09-18 03:42:59,495 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.53 sec -2013-09-18 03:43:00,500 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.53 sec -2013-09-18 03:43:01,505 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.53 sec -2013-09-18 03:43:02,510 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.53 sec -2013-09-18 03:43:03,515 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.82 sec -2013-09-18 03:43:04,520 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.82 sec -2013-09-18 03:43:05,527 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.82 sec -MapReduce Total cumulative CPU time: 18 seconds 820 msec -Ended Job = job_201309172235_0359 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 67.05 sec HDFS Read: 31344843 HDFS Write: 51717050 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.82 sec HDFS Read: 51717819 HDFS Write: 490 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 25 seconds 870 msec -OK -Time taken: 63.792 seconds, Fetched: 10 row(s) -hive> quit; - -times: 2 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23423@mturlrep13_201309180343_849801634.txt -hive> SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0360 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:43:14,871 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:43:22,908 Stage-1 map = 74%, reduce = 0% -2013-09-18 03:43:25,929 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 27.51 sec -2013-09-18 03:43:26,937 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.55 sec -2013-09-18 03:43:27,944 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.55 sec -2013-09-18 03:43:28,950 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.55 sec -2013-09-18 03:43:29,956 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.55 sec -2013-09-18 03:43:30,961 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.55 sec -2013-09-18 03:43:31,966 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.55 sec -2013-09-18 03:43:32,971 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 56.55 sec -2013-09-18 03:43:33,978 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 56.55 sec -2013-09-18 03:43:34,985 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 58.54 sec -2013-09-18 03:43:35,993 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 58.54 sec -2013-09-18 03:43:36,998 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 58.54 sec -2013-09-18 03:43:38,004 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 58.54 sec -2013-09-18 03:43:39,010 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 58.54 sec -2013-09-18 03:43:40,016 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 58.54 sec -2013-09-18 03:43:41,023 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.2 sec -2013-09-18 03:43:42,029 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.2 sec -2013-09-18 03:43:43,035 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.2 sec -MapReduce Total cumulative CPU time: 1 minutes 9 seconds 200 msec -Ended Job = job_201309172235_0360 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0361 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:43:45,535 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:43:57,575 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.46 sec -2013-09-18 03:43:58,580 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.46 sec -2013-09-18 03:43:59,588 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.46 sec -2013-09-18 03:44:00,592 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.46 sec -2013-09-18 03:44:01,597 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.46 sec -2013-09-18 03:44:02,601 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.46 sec -2013-09-18 03:44:03,606 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.46 sec -2013-09-18 03:44:04,612 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.46 sec -2013-09-18 03:44:05,617 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.46 sec -2013-09-18 03:44:06,621 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.66 sec -2013-09-18 03:44:07,642 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.66 sec -2013-09-18 03:44:08,647 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.66 sec -MapReduce Total cumulative CPU time: 18 seconds 660 msec -Ended Job = job_201309172235_0361 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 69.2 sec HDFS Read: 31344843 HDFS Write: 51717050 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.66 sec HDFS Read: 51717819 HDFS Write: 490 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 27 seconds 860 msec -OK -Time taken: 61.127 seconds, Fetched: 10 row(s) -hive> quit; - -times: 3 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25150@mturlrep13_201309180344_1265388056.txt -hive> SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0362 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:44:18,182 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:44:26,214 Stage-1 map = 63%, reduce = 0% -2013-09-18 03:44:29,227 Stage-1 map = 89%, reduce = 0% -2013-09-18 03:44:30,239 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 57.27 sec -2013-09-18 03:44:31,245 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 57.27 sec -2013-09-18 03:44:32,252 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 57.27 sec -2013-09-18 03:44:33,258 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 57.27 sec -2013-09-18 03:44:34,264 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 57.27 sec -2013-09-18 03:44:35,269 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 57.27 sec -2013-09-18 03:44:36,275 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 57.27 sec -2013-09-18 03:44:37,280 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.27 sec -2013-09-18 03:44:38,287 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 59.28 sec -2013-09-18 03:44:39,293 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 59.28 sec -2013-09-18 03:44:40,298 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 59.28 sec -2013-09-18 03:44:41,303 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 59.28 sec -2013-09-18 03:44:42,308 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 59.28 sec -2013-09-18 03:44:43,314 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 59.28 sec -2013-09-18 03:44:44,319 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 59.28 sec -2013-09-18 03:44:45,325 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.26 sec -2013-09-18 03:44:46,331 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.26 sec -MapReduce Total cumulative CPU time: 1 minutes 10 seconds 260 msec -Ended Job = job_201309172235_0362 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0363 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:44:49,785 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:45:00,823 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.52 sec -2013-09-18 03:45:01,829 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.52 sec -2013-09-18 03:45:02,834 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.52 sec -2013-09-18 03:45:03,903 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.52 sec -2013-09-18 03:45:04,908 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.52 sec -2013-09-18 03:45:05,912 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.52 sec -2013-09-18 03:45:06,916 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.52 sec -2013-09-18 03:45:07,921 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.52 sec -2013-09-18 03:45:08,927 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.52 sec -2013-09-18 03:45:09,932 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.85 sec -2013-09-18 03:45:10,938 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.85 sec -2013-09-18 03:45:11,943 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.85 sec -MapReduce Total cumulative CPU time: 19 seconds 850 msec -Ended Job = job_201309172235_0363 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 70.26 sec HDFS Read: 31344843 HDFS Write: 51717050 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 19.85 sec HDFS Read: 51717819 HDFS Write: 490 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 30 seconds 110 msec -OK -Time taken: 61.517 seconds, Fetched: 10 row(s) -hive> quit; - -times: 1 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27316@mturlrep13_201309180345_1510224697.txt -hive> ; -hive> quit; - -times: 2 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27776@mturlrep13_201309180345_1361207737.txt -hive> ; -hive> quit; - -times: 3 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27977@mturlrep13_201309180345_862396725.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28185@mturlrep13_201309180345_1903344467.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0364 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:45:40,647 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:45:48,681 Stage-1 map = 51%, reduce = 0% -2013-09-18 03:45:49,694 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 18.62 sec -2013-09-18 03:45:50,702 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.91 sec -2013-09-18 03:45:51,710 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.91 sec -2013-09-18 03:45:52,716 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.91 sec -2013-09-18 03:45:53,721 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.91 sec -2013-09-18 03:45:54,726 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.91 sec -2013-09-18 03:45:55,733 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.91 sec -2013-09-18 03:45:56,738 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.91 sec -2013-09-18 03:45:57,746 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 40.93 sec -2013-09-18 03:45:58,752 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.77 sec -2013-09-18 03:45:59,758 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.77 sec -MapReduce Total cumulative CPU time: 42 seconds 770 msec -Ended Job = job_201309172235_0364 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0365 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:46:03,281 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:46:04,286 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:46:05,292 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:46:06,300 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:46:07,305 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:46:08,309 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:46:09,313 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:46:10,318 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:46:11,323 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:46:12,328 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.28 sec -2013-09-18 03:46:13,335 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.28 sec -2013-09-18 03:46:14,340 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.28 sec -MapReduce Total cumulative CPU time: 2 seconds 280 msec -Ended Job = job_201309172235_0365 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.77 sec HDFS Read: 118784021 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.28 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 50 msec -OK -Time taken: 44.325 seconds -hive> quit; - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30170@mturlrep13_201309180346_1971537054.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0366 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:46:23,592 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:46:31,635 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 18.42 sec -2013-09-18 03:46:32,643 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.46 sec -2013-09-18 03:46:33,650 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.46 sec -2013-09-18 03:46:34,657 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.46 sec -2013-09-18 03:46:35,663 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.46 sec -2013-09-18 03:46:36,670 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.46 sec -2013-09-18 03:46:37,677 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.46 sec -2013-09-18 03:46:38,684 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.46 sec -2013-09-18 03:46:39,692 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 40.52 sec -2013-09-18 03:46:40,698 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.46 sec -2013-09-18 03:46:41,704 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.46 sec -MapReduce Total cumulative CPU time: 42 seconds 460 msec -Ended Job = job_201309172235_0366 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0367 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:46:45,150 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:46:46,156 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:46:47,162 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:46:48,167 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:46:49,172 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:46:50,177 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:46:51,183 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:46:52,188 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:46:53,192 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:46:54,198 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -2013-09-18 03:46:55,203 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -2013-09-18 03:46:56,209 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -MapReduce Total cumulative CPU time: 2 seconds 140 msec -Ended Job = job_201309172235_0367 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.46 sec HDFS Read: 118784021 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.14 sec HDFS Read: 959 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 600 msec -OK -Time taken: 39.995 seconds -hive> quit; - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32117@mturlrep13_201309180346_265757676.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0368 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:47:06,367 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:47:13,406 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 9.51 sec -2013-09-18 03:47:14,415 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.45 sec -2013-09-18 03:47:15,422 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.45 sec -2013-09-18 03:47:16,428 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.45 sec -2013-09-18 03:47:17,433 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.45 sec -2013-09-18 03:47:18,439 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.45 sec -2013-09-18 03:47:19,444 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.45 sec -2013-09-18 03:47:20,450 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.45 sec -2013-09-18 03:47:21,459 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 41.37 sec -2013-09-18 03:47:22,465 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.32 sec -2013-09-18 03:47:23,471 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.32 sec -MapReduce Total cumulative CPU time: 43 seconds 320 msec -Ended Job = job_201309172235_0368 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0369 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:47:26,930 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:47:27,935 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 03:47:28,941 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 03:47:29,947 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 03:47:30,952 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 03:47:31,956 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 03:47:32,961 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 03:47:33,966 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 03:47:34,971 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-18 03:47:35,977 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -2013-09-18 03:47:36,983 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -2013-09-18 03:47:37,988 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -MapReduce Total cumulative CPU time: 2 seconds 290 msec -Ended Job = job_201309172235_0369 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 43.32 sec HDFS Read: 118784021 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.29 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 610 msec -OK -Time taken: 39.979 seconds -hive> quit; - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1699@mturlrep13_201309180347_773881817.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0370 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:47:54,727 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:48:01,761 Stage-1 map = 66%, reduce = 0% -2013-09-18 03:48:03,778 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.99 sec -2013-09-18 03:48:04,785 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.99 sec -2013-09-18 03:48:05,792 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.99 sec -2013-09-18 03:48:06,799 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.99 sec -2013-09-18 03:48:07,804 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.99 sec -2013-09-18 03:48:08,811 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.99 sec -2013-09-18 03:48:09,818 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.99 sec -2013-09-18 03:48:10,824 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 38.99 sec -2013-09-18 03:48:11,833 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.36 sec -2013-09-18 03:48:12,840 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.36 sec -2013-09-18 03:48:13,847 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.36 sec -MapReduce Total cumulative CPU time: 43 seconds 360 msec -Ended Job = job_201309172235_0370 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0371 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:48:17,471 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:48:18,477 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 03:48:19,486 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 03:48:20,491 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 03:48:21,496 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 03:48:22,502 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 03:48:23,508 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 03:48:24,514 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 03:48:25,520 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-18 03:48:26,526 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -2013-09-18 03:48:27,532 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -2013-09-18 03:48:28,538 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -MapReduce Total cumulative CPU time: 2 seconds 230 msec -Ended Job = job_201309172235_0371 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 43.36 sec HDFS Read: 115339269 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.23 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 590 msec -OK -Time taken: 43.852 seconds -hive> quit; - -times: 2 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3593@mturlrep13_201309180348_533961775.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0372 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:48:38,968 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:48:46,007 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 18.26 sec -2013-09-18 03:48:47,015 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.04 sec -2013-09-18 03:48:48,023 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.04 sec -2013-09-18 03:48:49,029 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.04 sec -2013-09-18 03:48:50,035 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.04 sec -2013-09-18 03:48:51,040 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.04 sec -2013-09-18 03:48:52,047 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.04 sec -2013-09-18 03:48:53,053 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.04 sec -2013-09-18 03:48:54,062 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 39.06 sec -2013-09-18 03:48:55,069 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.09 sec -2013-09-18 03:48:56,076 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.09 sec -MapReduce Total cumulative CPU time: 41 seconds 90 msec -Ended Job = job_201309172235_0372 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0373 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:48:58,539 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:49:00,548 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 03:49:01,554 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 03:49:02,560 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 03:49:03,564 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 03:49:04,569 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 03:49:05,574 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 03:49:06,578 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-18 03:49:07,583 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.69 sec -2013-09-18 03:49:08,589 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -2013-09-18 03:49:09,594 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -2013-09-18 03:49:10,600 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.14 sec -MapReduce Total cumulative CPU time: 2 seconds 140 msec -Ended Job = job_201309172235_0373 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.09 sec HDFS Read: 115339269 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.14 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 230 msec -OK -Time taken: 40.178 seconds -hive> quit; - -times: 3 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5455@mturlrep13_201309180349_1828648291.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0374 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:49:21,008 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:49:28,046 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 8.89 sec -2013-09-18 03:49:29,054 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.98 sec -2013-09-18 03:49:30,062 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.98 sec -2013-09-18 03:49:31,067 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.98 sec -2013-09-18 03:49:32,073 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.98 sec -2013-09-18 03:49:33,079 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.98 sec -2013-09-18 03:49:34,084 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.98 sec -2013-09-18 03:49:35,090 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.98 sec -2013-09-18 03:49:36,096 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 37.98 sec -2013-09-18 03:49:37,104 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.56 sec -2013-09-18 03:49:38,109 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.56 sec -MapReduce Total cumulative CPU time: 42 seconds 560 msec -Ended Job = job_201309172235_0374 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0375 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:49:41,584 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:49:43,592 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:49:44,596 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:49:45,600 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:49:46,604 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:49:47,608 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:49:48,612 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:49:49,616 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:49:50,620 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -2013-09-18 03:49:51,625 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -2013-09-18 03:49:52,630 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -MapReduce Total cumulative CPU time: 2 seconds 100 msec -Ended Job = job_201309172235_0375 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.56 sec HDFS Read: 115339269 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.1 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 660 msec -OK -Time taken: 40.131 seconds -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7725@mturlrep13_201309180349_550066891.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0376 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:50:09,712 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:50:16,741 Stage-1 map = 44%, reduce = 0% -2013-09-18 03:50:18,758 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 34.83 sec -2013-09-18 03:50:19,766 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.89 sec -2013-09-18 03:50:20,774 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.89 sec -2013-09-18 03:50:21,781 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.89 sec -2013-09-18 03:50:22,787 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.89 sec -2013-09-18 03:50:23,793 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.89 sec -2013-09-18 03:50:24,799 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.89 sec -2013-09-18 03:50:25,805 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.89 sec -2013-09-18 03:50:26,813 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 41.94 sec -2013-09-18 03:50:27,820 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.02 sec -2013-09-18 03:50:28,827 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.02 sec -MapReduce Total cumulative CPU time: 44 seconds 20 msec -Ended Job = job_201309172235_0376 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0377 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:50:31,408 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:50:33,417 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:50:34,422 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:50:35,428 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:50:36,432 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:50:37,437 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:50:38,442 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:50:39,447 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.66 sec -2013-09-18 03:50:40,452 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.66 sec -2013-09-18 03:50:41,458 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -2013-09-18 03:50:42,463 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -2013-09-18 03:50:43,469 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.19 sec -MapReduce Total cumulative CPU time: 2 seconds 190 msec -Ended Job = job_201309172235_0377 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 44.02 sec HDFS Read: 118662691 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.19 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 46 seconds 210 msec -OK -Time taken: 44.493 seconds -hive> quit; - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10373@mturlrep13_201309180350_1849852052.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0378 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:50:53,794 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:51:00,831 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 9.24 sec -2013-09-18 03:51:01,838 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.73 sec -2013-09-18 03:51:02,846 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.73 sec -2013-09-18 03:51:03,852 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.73 sec -2013-09-18 03:51:04,858 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.73 sec -2013-09-18 03:51:05,865 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.73 sec -2013-09-18 03:51:06,871 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.73 sec -2013-09-18 03:51:07,876 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.73 sec -2013-09-18 03:51:08,883 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 40.86 sec -2013-09-18 03:51:09,890 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.85 sec -2013-09-18 03:51:10,897 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.85 sec -MapReduce Total cumulative CPU time: 42 seconds 850 msec -Ended Job = job_201309172235_0378 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0379 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:51:14,363 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:51:16,372 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:51:17,377 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:51:18,382 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:51:19,387 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:51:20,392 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:51:21,397 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:51:22,403 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:51:23,408 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -2013-09-18 03:51:24,414 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -2013-09-18 03:51:25,420 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -MapReduce Total cumulative CPU time: 2 seconds 160 msec -Ended Job = job_201309172235_0379 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.85 sec HDFS Read: 118662691 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.16 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 10 msec -OK -Time taken: 40.17 seconds -hive> quit; - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12355@mturlrep13_201309180351_396050441.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0380 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:51:34,723 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:51:42,768 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 19.49 sec -2013-09-18 03:51:43,777 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.68 sec -2013-09-18 03:51:44,785 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.68 sec -2013-09-18 03:51:45,791 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.68 sec -2013-09-18 03:51:46,798 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.68 sec -2013-09-18 03:51:47,804 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.68 sec -2013-09-18 03:51:48,810 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.68 sec -2013-09-18 03:51:49,818 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 40.68 sec -2013-09-18 03:51:50,825 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 40.68 sec -2013-09-18 03:51:51,834 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.72 sec -2013-09-18 03:51:52,841 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 44.72 sec -MapReduce Total cumulative CPU time: 44 seconds 720 msec -Ended Job = job_201309172235_0380 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0381 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:51:56,309 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:51:57,314 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:51:58,319 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:51:59,325 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:52:00,330 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:52:01,334 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:52:02,339 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:52:03,343 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:52:04,348 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:52:05,354 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -2013-09-18 03:52:06,360 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -2013-09-18 03:52:07,365 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -MapReduce Total cumulative CPU time: 2 seconds 160 msec -Ended Job = job_201309172235_0381 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 44.72 sec HDFS Read: 118662691 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.16 sec HDFS Read: 959 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 46 seconds 880 msec -OK -Time taken: 40.171 seconds -hive> quit; - -times: 1 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14305@mturlrep13_201309180352_966734049.txt -hive> SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; ; -FAILED: SemanticException [Error 10128]: Line 1:414 Not yet supported place for UDAF 'count' -hive> quit; - -times: 2 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14524@mturlrep13_201309180352_643113009.txt -hive> SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; ; -FAILED: SemanticException [Error 10128]: Line 1:414 Not yet supported place for UDAF 'count' -hive> quit; - -times: 3 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14731@mturlrep13_201309180352_231906667.txt -hive> SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*), if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') AS SRC FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, ''), URL ORDER BY count(*) DESC LIMIT 1000; ; -FAILED: SemanticException [Error 10128]: Line 1:414 Not yet supported place for UDAF 'count' -hive> quit; - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14942@mturlrep13_201309180352_606832963.txt -hive> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0382 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:52:47,876 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:52:54,907 Stage-1 map = 63%, reduce = 0% -2013-09-18 03:52:55,919 Stage-1 map = 72%, reduce = 0%, Cumulative CPU 9.33 sec -2013-09-18 03:52:56,927 Stage-1 map = 79%, reduce = 0%, Cumulative CPU 19.11 sec -2013-09-18 03:52:57,935 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.01 sec -2013-09-18 03:52:58,941 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.01 sec -2013-09-18 03:52:59,946 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.01 sec -2013-09-18 03:53:00,952 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.01 sec -2013-09-18 03:53:01,958 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.01 sec -2013-09-18 03:53:02,965 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 39.01 sec -2013-09-18 03:53:03,972 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 39.01 sec -2013-09-18 03:53:04,980 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.35 sec -2013-09-18 03:53:05,987 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.35 sec -MapReduce Total cumulative CPU time: 43 seconds 350 msec -Ended Job = job_201309172235_0382 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0383 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:53:09,467 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:53:11,476 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:53:12,482 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:53:13,487 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:53:14,491 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:53:15,496 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:53:16,513 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:53:17,519 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:53:18,524 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -2013-09-18 03:53:19,530 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -2013-09-18 03:53:20,536 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -MapReduce Total cumulative CPU time: 2 seconds 100 msec -Ended Job = job_201309172235_0383 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 43.35 sec HDFS Read: 148406904 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.1 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 450 msec -OK -Time taken: 43.025 seconds -hive> quit; - -times: 2 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16891@mturlrep13_201309180353_499012303.txt -hive> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0384 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:53:30,022 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:53:38,064 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 27.01 sec -2013-09-18 03:53:39,072 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.42 sec -2013-09-18 03:53:40,080 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.42 sec -2013-09-18 03:53:41,086 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.42 sec -2013-09-18 03:53:42,092 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.42 sec -2013-09-18 03:53:43,098 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.42 sec -2013-09-18 03:53:44,105 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.42 sec -2013-09-18 03:53:45,112 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.42 sec -2013-09-18 03:53:46,120 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.33 sec -2013-09-18 03:53:47,127 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.33 sec -2013-09-18 03:53:48,134 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.33 sec -MapReduce Total cumulative CPU time: 40 seconds 330 msec -Ended Job = job_201309172235_0384 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0385 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:53:50,608 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:53:52,616 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:53:53,622 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:53:54,627 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:53:55,631 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:53:56,636 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:53:57,641 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:53:58,646 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:53:59,652 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.65 sec -2013-09-18 03:54:00,658 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -2013-09-18 03:54:01,664 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -MapReduce Total cumulative CPU time: 2 seconds 160 msec -Ended Job = job_201309172235_0385 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.33 sec HDFS Read: 148406904 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.16 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 490 msec -OK -Time taken: 39.168 seconds -hive> quit; - -times: 3 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18843@mturlrep13_201309180354_379983066.txt -hive> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0386 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:54:10,812 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:54:18,851 Stage-1 map = 86%, reduce = 0%, Cumulative CPU 36.06 sec -2013-09-18 03:54:19,859 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.5 sec -2013-09-18 03:54:20,867 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.5 sec -2013-09-18 03:54:21,873 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.5 sec -2013-09-18 03:54:22,879 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.5 sec -2013-09-18 03:54:23,885 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.5 sec -2013-09-18 03:54:24,891 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 38.5 sec -2013-09-18 03:54:25,898 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.5 sec -2013-09-18 03:54:26,905 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.56 sec -2013-09-18 03:54:27,912 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.56 sec -2013-09-18 03:54:28,918 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.56 sec -MapReduce Total cumulative CPU time: 42 seconds 560 msec -Ended Job = job_201309172235_0386 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0387 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:54:32,443 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:54:33,449 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:54:34,455 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:54:35,460 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:54:36,465 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:54:37,470 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:54:38,475 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:54:39,479 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:54:40,484 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.65 sec -2013-09-18 03:54:41,490 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -2013-09-18 03:54:42,495 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -2013-09-18 03:54:43,500 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -MapReduce Total cumulative CPU time: 2 seconds 80 msec -Ended Job = job_201309172235_0387 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.56 sec HDFS Read: 148406904 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.08 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 640 msec -OK -Time taken: 40.042 seconds -hive> quit; - -times: 1 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20827@mturlrep13_201309180354_891781723.txt -hive> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0388 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:54:59,946 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:55:06,975 Stage-1 map = 56%, reduce = 0% -2013-09-18 03:55:08,992 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 17.58 sec -2013-09-18 03:55:09,999 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.11 sec -2013-09-18 03:55:11,007 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.11 sec -2013-09-18 03:55:12,014 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.11 sec -2013-09-18 03:55:13,020 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.11 sec -2013-09-18 03:55:14,027 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.11 sec -2013-09-18 03:55:15,033 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.11 sec -2013-09-18 03:55:16,039 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.11 sec -2013-09-18 03:55:17,047 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.2 sec -2013-09-18 03:55:18,053 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.2 sec -MapReduce Total cumulative CPU time: 41 seconds 200 msec -Ended Job = job_201309172235_0388 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0389 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:55:20,567 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:55:22,576 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:55:23,582 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:55:24,587 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:55:25,591 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:55:26,596 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:55:27,600 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:55:28,605 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:55:29,610 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.64 sec -2013-09-18 03:55:30,615 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -2013-09-18 03:55:31,620 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -2013-09-18 03:55:32,625 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -MapReduce Total cumulative CPU time: 2 seconds 100 msec -Ended Job = job_201309172235_0389 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.2 sec HDFS Read: 105631340 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.1 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 300 msec -OK -Time taken: 42.914 seconds -hive> quit; - -times: 2 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23548@mturlrep13_201309180355_61949059.txt -hive> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0390 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:55:41,925 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:55:49,963 Stage-1 map = 93%, reduce = 0%, Cumulative CPU 27.55 sec -2013-09-18 03:55:50,974 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.61 sec -2013-09-18 03:55:51,982 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.61 sec -2013-09-18 03:55:52,988 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.61 sec -2013-09-18 03:55:53,994 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.61 sec -2013-09-18 03:55:55,000 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.61 sec -2013-09-18 03:55:56,005 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.61 sec -2013-09-18 03:55:57,012 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.61 sec -2013-09-18 03:55:58,020 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 38.63 sec -2013-09-18 03:55:59,026 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.74 sec -2013-09-18 03:56:00,032 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.74 sec -MapReduce Total cumulative CPU time: 40 seconds 740 msec -Ended Job = job_201309172235_0390 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0391 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:56:03,570 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:56:05,578 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:56:06,583 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:56:07,588 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:56:08,593 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:56:09,598 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:56:10,603 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:56:11,608 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.67 sec -2013-09-18 03:56:12,614 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.67 sec -2013-09-18 03:56:13,620 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -2013-09-18 03:56:14,625 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.29 sec -MapReduce Total cumulative CPU time: 2 seconds 290 msec -Ended Job = job_201309172235_0391 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.74 sec HDFS Read: 105631340 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.29 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 30 msec -OK -Time taken: 40.129 seconds -hive> quit; - -times: 3 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25608@mturlrep13_201309180356_1104505823.txt -hive> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0392 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-18 03:56:24,831 Stage-1 map = 0%, reduce = 0% -2013-09-18 03:56:31,868 Stage-1 map = 97%, reduce = 0%, Cumulative CPU 26.64 sec -2013-09-18 03:56:32,876 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:56:33,883 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:56:34,889 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:56:35,895 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:56:36,901 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:56:37,907 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 36.09 sec -2013-09-18 03:56:38,914 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.09 sec -2013-09-18 03:56:39,922 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.24 sec -2013-09-18 03:56:40,930 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.24 sec -2013-09-18 03:56:41,935 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.24 sec -MapReduce Total cumulative CPU time: 40 seconds 240 msec -Ended Job = job_201309172235_0392 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309172235_0393 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-18 03:56:44,482 Stage-2 map = 0%, reduce = 0% -2013-09-18 03:56:46,491 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:56:47,497 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:56:48,502 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:56:49,507 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:56:50,512 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:56:51,516 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:56:52,521 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.64 sec -2013-09-18 03:56:53,527 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.64 sec -2013-09-18 03:56:54,532 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.0 sec -2013-09-18 03:56:55,541 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.0 sec -MapReduce Total cumulative CPU time: 2 seconds 0 msec -Ended Job = job_201309172235_0393 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.24 sec HDFS Read: 105631340 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.0 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 240 msec -OK -Time taken: 39.135 seconds -hive> quit; - -times: 1 -query: SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27654@mturlrep13_201309180357_1202049838.txt -hive> SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; ; -hive> quit; - -times: 2 -query: SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27867@mturlrep13_201309180357_2097495906.txt -hive> SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; ; -hive> quit; - -times: 3 -query: SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28078@mturlrep13_201309180357_1074846519.txt -hive> SELECT unix_timestamp(EventTime) - SECOND(EventTime), count(*) FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY m ORDER BY m; ; -hive> quit; -stop time: Ср. сент. 18 03:57:19 MSK 2013 diff --git a/benchmark/hive/log/log_10m/log_hits_10m b/benchmark/hive/log/log_10m/log_hits_10m deleted file mode 100644 index 9404479a423..00000000000 --- a/benchmark/hive/log/log_10m/log_hits_10m +++ /dev/null @@ -1,9306 +0,0 @@ -start time: Ср. сент. 11 18:04:18 MSK 2013 -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27940@mturlrep13_201309111804_333737265.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28397@mturlrep13_201309111804_2040079163.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0219 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:04:42,860 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:04:49,913 Stage-1 map = 4%, reduce = 0% -2013-09-11 18:04:52,925 Stage-1 map = 7%, reduce = 0% -2013-09-11 18:04:55,955 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:04:56,962 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:04:57,969 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:04:58,976 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:04:59,982 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:00,988 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:01,993 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:02,998 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:04,003 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:05,008 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:06,018 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:07,024 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:08,029 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:09,035 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:10,039 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:11,044 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 27.32 sec -2013-09-11 18:05:12,051 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 51.0 sec -2013-09-11 18:05:13,056 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.63 sec -2013-09-11 18:05:14,062 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.63 sec -2013-09-11 18:05:15,067 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.63 sec -2013-09-11 18:05:16,072 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.63 sec -2013-09-11 18:05:17,077 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.63 sec -2013-09-11 18:05:18,083 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.63 sec -2013-09-11 18:05:19,088 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.63 sec -2013-09-11 18:05:20,093 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:21,098 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:22,103 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:23,108 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:24,112 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:25,117 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:26,122 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:27,127 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:28,131 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:29,136 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:30,141 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:31,151 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:32,159 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:33,163 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:34,168 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:35,173 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:36,208 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:37,212 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:38,217 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:39,222 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 18:05:40,227 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 116.26 sec -2013-09-11 18:05:41,240 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.04 sec -2013-09-11 18:05:42,244 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.04 sec -2013-09-11 18:05:43,248 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.04 sec -2013-09-11 18:05:44,253 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.04 sec -2013-09-11 18:05:45,259 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 154.32 sec -2013-09-11 18:05:46,264 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 154.32 sec -2013-09-11 18:05:47,269 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 154.32 sec -MapReduce Total cumulative CPU time: 2 minutes 34 seconds 320 msec -Ended Job = job_201309101627_0219 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 154.32 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 34 seconds 320 msec -OK -10000000 -Time taken: 79.195 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31005@mturlrep13_201309111805_665303300.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31457@mturlrep13_201309111805_1086613.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0220 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:06:01,601 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:06:08,629 Stage-1 map = 7%, reduce = 0% -2013-09-11 18:06:11,641 Stage-1 map = 14%, reduce = 0% -2013-09-11 18:06:18,671 Stage-1 map = 22%, reduce = 0% -2013-09-11 18:06:21,683 Stage-1 map = 29%, reduce = 0% -2013-09-11 18:06:24,695 Stage-1 map = 36%, reduce = 0% -2013-09-11 18:06:27,707 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:06:28,718 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 34.66 sec -2013-09-11 18:06:29,725 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.93 sec -2013-09-11 18:06:30,731 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.93 sec -2013-09-11 18:06:31,736 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.93 sec -2013-09-11 18:06:32,742 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.93 sec -2013-09-11 18:06:33,747 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.93 sec -2013-09-11 18:06:34,752 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.93 sec -2013-09-11 18:06:35,757 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 78.93 sec -2013-09-11 18:06:36,762 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:37,767 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:38,772 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:39,778 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:40,783 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:41,787 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:42,793 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:43,798 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:44,803 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:45,808 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:46,867 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:47,872 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:48,877 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:49,888 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:50,893 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:51,916 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:52,921 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 78.93 sec -2013-09-11 18:06:53,925 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 112.4 sec -2013-09-11 18:06:54,931 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 112.4 sec -2013-09-11 18:06:55,940 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 154.14 sec -2013-09-11 18:06:56,945 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 154.14 sec -2013-09-11 18:06:57,950 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 157.93 sec -2013-09-11 18:06:58,954 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 157.93 sec -2013-09-11 18:06:59,959 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 157.93 sec -2013-09-11 18:07:00,963 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 157.93 sec -2013-09-11 18:07:01,970 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 159.3 sec -2013-09-11 18:07:02,975 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 159.3 sec -2013-09-11 18:07:03,987 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 159.3 sec -MapReduce Total cumulative CPU time: 2 minutes 39 seconds 300 msec -Ended Job = job_201309101627_0220 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 159.3 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 39 seconds 300 msec -OK -10000000 -Time taken: 71.026 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_836@mturlrep13_201309111807_561551889.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT count(*) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1313@mturlrep13_201309111807_1846613812.txt -hive> SELECT count(*) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0221 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:07:18,418 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:07:25,501 Stage-1 map = 7%, reduce = 0% -2013-09-11 18:07:31,523 Stage-1 map = 14%, reduce = 0% -2013-09-11 18:07:34,537 Stage-1 map = 22%, reduce = 0% -2013-09-11 18:07:37,547 Stage-1 map = 25%, reduce = 0% -2013-09-11 18:07:38,551 Stage-1 map = 29%, reduce = 0% -2013-09-11 18:07:40,559 Stage-1 map = 32%, reduce = 0% -2013-09-11 18:07:41,564 Stage-1 map = 36%, reduce = 0% -2013-09-11 18:07:43,571 Stage-1 map = 39%, reduce = 0% -2013-09-11 18:07:47,591 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 85.17 sec -2013-09-11 18:07:48,598 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 85.17 sec -2013-09-11 18:07:49,609 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 85.17 sec -2013-09-11 18:07:50,615 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 85.17 sec -2013-09-11 18:07:51,620 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 85.17 sec -2013-09-11 18:07:52,625 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 85.17 sec -2013-09-11 18:07:53,630 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 85.17 sec -2013-09-11 18:07:54,635 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 85.17 sec -2013-09-11 18:07:55,640 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:07:56,660 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:07:57,668 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:07:58,672 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:07:59,676 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:00,681 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:01,685 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:02,690 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:03,694 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:04,699 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:05,703 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:06,708 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:07,712 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:08,716 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:09,720 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:10,725 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:11,729 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 107.69 sec -2013-09-11 18:08:12,734 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 128.12 sec -2013-09-11 18:08:13,739 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 128.12 sec -2013-09-11 18:08:14,743 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.45 sec -2013-09-11 18:08:15,747 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.45 sec -2013-09-11 18:08:16,752 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.45 sec -2013-09-11 18:08:17,859 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 152.45 sec -2013-09-11 18:08:18,863 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 152.45 sec -2013-09-11 18:08:19,868 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 152.45 sec -2013-09-11 18:08:20,874 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 154.14 sec -2013-09-11 18:08:21,879 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 154.14 sec -MapReduce Total cumulative CPU time: 2 minutes 34 seconds 140 msec -Ended Job = job_201309101627_0221 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 154.14 sec HDFS Read: 1082943442 HDFS Write: 9 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 34 seconds 140 msec -OK -10000000 -Time taken: 71.919 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3224@mturlrep13_201309111808_410663518.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3703@mturlrep13_201309111808_1060047951.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0222 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:08:42,497 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:08:47,524 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.16 sec -2013-09-11 18:08:48,531 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.16 sec -2013-09-11 18:08:49,539 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.16 sec -2013-09-11 18:08:50,546 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.16 sec -2013-09-11 18:08:51,551 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.16 sec -2013-09-11 18:08:52,557 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.16 sec -2013-09-11 18:08:53,563 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.25 sec -2013-09-11 18:08:54,568 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.25 sec -2013-09-11 18:08:55,574 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 23.94 sec -2013-09-11 18:08:56,582 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.2 sec -2013-09-11 18:08:57,609 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.2 sec -MapReduce Total cumulative CPU time: 25 seconds 200 msec -Ended Job = job_201309101627_0222 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 25.2 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 25 seconds 200 msec -OK -171127 -Time taken: 24.984 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5325@mturlrep13_201309111808_1536943547.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5808@mturlrep13_201309111809_1398241978.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0223 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:09:11,922 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:09:15,946 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.33 sec -2013-09-11 18:09:16,954 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.33 sec -2013-09-11 18:09:17,965 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.33 sec -2013-09-11 18:09:18,971 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.33 sec -2013-09-11 18:09:19,977 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.33 sec -2013-09-11 18:09:20,983 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 17.02 sec -2013-09-11 18:09:21,989 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.14 sec -2013-09-11 18:09:22,995 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.14 sec -2013-09-11 18:09:24,000 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 23.14 sec -2013-09-11 18:09:25,007 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.1 sec -2013-09-11 18:09:26,014 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.1 sec -MapReduce Total cumulative CPU time: 25 seconds 100 msec -Ended Job = job_201309101627_0223 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 25.1 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 25 seconds 100 msec -OK -171127 -Time taken: 22.815 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7174@mturlrep13_201309111809_1436708878.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7626@mturlrep13_201309111809_1436982971.txt -hive> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0224 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:09:40,309 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:09:44,332 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 6.0 sec -2013-09-11 18:09:45,340 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.79 sec -2013-09-11 18:09:46,348 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.79 sec -2013-09-11 18:09:47,354 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.79 sec -2013-09-11 18:09:48,360 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.79 sec -2013-09-11 18:09:49,367 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.73 sec -2013-09-11 18:09:50,372 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.73 sec -2013-09-11 18:09:51,378 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.73 sec -2013-09-11 18:09:52,384 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 23.73 sec -2013-09-11 18:09:53,392 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.56 sec -2013-09-11 18:09:54,397 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 25.56 sec -MapReduce Total cumulative CPU time: 25 seconds 560 msec -Ended Job = job_201309101627_0224 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 25.56 sec HDFS Read: 907716 HDFS Write: 7 SUCCESS -Total MapReduce CPU Time Spent: 25 seconds 560 msec -OK -171127 -Time taken: 22.71 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9005@mturlrep13_201309111810_1951461595.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9573@mturlrep13_201309111810_1888842341.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0225 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:10:15,477 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:10:23,514 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.01 sec -2013-09-11 18:10:24,522 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.01 sec -2013-09-11 18:10:25,528 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.01 sec -2013-09-11 18:10:26,534 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.01 sec -2013-09-11 18:10:27,540 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.01 sec -2013-09-11 18:10:28,545 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.01 sec -2013-09-11 18:10:29,552 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.01 sec -2013-09-11 18:10:30,558 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 33.01 sec -2013-09-11 18:10:31,563 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 33.01 sec -2013-09-11 18:10:32,568 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 33.01 sec -2013-09-11 18:10:33,573 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 33.01 sec -2013-09-11 18:10:34,578 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 33.01 sec -2013-09-11 18:10:35,583 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 33.01 sec -2013-09-11 18:10:36,590 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 35.1 sec -2013-09-11 18:10:37,595 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 35.1 sec -MapReduce Total cumulative CPU time: 35 seconds 100 msec -Ended Job = job_201309101627_0225 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 35.1 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 35 seconds 100 msec -OK -Time taken: 32.289 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11687@mturlrep13_201309111810_1411263650.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12147@mturlrep13_201309111810_146667107.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0226 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:10:51,464 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:10:58,498 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-11 18:10:59,506 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-11 18:11:00,512 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-11 18:11:01,518 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-11 18:11:02,523 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-11 18:11:03,529 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.22 sec -2013-09-11 18:11:04,535 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 23.49 sec -2013-09-11 18:11:05,540 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 31.49 sec -2013-09-11 18:11:06,545 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.49 sec -2013-09-11 18:11:07,550 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.49 sec -2013-09-11 18:11:08,555 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.49 sec -2013-09-11 18:11:09,560 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.49 sec -2013-09-11 18:11:10,565 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.49 sec -2013-09-11 18:11:11,572 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.46 sec -2013-09-11 18:11:12,577 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.46 sec -2013-09-11 18:11:13,582 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 33.46 sec -MapReduce Total cumulative CPU time: 33 seconds 460 msec -Ended Job = job_201309101627_0226 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 33.46 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 33 seconds 460 msec -OK -Time taken: 30.316 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13663@mturlrep13_201309111811_609536671.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14110@mturlrep13_201309111811_1164319988.txt -hive> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0227 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:11:27,691 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:11:33,721 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.15 sec -2013-09-11 18:11:34,728 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.15 sec -2013-09-11 18:11:35,735 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.15 sec -2013-09-11 18:11:36,740 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.15 sec -2013-09-11 18:11:37,746 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.15 sec -2013-09-11 18:11:38,751 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.15 sec -2013-09-11 18:11:39,757 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.15 sec -2013-09-11 18:11:40,762 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 32.31 sec -2013-09-11 18:11:41,768 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.31 sec -2013-09-11 18:11:42,773 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.31 sec -2013-09-11 18:11:43,777 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.31 sec -2013-09-11 18:11:44,781 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.31 sec -2013-09-11 18:11:45,786 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.31 sec -2013-09-11 18:11:46,793 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.38 sec -2013-09-11 18:11:47,799 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.38 sec -2013-09-11 18:11:48,804 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 34.38 sec -MapReduce Total cumulative CPU time: 34 seconds 380 msec -Ended Job = job_201309101627_0227 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 34.38 sec HDFS Read: 8109219 HDFS Write: 30 SUCCESS -Total MapReduce CPU Time Spent: 34 seconds 380 msec -OK -Time taken: 29.584 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15640@mturlrep13_201309111811_1787412572.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16152@mturlrep13_201309111812_1844919592.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0228 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:12:10,652 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:12:17,682 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.4 sec -2013-09-11 18:12:18,689 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.4 sec -2013-09-11 18:12:19,695 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.4 sec -2013-09-11 18:12:20,701 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.4 sec -2013-09-11 18:12:21,706 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.4 sec -2013-09-11 18:12:22,711 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.4 sec -2013-09-11 18:12:23,717 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.4 sec -2013-09-11 18:12:24,722 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.64 sec -2013-09-11 18:12:25,728 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.64 sec -2013-09-11 18:12:26,732 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.64 sec -2013-09-11 18:12:27,737 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.64 sec -2013-09-11 18:12:28,741 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.64 sec -2013-09-11 18:12:29,745 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.64 sec -2013-09-11 18:12:30,752 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.61 sec -2013-09-11 18:12:31,758 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.61 sec -2013-09-11 18:12:32,762 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.61 sec -MapReduce Total cumulative CPU time: 32 seconds 610 msec -Ended Job = job_201309101627_0228 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 32.61 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 32 seconds 610 msec -OK --4662894107982093709 -Time taken: 32.018 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17658@mturlrep13_201309111812_1917152659.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18081@mturlrep13_201309111812_969986372.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0229 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:12:46,636 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:12:52,673 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 7.62 sec -2013-09-11 18:12:53,681 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.58 sec -2013-09-11 18:12:54,687 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.58 sec -2013-09-11 18:12:55,692 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.58 sec -2013-09-11 18:12:56,698 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.58 sec -2013-09-11 18:12:57,704 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.58 sec -2013-09-11 18:12:58,710 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 22.97 sec -2013-09-11 18:12:59,715 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.12 sec -2013-09-11 18:13:00,721 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.12 sec -2013-09-11 18:13:01,726 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.12 sec -2013-09-11 18:13:02,731 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.12 sec -2013-09-11 18:13:03,736 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.12 sec -2013-09-11 18:13:04,740 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.12 sec -2013-09-11 18:13:05,747 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.15 sec -2013-09-11 18:13:06,752 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.15 sec -2013-09-11 18:13:07,757 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 32.15 sec -MapReduce Total cumulative CPU time: 32 seconds 150 msec -Ended Job = job_201309101627_0229 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 32.15 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 32 seconds 150 msec -OK --4662894107982093709 -Time taken: 29.424 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19726@mturlrep13_201309111813_601476946.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT sum(UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20163@mturlrep13_201309111813_458490060.txt -hive> SELECT sum(UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0230 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:13:22,398 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:13:28,444 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:13:29,452 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:13:30,459 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:13:31,464 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:13:32,470 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:13:33,476 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:13:34,481 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 29.52 sec -2013-09-11 18:13:35,486 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 29.52 sec -2013-09-11 18:13:36,493 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.4 sec -2013-09-11 18:13:37,500 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 31.4 sec -MapReduce Total cumulative CPU time: 31 seconds 400 msec -Ended Job = job_201309101627_0230 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 31.4 sec HDFS Read: 57312623 HDFS Write: 21 SUCCESS -Total MapReduce CPU Time Spent: 31 seconds 400 msec -OK --4662894107982093709 -Time taken: 23.314 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21535@mturlrep13_201309111813_1532287977.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22021@mturlrep13_201309111813_1807628976.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0231 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:13:58,394 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:14:05,419 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:14:08,436 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.36 sec -2013-09-11 18:14:09,442 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.36 sec -2013-09-11 18:14:10,449 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.36 sec -2013-09-11 18:14:11,454 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.36 sec -2013-09-11 18:14:12,460 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.36 sec -2013-09-11 18:14:13,466 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.36 sec -2013-09-11 18:14:14,471 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.36 sec -2013-09-11 18:14:15,518 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.36 sec -2013-09-11 18:14:16,523 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.36 sec -2013-09-11 18:14:17,529 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.36 sec -2013-09-11 18:14:18,535 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 40.6 sec -2013-09-11 18:14:19,551 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.46 sec -2013-09-11 18:14:20,556 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.46 sec -2013-09-11 18:14:21,561 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.46 sec -2013-09-11 18:14:22,566 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.46 sec -2013-09-11 18:14:23,572 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.46 sec -2013-09-11 18:14:24,576 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.46 sec -2013-09-11 18:14:25,581 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 55.46 sec -2013-09-11 18:14:26,589 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.41 sec -2013-09-11 18:14:27,646 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.41 sec -MapReduce Total cumulative CPU time: 1 minutes 3 seconds 410 msec -Ended Job = job_201309101627_0231 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 63.41 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 3 seconds 410 msec -OK -2037258 -Time taken: 39.018 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23547@mturlrep13_201309111814_32240262.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24000@mturlrep13_201309111814_899622484.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0232 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:14:41,528 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:14:48,560 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:14:50,573 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 14.3 sec -2013-09-11 18:14:51,581 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.07 sec -2013-09-11 18:14:52,588 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.07 sec -2013-09-11 18:14:53,594 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.07 sec -2013-09-11 18:14:54,599 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.07 sec -2013-09-11 18:14:55,606 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.07 sec -2013-09-11 18:14:56,613 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.07 sec -2013-09-11 18:14:57,619 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.07 sec -2013-09-11 18:14:58,625 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.07 sec -2013-09-11 18:14:59,630 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.07 sec -2013-09-11 18:15:00,635 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 18:15:01,640 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 18:15:02,645 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 18:15:03,651 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 18:15:04,656 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 18:15:05,661 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 18:15:06,667 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 18:15:07,672 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 55.82 sec -2013-09-11 18:15:08,679 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.98 sec -2013-09-11 18:15:09,685 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.98 sec -2013-09-11 18:15:10,690 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.98 sec -MapReduce Total cumulative CPU time: 1 minutes 3 seconds 980 msec -Ended Job = job_201309101627_0232 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 63.98 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 3 seconds 980 msec -OK -2037258 -Time taken: 37.449 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25822@mturlrep13_201309111815_360131093.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26668@mturlrep13_201309111815_984788128.txt -hive> SELECT count(DISTINCT UserID) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0233 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:15:23,620 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:15:31,655 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:15:33,669 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.26 sec -2013-09-11 18:15:34,676 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.26 sec -2013-09-11 18:15:35,683 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.26 sec -2013-09-11 18:15:36,689 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.26 sec -2013-09-11 18:15:37,695 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.26 sec -2013-09-11 18:15:38,702 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.26 sec -2013-09-11 18:15:39,708 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.26 sec -2013-09-11 18:15:40,713 Stage-1 map = 72%, reduce = 17%, Cumulative CPU 24.26 sec -2013-09-11 18:15:41,718 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 24.26 sec -2013-09-11 18:15:42,723 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 36.34 sec -2013-09-11 18:15:43,727 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.93 sec -2013-09-11 18:15:44,732 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.93 sec -2013-09-11 18:15:45,737 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.93 sec -2013-09-11 18:15:46,741 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.93 sec -2013-09-11 18:15:47,746 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.93 sec -2013-09-11 18:15:48,751 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.93 sec -2013-09-11 18:15:49,755 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 49.93 sec -2013-09-11 18:15:50,762 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.65 sec -2013-09-11 18:15:51,767 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.65 sec -2013-09-11 18:15:52,773 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.65 sec -MapReduce Total cumulative CPU time: 57 seconds 650 msec -Ended Job = job_201309101627_0233 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 57.65 sec HDFS Read: 57312623 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 57 seconds 650 msec -OK -2037258 -Time taken: 36.433 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28232@mturlrep13_201309111815_1215288015.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28709@mturlrep13_201309111816_1499620624.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0234 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:16:13,939 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:16:20,965 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:16:22,978 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.31 sec -2013-09-11 18:16:23,985 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.31 sec -2013-09-11 18:16:24,992 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.31 sec -2013-09-11 18:16:25,998 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.31 sec -2013-09-11 18:16:27,004 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.31 sec -2013-09-11 18:16:28,010 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.31 sec -2013-09-11 18:16:29,016 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.31 sec -2013-09-11 18:16:30,021 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 22.31 sec -2013-09-11 18:16:31,026 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 32.71 sec -2013-09-11 18:16:32,031 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.17 sec -2013-09-11 18:16:33,035 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.17 sec -2013-09-11 18:16:34,040 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.17 sec -2013-09-11 18:16:35,045 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.17 sec -2013-09-11 18:16:36,050 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.17 sec -2013-09-11 18:16:37,055 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.17 sec -2013-09-11 18:16:38,060 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.17 sec -2013-09-11 18:16:39,065 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 44.17 sec -2013-09-11 18:16:40,073 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 51.78 sec -2013-09-11 18:16:41,078 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 51.78 sec -2013-09-11 18:16:42,083 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 51.78 sec -MapReduce Total cumulative CPU time: 51 seconds 780 msec -Ended Job = job_201309101627_0234 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 51.78 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 51 seconds 780 msec -OK -1110413 -Time taken: 38.322 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30252@mturlrep13_201309111816_93736906.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30689@mturlrep13_201309111816_96459731.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0235 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:16:55,164 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:17:03,202 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:17:04,212 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-11 18:17:05,219 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-11 18:17:06,226 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-11 18:17:07,232 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-11 18:17:08,237 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-11 18:17:09,243 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-11 18:17:10,250 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.55 sec -2013-09-11 18:17:11,255 Stage-1 map = 72%, reduce = 17%, Cumulative CPU 22.55 sec -2013-09-11 18:17:12,261 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.9 sec -2013-09-11 18:17:13,266 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.9 sec -2013-09-11 18:17:14,271 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.9 sec -2013-09-11 18:17:15,275 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.9 sec -2013-09-11 18:17:16,280 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.9 sec -2013-09-11 18:17:17,285 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.9 sec -2013-09-11 18:17:18,290 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.9 sec -2013-09-11 18:17:19,296 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.9 sec -2013-09-11 18:17:20,301 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 44.9 sec -2013-09-11 18:17:21,309 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.41 sec -2013-09-11 18:17:22,314 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.41 sec -MapReduce Total cumulative CPU time: 52 seconds 410 msec -Ended Job = job_201309101627_0235 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 52.41 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 52 seconds 410 msec -OK -1110413 -Time taken: 34.532 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32218@mturlrep13_201309111817_1630955863.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32645@mturlrep13_201309111817_1064457391.txt -hive> SELECT count(DISTINCT SearchPhrase) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0236 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:17:35,486 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:17:43,516 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:17:44,526 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.45 sec -2013-09-11 18:17:45,533 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.45 sec -2013-09-11 18:17:46,539 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.45 sec -2013-09-11 18:17:47,545 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.45 sec -2013-09-11 18:17:48,550 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.45 sec -2013-09-11 18:17:49,556 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.45 sec -2013-09-11 18:17:50,563 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.45 sec -2013-09-11 18:17:51,568 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 21.45 sec -2013-09-11 18:17:52,573 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.27 sec -2013-09-11 18:17:53,578 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.27 sec -2013-09-11 18:17:54,583 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.27 sec -2013-09-11 18:17:55,588 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.27 sec -2013-09-11 18:17:56,593 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.2 sec -2013-09-11 18:17:57,598 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.2 sec -2013-09-11 18:17:58,604 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.2 sec -2013-09-11 18:17:59,609 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 44.2 sec -2013-09-11 18:18:00,614 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 44.2 sec -2013-09-11 18:18:01,621 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.93 sec -2013-09-11 18:18:02,626 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 50.93 sec -MapReduce Total cumulative CPU time: 50 seconds 930 msec -Ended Job = job_201309101627_0236 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 50.93 sec HDFS Read: 27820105 HDFS Write: 8 SUCCESS -Total MapReduce CPU Time Spent: 50 seconds 930 msec -OK -1110413 -Time taken: 34.523 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1798@mturlrep13_201309111818_1042447198.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2299@mturlrep13_201309111818_765489992.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0237 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:18:24,738 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:18:30,765 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.26 sec -2013-09-11 18:18:31,772 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.26 sec -2013-09-11 18:18:32,778 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.26 sec -2013-09-11 18:18:33,784 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.26 sec -2013-09-11 18:18:34,789 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.26 sec -2013-09-11 18:18:35,795 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.26 sec -2013-09-11 18:18:36,801 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.81 sec -2013-09-11 18:18:37,806 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 27.81 sec -2013-09-11 18:18:38,811 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 27.81 sec -2013-09-11 18:18:39,816 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 27.81 sec -2013-09-11 18:18:40,821 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 27.81 sec -2013-09-11 18:18:41,826 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 27.81 sec -2013-09-11 18:18:42,832 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 27.81 sec -2013-09-11 18:18:43,839 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.81 sec -2013-09-11 18:18:44,844 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.81 sec -2013-09-11 18:18:45,849 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 29.81 sec -MapReduce Total cumulative CPU time: 29 seconds 810 msec -Ended Job = job_201309101627_0237 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 29.81 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 29 seconds 810 msec -OK -Time taken: 31.306 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3746@mturlrep13_201309111818_1331650833.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4171@mturlrep13_201309111818_897231302.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0238 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:18:59,980 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:19:05,006 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.74 sec -2013-09-11 18:19:06,013 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.74 sec -2013-09-11 18:19:07,020 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.74 sec -2013-09-11 18:19:08,025 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.74 sec -2013-09-11 18:19:09,030 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.74 sec -2013-09-11 18:19:10,036 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.74 sec -2013-09-11 18:19:11,041 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.68 sec -2013-09-11 18:19:12,046 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.68 sec -2013-09-11 18:19:13,054 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 26.68 sec -2013-09-11 18:19:14,061 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.62 sec -2013-09-11 18:19:15,068 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.62 sec -MapReduce Total cumulative CPU time: 28 seconds 620 msec -Ended Job = job_201309101627_0238 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 28.62 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 28 seconds 620 msec -OK -Time taken: 23.519 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5847@mturlrep13_201309111819_1092177982.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6293@mturlrep13_201309111819_874118653.txt -hive> SELECT min(EventDate), max(EventDate) FROM hits_10m;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0239 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 18:19:28,271 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:19:34,299 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.64 sec -2013-09-11 18:19:35,306 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.64 sec -2013-09-11 18:19:36,314 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.64 sec -2013-09-11 18:19:37,319 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.64 sec -2013-09-11 18:19:38,324 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.64 sec -2013-09-11 18:19:39,330 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 13.64 sec -2013-09-11 18:19:40,336 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.97 sec -2013-09-11 18:19:41,341 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 26.97 sec -2013-09-11 18:19:42,347 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 26.97 sec -2013-09-11 18:19:43,354 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.77 sec -2013-09-11 18:19:44,360 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.77 sec -MapReduce Total cumulative CPU time: 28 seconds 770 msec -Ended Job = job_201309101627_0239 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 28.77 sec HDFS Read: 597016 HDFS Write: 6 SUCCESS -Total MapReduce CPU Time Spent: 28 seconds 770 msec -OK -Time taken: 23.402 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7692@mturlrep13_201309111819_276347275.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8135@mturlrep13_201309111819_385716024.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0240 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:20:04,638 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:20:10,670 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.51 sec -2013-09-11 18:20:11,678 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.51 sec -2013-09-11 18:20:12,685 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.51 sec -2013-09-11 18:20:13,692 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.51 sec -2013-09-11 18:20:14,698 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.51 sec -2013-09-11 18:20:15,704 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.51 sec -2013-09-11 18:20:16,710 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 24.33 sec -2013-09-11 18:20:17,716 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 24.33 sec -2013-09-11 18:20:18,724 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.27 sec -2013-09-11 18:20:19,731 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.27 sec -2013-09-11 18:20:20,737 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 28.27 sec -MapReduce Total cumulative CPU time: 28 seconds 270 msec -Ended Job = job_201309101627_0240 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0241 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:20:24,339 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:20:25,345 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 18:20:26,351 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 18:20:27,356 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 18:20:28,361 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 18:20:29,366 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 18:20:30,371 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 18:20:31,376 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 18:20:32,391 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 18:20:33,396 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.18 sec -2013-09-11 18:20:34,401 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.18 sec -2013-09-11 18:20:35,407 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.18 sec -MapReduce Total cumulative CPU time: 2 seconds 180 msec -Ended Job = job_201309101627_0241 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 28.27 sec HDFS Read: 907716 HDFS Write: 384 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.18 sec HDFS Read: 1153 HDFS Write: 60 SUCCESS -Total MapReduce CPU Time Spent: 30 seconds 450 msec -OK -Time taken: 40.897 seconds, Fetched: 9 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10933@mturlrep13_201309111820_1327597176.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11362@mturlrep13_201309111820_1775916404.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0242 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:20:50,332 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:20:54,363 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 5.92 sec -2013-09-11 18:20:55,371 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.99 sec -2013-09-11 18:20:56,378 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.99 sec -2013-09-11 18:20:57,385 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.99 sec -2013-09-11 18:20:58,392 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.99 sec -2013-09-11 18:20:59,398 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.48 sec -2013-09-11 18:21:00,404 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.48 sec -2013-09-11 18:21:01,410 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.48 sec -2013-09-11 18:21:02,418 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 25.33 sec -2013-09-11 18:21:03,425 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.22 sec -2013-09-11 18:21:04,432 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.22 sec -MapReduce Total cumulative CPU time: 27 seconds 220 msec -Ended Job = job_201309101627_0242 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0243 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:21:06,978 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:21:08,988 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:09,993 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:10,998 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:12,002 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:13,019 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:14,024 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:15,029 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:16,035 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.77 sec -2013-09-11 18:21:17,040 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -2013-09-11 18:21:18,046 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -2013-09-11 18:21:19,051 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.16 sec -MapReduce Total cumulative CPU time: 2 seconds 160 msec -Ended Job = job_201309101627_0243 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 27.22 sec HDFS Read: 907716 HDFS Write: 384 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.16 sec HDFS Read: 1153 HDFS Write: 60 SUCCESS -Total MapReduce CPU Time Spent: 29 seconds 380 msec -OK -Time taken: 36.981 seconds, Fetched: 9 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13448@mturlrep13_201309111821_1010315846.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13877@mturlrep13_201309111821_1055383868.txt -hive> SELECT AdvEngineID, count(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c DESC;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0244 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:21:32,304 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:21:38,338 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.77 sec -2013-09-11 18:21:39,346 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.77 sec -2013-09-11 18:21:40,353 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.77 sec -2013-09-11 18:21:41,360 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 11.77 sec -2013-09-11 18:21:42,366 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 17.34 sec -2013-09-11 18:21:43,373 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.62 sec -2013-09-11 18:21:44,379 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 23.62 sec -2013-09-11 18:21:45,385 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 23.62 sec -2013-09-11 18:21:46,393 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.25 sec -2013-09-11 18:21:47,400 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 27.25 sec -MapReduce Total cumulative CPU time: 27 seconds 250 msec -Ended Job = job_201309101627_0244 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0245 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:21:50,923 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:21:52,932 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:53,938 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:54,943 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:55,948 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:56,954 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:57,960 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:58,964 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 18:21:59,970 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -2013-09-11 18:22:00,976 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -2013-09-11 18:22:01,982 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -MapReduce Total cumulative CPU time: 2 seconds 130 msec -Ended Job = job_201309101627_0245 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 27.25 sec HDFS Read: 907716 HDFS Write: 384 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.13 sec HDFS Read: 1153 HDFS Write: 60 SUCCESS -Total MapReduce CPU Time Spent: 29 seconds 380 msec -OK -Time taken: 37.244 seconds, Fetched: 9 row(s) -hive> quit; --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15957@mturlrep13_201309111822_240108876.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16471@mturlrep13_201309111822_1455089061.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0246 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:22:22,925 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:22:29,950 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:22:32,967 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 14.49 sec -2013-09-11 18:22:33,975 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.57 sec -2013-09-11 18:22:34,984 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.57 sec -2013-09-11 18:22:35,990 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.57 sec -2013-09-11 18:22:36,996 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.57 sec -2013-09-11 18:22:38,003 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.57 sec -2013-09-11 18:22:39,008 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.57 sec -2013-09-11 18:22:40,015 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.57 sec -2013-09-11 18:22:41,021 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.57 sec -2013-09-11 18:22:42,026 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.57 sec -2013-09-11 18:22:43,032 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.57 sec -2013-09-11 18:22:44,038 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 18:22:45,043 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 18:22:46,049 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 18:22:47,055 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 18:22:48,060 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 18:22:49,067 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 63.29 sec -2013-09-11 18:22:50,073 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 63.29 sec -2013-09-11 18:22:51,079 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.54 sec -2013-09-11 18:22:52,084 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.54 sec -MapReduce Total cumulative CPU time: 1 minutes 10 seconds 540 msec -Ended Job = job_201309101627_0246 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0247 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:22:54,572 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:22:56,582 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:22:57,589 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:22:58,594 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:22:59,600 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:23:00,605 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:23:01,610 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:23:02,688 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:23:03,693 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:23:04,698 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.16 sec -2013-09-11 18:23:05,704 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.16 sec -2013-09-11 18:23:06,709 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.16 sec -MapReduce Total cumulative CPU time: 3 seconds 160 msec -Ended Job = job_201309101627_0247 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 70.54 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.16 sec HDFS Read: 100911 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 13 seconds 700 msec -OK -Time taken: 53.83 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18961@mturlrep13_201309111823_617403759.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19391@mturlrep13_201309111823_22985485.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0248 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:23:19,790 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:23:27,823 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:23:29,837 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 14.01 sec -2013-09-11 18:23:30,846 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.85 sec -2013-09-11 18:23:31,853 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.85 sec -2013-09-11 18:23:32,859 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.85 sec -2013-09-11 18:23:33,865 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.85 sec -2013-09-11 18:23:34,872 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.85 sec -2013-09-11 18:23:35,878 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.85 sec -2013-09-11 18:23:36,884 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.85 sec -2013-09-11 18:23:37,890 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.85 sec -2013-09-11 18:23:38,896 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.85 sec -2013-09-11 18:23:39,902 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 43.08 sec -2013-09-11 18:23:40,908 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 18:23:41,913 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 18:23:42,919 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 18:23:43,925 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 18:23:44,930 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 18:23:45,937 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 63.27 sec -2013-09-11 18:23:46,943 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 63.27 sec -2013-09-11 18:23:47,949 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.4 sec -2013-09-11 18:23:48,954 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.4 sec -MapReduce Total cumulative CPU time: 1 minutes 10 seconds 400 msec -Ended Job = job_201309101627_0248 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0249 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:23:51,398 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:23:54,410 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.63 sec -2013-09-11 18:23:55,416 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.63 sec -2013-09-11 18:23:56,421 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.63 sec -2013-09-11 18:23:57,427 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.63 sec -2013-09-11 18:23:58,432 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.63 sec -2013-09-11 18:23:59,438 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.63 sec -2013-09-11 18:24:00,443 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.63 sec -2013-09-11 18:24:01,464 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.63 sec -2013-09-11 18:24:02,470 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.23 sec -2013-09-11 18:24:03,476 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.23 sec -MapReduce Total cumulative CPU time: 3 seconds 230 msec -Ended Job = job_201309101627_0249 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 70.4 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.23 sec HDFS Read: 100911 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 13 seconds 630 msec -OK -Time taken: 51.073 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21591@mturlrep13_201309111824_608372135.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22042@mturlrep13_201309111824_581555967.txt -hive> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0250 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:24:16,471 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:24:24,501 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:24:26,515 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 14.52 sec -2013-09-11 18:24:27,522 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.86 sec -2013-09-11 18:24:28,532 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.86 sec -2013-09-11 18:24:29,538 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.86 sec -2013-09-11 18:24:30,544 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.86 sec -2013-09-11 18:24:31,551 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.86 sec -2013-09-11 18:24:32,557 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.86 sec -2013-09-11 18:24:33,563 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 28.86 sec -2013-09-11 18:24:34,568 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.86 sec -2013-09-11 18:24:35,573 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.86 sec -2013-09-11 18:24:36,578 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 28.86 sec -2013-09-11 18:24:37,584 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 58.7 sec -2013-09-11 18:24:38,589 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 58.7 sec -2013-09-11 18:24:39,595 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 58.7 sec -2013-09-11 18:24:40,600 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 58.7 sec -2013-09-11 18:24:41,605 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 58.7 sec -2013-09-11 18:24:42,613 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 64.21 sec -2013-09-11 18:24:43,618 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 64.21 sec -2013-09-11 18:24:44,625 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 71.23 sec -2013-09-11 18:24:45,630 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 71.23 sec -MapReduce Total cumulative CPU time: 1 minutes 11 seconds 230 msec -Ended Job = job_201309101627_0250 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0251 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:24:49,129 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:24:51,138 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.5 sec -2013-09-11 18:24:52,144 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.5 sec -2013-09-11 18:24:53,149 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.5 sec -2013-09-11 18:24:54,154 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.5 sec -2013-09-11 18:24:55,160 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.5 sec -2013-09-11 18:24:56,165 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.5 sec -2013-09-11 18:24:57,171 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.5 sec -2013-09-11 18:24:58,176 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.01 sec -2013-09-11 18:24:59,181 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.01 sec -2013-09-11 18:25:00,186 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.01 sec -MapReduce Total cumulative CPU time: 3 seconds 10 msec -Ended Job = job_201309101627_0251 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 71.23 sec HDFS Read: 67340015 HDFS Write: 100142 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.01 sec HDFS Read: 100911 HDFS Write: 96 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 14 seconds 240 msec -OK -Time taken: 51.083 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24272@mturlrep13_201309111825_819364546.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25048@mturlrep13_201309111825_117221208.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0252 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:25:21,952 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:25:28,978 Stage-1 map = 29%, reduce = 0% -2013-09-11 18:25:31,990 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:25:35,008 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.17 sec -2013-09-11 18:25:36,015 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.17 sec -2013-09-11 18:25:37,023 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.17 sec -2013-09-11 18:25:38,029 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.17 sec -2013-09-11 18:25:39,034 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.17 sec -2013-09-11 18:25:40,040 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.17 sec -2013-09-11 18:25:41,045 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.17 sec -2013-09-11 18:25:42,050 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 35.17 sec -2013-09-11 18:25:43,055 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 35.17 sec -2013-09-11 18:25:44,060 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 35.17 sec -2013-09-11 18:25:45,065 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 35.17 sec -2013-09-11 18:25:46,070 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 35.17 sec -2013-09-11 18:25:47,098 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 35.17 sec -2013-09-11 18:25:48,104 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-11 18:25:49,109 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-11 18:25:50,114 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-11 18:25:51,119 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-11 18:25:52,124 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-11 18:25:53,129 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-11 18:25:54,135 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-11 18:25:55,140 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-11 18:25:56,146 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.29 sec -2013-09-11 18:25:57,154 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 75.46 sec -2013-09-11 18:25:58,160 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 83.41 sec -2013-09-11 18:25:59,225 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 84.91 sec -2013-09-11 18:26:00,231 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 84.91 sec -MapReduce Total cumulative CPU time: 1 minutes 24 seconds 910 msec -Ended Job = job_201309101627_0252 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0253 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:26:03,726 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:26:05,734 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.54 sec -2013-09-11 18:26:06,738 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.54 sec -2013-09-11 18:26:07,742 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.54 sec -2013-09-11 18:26:08,747 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.54 sec -2013-09-11 18:26:09,786 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.54 sec -2013-09-11 18:26:10,790 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.54 sec -2013-09-11 18:26:11,795 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.54 sec -2013-09-11 18:26:12,800 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 1.54 sec -2013-09-11 18:26:13,805 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.09 sec -2013-09-11 18:26:14,810 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.09 sec -MapReduce Total cumulative CPU time: 3 seconds 90 msec -Ended Job = job_201309101627_0253 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 84.91 sec HDFS Read: 74853201 HDFS Write: 148871 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.09 sec HDFS Read: 149640 HDFS Write: 414 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 28 seconds 0 msec -OK -Time taken: 63.196 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27967@mturlrep13_201309111826_889850944.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28412@mturlrep13_201309111826_925180195.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0254 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:26:33,670 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:26:40,696 Stage-1 map = 29%, reduce = 0% -2013-09-11 18:26:43,708 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:26:46,726 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.26 sec -2013-09-11 18:26:47,733 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.26 sec -2013-09-11 18:26:48,741 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.26 sec -2013-09-11 18:26:49,748 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.26 sec -2013-09-11 18:26:50,753 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.26 sec -2013-09-11 18:26:51,760 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.26 sec -2013-09-11 18:26:52,765 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.26 sec -2013-09-11 18:26:53,771 Stage-1 map = 80%, reduce = 8%, Cumulative CPU 33.26 sec -2013-09-11 18:26:54,795 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 33.26 sec -2013-09-11 18:26:55,800 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 33.26 sec -2013-09-11 18:26:56,806 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.26 sec -2013-09-11 18:26:57,813 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 60.67 sec -2013-09-11 18:26:58,820 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 60.67 sec -2013-09-11 18:26:59,826 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:27:00,831 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:27:01,837 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:27:02,843 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:27:03,849 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:27:04,854 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:27:05,859 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:27:06,865 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:27:07,870 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:27:08,878 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 75.48 sec -2013-09-11 18:27:09,884 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 75.48 sec -2013-09-11 18:27:10,890 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 84.13 sec -2013-09-11 18:27:11,896 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 84.13 sec -MapReduce Total cumulative CPU time: 1 minutes 24 seconds 130 msec -Ended Job = job_201309101627_0254 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0255 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:27:15,439 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:27:17,447 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:27:18,451 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:27:19,457 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:27:20,462 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:27:21,467 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:27:22,472 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:27:23,477 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.56 sec -2013-09-11 18:27:24,482 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.15 sec -2013-09-11 18:27:25,487 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.15 sec -2013-09-11 18:27:26,493 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.15 sec -MapReduce Total cumulative CPU time: 3 seconds 150 msec -Ended Job = job_201309101627_0255 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 84.13 sec HDFS Read: 74853201 HDFS Write: 148871 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.15 sec HDFS Read: 149640 HDFS Write: 414 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 27 seconds 280 msec -OK -Time taken: 65.921 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31044@mturlrep13_201309111827_588046604.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31497@mturlrep13_201309111827_1421279621.txt -hive> SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0256 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:27:41,761 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:27:49,791 Stage-1 map = 36%, reduce = 0% -2013-09-11 18:27:52,803 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:27:54,818 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.22 sec -2013-09-11 18:27:55,825 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.22 sec -2013-09-11 18:27:56,833 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.22 sec -2013-09-11 18:27:57,840 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.22 sec -2013-09-11 18:27:58,846 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.22 sec -2013-09-11 18:27:59,852 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.22 sec -2013-09-11 18:28:00,858 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.22 sec -2013-09-11 18:28:01,864 Stage-1 map = 88%, reduce = 8%, Cumulative CPU 34.22 sec -2013-09-11 18:28:02,869 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.22 sec -2013-09-11 18:28:03,875 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.22 sec -2013-09-11 18:28:04,881 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.22 sec -2013-09-11 18:28:05,887 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.22 sec -2013-09-11 18:28:06,892 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 50.09 sec -2013-09-11 18:28:07,897 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:28:08,903 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:28:09,908 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:28:10,914 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:28:11,920 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:28:12,925 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:28:13,930 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:28:14,936 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:28:15,941 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.82 sec -2013-09-11 18:28:16,948 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 76.29 sec -2013-09-11 18:28:17,953 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 76.29 sec -2013-09-11 18:28:19,811 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 76.29 sec -2013-09-11 18:28:20,994 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 86.22 sec -2013-09-11 18:28:22,073 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 86.22 sec -2013-09-11 18:28:23,078 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 86.22 sec -2013-09-11 18:28:24,083 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 86.22 sec -MapReduce Total cumulative CPU time: 1 minutes 26 seconds 220 msec -Ended Job = job_201309101627_0256 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0257 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:28:30,225 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:28:35,317 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:36,331 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:37,335 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:38,340 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:39,345 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:40,350 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:41,354 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:42,359 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:43,364 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:44,369 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:45,374 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 1.67 sec -2013-09-11 18:28:46,379 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.31 sec -2013-09-11 18:28:47,384 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.31 sec -2013-09-11 18:28:48,389 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.31 sec -2013-09-11 18:28:49,395 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.31 sec -2013-09-11 18:28:50,400 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 3.31 sec -MapReduce Total cumulative CPU time: 3 seconds 310 msec -Ended Job = job_201309101627_0257 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 86.22 sec HDFS Read: 74853201 HDFS Write: 148871 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 3.31 sec HDFS Read: 149640 HDFS Write: 414 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 29 seconds 530 msec -OK -Time taken: 77.923 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2540@mturlrep13_201309111828_1228944488.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3275@mturlrep13_201309111829_1496747935.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0258 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:29:20,524 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:29:27,560 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 6.02 sec -2013-09-11 18:29:28,569 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.58 sec -2013-09-11 18:29:29,577 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.58 sec -2013-09-11 18:29:30,585 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.58 sec -2013-09-11 18:29:31,592 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.58 sec -2013-09-11 18:29:32,598 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.58 sec -2013-09-11 18:29:33,605 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.58 sec -2013-09-11 18:29:34,611 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.58 sec -2013-09-11 18:29:35,618 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.88 sec -2013-09-11 18:29:36,623 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.88 sec -2013-09-11 18:29:37,629 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.88 sec -2013-09-11 18:29:38,634 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.88 sec -2013-09-11 18:29:39,640 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.88 sec -2013-09-11 18:29:40,645 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.88 sec -2013-09-11 18:29:41,653 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 32.47 sec -2013-09-11 18:29:42,660 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 36.02 sec -2013-09-11 18:29:43,666 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 36.02 sec -MapReduce Total cumulative CPU time: 36 seconds 20 msec -Ended Job = job_201309101627_0258 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0259 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:29:49,843 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:29:52,856 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:29:53,926 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:29:54,932 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:29:55,937 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:29:56,943 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:29:57,948 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:29:58,953 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:29:59,958 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:30:00,963 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:30:01,969 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:30:02,974 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:30:03,980 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:30:04,985 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:30:05,990 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.43 sec -2013-09-11 18:30:06,995 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.43 sec -2013-09-11 18:30:08,001 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.43 sec -2013-09-11 18:30:09,008 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.43 sec -2013-09-11 18:30:10,015 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.43 sec -MapReduce Total cumulative CPU time: 2 seconds 430 msec -Ended Job = job_201309101627_0259 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 36.02 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.43 sec HDFS Read: 21897 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 38 seconds 450 msec -OK -Time taken: 63.358 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7226@mturlrep13_201309111830_168478607.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8250@mturlrep13_201309111830_1240423626.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0260 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:30:47,475 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:30:55,514 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:30:56,528 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.56 sec -2013-09-11 18:30:57,535 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.56 sec -2013-09-11 18:30:58,544 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.56 sec -2013-09-11 18:30:59,550 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.56 sec -2013-09-11 18:31:00,556 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.56 sec -2013-09-11 18:31:01,563 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.56 sec -2013-09-11 18:31:02,574 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.56 sec -2013-09-11 18:31:03,580 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 25.74 sec -2013-09-11 18:31:04,587 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 33.71 sec -2013-09-11 18:31:05,593 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 33.71 sec -2013-09-11 18:31:06,599 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 33.71 sec -2013-09-11 18:31:07,605 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 33.71 sec -2013-09-11 18:31:08,610 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 33.71 sec -2013-09-11 18:31:09,617 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 33.71 sec -2013-09-11 18:31:10,623 Stage-1 map = 100%, reduce = 42%, Cumulative CPU 33.71 sec -2013-09-11 18:31:11,643 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.64 sec -2013-09-11 18:31:12,649 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.64 sec -2013-09-11 18:31:13,655 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.64 sec -2013-09-11 18:31:14,661 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.64 sec -2013-09-11 18:31:15,668 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.64 sec -2013-09-11 18:31:16,675 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.64 sec -MapReduce Total cumulative CPU time: 40 seconds 640 msec -Ended Job = job_201309101627_0260 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0261 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:31:21,465 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:31:23,474 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:31:24,479 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:31:25,485 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:31:26,490 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:31:27,495 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:31:28,501 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:31:29,506 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:31:30,511 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.51 sec -2013-09-11 18:31:31,516 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.51 sec -2013-09-11 18:31:32,521 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.51 sec -2013-09-11 18:31:33,527 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.51 sec -MapReduce Total cumulative CPU time: 2 seconds 510 msec -Ended Job = job_201309101627_0261 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.64 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.51 sec HDFS Read: 21897 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 150 msec -OK -Time taken: 71.328 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11493@mturlrep13_201309111831_1720637849.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12279@mturlrep13_201309111831_901895827.txt -hive> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0262 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:32:03,855 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:32:10,590 Stage-1 map = 36%, reduce = 0% -2013-09-11 18:32:12,950 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:13,958 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:14,965 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:15,972 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:16,978 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:17,984 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:18,990 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:19,997 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:21,002 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:22,009 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:23,014 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:24,021 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:25,027 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:26,034 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:27,040 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 12.86 sec -2013-09-11 18:32:28,046 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 20.25 sec -2013-09-11 18:32:29,052 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.79 sec -2013-09-11 18:32:30,059 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.79 sec -2013-09-11 18:32:31,064 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.79 sec -2013-09-11 18:32:32,070 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.79 sec -2013-09-11 18:32:33,076 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.79 sec -2013-09-11 18:32:34,081 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.79 sec -2013-09-11 18:32:35,088 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 28.79 sec -2013-09-11 18:32:36,120 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 35.98 sec -2013-09-11 18:32:37,126 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 35.98 sec -2013-09-11 18:32:38,133 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 35.98 sec -2013-09-11 18:32:39,226 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 35.98 sec -MapReduce Total cumulative CPU time: 35 seconds 980 msec -Ended Job = job_201309101627_0262 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0263 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:32:45,523 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:32:47,542 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 18:32:48,547 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 18:32:50,032 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 18:32:51,036 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 18:32:52,041 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 18:32:53,046 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 18:32:54,050 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 18:32:55,055 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.93 sec -2013-09-11 18:32:56,061 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.93 sec -2013-09-11 18:32:57,066 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -2013-09-11 18:32:58,071 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -2013-09-11 18:32:59,076 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -2013-09-11 18:33:00,081 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -2013-09-11 18:33:01,087 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -2013-09-11 18:33:02,092 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -MapReduce Total cumulative CPU time: 2 seconds 440 msec -Ended Job = job_201309101627_0263 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 35.98 sec HDFS Read: 58273488 HDFS Write: 21128 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.44 sec HDFS Read: 21897 HDFS Write: 127 SUCCESS -Total MapReduce CPU Time Spent: 38 seconds 420 msec -OK -Time taken: 75.971 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по строкам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16116@mturlrep13_201309111833_605870459.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16886@mturlrep13_201309111833_202130159.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0264 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:33:29,621 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:33:37,775 Stage-1 map = 36%, reduce = 0% -2013-09-11 18:33:38,846 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:39,854 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:40,861 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:41,868 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:42,875 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:43,882 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:44,889 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:45,895 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:46,900 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:47,906 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:48,913 Stage-1 map = 88%, reduce = 0%, Cumulative CPU 16.67 sec -2013-09-11 18:33:49,918 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.02 sec -2013-09-11 18:33:50,924 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.02 sec -2013-09-11 18:33:51,930 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.02 sec -2013-09-11 18:33:52,936 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.02 sec -2013-09-11 18:33:53,941 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 33.02 sec -2013-09-11 18:33:55,094 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 33.02 sec -2013-09-11 18:33:56,113 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.54 sec -2013-09-11 18:33:57,119 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.54 sec -2013-09-11 18:33:58,125 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.54 sec -2013-09-11 18:33:59,132 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.54 sec -2013-09-11 18:34:00,138 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.54 sec -2013-09-11 18:34:01,144 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.54 sec -MapReduce Total cumulative CPU time: 40 seconds 540 msec -Ended Job = job_201309101627_0264 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0265 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:34:04,726 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:34:06,735 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:34:07,741 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:34:08,746 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:34:09,751 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:34:10,756 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:34:11,761 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:34:12,766 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:34:13,771 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.95 sec -2013-09-11 18:34:14,777 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -2013-09-11 18:34:15,783 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -2013-09-11 18:34:16,788 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -MapReduce Total cumulative CPU time: 2 seconds 450 msec -Ended Job = job_201309101627_0265 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.54 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.45 sec HDFS Read: 23479 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 990 msec -OK -Time taken: 60.019 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20052@mturlrep13_201309111834_786769685.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20585@mturlrep13_201309111834_648955611.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0266 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:34:37,882 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:34:45,163 Stage-1 map = 21%, reduce = 0% -2013-09-11 18:34:46,294 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.03 sec -2013-09-11 18:34:47,302 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.03 sec -2013-09-11 18:34:48,310 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.03 sec -2013-09-11 18:34:49,317 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.03 sec -2013-09-11 18:34:50,323 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.03 sec -2013-09-11 18:34:51,330 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.03 sec -2013-09-11 18:34:52,338 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.03 sec -2013-09-11 18:34:53,345 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.03 sec -2013-09-11 18:34:54,351 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.03 sec -2013-09-11 18:34:55,357 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 15.03 sec -2013-09-11 18:34:56,364 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 15.03 sec -2013-09-11 18:34:57,370 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.28 sec -2013-09-11 18:34:58,377 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 31.28 sec -2013-09-11 18:34:59,383 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.94 sec -2013-09-11 18:35:00,389 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.94 sec -2013-09-11 18:35:01,395 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.94 sec -2013-09-11 18:35:02,401 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 32.94 sec -2013-09-11 18:35:03,410 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.58 sec -2013-09-11 18:35:04,417 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.58 sec -2013-09-11 18:35:05,447 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 39.58 sec -MapReduce Total cumulative CPU time: 39 seconds 580 msec -Ended Job = job_201309101627_0266 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0267 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:35:12,739 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:35:14,998 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:35:16,004 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:35:17,010 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:35:18,015 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:35:19,020 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:35:20,025 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:35:21,031 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:35:22,036 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:35:23,042 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.97 sec -2013-09-11 18:35:24,048 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -2013-09-11 18:35:25,053 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -2013-09-11 18:35:26,058 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -2013-09-11 18:35:27,064 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.45 sec -MapReduce Total cumulative CPU time: 2 seconds 450 msec -Ended Job = job_201309101627_0267 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 39.58 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.45 sec HDFS Read: 23479 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 30 msec -OK -Time taken: 59.993 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24080@mturlrep13_201309111835_1895693790.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24734@mturlrep13_201309111835_1260739241.txt -hive> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0268 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:35:59,129 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:36:06,189 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:36:07,197 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:36:08,207 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:36:09,214 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:36:10,222 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:36:11,228 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:36:12,234 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 14.87 sec -2013-09-11 18:36:13,241 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 30.46 sec -2013-09-11 18:36:14,248 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.46 sec -2013-09-11 18:36:15,254 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.46 sec -2013-09-11 18:36:16,260 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.46 sec -2013-09-11 18:36:17,266 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.46 sec -2013-09-11 18:36:18,272 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.46 sec -2013-09-11 18:36:19,278 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 30.46 sec -2013-09-11 18:36:20,286 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.87 sec -2013-09-11 18:36:21,292 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.87 sec -2013-09-11 18:36:22,300 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.87 sec -MapReduce Total cumulative CPU time: 37 seconds 870 msec -Ended Job = job_201309101627_0268 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0269 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:36:25,859 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:36:27,868 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.86 sec -2013-09-11 18:36:28,873 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.86 sec -2013-09-11 18:36:29,879 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.86 sec -2013-09-11 18:36:30,885 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.86 sec -2013-09-11 18:36:31,890 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.86 sec -2013-09-11 18:36:32,895 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.86 sec -2013-09-11 18:36:33,900 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.86 sec -2013-09-11 18:36:34,906 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.28 sec -2013-09-11 18:36:35,911 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.28 sec -2013-09-11 18:36:36,917 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.28 sec -MapReduce Total cumulative CPU time: 2 seconds 280 msec -Ended Job = job_201309101627_0269 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 37.87 sec HDFS Read: 59259422 HDFS Write: 22710 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.28 sec HDFS Read: 23479 HDFS Write: 149 SUCCESS -Total MapReduce CPU Time Spent: 40 seconds 150 msec -OK -Time taken: 52.093 seconds, Fetched: 10 row(s) -hive> quit; --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27342@mturlrep13_201309111836_1910306053.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27811@mturlrep13_201309111836_1213140572.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0270 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:36:57,519 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:37:04,550 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:37:05,563 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.33 sec -2013-09-11 18:37:06,570 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.33 sec -2013-09-11 18:37:07,577 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.33 sec -2013-09-11 18:37:08,583 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.33 sec -2013-09-11 18:37:09,589 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.33 sec -2013-09-11 18:37:10,595 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.33 sec -2013-09-11 18:37:11,602 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.33 sec -2013-09-11 18:37:12,608 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 20.33 sec -2013-09-11 18:37:13,613 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.89 sec -2013-09-11 18:37:14,618 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.89 sec -2013-09-11 18:37:15,623 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.89 sec -2013-09-11 18:37:16,629 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.89 sec -2013-09-11 18:37:17,635 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.89 sec -2013-09-11 18:37:18,640 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.89 sec -2013-09-11 18:37:19,646 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.89 sec -2013-09-11 18:37:20,652 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.89 sec -2013-09-11 18:37:21,659 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 47.0 sec -2013-09-11 18:37:22,665 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 54.11 sec -2013-09-11 18:37:23,671 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 54.11 sec -MapReduce Total cumulative CPU time: 54 seconds 110 msec -Ended Job = job_201309101627_0270 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0271 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:37:26,224 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:37:34,251 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:37:36,259 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.77 sec -2013-09-11 18:37:37,265 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.77 sec -2013-09-11 18:37:38,269 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.77 sec -2013-09-11 18:37:39,274 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.77 sec -2013-09-11 18:37:40,279 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.77 sec -2013-09-11 18:37:41,284 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.77 sec -2013-09-11 18:37:42,289 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.77 sec -2013-09-11 18:37:43,294 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.77 sec -2013-09-11 18:37:44,300 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.77 sec -2013-09-11 18:37:45,306 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.08 sec -2013-09-11 18:37:46,312 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.08 sec -2013-09-11 18:37:47,317 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.08 sec -MapReduce Total cumulative CPU time: 18 seconds 80 msec -Ended Job = job_201309101627_0271 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 54.11 sec HDFS Read: 27820105 HDFS Write: 79726641 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.08 sec HDFS Read: 79727410 HDFS Write: 275 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 12 seconds 190 msec -OK -Time taken: 59.798 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30130@mturlrep13_201309111837_1070436156.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30577@mturlrep13_201309111837_1857751509.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0272 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:38:00,966 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:38:09,007 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.49 sec -2013-09-11 18:38:10,015 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.49 sec -2013-09-11 18:38:11,023 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.49 sec -2013-09-11 18:38:12,029 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.49 sec -2013-09-11 18:38:13,036 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.49 sec -2013-09-11 18:38:14,042 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.49 sec -2013-09-11 18:38:15,050 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.49 sec -2013-09-11 18:38:16,057 Stage-1 map = 97%, reduce = 8%, Cumulative CPU 30.84 sec -2013-09-11 18:38:17,063 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.01 sec -2013-09-11 18:38:18,069 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.01 sec -2013-09-11 18:38:19,075 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.01 sec -2013-09-11 18:38:20,081 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.01 sec -2013-09-11 18:38:21,087 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.01 sec -2013-09-11 18:38:22,093 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.01 sec -2013-09-11 18:38:23,100 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.01 sec -2013-09-11 18:38:24,106 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.01 sec -2013-09-11 18:38:25,113 Stage-1 map = 100%, reduce = 56%, Cumulative CPU 41.01 sec -2013-09-11 18:38:26,122 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.19 sec -2013-09-11 18:38:27,128 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 56.19 sec -MapReduce Total cumulative CPU time: 56 seconds 190 msec -Ended Job = job_201309101627_0272 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0273 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:38:29,657 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:38:37,684 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:38:39,692 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.16 sec -2013-09-11 18:38:40,697 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.16 sec -2013-09-11 18:38:41,702 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.16 sec -2013-09-11 18:38:42,707 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.16 sec -2013-09-11 18:38:43,712 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.16 sec -2013-09-11 18:38:44,716 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.16 sec -2013-09-11 18:38:45,721 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.16 sec -2013-09-11 18:38:46,726 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.16 sec -2013-09-11 18:38:47,730 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.16 sec -2013-09-11 18:38:48,736 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.7 sec -2013-09-11 18:38:49,742 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.7 sec -2013-09-11 18:38:50,747 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.7 sec -MapReduce Total cumulative CPU time: 17 seconds 700 msec -Ended Job = job_201309101627_0273 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 56.19 sec HDFS Read: 27820105 HDFS Write: 79726641 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.7 sec HDFS Read: 79727410 HDFS Write: 275 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 13 seconds 890 msec -OK -Time taken: 57.143 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_415@mturlrep13_201309111838_1362426037.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_874@mturlrep13_201309111838_1940288899.txt -hive> SELECT SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0274 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:39:05,176 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:39:12,209 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.99 sec -2013-09-11 18:39:13,217 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.99 sec -2013-09-11 18:39:14,223 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.99 sec -2013-09-11 18:39:15,230 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.99 sec -2013-09-11 18:39:16,235 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.99 sec -2013-09-11 18:39:17,241 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.99 sec -2013-09-11 18:39:18,251 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.99 sec -2013-09-11 18:39:19,258 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 30.52 sec -2013-09-11 18:39:20,264 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.59 sec -2013-09-11 18:39:21,270 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.59 sec -2013-09-11 18:39:22,275 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.59 sec -2013-09-11 18:39:23,281 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.59 sec -2013-09-11 18:39:24,287 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.59 sec -2013-09-11 18:39:25,293 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.59 sec -2013-09-11 18:39:26,298 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.59 sec -2013-09-11 18:39:27,304 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.59 sec -2013-09-11 18:39:28,310 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.59 sec -2013-09-11 18:39:29,319 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 55.31 sec -2013-09-11 18:39:30,325 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 55.31 sec -2013-09-11 18:39:31,332 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 55.31 sec -MapReduce Total cumulative CPU time: 55 seconds 310 msec -Ended Job = job_201309101627_0274 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0275 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:39:33,849 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:39:41,877 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:39:43,918 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.1 sec -2013-09-11 18:39:44,924 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.1 sec -2013-09-11 18:39:45,929 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.1 sec -2013-09-11 18:39:46,934 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.1 sec -2013-09-11 18:39:47,939 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.1 sec -2013-09-11 18:39:48,944 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.1 sec -2013-09-11 18:39:49,948 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.1 sec -2013-09-11 18:39:50,954 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.1 sec -2013-09-11 18:39:52,010 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.1 sec -2013-09-11 18:39:53,015 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.36 sec -2013-09-11 18:39:54,020 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.36 sec -MapReduce Total cumulative CPU time: 17 seconds 360 msec -Ended Job = job_201309101627_0275 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 55.31 sec HDFS Read: 27820105 HDFS Write: 79726641 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.36 sec HDFS Read: 79727410 HDFS Write: 275 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 12 seconds 670 msec -OK -Time taken: 57.089 seconds, Fetched: 10 row(s) -hive> quit; --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3290@mturlrep13_201309111840_695958684.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3872@mturlrep13_201309111840_1093049561.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0276 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:40:17,267 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:40:24,294 Stage-1 map = 36%, reduce = 0% -2013-09-11 18:40:26,312 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.08 sec -2013-09-11 18:40:27,320 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.08 sec -2013-09-11 18:40:28,328 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.08 sec -2013-09-11 18:40:29,334 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.08 sec -2013-09-11 18:40:30,340 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.08 sec -2013-09-11 18:40:31,346 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.08 sec -2013-09-11 18:40:32,354 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.08 sec -2013-09-11 18:40:33,360 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.08 sec -2013-09-11 18:40:34,366 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 23.08 sec -2013-09-11 18:40:35,372 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 35.37 sec -2013-09-11 18:40:36,377 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.42 sec -2013-09-11 18:40:37,385 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.42 sec -2013-09-11 18:40:38,390 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.42 sec -2013-09-11 18:40:39,396 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.42 sec -2013-09-11 18:40:40,402 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.42 sec -2013-09-11 18:40:41,409 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.42 sec -2013-09-11 18:40:42,415 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.42 sec -2013-09-11 18:40:43,424 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 55.41 sec -2013-09-11 18:40:44,430 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.73 sec -2013-09-11 18:40:45,435 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.73 sec -MapReduce Total cumulative CPU time: 1 minutes 3 seconds 730 msec -Ended Job = job_201309101627_0276 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0277 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:40:48,962 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:40:55,985 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:40:57,993 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.39 sec -2013-09-11 18:40:58,999 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.39 sec -2013-09-11 18:41:00,004 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.39 sec -2013-09-11 18:41:01,009 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.39 sec -2013-09-11 18:41:02,013 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.39 sec -2013-09-11 18:41:03,018 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.39 sec -2013-09-11 18:41:04,023 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.39 sec -2013-09-11 18:41:05,028 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.39 sec -2013-09-11 18:41:06,034 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.39 sec -2013-09-11 18:41:07,040 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.97 sec -2013-09-11 18:41:08,046 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.97 sec -2013-09-11 18:41:09,051 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.97 sec -MapReduce Total cumulative CPU time: 17 seconds 970 msec -Ended Job = job_201309101627_0277 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 63.73 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.97 sec HDFS Read: 79727313 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 21 seconds 700 msec -OK -Time taken: 62.137 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7182@mturlrep13_201309111841_759539300.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7624@mturlrep13_201309111841_70388529.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0278 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:41:23,258 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:41:30,293 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:41:31,306 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.41 sec -2013-09-11 18:41:32,314 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.41 sec -2013-09-11 18:41:33,321 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.41 sec -2013-09-11 18:41:34,328 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.41 sec -2013-09-11 18:41:35,334 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.41 sec -2013-09-11 18:41:36,341 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.41 sec -2013-09-11 18:41:37,348 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.41 sec -2013-09-11 18:41:38,355 Stage-1 map = 88%, reduce = 8%, Cumulative CPU 22.41 sec -2013-09-11 18:41:39,362 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 33.84 sec -2013-09-11 18:41:40,368 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.56 sec -2013-09-11 18:41:41,374 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.56 sec -2013-09-11 18:41:42,380 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.56 sec -2013-09-11 18:41:43,385 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.56 sec -2013-09-11 18:41:44,391 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.56 sec -2013-09-11 18:41:45,398 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.56 sec -2013-09-11 18:41:46,404 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 45.56 sec -2013-09-11 18:41:47,410 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 45.56 sec -2013-09-11 18:41:48,416 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 45.56 sec -2013-09-11 18:41:49,424 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.28 sec -2013-09-11 18:41:50,430 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.28 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 280 msec -Ended Job = job_201309101627_0278 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0279 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:41:53,909 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:42:00,937 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:42:02,947 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.32 sec -2013-09-11 18:42:03,953 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.32 sec -2013-09-11 18:42:04,958 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.32 sec -2013-09-11 18:42:05,962 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.32 sec -2013-09-11 18:42:06,967 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.32 sec -2013-09-11 18:42:07,972 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.32 sec -2013-09-11 18:42:08,976 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.32 sec -2013-09-11 18:42:09,981 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.32 sec -2013-09-11 18:42:10,987 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.32 sec -2013-09-11 18:42:11,992 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.74 sec -2013-09-11 18:42:12,998 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.74 sec -2013-09-11 18:42:14,004 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.74 sec -MapReduce Total cumulative CPU time: 17 seconds 740 msec -Ended Job = job_201309101627_0279 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.28 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.74 sec HDFS Read: 79727313 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 20 seconds 20 msec -OK -Time taken: 59.295 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9991@mturlrep13_201309111842_1002615077.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10431@mturlrep13_201309111842_612246455.txt -hive> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0280 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:42:27,139 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:42:35,171 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:42:36,186 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.88 sec -2013-09-11 18:42:37,194 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.88 sec -2013-09-11 18:42:38,202 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.88 sec -2013-09-11 18:42:39,209 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.88 sec -2013-09-11 18:42:40,215 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.88 sec -2013-09-11 18:42:41,222 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.88 sec -2013-09-11 18:42:42,229 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 22.88 sec -2013-09-11 18:42:43,236 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 22.88 sec -2013-09-11 18:42:44,242 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 34.43 sec -2013-09-11 18:42:45,247 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.67 sec -2013-09-11 18:42:46,277 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.67 sec -2013-09-11 18:42:47,283 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.67 sec -2013-09-11 18:42:48,288 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.67 sec -2013-09-11 18:42:49,294 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.67 sec -2013-09-11 18:42:50,299 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.67 sec -2013-09-11 18:42:51,305 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.67 sec -2013-09-11 18:42:52,311 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 46.67 sec -2013-09-11 18:42:53,319 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 54.9 sec -2013-09-11 18:42:54,326 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.26 sec -2013-09-11 18:42:55,333 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 63.26 sec -MapReduce Total cumulative CPU time: 1 minutes 3 seconds 260 msec -Ended Job = job_201309101627_0280 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0281 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:42:57,815 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:43:05,847 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:43:07,856 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.3 sec -2013-09-11 18:43:08,862 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.3 sec -2013-09-11 18:43:09,867 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.3 sec -2013-09-11 18:43:10,872 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.3 sec -2013-09-11 18:43:11,877 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.3 sec -2013-09-11 18:43:12,883 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.3 sec -2013-09-11 18:43:13,888 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.3 sec -2013-09-11 18:43:14,893 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.3 sec -2013-09-11 18:43:15,898 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.3 sec -2013-09-11 18:43:16,904 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.84 sec -2013-09-11 18:43:17,909 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.84 sec -2013-09-11 18:43:18,915 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 17.84 sec -MapReduce Total cumulative CPU time: 17 seconds 840 msec -Ended Job = job_201309101627_0281 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 63.26 sec HDFS Read: 84536695 HDFS Write: 79726544 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 17.84 sec HDFS Read: 79727313 HDFS Write: 293 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 21 seconds 100 msec -OK -Time taken: 59.238 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация чуть сложнее.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12833@mturlrep13_201309111843_203223094.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13310@mturlrep13_201309111843_869968579.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0282 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:43:40,513 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:43:47,543 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:43:48,556 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.59 sec -2013-09-11 18:43:49,563 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.59 sec -2013-09-11 18:43:50,572 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.59 sec -2013-09-11 18:43:51,578 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.59 sec -2013-09-11 18:43:52,585 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.59 sec -2013-09-11 18:43:53,592 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.59 sec -2013-09-11 18:43:54,598 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.59 sec -2013-09-11 18:43:55,605 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.59 sec -2013-09-11 18:43:56,611 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 20.59 sec -2013-09-11 18:43:57,617 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.09 sec -2013-09-11 18:43:58,623 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.09 sec -2013-09-11 18:43:59,629 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.09 sec -2013-09-11 18:44:00,636 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.79 sec -2013-09-11 18:44:01,642 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.79 sec -2013-09-11 18:44:02,648 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.79 sec -2013-09-11 18:44:03,655 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.79 sec -2013-09-11 18:44:04,661 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.79 sec -2013-09-11 18:44:05,670 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 50.58 sec -2013-09-11 18:44:06,676 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.28 sec -2013-09-11 18:44:07,682 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 57.28 sec -MapReduce Total cumulative CPU time: 57 seconds 280 msec -Ended Job = job_201309101627_0282 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0283 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:44:11,283 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:44:18,307 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:44:21,319 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.71 sec -2013-09-11 18:44:22,324 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.71 sec -2013-09-11 18:44:23,329 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.71 sec -2013-09-11 18:44:24,333 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.71 sec -2013-09-11 18:44:25,338 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.71 sec -2013-09-11 18:44:26,343 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.71 sec -2013-09-11 18:44:27,347 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.71 sec -2013-09-11 18:44:28,352 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.71 sec -2013-09-11 18:44:29,357 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.71 sec -2013-09-11 18:44:30,362 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.74 sec -2013-09-11 18:44:31,368 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.74 sec -2013-09-11 18:44:32,374 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.74 sec -MapReduce Total cumulative CPU time: 20 seconds 740 msec -Ended Job = job_201309101627_0283 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 57.28 sec HDFS Read: 30310112 HDFS Write: 84160093 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 20.74 sec HDFS Read: 84160862 HDFS Write: 297 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 18 seconds 20 msec -OK -Time taken: 61.858 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15620@mturlrep13_201309111844_1876164990.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16081@mturlrep13_201309111844_1689787397.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0284 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:44:45,391 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:44:53,428 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.0 sec -2013-09-11 18:44:54,436 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.0 sec -2013-09-11 18:44:55,445 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.0 sec -2013-09-11 18:44:56,452 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.0 sec -2013-09-11 18:44:57,458 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.0 sec -2013-09-11 18:44:58,465 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.0 sec -2013-09-11 18:44:59,471 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.0 sec -2013-09-11 18:45:00,479 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.0 sec -2013-09-11 18:45:01,485 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 33.06 sec -2013-09-11 18:45:02,492 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.76 sec -2013-09-11 18:45:03,498 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.76 sec -2013-09-11 18:45:04,504 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.76 sec -2013-09-11 18:45:05,510 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.76 sec -2013-09-11 18:45:06,515 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.76 sec -2013-09-11 18:45:07,521 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.76 sec -2013-09-11 18:45:08,527 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.76 sec -2013-09-11 18:45:09,533 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 43.76 sec -2013-09-11 18:45:10,539 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 43.76 sec -2013-09-11 18:45:11,546 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 59.3 sec -2013-09-11 18:45:12,553 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 59.3 sec -2013-09-11 18:45:13,559 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 59.3 sec -MapReduce Total cumulative CPU time: 59 seconds 300 msec -Ended Job = job_201309101627_0284 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0285 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:45:16,070 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:45:24,097 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:45:27,108 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 18:45:28,114 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 18:45:29,119 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 18:45:30,123 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 18:45:31,128 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 18:45:32,133 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 18:45:33,139 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 18:45:34,144 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.71 sec -2013-09-11 18:45:35,149 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.71 sec -2013-09-11 18:45:36,155 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 16.95 sec -2013-09-11 18:45:37,160 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 16.95 sec -MapReduce Total cumulative CPU time: 16 seconds 950 msec -Ended Job = job_201309101627_0285 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 59.3 sec HDFS Read: 30310112 HDFS Write: 84160093 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 16.95 sec HDFS Read: 84160862 HDFS Write: 297 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 16 seconds 250 msec -OK -Time taken: 59.204 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19315@mturlrep13_201309111845_275707985.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19758@mturlrep13_201309111845_1428482437.txt -hive> SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0286 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:45:50,307 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:45:58,346 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.37 sec -2013-09-11 18:45:59,353 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.37 sec -2013-09-11 18:46:00,362 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.37 sec -2013-09-11 18:46:01,368 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.37 sec -2013-09-11 18:46:02,375 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.37 sec -2013-09-11 18:46:03,381 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.37 sec -2013-09-11 18:46:04,387 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.37 sec -2013-09-11 18:46:05,394 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 21.37 sec -2013-09-11 18:46:06,401 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 31.78 sec -2013-09-11 18:46:07,410 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.63 sec -2013-09-11 18:46:08,416 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.63 sec -2013-09-11 18:46:09,422 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.63 sec -2013-09-11 18:46:10,428 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.63 sec -2013-09-11 18:46:11,434 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.63 sec -2013-09-11 18:46:12,440 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.63 sec -2013-09-11 18:46:13,447 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.63 sec -2013-09-11 18:46:14,453 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 42.63 sec -2013-09-11 18:46:15,462 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 50.32 sec -2013-09-11 18:46:16,468 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 58.04 sec -2013-09-11 18:46:17,474 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 58.04 sec -MapReduce Total cumulative CPU time: 58 seconds 40 msec -Ended Job = job_201309101627_0286 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0287 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:46:20,954 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:46:27,976 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:46:30,987 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.59 sec -2013-09-11 18:46:31,992 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.59 sec -2013-09-11 18:46:32,998 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.59 sec -2013-09-11 18:46:34,003 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.59 sec -2013-09-11 18:46:35,008 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.59 sec -2013-09-11 18:46:36,013 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.59 sec -2013-09-11 18:46:37,017 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.59 sec -2013-09-11 18:46:38,023 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 13.59 sec -2013-09-11 18:46:39,028 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 13.59 sec -2013-09-11 18:46:40,033 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.25 sec -2013-09-11 18:46:41,039 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.25 sec -2013-09-11 18:46:42,044 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 18.25 sec -MapReduce Total cumulative CPU time: 18 seconds 250 msec -Ended Job = job_201309101627_0287 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 58.04 sec HDFS Read: 30310112 HDFS Write: 84160093 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 18.25 sec HDFS Read: 84160862 HDFS Write: 297 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 16 seconds 290 msec -OK -Time taken: 59.141 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по числу и строке, большое количество ключей.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22085@mturlrep13_201309111846_197414974.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22551@mturlrep13_201309111846_356966815.txt -hive> SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0288 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:47:04,259 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:47:11,285 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:47:14,304 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.67 sec -2013-09-11 18:47:15,311 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.67 sec -2013-09-11 18:47:16,320 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.67 sec -2013-09-11 18:47:17,326 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.67 sec -2013-09-11 18:47:18,332 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.67 sec -2013-09-11 18:47:19,339 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.67 sec -2013-09-11 18:47:20,345 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.67 sec -2013-09-11 18:47:21,351 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.67 sec -2013-09-11 18:47:22,356 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.67 sec -2013-09-11 18:47:23,387 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.67 sec -2013-09-11 18:47:24,392 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 40.98 sec -2013-09-11 18:47:25,398 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.58 sec -2013-09-11 18:47:26,403 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.58 sec -2013-09-11 18:47:27,408 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.58 sec -2013-09-11 18:47:28,414 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.58 sec -2013-09-11 18:47:29,419 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.58 sec -2013-09-11 18:47:30,424 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 55.58 sec -2013-09-11 18:47:31,432 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 62.71 sec -2013-09-11 18:47:32,438 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.85 sec -2013-09-11 18:47:33,443 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 69.85 sec -MapReduce Total cumulative CPU time: 1 minutes 9 seconds 850 msec -Ended Job = job_201309101627_0288 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0289 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:47:37,065 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:47:47,092 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:47:51,104 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.97 sec -2013-09-11 18:47:52,109 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.97 sec -2013-09-11 18:47:53,113 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.97 sec -2013-09-11 18:47:54,117 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.97 sec -2013-09-11 18:47:55,122 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.97 sec -2013-09-11 18:47:56,127 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.97 sec -2013-09-11 18:47:57,131 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.97 sec -2013-09-11 18:47:58,136 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.97 sec -2013-09-11 18:47:59,141 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.97 sec -2013-09-11 18:48:00,146 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.32 sec -2013-09-11 18:48:01,150 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.32 sec -MapReduce Total cumulative CPU time: 22 seconds 320 msec -Ended Job = job_201309101627_0289 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 69.85 sec HDFS Read: 57312623 HDFS Write: 55475412 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 22.32 sec HDFS Read: 55476177 HDFS Write: 246 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 32 seconds 170 msec -OK -Time taken: 66.72 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24895@mturlrep13_201309111848_816582111.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_25356@mturlrep13_201309111848_354267688.txt -hive> SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0290 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:48:14,183 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:48:22,214 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:48:24,229 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.34 sec -2013-09-11 18:48:25,236 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.34 sec -2013-09-11 18:48:26,243 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.34 sec -2013-09-11 18:48:27,249 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.34 sec -2013-09-11 18:48:28,256 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.34 sec -2013-09-11 18:48:29,262 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.34 sec -2013-09-11 18:48:30,267 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 26.34 sec -2013-09-11 18:48:31,273 Stage-1 map = 96%, reduce = 8%, Cumulative CPU 26.34 sec -2013-09-11 18:48:32,277 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 26.34 sec -2013-09-11 18:48:33,282 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 39.02 sec -2013-09-11 18:48:34,288 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.45 sec -2013-09-11 18:48:35,293 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.45 sec -2013-09-11 18:48:36,298 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.45 sec -2013-09-11 18:48:37,304 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.45 sec -2013-09-11 18:48:38,309 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.45 sec -2013-09-11 18:48:39,315 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 54.45 sec -2013-09-11 18:48:40,320 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 54.45 sec -2013-09-11 18:48:41,328 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.84 sec -2013-09-11 18:48:42,334 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.84 sec -2013-09-11 18:48:43,340 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 68.84 sec -MapReduce Total cumulative CPU time: 1 minutes 8 seconds 840 msec -Ended Job = job_201309101627_0290 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0291 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:48:45,869 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:48:56,900 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:49:00,912 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 18:49:01,917 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 18:49:02,922 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 18:49:03,927 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 18:49:04,931 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 18:49:05,936 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 18:49:06,941 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 18:49:07,946 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.7 sec -2013-09-11 18:49:08,951 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.7 sec -2013-09-11 18:49:10,270 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.14 sec -2013-09-11 18:49:11,275 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 23.14 sec -MapReduce Total cumulative CPU time: 23 seconds 140 msec -Ended Job = job_201309101627_0291 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 68.84 sec HDFS Read: 57312623 HDFS Write: 55475412 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 23.14 sec HDFS Read: 55476181 HDFS Write: 246 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 31 seconds 980 msec -OK -Time taken: 64.324 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27717@mturlrep13_201309111849_53528863.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28180@mturlrep13_201309111849_1908133462.txt -hive> SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0292 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:49:25,328 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:49:32,358 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:49:34,373 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.15 sec -2013-09-11 18:49:35,381 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.15 sec -2013-09-11 18:49:36,388 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.15 sec -2013-09-11 18:49:37,394 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.15 sec -2013-09-11 18:49:38,399 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.15 sec -2013-09-11 18:49:39,406 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.15 sec -2013-09-11 18:49:40,413 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.15 sec -2013-09-11 18:49:41,430 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 27.15 sec -2013-09-11 18:49:42,441 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.15 sec -2013-09-11 18:49:43,447 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 27.15 sec -2013-09-11 18:49:44,453 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 39.96 sec -2013-09-11 18:49:45,458 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.29 sec -2013-09-11 18:49:46,464 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.29 sec -2013-09-11 18:49:47,469 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.29 sec -2013-09-11 18:49:48,475 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.29 sec -2013-09-11 18:49:49,480 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.29 sec -2013-09-11 18:49:50,486 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 56.29 sec -2013-09-11 18:49:51,494 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 63.39 sec -2013-09-11 18:49:52,500 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.79 sec -2013-09-11 18:49:53,506 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 70.79 sec -MapReduce Total cumulative CPU time: 1 minutes 10 seconds 790 msec -Ended Job = job_201309101627_0292 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0293 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:49:57,008 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:50:07,044 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:50:11,059 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.15 sec -2013-09-11 18:50:12,064 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.15 sec -2013-09-11 18:50:13,069 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.15 sec -2013-09-11 18:50:14,074 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.15 sec -2013-09-11 18:50:15,078 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.15 sec -2013-09-11 18:50:16,083 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.15 sec -2013-09-11 18:50:17,088 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.15 sec -2013-09-11 18:50:18,093 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.15 sec -2013-09-11 18:50:19,100 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.15 sec -2013-09-11 18:50:20,106 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.72 sec -2013-09-11 18:50:21,111 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.72 sec -2013-09-11 18:50:22,117 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.72 sec -MapReduce Total cumulative CPU time: 22 seconds 720 msec -Ended Job = job_201309101627_0293 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 70.79 sec HDFS Read: 57312623 HDFS Write: 55475412 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 22.72 sec HDFS Read: 55476181 HDFS Write: 246 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 33 seconds 510 msec -OK -Time taken: 65.232 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31241@mturlrep13_201309111850_2087772278.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31725@mturlrep13_201309111850_1907040905.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0294 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:50:43,595 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:50:50,623 Stage-1 map = 36%, reduce = 0% -2013-09-11 18:50:53,635 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:50:56,653 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.96 sec -2013-09-11 18:50:57,661 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.96 sec -2013-09-11 18:50:58,669 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.96 sec -2013-09-11 18:50:59,675 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.96 sec -2013-09-11 18:51:00,682 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.96 sec -2013-09-11 18:51:01,689 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.96 sec -2013-09-11 18:51:02,695 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.96 sec -2013-09-11 18:51:03,727 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.96 sec -2013-09-11 18:51:04,733 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.96 sec -2013-09-11 18:51:05,739 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.96 sec -2013-09-11 18:51:06,745 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 34.96 sec -2013-09-11 18:51:07,751 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.96 sec -2013-09-11 18:51:08,756 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.96 sec -2013-09-11 18:51:09,762 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.89 sec -2013-09-11 18:51:10,767 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.89 sec -2013-09-11 18:51:11,772 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.89 sec -2013-09-11 18:51:12,778 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.89 sec -2013-09-11 18:51:13,783 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.89 sec -2013-09-11 18:51:14,788 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.89 sec -2013-09-11 18:51:15,794 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.89 sec -2013-09-11 18:51:16,800 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.89 sec -2013-09-11 18:51:17,805 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.89 sec -2013-09-11 18:51:18,811 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.89 sec -2013-09-11 18:51:19,817 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 69.89 sec -2013-09-11 18:51:20,822 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 69.89 sec -2013-09-11 18:51:21,830 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 91.37 sec -2013-09-11 18:51:22,836 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 91.37 sec -MapReduce Total cumulative CPU time: 1 minutes 31 seconds 370 msec -Ended Job = job_201309101627_0294 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0295 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:51:25,350 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:51:36,383 Stage-2 map = 46%, reduce = 0% -2013-09-11 18:51:39,394 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:51:42,404 Stage-2 map = 96%, reduce = 0% -2013-09-11 18:51:44,411 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.3 sec -2013-09-11 18:51:45,416 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.3 sec -2013-09-11 18:51:46,421 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.3 sec -2013-09-11 18:51:47,425 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.3 sec -2013-09-11 18:51:48,430 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.3 sec -2013-09-11 18:51:49,435 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.3 sec -2013-09-11 18:51:50,439 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.3 sec -2013-09-11 18:51:51,444 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.3 sec -2013-09-11 18:51:52,449 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.3 sec -2013-09-11 18:51:53,454 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.3 sec -2013-09-11 18:51:54,460 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 27.86 sec -2013-09-11 18:51:55,465 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 27.86 sec -2013-09-11 18:51:56,470 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 27.86 sec -MapReduce Total cumulative CPU time: 27 seconds 860 msec -Ended Job = job_201309101627_0295 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 91.37 sec HDFS Read: 84536695 HDFS Write: 146202868 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 27.86 sec HDFS Read: 146210119 HDFS Write: 256 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 59 seconds 230 msec -OK -Time taken: 82.786 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2016@mturlrep13_201309111851_1953281129.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2464@mturlrep13_201309111852_1098803176.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0296 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:52:09,495 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:52:17,540 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:52:21,561 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.76 sec -2013-09-11 18:52:22,568 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.76 sec -2013-09-11 18:52:23,576 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.76 sec -2013-09-11 18:52:24,583 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.76 sec -2013-09-11 18:52:25,589 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.76 sec -2013-09-11 18:52:26,595 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.76 sec -2013-09-11 18:52:27,600 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.76 sec -2013-09-11 18:52:28,606 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.76 sec -2013-09-11 18:52:29,611 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.76 sec -2013-09-11 18:52:30,616 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.76 sec -2013-09-11 18:52:31,622 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.76 sec -2013-09-11 18:52:32,627 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.76 sec -2013-09-11 18:52:33,633 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.13 sec -2013-09-11 18:52:34,638 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.13 sec -2013-09-11 18:52:35,643 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.13 sec -2013-09-11 18:52:36,648 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.13 sec -2013-09-11 18:52:37,654 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.13 sec -2013-09-11 18:52:38,659 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 68.13 sec -2013-09-11 18:52:39,665 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 68.13 sec -2013-09-11 18:52:40,671 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 68.13 sec -2013-09-11 18:52:41,677 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 68.13 sec -2013-09-11 18:52:42,683 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 68.13 sec -2013-09-11 18:52:43,688 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 68.13 sec -2013-09-11 18:52:44,694 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 68.13 sec -2013-09-11 18:52:45,700 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 68.13 sec -2013-09-11 18:52:46,708 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.71 sec -2013-09-11 18:52:47,714 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.71 sec -2013-09-11 18:52:48,720 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.71 sec -MapReduce Total cumulative CPU time: 1 minutes 29 seconds 710 msec -Ended Job = job_201309101627_0296 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0297 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:52:52,227 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:53:02,258 Stage-2 map = 46%, reduce = 0%, Cumulative CPU 10.28 sec -2013-09-11 18:53:03,263 Stage-2 map = 46%, reduce = 0%, Cumulative CPU 10.28 sec -2013-09-11 18:53:04,267 Stage-2 map = 46%, reduce = 0%, Cumulative CPU 10.28 sec -2013-09-11 18:53:05,272 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 10.28 sec -2013-09-11 18:53:06,276 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 10.28 sec -2013-09-11 18:53:07,281 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 10.28 sec -2013-09-11 18:53:08,286 Stage-2 map = 96%, reduce = 0%, Cumulative CPU 10.28 sec -2013-09-11 18:53:09,291 Stage-2 map = 96%, reduce = 0%, Cumulative CPU 10.28 sec -2013-09-11 18:53:10,296 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.23 sec -2013-09-11 18:53:11,300 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.23 sec -2013-09-11 18:53:12,305 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.23 sec -2013-09-11 18:53:13,309 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.23 sec -2013-09-11 18:53:14,313 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.23 sec -2013-09-11 18:53:15,318 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.23 sec -2013-09-11 18:53:16,322 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 23.23 sec -2013-09-11 18:53:17,327 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.23 sec -2013-09-11 18:53:18,331 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.23 sec -2013-09-11 18:53:19,336 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 23.23 sec -2013-09-11 18:53:20,341 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 27.69 sec -2013-09-11 18:53:21,347 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 27.69 sec -2013-09-11 18:53:22,352 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 27.69 sec -MapReduce Total cumulative CPU time: 27 seconds 690 msec -Ended Job = job_201309101627_0297 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 89.71 sec HDFS Read: 84536695 HDFS Write: 146202868 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 27.69 sec HDFS Read: 146210123 HDFS Write: 256 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 57 seconds 400 msec -OK -Time taken: 80.259 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5290@mturlrep13_201309111853_777112224.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5789@mturlrep13_201309111853_1022585979.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0298 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:53:36,406 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:53:43,433 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:53:47,454 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 16.37 sec -2013-09-11 18:53:48,462 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-11 18:53:49,469 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-11 18:53:50,476 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-11 18:53:51,485 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-11 18:53:52,495 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-11 18:53:53,500 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-11 18:53:54,506 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 34.32 sec -2013-09-11 18:53:55,512 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.32 sec -2013-09-11 18:53:56,519 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.32 sec -2013-09-11 18:53:57,525 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.32 sec -2013-09-11 18:53:58,531 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 34.32 sec -2013-09-11 18:53:59,537 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 68.03 sec -2013-09-11 18:54:00,543 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.66 sec -2013-09-11 18:54:01,548 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.66 sec -2013-09-11 18:54:02,554 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.66 sec -2013-09-11 18:54:03,560 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.66 sec -2013-09-11 18:54:04,566 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.66 sec -2013-09-11 18:54:05,572 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.66 sec -2013-09-11 18:54:06,578 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.66 sec -2013-09-11 18:54:07,584 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.66 sec -2013-09-11 18:54:08,590 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.66 sec -2013-09-11 18:54:09,596 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 69.66 sec -2013-09-11 18:54:10,602 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 69.66 sec -2013-09-11 18:54:11,608 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 69.66 sec -2013-09-11 18:54:12,616 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.68 sec -2013-09-11 18:54:13,622 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.68 sec -2013-09-11 18:54:14,628 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 89.68 sec -MapReduce Total cumulative CPU time: 1 minutes 29 seconds 680 msec -Ended Job = job_201309101627_0298 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0299 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:54:17,084 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:54:28,119 Stage-2 map = 46%, reduce = 0% -2013-09-11 18:54:31,128 Stage-2 map = 50%, reduce = 0% -2013-09-11 18:54:34,137 Stage-2 map = 96%, reduce = 0% -2013-09-11 18:54:36,145 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.19 sec -2013-09-11 18:54:37,151 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.19 sec -2013-09-11 18:54:38,156 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.19 sec -2013-09-11 18:54:39,161 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.19 sec -2013-09-11 18:54:40,166 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.19 sec -2013-09-11 18:54:41,171 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.19 sec -2013-09-11 18:54:42,176 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.19 sec -2013-09-11 18:54:43,181 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 24.19 sec -2013-09-11 18:54:44,187 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.19 sec -2013-09-11 18:54:45,192 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.19 sec -2013-09-11 18:54:46,198 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 24.19 sec -2013-09-11 18:54:47,204 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.71 sec -2013-09-11 18:54:48,210 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 28.71 sec -MapReduce Total cumulative CPU time: 28 seconds 710 msec -Ended Job = job_201309101627_0299 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 89.68 sec HDFS Read: 84536695 HDFS Write: 146202868 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 28.71 sec HDFS Read: 146210123 HDFS Write: 256 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 58 seconds 390 msec -OK -Time taken: 80.119 seconds, Fetched: 10 row(s) -hive> quit; --- ещё более сложная агрегация.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8402@mturlrep13_201309111854_283727002.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8865@mturlrep13_201309111855_86754766.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0300 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:55:11,854 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:55:18,883 Stage-1 map = 36%, reduce = 0% -2013-09-11 18:55:21,896 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:55:24,917 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.54 sec -2013-09-11 18:55:25,924 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.54 sec -2013-09-11 18:55:26,932 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.54 sec -2013-09-11 18:55:27,938 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.54 sec -2013-09-11 18:55:28,944 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.54 sec -2013-09-11 18:55:29,950 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.54 sec -2013-09-11 18:55:30,955 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 33.54 sec -2013-09-11 18:55:31,961 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 33.54 sec -2013-09-11 18:55:32,967 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 33.54 sec -2013-09-11 18:55:33,973 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 33.54 sec -2013-09-11 18:55:34,979 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.54 sec -2013-09-11 18:55:35,985 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 33.54 sec -2013-09-11 18:55:36,991 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 49.64 sec -2013-09-11 18:55:37,996 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.87 sec -2013-09-11 18:55:39,002 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.87 sec -2013-09-11 18:55:40,008 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.87 sec -2013-09-11 18:55:41,013 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.87 sec -2013-09-11 18:55:42,019 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.87 sec -2013-09-11 18:55:43,025 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 67.87 sec -2013-09-11 18:55:44,032 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 72.63 sec -2013-09-11 18:55:45,037 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.38 sec -2013-09-11 18:55:46,042 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 77.38 sec -MapReduce Total cumulative CPU time: 1 minutes 17 seconds 380 msec -Ended Job = job_201309101627_0300 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 77.38 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 380 msec -OK -Time taken: 45.248 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11369@mturlrep13_201309111855_856551231.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11797@mturlrep13_201309111855_1646605999.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0301 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:56:00,548 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:56:07,575 Stage-1 map = 39%, reduce = 0% -2013-09-11 18:56:10,588 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:56:12,602 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.19 sec -2013-09-11 18:56:13,609 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.19 sec -2013-09-11 18:56:14,617 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.19 sec -2013-09-11 18:56:15,624 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.19 sec -2013-09-11 18:56:16,630 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.19 sec -2013-09-11 18:56:17,636 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.19 sec -2013-09-11 18:56:18,641 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.19 sec -2013-09-11 18:56:19,647 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 31.19 sec -2013-09-11 18:56:20,653 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 31.19 sec -2013-09-11 18:56:21,659 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 31.19 sec -2013-09-11 18:56:22,665 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 31.19 sec -2013-09-11 18:56:23,671 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 46.73 sec -2013-09-11 18:56:24,676 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 64.93 sec -2013-09-11 18:56:25,682 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 64.93 sec -2013-09-11 18:56:26,687 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 64.93 sec -2013-09-11 18:56:27,693 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 64.93 sec -2013-09-11 18:56:28,698 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 64.93 sec -2013-09-11 18:56:29,703 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 64.93 sec -2013-09-11 18:56:30,709 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 64.93 sec -2013-09-11 18:56:31,717 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 69.62 sec -2013-09-11 18:56:32,722 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 74.09 sec -2013-09-11 18:56:33,728 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 74.09 sec -MapReduce Total cumulative CPU time: 1 minutes 14 seconds 90 msec -Ended Job = job_201309101627_0301 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 74.09 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 14 seconds 90 msec -OK -Time taken: 41.584 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13564@mturlrep13_201309111856_1947594684.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14004@mturlrep13_201309111856_1046197498.txt -hive> SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0302 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:56:47,624 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:56:54,649 Stage-1 map = 43%, reduce = 0% -2013-09-11 18:56:58,669 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.32 sec -2013-09-11 18:56:59,676 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.32 sec -2013-09-11 18:57:00,687 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.32 sec -2013-09-11 18:57:01,693 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.32 sec -2013-09-11 18:57:02,701 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.32 sec -2013-09-11 18:57:03,707 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.32 sec -2013-09-11 18:57:04,713 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.32 sec -2013-09-11 18:57:05,718 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 32.32 sec -2013-09-11 18:57:06,723 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 32.32 sec -2013-09-11 18:57:07,729 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 32.32 sec -2013-09-11 18:57:08,734 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 32.32 sec -2013-09-11 18:57:09,740 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 32.32 sec -2013-09-11 18:57:10,745 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.85 sec -2013-09-11 18:57:11,749 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.85 sec -2013-09-11 18:57:12,754 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.85 sec -2013-09-11 18:57:13,758 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.85 sec -2013-09-11 18:57:14,763 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.85 sec -2013-09-11 18:57:15,769 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.85 sec -2013-09-11 18:57:16,774 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.85 sec -2013-09-11 18:57:17,780 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.85 sec -2013-09-11 18:57:18,787 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 73.17 sec -2013-09-11 18:57:19,793 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 73.17 sec -2013-09-11 18:57:20,799 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 73.17 sec -MapReduce Total cumulative CPU time: 1 minutes 13 seconds 170 msec -Ended Job = job_201309101627_0302 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 73.17 sec HDFS Read: 84536695 HDFS Write: 889 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 13 seconds 170 msec -OK -Time taken: 41.413 seconds, Fetched: 10 row(s) -hive> quit; --- то же самое, но без сортировки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15789@mturlrep13_201309111857_1616448565.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16290@mturlrep13_201309111857_668645670.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0303 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:57:42,641 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:57:49,670 Stage-1 map = 7%, reduce = 0% -2013-09-11 18:57:52,683 Stage-1 map = 22%, reduce = 0% -2013-09-11 18:57:55,696 Stage-1 map = 29%, reduce = 0% -2013-09-11 18:57:58,710 Stage-1 map = 36%, reduce = 0% -2013-09-11 18:58:01,726 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 43.37 sec -2013-09-11 18:58:02,733 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.76 sec -2013-09-11 18:58:03,741 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.76 sec -2013-09-11 18:58:04,747 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.76 sec -2013-09-11 18:58:05,753 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.76 sec -2013-09-11 18:58:06,758 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.76 sec -2013-09-11 18:58:07,763 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.76 sec -2013-09-11 18:58:08,769 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.76 sec -2013-09-11 18:58:09,775 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.76 sec -2013-09-11 18:58:10,781 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:11,788 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:12,793 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:13,799 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:14,805 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:15,811 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:16,816 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:17,822 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:18,827 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:19,833 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:20,839 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:21,845 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 53.76 sec -2013-09-11 18:58:22,850 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 78.8 sec -2013-09-11 18:58:23,855 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 105.63 sec -2013-09-11 18:58:24,860 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 105.63 sec -2013-09-11 18:58:25,866 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 105.63 sec -2013-09-11 18:58:26,871 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 105.63 sec -2013-09-11 18:58:27,877 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 105.63 sec -2013-09-11 18:58:28,882 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 105.63 sec -2013-09-11 18:58:29,888 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 105.63 sec -2013-09-11 18:58:30,893 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 105.63 sec -2013-09-11 18:58:31,899 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 105.63 sec -2013-09-11 18:58:32,905 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 105.63 sec -2013-09-11 18:58:34,657 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 105.63 sec -2013-09-11 18:58:35,663 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 105.63 sec -2013-09-11 18:58:36,668 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 105.63 sec -2013-09-11 18:58:37,673 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 105.63 sec -2013-09-11 18:58:38,679 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 105.63 sec -2013-09-11 18:58:39,684 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 105.63 sec -2013-09-11 18:58:40,690 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 105.63 sec -2013-09-11 18:58:41,695 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 105.63 sec -2013-09-11 18:58:42,702 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 127.04 sec -2013-09-11 18:58:43,707 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 148.11 sec -2013-09-11 18:58:44,713 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 148.11 sec -MapReduce Total cumulative CPU time: 2 minutes 28 seconds 110 msec -Ended Job = job_201309101627_0303 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0304 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 18:58:47,302 Stage-2 map = 0%, reduce = 0% -2013-09-11 18:59:01,348 Stage-2 map = 28%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:02,353 Stage-2 map = 28%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:03,358 Stage-2 map = 28%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:04,362 Stage-2 map = 28%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:05,370 Stage-2 map = 28%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:06,375 Stage-2 map = 28%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:07,381 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:08,385 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:09,390 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:10,395 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:11,400 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:12,405 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:13,410 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:14,415 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:15,419 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:16,424 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:17,429 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:18,434 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:19,438 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:20,452 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:21,457 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:22,461 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:23,466 Stage-2 map = 78%, reduce = 0%, Cumulative CPU 17.05 sec -2013-09-11 18:59:24,471 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.88 sec -2013-09-11 18:59:25,476 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.88 sec -2013-09-11 18:59:26,480 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.88 sec -2013-09-11 18:59:27,485 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.88 sec -2013-09-11 18:59:28,489 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.88 sec -2013-09-11 18:59:29,494 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.88 sec -2013-09-11 18:59:30,498 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.88 sec -2013-09-11 18:59:31,503 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.88 sec -2013-09-11 18:59:32,507 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.88 sec -2013-09-11 18:59:33,511 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 42.88 sec -2013-09-11 18:59:34,515 Stage-2 map = 100%, reduce = 69%, Cumulative CPU 42.88 sec -2013-09-11 18:59:35,520 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.19 sec -2013-09-11 18:59:36,525 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.19 sec -2013-09-11 18:59:37,530 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 49.19 sec -MapReduce Total cumulative CPU time: 49 seconds 190 msec -Ended Job = job_201309101627_0304 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 148.11 sec HDFS Read: 84944733 HDFS Write: 241346048 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 49.19 sec HDFS Read: 241349358 HDFS Write: 268 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 17 seconds 300 msec -OK -Time taken: 124.833 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19630@mturlrep13_201309111859_2048292740.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20052@mturlrep13_201309111859_1861199290.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0305 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 18:59:51,492 Stage-1 map = 0%, reduce = 0% -2013-09-11 18:59:58,519 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:00:01,534 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 20.11 sec -2013-09-11 19:00:02,541 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 20.11 sec -2013-09-11 19:00:03,550 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 20.11 sec -2013-09-11 19:00:04,556 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 20.11 sec -2013-09-11 19:00:05,562 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 20.11 sec -2013-09-11 19:00:06,568 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 20.11 sec -2013-09-11 19:00:07,574 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 20.11 sec -2013-09-11 19:00:08,580 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 20.11 sec -2013-09-11 19:00:09,586 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 20.11 sec -2013-09-11 19:00:10,591 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 20.11 sec -2013-09-11 19:00:11,599 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.39 sec -2013-09-11 19:00:12,604 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.39 sec -2013-09-11 19:00:13,610 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.39 sec -2013-09-11 19:00:14,616 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.39 sec -2013-09-11 19:00:15,621 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.39 sec -2013-09-11 19:00:16,627 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.39 sec -2013-09-11 19:00:17,633 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.39 sec -2013-09-11 19:00:18,638 Stage-1 map = 54%, reduce = 8%, Cumulative CPU 54.39 sec -2013-09-11 19:00:19,644 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:20,649 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:21,655 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:22,660 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:23,666 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:24,671 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:25,676 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:26,682 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:27,687 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:28,692 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:29,698 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 54.39 sec -2013-09-11 19:00:30,703 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 78.63 sec -2013-09-11 19:00:31,708 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.04 sec -2013-09-11 19:00:32,713 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.04 sec -2013-09-11 19:00:33,717 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 106.04 sec -2013-09-11 19:00:34,722 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 106.04 sec -2013-09-11 19:00:35,727 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 106.04 sec -2013-09-11 19:00:36,732 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 106.04 sec -2013-09-11 19:00:37,736 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 106.04 sec -2013-09-11 19:00:38,742 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 106.04 sec -2013-09-11 19:00:39,746 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 106.04 sec -2013-09-11 19:00:40,751 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 106.04 sec -2013-09-11 19:00:41,756 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 106.04 sec -2013-09-11 19:00:42,762 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 106.04 sec -2013-09-11 19:00:43,767 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 106.04 sec -2013-09-11 19:00:45,242 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 106.04 sec -2013-09-11 19:00:46,247 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 106.04 sec -2013-09-11 19:00:47,253 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 106.04 sec -2013-09-11 19:00:48,259 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 106.04 sec -2013-09-11 19:00:49,264 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 106.04 sec -2013-09-11 19:00:50,275 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 106.04 sec -2013-09-11 19:00:51,283 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.57 sec -2013-09-11 19:00:52,288 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.57 sec -MapReduce Total cumulative CPU time: 2 minutes 27 seconds 570 msec -Ended Job = job_201309101627_0305 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0306 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:00:54,856 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:01:08,898 Stage-2 map = 28%, reduce = 0% -2013-09-11 19:01:15,016 Stage-2 map = 50%, reduce = 0% -2013-09-11 19:01:24,046 Stage-2 map = 78%, reduce = 0% -2013-09-11 19:01:32,074 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:33,080 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:34,085 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:35,090 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:36,095 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:37,100 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:38,105 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:39,110 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:40,116 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:41,121 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:42,126 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.92 sec -2013-09-11 19:01:43,131 Stage-2 map = 100%, reduce = 69%, Cumulative CPU 43.92 sec -2013-09-11 19:01:44,142 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.31 sec -2013-09-11 19:01:45,148 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.31 sec -MapReduce Total cumulative CPU time: 50 seconds 310 msec -Ended Job = job_201309101627_0306 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 147.57 sec HDFS Read: 84944733 HDFS Write: 241346048 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 50.31 sec HDFS Read: 241349354 HDFS Write: 268 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 17 seconds 880 msec -OK -Time taken: 122.023 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23810@mturlrep13_201309111901_515641948.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24270@mturlrep13_201309111901_54212452.txt -hive> SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0307 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:01:57,968 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:02:06,001 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:02:09,013 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:02:12,025 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:02:15,039 Stage-1 map = 39%, reduce = 0% -2013-09-11 19:02:18,050 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:02:19,060 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.29 sec -2013-09-11 19:02:20,067 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.29 sec -2013-09-11 19:02:21,073 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.29 sec -2013-09-11 19:02:22,078 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.29 sec -2013-09-11 19:02:23,083 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.29 sec -2013-09-11 19:02:24,089 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.29 sec -2013-09-11 19:02:25,094 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.29 sec -2013-09-11 19:02:26,099 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:27,104 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:28,109 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:29,114 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:30,119 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:31,124 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:32,130 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:33,135 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:34,140 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:35,146 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:36,151 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 55.29 sec -2013-09-11 19:02:37,156 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 78.76 sec -2013-09-11 19:02:38,160 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 78.76 sec -2013-09-11 19:02:39,165 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 105.48 sec -2013-09-11 19:02:40,169 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 105.48 sec -2013-09-11 19:02:41,174 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 105.48 sec -2013-09-11 19:02:42,178 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 105.48 sec -2013-09-11 19:02:43,183 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 105.48 sec -2013-09-11 19:02:44,188 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 105.48 sec -2013-09-11 19:02:45,193 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 105.48 sec -2013-09-11 19:02:46,198 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 105.48 sec -2013-09-11 19:02:47,202 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 105.48 sec -2013-09-11 19:02:48,207 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 105.48 sec -2013-09-11 19:02:49,212 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 105.48 sec -2013-09-11 19:02:50,845 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 105.48 sec -2013-09-11 19:02:51,850 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 105.48 sec -2013-09-11 19:02:52,855 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 105.48 sec -2013-09-11 19:02:53,861 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 105.48 sec -2013-09-11 19:02:54,866 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 105.48 sec -2013-09-11 19:02:55,871 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 105.48 sec -2013-09-11 19:02:56,875 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 105.48 sec -2013-09-11 19:02:57,883 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 105.48 sec -2013-09-11 19:02:58,890 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.84 sec -2013-09-11 19:02:59,895 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 147.84 sec -MapReduce Total cumulative CPU time: 2 minutes 27 seconds 840 msec -Ended Job = job_201309101627_0307 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0308 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:03:03,355 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:03:16,392 Stage-2 map = 28%, reduce = 0% -2013-09-11 19:03:22,408 Stage-2 map = 50%, reduce = 0% -2013-09-11 19:03:31,433 Stage-2 map = 78%, reduce = 0% -2013-09-11 19:03:39,456 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.73 sec -2013-09-11 19:03:40,460 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.73 sec -2013-09-11 19:03:41,464 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.73 sec -2013-09-11 19:03:42,469 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.73 sec -2013-09-11 19:03:43,473 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.73 sec -2013-09-11 19:03:44,478 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.73 sec -2013-09-11 19:03:45,482 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.73 sec -2013-09-11 19:03:46,487 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.73 sec -2013-09-11 19:03:47,491 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.73 sec -2013-09-11 19:03:48,496 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 43.73 sec -2013-09-11 19:03:49,500 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 43.73 sec -2013-09-11 19:03:50,504 Stage-2 map = 100%, reduce = 68%, Cumulative CPU 43.73 sec -2013-09-11 19:03:51,509 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.23 sec -2013-09-11 19:03:52,514 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 50.23 sec -MapReduce Total cumulative CPU time: 50 seconds 230 msec -Ended Job = job_201309101627_0308 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 147.84 sec HDFS Read: 84944733 HDFS Write: 241346048 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 50.23 sec HDFS Read: 241349358 HDFS Write: 268 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 18 seconds 70 msec -OK -Time taken: 121.826 seconds, Fetched: 10 row(s) -hive> quit; --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27368@mturlrep13_201309111904_1529836462.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27852@mturlrep13_201309111904_1081537639.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0309 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-11 19:04:14,784 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:04:19,814 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.75 sec -2013-09-11 19:04:20,820 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.75 sec -2013-09-11 19:04:21,827 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.75 sec -2013-09-11 19:04:22,832 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.75 sec -2013-09-11 19:04:23,838 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.75 sec -2013-09-11 19:04:24,843 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.39 sec -2013-09-11 19:04:25,848 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.39 sec -2013-09-11 19:04:26,854 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 15.39 sec -MapReduce Total cumulative CPU time: 15 seconds 390 msec -Ended Job = job_201309101627_0309 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 15.39 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 15 seconds 390 msec -OK -Time taken: 21.676 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29092@mturlrep13_201309111904_253068464.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29537@mturlrep13_201309111904_1909407625.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0310 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-11 19:04:40,721 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:04:44,746 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.65 sec -2013-09-11 19:04:45,753 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.65 sec -2013-09-11 19:04:46,761 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.65 sec -2013-09-11 19:04:47,767 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.65 sec -2013-09-11 19:04:48,774 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 11.23 sec -2013-09-11 19:04:49,780 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.19 sec -2013-09-11 19:04:50,786 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 15.19 sec -MapReduce Total cumulative CPU time: 15 seconds 190 msec -Ended Job = job_201309101627_0310 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 15.19 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 15 seconds 190 msec -OK -Time taken: 18.26 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30764@mturlrep13_201309111904_2146737469.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31195@mturlrep13_201309111904_832602397.txt -hive> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks is set to 0 since there's no reduce operator -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0311 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 0 -2013-09-11 19:05:04,452 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:05:09,482 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.95 sec -2013-09-11 19:05:10,490 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.95 sec -2013-09-11 19:05:11,498 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.95 sec -2013-09-11 19:05:12,504 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 7.95 sec -2013-09-11 19:05:13,510 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.6 sec -2013-09-11 19:05:14,516 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 15.6 sec -2013-09-11 19:05:15,522 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 15.6 sec -MapReduce Total cumulative CPU time: 15 seconds 600 msec -Ended Job = job_201309101627_0311 -MapReduce Jobs Launched: -Job 0: Map: 4 Cumulative CPU: 15.6 sec HDFS Read: 57312623 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 15 seconds 600 msec -OK -Time taken: 19.128 seconds -hive> quit; --- мощная фильтрация по столбцу типа UInt64.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_667@mturlrep13_201309111905_1082929224.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1182@mturlrep13_201309111905_294903948.txt -hive> SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0312 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:05:36,814 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:05:43,841 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:05:44,854 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.28 sec -2013-09-11 19:05:45,861 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.28 sec -2013-09-11 19:05:46,869 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.28 sec -2013-09-11 19:05:47,876 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.28 sec -2013-09-11 19:05:48,882 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.28 sec -2013-09-11 19:05:49,888 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.28 sec -2013-09-11 19:05:50,894 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.28 sec -2013-09-11 19:05:51,901 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.6 sec -2013-09-11 19:05:52,906 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.6 sec -2013-09-11 19:05:53,912 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.6 sec -2013-09-11 19:05:54,917 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.6 sec -2013-09-11 19:05:55,923 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.6 sec -2013-09-11 19:05:56,928 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.6 sec -2013-09-11 19:05:57,935 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.72 sec -2013-09-11 19:05:58,941 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.72 sec -MapReduce Total cumulative CPU time: 37 seconds 720 msec -Ended Job = job_201309101627_0312 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 37.72 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 37 seconds 720 msec -OK -8428 -Time taken: 32.101 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2715@mturlrep13_201309111906_1847551068.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3181@mturlrep13_201309111906_1226066334.txt -hive> SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0313 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:06:12,741 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:06:19,778 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.88 sec -2013-09-11 19:06:20,785 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.88 sec -2013-09-11 19:06:21,792 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.88 sec -2013-09-11 19:06:22,798 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.88 sec -2013-09-11 19:06:23,804 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.88 sec -2013-09-11 19:06:24,810 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.88 sec -2013-09-11 19:06:25,816 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 17.88 sec -2013-09-11 19:06:26,821 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 26.71 sec -2013-09-11 19:06:27,827 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.82 sec -2013-09-11 19:06:28,832 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.82 sec -2013-09-11 19:06:29,837 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.82 sec -2013-09-11 19:06:30,842 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.82 sec -2013-09-11 19:06:31,847 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.82 sec -2013-09-11 19:06:32,855 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 38.11 sec -2013-09-11 19:06:33,861 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 38.11 sec -2013-09-11 19:06:34,866 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 38.11 sec -MapReduce Total cumulative CPU time: 38 seconds 110 msec -Ended Job = job_201309101627_0313 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 38.11 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 38 seconds 110 msec -OK -8428 -Time taken: 30.408 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4693@mturlrep13_201309111906_1529180994.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5305@mturlrep13_201309111906_779445337.txt -hive> SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%';; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0314 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:06:48,153 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:06:56,192 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.5 sec -2013-09-11 19:06:57,200 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.5 sec -2013-09-11 19:06:58,207 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.5 sec -2013-09-11 19:06:59,213 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.5 sec -2013-09-11 19:07:00,219 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.5 sec -2013-09-11 19:07:01,225 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.5 sec -2013-09-11 19:07:02,232 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.5 sec -2013-09-11 19:07:03,239 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.91 sec -2013-09-11 19:07:04,244 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.91 sec -2013-09-11 19:07:05,250 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.91 sec -2013-09-11 19:07:06,255 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.91 sec -2013-09-11 19:07:07,261 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.91 sec -2013-09-11 19:07:08,266 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.91 sec -2013-09-11 19:07:09,274 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.92 sec -2013-09-11 19:07:10,280 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.92 sec -2013-09-11 19:07:11,285 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 37.92 sec -MapReduce Total cumulative CPU time: 37 seconds 920 msec -Ended Job = job_201309101627_0314 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 37.92 sec HDFS Read: 109451651 HDFS Write: 5 SUCCESS -Total MapReduce CPU Time Spent: 37 seconds 920 msec -OK -8428 -Time taken: 30.38 seconds, Fetched: 1 row(s) -hive> quit; --- фильтрация по поиску подстроки в строке.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6820@mturlrep13_201309111907_920541373.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7269@mturlrep13_201309111907_775629306.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0315 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:07:32,401 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:07:39,430 Stage-1 map = 32%, reduce = 0% -2013-09-11 19:07:40,443 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 9.06 sec -2013-09-11 19:07:41,451 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 19:07:42,458 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 19:07:43,465 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 19:07:44,471 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 19:07:45,478 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 19:07:46,485 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 19:07:47,492 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 19:07:48,498 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 18.91 sec -2013-09-11 19:07:49,504 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.92 sec -2013-09-11 19:07:50,509 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.92 sec -2013-09-11 19:07:51,515 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.92 sec -2013-09-11 19:07:52,521 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.92 sec -2013-09-11 19:07:53,527 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.92 sec -2013-09-11 19:07:54,534 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.18 sec -2013-09-11 19:07:55,540 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.18 sec -MapReduce Total cumulative CPU time: 42 seconds 180 msec -Ended Job = job_201309101627_0315 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0316 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:07:59,142 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:08:01,152 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-11 19:08:02,158 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-11 19:08:03,163 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-11 19:08:04,168 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-11 19:08:05,174 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-11 19:08:06,181 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-11 19:08:07,186 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.82 sec -2013-09-11 19:08:08,191 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.24 sec -2013-09-11 19:08:09,197 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.24 sec -2013-09-11 19:08:10,203 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.24 sec -MapReduce Total cumulative CPU time: 2 seconds 240 msec -Ended Job = job_201309101627_0316 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.18 sec HDFS Read: 136675723 HDFS Write: 5172 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.24 sec HDFS Read: 5941 HDFS Write: 984 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 420 msec -OK -Time taken: 47.807 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9501@mturlrep13_201309111908_1204289107.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9961@mturlrep13_201309111908_623470333.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0317 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:08:23,302 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:08:31,343 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.52 sec -2013-09-11 19:08:32,351 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.52 sec -2013-09-11 19:08:33,359 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.52 sec -2013-09-11 19:08:34,366 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.52 sec -2013-09-11 19:08:35,373 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.52 sec -2013-09-11 19:08:36,379 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.52 sec -2013-09-11 19:08:37,386 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.52 sec -2013-09-11 19:08:38,394 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.52 sec -2013-09-11 19:08:39,400 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 19:08:40,406 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 19:08:41,412 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 19:08:42,419 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 19:08:43,426 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 19:08:44,433 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 39.17 sec -2013-09-11 19:08:45,440 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.39 sec -2013-09-11 19:08:46,446 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.39 sec -MapReduce Total cumulative CPU time: 41 seconds 390 msec -Ended Job = job_201309101627_0317 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0318 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:08:48,932 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:08:50,941 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-11 19:08:51,947 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-11 19:08:52,954 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-11 19:08:53,960 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-11 19:08:54,965 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-11 19:08:55,971 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-11 19:08:56,976 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.78 sec -2013-09-11 19:08:57,983 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.78 sec -2013-09-11 19:08:58,989 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.3 sec -2013-09-11 19:08:59,995 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.3 sec -2013-09-11 19:09:01,000 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.3 sec -MapReduce Total cumulative CPU time: 2 seconds 300 msec -Ended Job = job_201309101627_0318 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.39 sec HDFS Read: 136675723 HDFS Write: 5172 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.3 sec HDFS Read: 5941 HDFS Write: 984 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 690 msec -OK -Time taken: 45.178 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12180@mturlrep13_201309111909_1253260332.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12625@mturlrep13_201309111909_397580064.txt -hive> SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0319 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:09:13,988 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:09:22,031 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.22 sec -2013-09-11 19:09:23,039 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.22 sec -2013-09-11 19:09:24,047 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.22 sec -2013-09-11 19:09:25,054 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.22 sec -2013-09-11 19:09:26,060 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.22 sec -2013-09-11 19:09:27,066 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.22 sec -2013-09-11 19:09:28,072 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.22 sec -2013-09-11 19:09:29,080 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.22 sec -2013-09-11 19:09:30,086 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.92 sec -2013-09-11 19:09:31,091 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.92 sec -2013-09-11 19:09:32,097 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.92 sec -2013-09-11 19:09:33,103 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.92 sec -2013-09-11 19:09:34,108 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.92 sec -2013-09-11 19:09:35,116 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 38.05 sec -2013-09-11 19:09:36,160 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.26 sec -2013-09-11 19:09:37,167 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.26 sec -MapReduce Total cumulative CPU time: 40 seconds 260 msec -Ended Job = job_201309101627_0319 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0320 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:09:39,694 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:09:41,703 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:09:42,708 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:09:43,713 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:09:44,718 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:09:45,722 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:09:46,727 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:09:47,732 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:09:48,737 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:09:49,743 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -2013-09-11 19:09:50,749 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.23 sec -MapReduce Total cumulative CPU time: 2 seconds 230 msec -Ended Job = job_201309101627_0320 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.26 sec HDFS Read: 136675723 HDFS Write: 5172 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.23 sec HDFS Read: 5941 HDFS Write: 984 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 490 msec -OK -Time taken: 44.192 seconds, Fetched: 10 row(s) -hive> quit; --- вынимаем большие столбцы, фильтрация по строке.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14855@mturlrep13_201309111909_548656441.txt -hive> ; -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15305@mturlrep13_201309111910_808798094.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0321 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:10:12,656 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:10:19,683 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:10:22,695 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:10:23,708 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.32 sec -2013-09-11 19:10:24,746 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.32 sec -2013-09-11 19:10:25,755 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.32 sec -2013-09-11 19:10:26,761 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.32 sec -2013-09-11 19:10:27,769 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.32 sec -2013-09-11 19:10:28,775 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.32 sec -2013-09-11 19:10:29,781 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.32 sec -2013-09-11 19:10:30,787 Stage-1 map = 73%, reduce = 8%, Cumulative CPU 24.32 sec -2013-09-11 19:10:31,793 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 24.32 sec -2013-09-11 19:10:32,799 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 24.32 sec -2013-09-11 19:10:33,804 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 36.21 sec -2013-09-11 19:10:34,810 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.39 sec -2013-09-11 19:10:35,816 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.39 sec -2013-09-11 19:10:36,824 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 50.53 sec -2013-09-11 19:10:37,829 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 50.53 sec -2013-09-11 19:10:38,835 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 50.53 sec -2013-09-11 19:10:39,841 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 50.53 sec -2013-09-11 19:10:40,848 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 50.53 sec -2013-09-11 19:10:41,855 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.6 sec -2013-09-11 19:10:42,861 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.6 sec -2013-09-11 19:10:43,867 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 52.6 sec -MapReduce Total cumulative CPU time: 52 seconds 600 msec -Ended Job = job_201309101627_0321 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0322 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:10:47,360 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:10:48,366 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:10:49,371 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:10:50,377 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:10:51,382 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:10:52,387 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:10:53,392 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:10:54,396 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:10:55,402 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:10:56,407 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -2013-09-11 19:10:57,412 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -2013-09-11 19:10:58,418 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -MapReduce Total cumulative CPU time: 2 seconds 200 msec -Ended Job = job_201309101627_0322 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 52.6 sec HDFS Read: 298803179 HDFS Write: 12221 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.2 sec HDFS Read: 12990 HDFS Write: 2646 SUCCESS -Total MapReduce CPU Time Spent: 54 seconds 800 msec -OK -Time taken: 56.389 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18714@mturlrep13_201309111911_81889019.txt -hive> ; -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19205@mturlrep13_201309111911_2026522553.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0323 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:11:11,864 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:11:19,897 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:11:22,917 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.28 sec -2013-09-11 19:11:23,924 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.28 sec -2013-09-11 19:11:24,933 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.28 sec -2013-09-11 19:11:25,939 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.28 sec -2013-09-11 19:11:26,946 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.28 sec -2013-09-11 19:11:27,952 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.28 sec -2013-09-11 19:11:28,958 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.28 sec -2013-09-11 19:11:29,963 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 25.28 sec -2013-09-11 19:11:30,969 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 25.28 sec -2013-09-11 19:11:31,974 Stage-1 map = 89%, reduce = 17%, Cumulative CPU 36.84 sec -2013-09-11 19:11:32,980 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.31 sec -2013-09-11 19:11:33,985 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.31 sec -2013-09-11 19:11:34,991 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 49.31 sec -2013-09-11 19:11:35,998 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.48 sec -2013-09-11 19:11:37,004 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.48 sec -2013-09-11 19:11:38,011 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.48 sec -MapReduce Total cumulative CPU time: 53 seconds 480 msec -Ended Job = job_201309101627_0323 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0324 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:11:40,494 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:11:42,503 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:11:43,508 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:11:44,514 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:11:45,519 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:11:46,524 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:11:47,529 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:11:48,534 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:11:49,540 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.76 sec -2013-09-11 19:11:50,545 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.15 sec -2013-09-11 19:11:51,551 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.15 sec -MapReduce Total cumulative CPU time: 2 seconds 150 msec -Ended Job = job_201309101627_0324 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 53.48 sec HDFS Read: 298803179 HDFS Write: 12221 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.15 sec HDFS Read: 12988 HDFS Write: 2646 SUCCESS -Total MapReduce CPU Time Spent: 55 seconds 630 msec -OK -Time taken: 47.357 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21577@mturlrep13_201309111911_1538548655.txt -hive> ; -hive> quit; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22017@mturlrep13_201309111911_1851586484.txt -hive> 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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0325 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:12:06,075 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:12:13,105 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:12:15,124 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 12.29 sec -2013-09-11 19:12:16,133 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.56 sec -2013-09-11 19:12:17,141 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.56 sec -2013-09-11 19:12:18,148 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.56 sec -2013-09-11 19:12:19,155 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.56 sec -2013-09-11 19:12:20,162 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.56 sec -2013-09-11 19:12:21,169 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.56 sec -2013-09-11 19:12:22,176 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.56 sec -2013-09-11 19:12:23,182 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 24.56 sec -2013-09-11 19:12:24,188 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 24.56 sec -2013-09-11 19:12:25,194 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 24.56 sec -2013-09-11 19:12:26,209 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.93 sec -2013-09-11 19:12:27,215 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.93 sec -2013-09-11 19:12:28,221 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.93 sec -2013-09-11 19:12:29,229 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.09 sec -2013-09-11 19:12:30,235 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 53.09 sec -MapReduce Total cumulative CPU time: 53 seconds 90 msec -Ended Job = job_201309101627_0325 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0326 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:12:33,748 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:12:34,753 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-11 19:12:35,759 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-11 19:12:36,764 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-11 19:12:37,769 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-11 19:12:38,775 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-11 19:12:39,780 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-11 19:12:40,785 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-11 19:12:41,790 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.88 sec -2013-09-11 19:12:42,796 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.28 sec -2013-09-11 19:12:43,802 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.28 sec -2013-09-11 19:12:44,808 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.28 sec -MapReduce Total cumulative CPU time: 2 seconds 280 msec -Ended Job = job_201309101627_0326 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 53.09 sec HDFS Read: 298803179 HDFS Write: 12221 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.28 sec HDFS Read: 12990 HDFS Write: 2646 SUCCESS -Total MapReduce CPU Time Spent: 55 seconds 370 msec -OK -Time taken: 47.653 seconds, Fetched: 10 row(s) -hive> quit; --- чуть больше столбцы.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24402@mturlrep13_201309111912_1264895416.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24883@mturlrep13_201309111912_1271282115.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0327 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:13:05,825 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:13:15,872 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:13:18,884 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:13:21,898 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:13:27,920 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:13:30,932 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:13:36,953 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:13:37,964 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 93.08 sec -2013-09-11 19:13:38,971 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 93.08 sec -2013-09-11 19:13:39,978 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 93.08 sec -2013-09-11 19:13:40,984 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 93.08 sec -2013-09-11 19:13:41,990 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 93.08 sec -2013-09-11 19:13:42,995 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 93.08 sec -2013-09-11 19:13:44,001 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 93.08 sec -2013-09-11 19:13:45,007 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 93.08 sec -2013-09-11 19:13:46,011 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:47,016 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:48,020 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:49,060 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:50,074 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:51,078 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:52,082 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:53,086 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:54,090 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:55,095 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:56,100 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:57,105 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:58,110 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:13:59,115 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:14:00,120 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:14:01,125 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:14:02,130 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 93.08 sec -2013-09-11 19:14:03,140 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 155.04 sec -2013-09-11 19:14:04,145 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 155.04 sec -2013-09-11 19:14:05,153 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 159.81 sec -2013-09-11 19:14:06,158 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 159.81 sec -2013-09-11 19:14:07,163 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 159.81 sec -2013-09-11 19:14:08,168 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 159.81 sec -2013-09-11 19:14:09,173 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 159.81 sec -2013-09-11 19:14:10,178 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 175.38 sec -2013-09-11 19:14:11,183 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 175.38 sec -2013-09-11 19:14:12,188 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 175.38 sec -2013-09-11 19:14:13,193 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 175.38 sec -2013-09-11 19:14:14,197 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 175.38 sec -2013-09-11 19:14:15,202 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 175.38 sec -2013-09-11 19:14:16,210 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 177.7 sec -2013-09-11 19:14:17,215 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 177.7 sec -2013-09-11 19:14:18,221 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 177.7 sec -MapReduce Total cumulative CPU time: 2 minutes 57 seconds 700 msec -Ended Job = job_201309101627_0327 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 177.7 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 57 seconds 700 msec -OK -Time taken: 82.942 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26833@mturlrep13_201309111914_284626838.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27262@mturlrep13_201309111914_572123966.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0328 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:14:31,952 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:14:39,983 Stage-1 map = 4%, reduce = 0% -2013-09-11 19:14:42,994 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:14:46,006 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:14:49,018 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:14:52,064 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:14:55,075 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:15:01,097 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:15:02,183 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:03,191 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:04,196 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:05,202 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:06,208 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:07,213 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:08,219 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:09,224 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:10,230 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:11,235 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:12,240 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 76.71 sec -2013-09-11 19:15:13,247 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:14,252 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:15,257 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:16,263 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:17,267 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:18,272 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:19,276 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:20,281 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:21,286 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:22,291 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:23,296 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:24,306 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:25,311 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:26,316 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:27,381 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:28,387 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:29,392 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:30,396 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:31,401 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:32,406 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:33,411 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:34,416 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:35,421 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:36,425 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:37,429 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 76.71 sec -2013-09-11 19:15:38,434 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 111.68 sec -2013-09-11 19:15:39,439 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 111.68 sec -2013-09-11 19:15:40,443 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 150.15 sec -2013-09-11 19:15:41,447 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 150.15 sec -2013-09-11 19:15:42,452 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 150.15 sec -2013-09-11 19:15:43,457 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 150.15 sec -2013-09-11 19:15:44,462 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 150.15 sec -2013-09-11 19:15:45,466 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 150.15 sec -2013-09-11 19:15:46,501 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 153.5 sec -2013-09-11 19:15:47,506 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 153.5 sec -2013-09-11 19:15:48,511 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 153.5 sec -2013-09-11 19:15:49,516 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 153.5 sec -MapReduce Total cumulative CPU time: 2 minutes 33 seconds 500 msec -Ended Job = job_201309101627_0328 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 153.5 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 2 minutes 33 seconds 500 msec -OK -Time taken: 85.578 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30831@mturlrep13_201309111915_2050397294.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31290@mturlrep13_201309111915_470444048.txt -hive> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0329 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:16:06,376 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:16:16,414 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:16:19,425 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:16:22,437 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:16:25,449 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:16:31,471 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:16:34,482 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:16:37,501 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 94.38 sec -2013-09-11 19:16:38,508 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 94.38 sec -2013-09-11 19:16:39,514 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 94.38 sec -2013-09-11 19:16:40,521 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 94.38 sec -2013-09-11 19:16:41,526 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 94.38 sec -2013-09-11 19:16:42,532 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 94.38 sec -2013-09-11 19:16:43,538 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 94.38 sec -2013-09-11 19:16:44,543 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:45,549 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:46,554 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:47,559 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:48,565 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:49,570 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:50,575 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:51,580 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:52,585 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:53,590 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:54,595 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:55,600 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:56,605 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:57,610 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:58,616 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:16:59,621 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:17:00,626 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:17:01,630 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:17:02,644 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 94.38 sec -2013-09-11 19:17:03,668 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 164.11 sec -2013-09-11 19:17:04,676 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 164.11 sec -2013-09-11 19:17:05,681 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 172.49 sec -2013-09-11 19:17:06,687 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 172.49 sec -2013-09-11 19:17:07,691 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 179.48 sec -2013-09-11 19:17:08,696 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 179.48 sec -2013-09-11 19:17:09,701 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 179.48 sec -2013-09-11 19:17:10,707 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 181.61 sec -2013-09-11 19:17:11,712 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 181.61 sec -2013-09-11 19:17:12,717 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 181.61 sec -MapReduce Total cumulative CPU time: 3 minutes 1 seconds 610 msec -Ended Job = job_201309101627_0329 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 181.61 sec HDFS Read: 1082943442 HDFS Write: 5318 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 1 seconds 610 msec -OK -Time taken: 77.012 seconds, Fetched: 10 row(s) -hive> quit; --- плохой запрос - вынимаем все столбцы.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_745@mturlrep13_201309111917_444909204.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1250@mturlrep13_201309111917_461322150.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0330 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:17:33,067 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:17:40,099 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.68 sec -2013-09-11 19:17:41,107 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.68 sec -2013-09-11 19:17:42,114 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.68 sec -2013-09-11 19:17:43,120 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.68 sec -2013-09-11 19:17:44,125 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.68 sec -2013-09-11 19:17:45,131 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.68 sec -2013-09-11 19:17:46,136 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.68 sec -2013-09-11 19:17:47,142 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.68 sec -2013-09-11 19:17:48,148 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 28.05 sec -2013-09-11 19:17:49,153 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-11 19:17:50,158 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-11 19:17:51,163 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-11 19:17:52,168 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-11 19:17:53,173 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-11 19:17:54,179 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.3 sec -2013-09-11 19:17:55,185 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.51 sec -2013-09-11 19:17:56,191 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.51 sec -2013-09-11 19:17:57,196 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 43.51 sec -MapReduce Total cumulative CPU time: 43 seconds 510 msec -Ended Job = job_201309101627_0330 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 43.51 sec HDFS Read: 28228143 HDFS Write: 766 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 510 msec -OK -Time taken: 33.875 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2899@mturlrep13_201309111917_223797260.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3331@mturlrep13_201309111918_1567198931.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0331 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:18:10,975 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:18:18,012 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.48 sec -2013-09-11 19:18:19,020 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.48 sec -2013-09-11 19:18:20,027 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.48 sec -2013-09-11 19:18:21,033 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.48 sec -2013-09-11 19:18:22,039 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.48 sec -2013-09-11 19:18:23,045 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.48 sec -2013-09-11 19:18:24,051 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.48 sec -2013-09-11 19:18:25,056 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 37.82 sec -2013-09-11 19:18:26,062 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.82 sec -2013-09-11 19:18:27,067 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.82 sec -2013-09-11 19:18:28,072 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.82 sec -2013-09-11 19:18:29,077 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.82 sec -2013-09-11 19:18:30,082 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.82 sec -2013-09-11 19:18:31,087 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.82 sec -2013-09-11 19:18:32,092 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.82 sec -2013-09-11 19:18:33,099 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.82 sec -2013-09-11 19:18:34,105 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.82 sec -MapReduce Total cumulative CPU time: 42 seconds 820 msec -Ended Job = job_201309101627_0331 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 42.82 sec HDFS Read: 28228143 HDFS Write: 766 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 820 msec -OK -Time taken: 31.269 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5060@mturlrep13_201309111918_467637722.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5623@mturlrep13_201309111918_113416530.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0332 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:18:47,895 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:18:54,933 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 19:18:55,941 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 19:18:56,948 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 19:18:57,955 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 19:18:58,961 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 19:18:59,966 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 19:19:00,977 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 19:19:01,983 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.79 sec -2013-09-11 19:19:02,990 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.79 sec -2013-09-11 19:19:03,996 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.65 sec -2013-09-11 19:19:05,001 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.65 sec -2013-09-11 19:19:06,007 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.65 sec -2013-09-11 19:19:07,012 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.65 sec -2013-09-11 19:19:08,018 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.65 sec -2013-09-11 19:19:09,024 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.65 sec -2013-09-11 19:19:10,031 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.96 sec -2013-09-11 19:19:11,037 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.96 sec -MapReduce Total cumulative CPU time: 41 seconds 960 msec -Ended Job = job_201309101627_0332 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 41.96 sec HDFS Read: 28228143 HDFS Write: 766 SUCCESS -Total MapReduce CPU Time Spent: 41 seconds 960 msec -OK -Time taken: 31.323 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7228@mturlrep13_201309111919_1142490525.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7745@mturlrep13_201309111919_1527730811.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0333 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:19:32,813 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:19:39,842 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:19:40,855 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.55 sec -2013-09-11 19:19:41,862 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.55 sec -2013-09-11 19:19:42,869 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.55 sec -2013-09-11 19:19:43,875 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.55 sec -2013-09-11 19:19:44,881 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.55 sec -2013-09-11 19:19:45,887 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.55 sec -2013-09-11 19:19:46,893 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.55 sec -2013-09-11 19:19:47,900 Stage-1 map = 97%, reduce = 8%, Cumulative CPU 30.4 sec -2013-09-11 19:19:48,906 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 40.45 sec -2013-09-11 19:19:49,911 Stage-1 map = 100%, reduce = 8%, Cumulative CPU 40.45 sec -2013-09-11 19:19:50,916 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.45 sec -2013-09-11 19:19:51,921 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.45 sec -2013-09-11 19:19:52,926 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.45 sec -2013-09-11 19:19:53,931 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 40.45 sec -2013-09-11 19:19:54,936 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 40.45 sec -2013-09-11 19:19:55,942 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 40.45 sec -2013-09-11 19:19:56,947 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.45 sec -2013-09-11 19:19:57,955 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 46.75 sec -2013-09-11 19:19:58,961 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 46.75 sec -MapReduce Total cumulative CPU time: 46 seconds 750 msec -Ended Job = job_201309101627_0333 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 46.75 sec HDFS Read: 27820105 HDFS Write: 666 SUCCESS -Total MapReduce CPU Time Spent: 46 seconds 750 msec -OK -ялта интурист -! как одеть трехнедельного ребенка при температуре 20 градусов -! отель rattana beach hotel 3* -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -!( центробежный скважинный калибр форумы) -!(!(storm master silmarils)) -!(!(storm master silmarils)) -!(!(title:(схема sputnik hi 4000))) -!(44-фз о контрактной системе) -Time taken: 35.874 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9340@mturlrep13_201309111920_1347531078.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9842@mturlrep13_201309111920_1738943444.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0334 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:20:13,005 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:20:20,042 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.62 sec -2013-09-11 19:20:21,051 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.62 sec -2013-09-11 19:20:22,057 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.62 sec -2013-09-11 19:20:23,064 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.62 sec -2013-09-11 19:20:24,070 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.62 sec -2013-09-11 19:20:25,076 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.62 sec -2013-09-11 19:20:26,082 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.62 sec -2013-09-11 19:20:27,088 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.62 sec -2013-09-11 19:20:28,094 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.25 sec -2013-09-11 19:20:29,099 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.25 sec -2013-09-11 19:20:30,104 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.25 sec -2013-09-11 19:20:31,109 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.25 sec -2013-09-11 19:20:32,114 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.25 sec -2013-09-11 19:20:33,120 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.25 sec -2013-09-11 19:20:34,125 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.25 sec -2013-09-11 19:20:35,131 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.25 sec -2013-09-11 19:20:36,138 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.44 sec -2013-09-11 19:20:37,144 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.44 sec -MapReduce Total cumulative CPU time: 45 seconds 440 msec -Ended Job = job_201309101627_0334 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 45.44 sec HDFS Read: 27820105 HDFS Write: 666 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 440 msec -OK -ялта интурист -! как одеть трехнедельного ребенка при температуре 20 градусов -! отель rattana beach hotel 3* -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -!( центробежный скважинный калибр форумы) -!(!(storm master silmarils)) -!(!(storm master silmarils)) -!(!(title:(схема sputnik hi 4000))) -!(44-фз о контрактной системе) -Time taken: 32.533 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_11939@mturlrep13_201309111920_1151781772.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_12380@mturlrep13_201309111920_806872053.txt -hive> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0335 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:20:49,926 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:20:57,966 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.91 sec -2013-09-11 19:20:58,974 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.91 sec -2013-09-11 19:20:59,981 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.91 sec -2013-09-11 19:21:00,987 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.91 sec -2013-09-11 19:21:01,993 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.91 sec -2013-09-11 19:21:02,999 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.91 sec -2013-09-11 19:21:04,006 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.91 sec -2013-09-11 19:21:05,013 Stage-1 map = 75%, reduce = 0%, Cumulative CPU 29.4 sec -2013-09-11 19:21:06,019 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.14 sec -2013-09-11 19:21:07,024 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.14 sec -2013-09-11 19:21:08,030 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.14 sec -2013-09-11 19:21:09,036 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.14 sec -2013-09-11 19:21:10,041 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.14 sec -2013-09-11 19:21:11,047 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.14 sec -2013-09-11 19:21:12,052 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.14 sec -2013-09-11 19:21:13,058 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.14 sec -2013-09-11 19:21:14,066 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.25 sec -2013-09-11 19:21:15,072 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 45.25 sec -MapReduce Total cumulative CPU time: 45 seconds 250 msec -Ended Job = job_201309101627_0335 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 45.25 sec HDFS Read: 27820105 HDFS Write: 666 SUCCESS -Total MapReduce CPU Time Spent: 45 seconds 250 msec -OK -ялта интурист -! как одеть трехнедельного ребенка при температуре 20 градусов -! отель rattana beach hotel 3* -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -! официальный сайт ооо "группа аист"г москва, ул коцюбинского, д 4, офис 343 -!( центробежный скважинный калибр форумы) -!(!(storm master silmarils)) -!(!(storm master silmarils)) -!(!(title:(схема sputnik hi 4000))) -!(44-фз о контрактной системе) -Time taken: 32.402 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка по строкам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14015@mturlrep13_201309111921_1013339895.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_14464@mturlrep13_201309111921_1424399418.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0336 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:21:37,862 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:21:44,890 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:21:45,902 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.73 sec -2013-09-11 19:21:46,908 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.73 sec -2013-09-11 19:21:47,915 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.73 sec -2013-09-11 19:21:48,921 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.73 sec -2013-09-11 19:21:49,926 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.73 sec -2013-09-11 19:21:50,932 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.73 sec -2013-09-11 19:21:51,938 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.73 sec -2013-09-11 19:21:52,944 Stage-1 map = 72%, reduce = 17%, Cumulative CPU 20.73 sec -2013-09-11 19:21:53,953 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.75 sec -2013-09-11 19:21:54,959 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.75 sec -2013-09-11 19:21:55,964 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.75 sec -2013-09-11 19:21:56,970 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.75 sec -2013-09-11 19:21:57,975 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.75 sec -2013-09-11 19:21:58,980 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.75 sec -2013-09-11 19:21:59,984 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.75 sec -2013-09-11 19:22:00,989 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 40.75 sec -2013-09-11 19:22:01,996 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 47.37 sec -2013-09-11 19:22:03,001 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 47.37 sec -2013-09-11 19:22:04,007 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 47.37 sec -MapReduce Total cumulative CPU time: 47 seconds 370 msec -Ended Job = job_201309101627_0336 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 47.37 sec HDFS Read: 28228143 HDFS Write: 762 SUCCESS -Total MapReduce CPU Time Spent: 47 seconds 370 msec -OK -Time taken: 36.108 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16194@mturlrep13_201309111922_1473276999.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16658@mturlrep13_201309111922_883471210.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0337 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:22:16,915 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:22:24,952 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.15 sec -2013-09-11 19:22:25,959 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.15 sec -2013-09-11 19:22:26,966 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.15 sec -2013-09-11 19:22:27,972 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.15 sec -2013-09-11 19:22:28,978 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.15 sec -2013-09-11 19:22:29,984 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.15 sec -2013-09-11 19:22:30,990 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.15 sec -2013-09-11 19:22:32,000 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.15 sec -2013-09-11 19:22:33,005 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 19:22:34,011 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 19:22:35,016 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 19:22:36,022 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 19:22:37,027 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 19:22:38,032 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 19:22:39,038 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 19:22:40,043 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 19:22:41,051 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 46.37 sec -2013-09-11 19:22:42,056 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 46.37 sec -MapReduce Total cumulative CPU time: 46 seconds 370 msec -Ended Job = job_201309101627_0337 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 46.37 sec HDFS Read: 28228143 HDFS Write: 762 SUCCESS -Total MapReduce CPU Time Spent: 46 seconds 370 msec -OK -Time taken: 32.47 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18370@mturlrep13_201309111922_201833234.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18842@mturlrep13_201309111922_1636378734.txt -hive> SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0338 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:22:56,099 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:23:03,136 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.89 sec -2013-09-11 19:23:04,144 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.89 sec -2013-09-11 19:23:05,151 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.89 sec -2013-09-11 19:23:06,158 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.89 sec -2013-09-11 19:23:07,164 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.89 sec -2013-09-11 19:23:08,170 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.89 sec -2013-09-11 19:23:09,177 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.89 sec -2013-09-11 19:23:10,183 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 20.89 sec -2013-09-11 19:23:11,190 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.16 sec -2013-09-11 19:23:12,195 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.16 sec -2013-09-11 19:23:13,200 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.16 sec -2013-09-11 19:23:14,205 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.16 sec -2013-09-11 19:23:15,210 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.16 sec -2013-09-11 19:23:16,216 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.16 sec -2013-09-11 19:23:17,223 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.16 sec -2013-09-11 19:23:18,229 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 41.16 sec -2013-09-11 19:23:19,236 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 47.73 sec -2013-09-11 19:23:20,242 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 47.73 sec -MapReduce Total cumulative CPU time: 47 seconds 730 msec -Ended Job = job_201309101627_0338 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 47.73 sec HDFS Read: 28228143 HDFS Write: 762 SUCCESS -Total MapReduce CPU Time Spent: 47 seconds 730 msec -OK -Time taken: 32.441 seconds, Fetched: 10 row(s) -hive> quit; --- большая сортировка по кортежу.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20518@mturlrep13_201309111923_1417644101.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21000@mturlrep13_201309111923_777732561.txt -hive> SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0339 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:23:40,966 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:23:47,994 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:23:51,006 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:23:54,019 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:23:57,032 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:24:00,051 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.36 sec -2013-09-11 19:24:01,058 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.36 sec -2013-09-11 19:24:02,065 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.36 sec -2013-09-11 19:24:03,071 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.36 sec -2013-09-11 19:24:04,078 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.36 sec -2013-09-11 19:24:05,084 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.36 sec -2013-09-11 19:24:06,090 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.36 sec -2013-09-11 19:24:07,096 Stage-1 map = 65%, reduce = 4%, Cumulative CPU 46.36 sec -2013-09-11 19:24:08,101 Stage-1 map = 65%, reduce = 13%, Cumulative CPU 46.36 sec -2013-09-11 19:24:09,107 Stage-1 map = 65%, reduce = 13%, Cumulative CPU 46.36 sec -2013-09-11 19:24:10,113 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 46.36 sec -2013-09-11 19:24:11,119 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 46.36 sec -2013-09-11 19:24:12,124 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 46.36 sec -2013-09-11 19:24:13,130 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 46.36 sec -2013-09-11 19:24:14,136 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 46.36 sec -2013-09-11 19:24:15,142 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 46.36 sec -2013-09-11 19:24:16,148 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 46.36 sec -2013-09-11 19:24:17,153 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 66.94 sec -2013-09-11 19:24:18,159 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 66.94 sec -2013-09-11 19:24:19,164 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 90.24 sec -2013-09-11 19:24:20,170 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 90.24 sec -2013-09-11 19:24:21,175 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 90.24 sec -2013-09-11 19:24:22,180 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 90.24 sec -2013-09-11 19:24:23,185 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 90.24 sec -2013-09-11 19:24:24,190 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 90.24 sec -2013-09-11 19:24:25,196 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 90.24 sec -2013-09-11 19:24:26,202 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 90.24 sec -2013-09-11 19:24:27,207 Stage-1 map = 100%, reduce = 51%, Cumulative CPU 90.24 sec -2013-09-11 19:24:28,212 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 90.24 sec -2013-09-11 19:24:29,219 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 99.81 sec -2013-09-11 19:24:30,224 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 99.81 sec -2013-09-11 19:24:31,230 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 99.81 sec -2013-09-11 19:24:32,468 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 99.81 sec -2013-09-11 19:24:33,480 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 99.81 sec -2013-09-11 19:24:34,496 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 99.81 sec -2013-09-11 19:24:35,501 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 116.36 sec -2013-09-11 19:24:36,506 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 116.36 sec -2013-09-11 19:24:37,511 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 116.36 sec -MapReduce Total cumulative CPU time: 1 minutes 56 seconds 360 msec -Ended Job = job_201309101627_0339 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0340 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:24:41,030 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:24:43,038 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 19:24:44,043 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 19:24:45,048 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 19:24:46,052 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 19:24:47,056 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 19:24:48,060 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 19:24:49,065 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.93 sec -2013-09-11 19:24:50,070 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -2013-09-11 19:24:51,075 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -2013-09-11 19:24:52,080 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.44 sec -MapReduce Total cumulative CPU time: 2 seconds 440 msec -Ended Job = job_201309101627_0340 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 116.36 sec HDFS Read: 117363067 HDFS Write: 794 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.44 sec HDFS Read: 1563 HDFS Write: 571 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 58 seconds 800 msec -OK -Time taken: 81.295 seconds, Fetched: 19 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23567@mturlrep13_201309111924_1954971285.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23997@mturlrep13_201309111924_515409695.txt -hive> SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0341 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:25:06,659 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:25:13,685 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:25:16,697 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:25:19,708 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:25:22,720 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:25:24,736 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.82 sec -2013-09-11 19:25:25,743 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.82 sec -2013-09-11 19:25:26,752 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.82 sec -2013-09-11 19:25:27,758 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.82 sec -2013-09-11 19:25:28,764 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.82 sec -2013-09-11 19:25:29,770 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.82 sec -2013-09-11 19:25:30,776 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.82 sec -2013-09-11 19:25:31,782 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 46.82 sec -2013-09-11 19:25:32,788 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 46.82 sec -2013-09-11 19:25:33,793 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 46.82 sec -2013-09-11 19:25:34,799 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 46.82 sec -2013-09-11 19:25:35,804 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 46.82 sec -2013-09-11 19:25:36,809 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 46.82 sec -2013-09-11 19:25:37,815 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 46.82 sec -2013-09-11 19:25:38,821 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 46.82 sec -2013-09-11 19:25:39,827 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 46.82 sec -2013-09-11 19:25:40,832 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 46.82 sec -2013-09-11 19:25:41,838 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 67.29 sec -2013-09-11 19:25:42,843 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 89.57 sec -2013-09-11 19:25:43,848 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 89.57 sec -2013-09-11 19:25:44,854 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 89.57 sec -2013-09-11 19:25:45,860 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 89.57 sec -2013-09-11 19:25:46,865 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 89.57 sec -2013-09-11 19:25:47,871 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 89.57 sec -2013-09-11 19:25:48,876 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 89.57 sec -2013-09-11 19:25:49,884 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 89.57 sec -2013-09-11 19:25:50,891 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 89.57 sec -2013-09-11 19:25:51,896 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 89.57 sec -2013-09-11 19:25:52,901 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 89.57 sec -2013-09-11 19:25:53,906 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 89.57 sec -2013-09-11 19:25:54,913 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 99.36 sec -2013-09-11 19:25:55,919 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 99.36 sec -2013-09-11 19:25:56,925 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 99.36 sec -2013-09-11 19:25:57,931 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 99.36 sec -2013-09-11 19:25:58,936 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 99.36 sec -2013-09-11 19:25:59,952 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 114.98 sec -2013-09-11 19:26:00,959 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 114.98 sec -MapReduce Total cumulative CPU time: 1 minutes 54 seconds 980 msec -Ended Job = job_201309101627_0341 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0342 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:26:03,406 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:26:05,415 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:26:06,420 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:26:07,426 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:26:08,431 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:26:09,436 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:26:10,441 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:26:11,445 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.76 sec -2013-09-11 19:26:12,450 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.76 sec -2013-09-11 19:26:13,456 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.17 sec -2013-09-11 19:26:14,461 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.17 sec -MapReduce Total cumulative CPU time: 2 seconds 170 msec -Ended Job = job_201309101627_0342 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 114.98 sec HDFS Read: 117363067 HDFS Write: 794 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.17 sec HDFS Read: 1563 HDFS Write: 571 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 57 seconds 150 msec -OK -Time taken: 76.426 seconds, Fetched: 19 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27255@mturlrep13_201309111926_1222835314.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27706@mturlrep13_201309111926_904396134.txt -hive> SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0343 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:26:27,528 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:26:35,559 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:26:38,572 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:26:41,585 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:26:44,599 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:26:45,611 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.89 sec -2013-09-11 19:26:46,618 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.89 sec -2013-09-11 19:26:47,627 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.89 sec -2013-09-11 19:26:48,633 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.89 sec -2013-09-11 19:26:49,638 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.89 sec -2013-09-11 19:26:50,644 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.89 sec -2013-09-11 19:26:51,649 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 44.89 sec -2013-09-11 19:26:52,655 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 44.89 sec -2013-09-11 19:26:53,661 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 44.89 sec -2013-09-11 19:26:54,666 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 44.89 sec -2013-09-11 19:26:55,673 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 44.89 sec -2013-09-11 19:26:56,679 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 44.89 sec -2013-09-11 19:26:57,685 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 44.89 sec -2013-09-11 19:26:58,691 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 44.89 sec -2013-09-11 19:26:59,696 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 44.89 sec -2013-09-11 19:27:00,717 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 44.89 sec -2013-09-11 19:27:01,723 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 44.89 sec -2013-09-11 19:27:02,728 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 65.71 sec -2013-09-11 19:27:03,734 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 88.34 sec -2013-09-11 19:27:04,740 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 92.03 sec -2013-09-11 19:27:05,745 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 92.03 sec -2013-09-11 19:27:06,751 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 92.03 sec -2013-09-11 19:27:07,757 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 92.03 sec -2013-09-11 19:27:08,763 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 92.03 sec -2013-09-11 19:27:09,769 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 92.03 sec -2013-09-11 19:27:10,775 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 92.03 sec -2013-09-11 19:27:11,781 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 92.03 sec -2013-09-11 19:27:12,786 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 92.03 sec -2013-09-11 19:27:13,792 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 92.03 sec -2013-09-11 19:27:14,799 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 100.81 sec -2013-09-11 19:27:15,805 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 100.81 sec -2013-09-11 19:27:16,811 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 100.81 sec -2013-09-11 19:27:17,818 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 100.81 sec -2013-09-11 19:27:18,823 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 100.81 sec -2013-09-11 19:27:19,829 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 100.81 sec -2013-09-11 19:27:20,846 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 114.65 sec -2013-09-11 19:27:21,852 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 114.65 sec -MapReduce Total cumulative CPU time: 1 minutes 54 seconds 650 msec -Ended Job = job_201309101627_0343 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0344 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:27:24,306 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:27:26,313 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:27:27,318 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:27:28,323 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:27:29,327 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:27:30,331 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:27:31,335 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:27:32,340 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 19:27:33,345 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.75 sec -2013-09-11 19:27:34,351 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.15 sec -2013-09-11 19:27:35,356 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.15 sec -MapReduce Total cumulative CPU time: 2 seconds 150 msec -Ended Job = job_201309101627_0344 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 114.65 sec HDFS Read: 117363067 HDFS Write: 794 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.15 sec HDFS Read: 1563 HDFS Write: 571 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 56 seconds 800 msec -OK -Time taken: 75.21 seconds, Fetched: 19 row(s) -hive> quit; --- считаем средние длины URL для крупных счётчиков.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30248@mturlrep13_201309111927_36693068.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30881@mturlrep13_201309111927_1823096494.txt -hive> SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -FAILED: SemanticException [Error 10011]: Line 1:336 Invalid function 'GREATEST' -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31193@mturlrep13_201309111927_277766864.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31619@mturlrep13_201309111927_804737407.txt -hive> SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -FAILED: SemanticException [Error 10011]: Line 1:336 Invalid function 'GREATEST' -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31900@mturlrep13_201309111928_2860526.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32327@mturlrep13_201309111928_474072567.txt -hive> SELECT SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2), 1, GREATEST(0, FIND_IN_SET('/', SUBSTRING(Referer, FIND_IN_SET('//', Referer) + 2)) - 1)) HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25;; -FAILED: SemanticException [Error 10011]: Line 1:336 Invalid function 'GREATEST' -hive> quit; --- то же самое, но с разбивкой по доменам.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_32599@mturlrep13_201309111928_1292767865.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_624@mturlrep13_201309111928_489280386.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0345 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:28:32,803 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:28:45,869 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:28:51,894 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:28:57,917 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:29:04,947 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:05,953 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:06,959 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:07,965 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:08,970 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:09,987 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:10,992 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:11,998 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:13,003 Stage-1 map = 32%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:14,008 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:15,013 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:16,017 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:17,022 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:18,027 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:19,031 Stage-1 map = 39%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:20,036 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:21,040 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:22,045 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:23,050 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:24,055 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 69.54 sec -2013-09-11 19:29:25,080 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 90.43 sec -2013-09-11 19:29:26,085 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.27 sec -2013-09-11 19:29:27,090 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.27 sec -2013-09-11 19:29:28,095 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.27 sec -2013-09-11 19:29:29,100 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.27 sec -2013-09-11 19:29:30,105 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.27 sec -2013-09-11 19:29:31,109 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.27 sec -2013-09-11 19:29:32,114 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:33,119 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:34,123 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:35,177 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:36,182 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:37,186 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:38,191 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:39,195 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:40,226 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:41,230 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:42,234 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:43,239 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:44,243 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:45,267 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:46,271 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:47,276 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:48,280 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:49,284 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:50,289 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:51,293 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:52,297 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:53,301 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:54,306 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:55,311 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:56,316 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:57,321 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:58,332 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:29:59,337 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:30:00,342 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:30:01,347 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:30:02,352 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:30:03,357 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 112.27 sec -2013-09-11 19:30:04,363 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 195.43 sec -2013-09-11 19:30:05,368 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 195.43 sec -2013-09-11 19:30:06,374 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 195.43 sec -2013-09-11 19:30:07,379 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 195.43 sec -2013-09-11 19:30:08,384 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 195.43 sec -2013-09-11 19:30:09,389 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 195.43 sec -2013-09-11 19:30:10,395 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 195.43 sec -2013-09-11 19:30:11,400 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 195.43 sec -2013-09-11 19:30:12,405 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 195.43 sec -2013-09-11 19:30:13,411 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 205.31 sec -2013-09-11 19:30:14,416 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 205.31 sec -2013-09-11 19:30:15,421 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 205.31 sec -2013-09-11 19:30:16,426 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 205.31 sec -2013-09-11 19:30:17,431 Stage-1 map = 97%, reduce = 25%, Cumulative CPU 205.31 sec -2013-09-11 19:30:18,436 Stage-1 map = 97%, reduce = 25%, Cumulative CPU 205.31 sec -2013-09-11 19:30:19,441 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 222.12 sec -2013-09-11 19:30:20,446 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 222.12 sec -2013-09-11 19:30:21,451 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 222.12 sec -2013-09-11 19:30:22,456 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 222.12 sec -2013-09-11 19:30:23,463 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 224.88 sec -2013-09-11 19:30:24,469 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 224.88 sec -2013-09-11 19:30:25,475 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 224.88 sec -MapReduce Total cumulative CPU time: 3 minutes 44 seconds 880 msec -Ended Job = job_201309101627_0345 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 224.88 sec HDFS Read: 7797536 HDFS Write: 1080 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 44 seconds 880 msec -OK -Time taken: 123.601 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3693@mturlrep13_201309111930_78582272.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4150@mturlrep13_201309111930_88890337.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0346 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:30:39,700 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:30:50,746 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:30:59,777 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:31:05,802 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:06,807 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:07,813 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:08,820 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:09,825 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:10,829 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:11,833 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:12,838 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:13,842 Stage-1 map = 25%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:14,847 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:15,852 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:16,857 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:17,861 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:18,866 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:19,870 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:20,875 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:21,879 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:22,883 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:23,887 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:24,892 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:25,896 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:26,900 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:27,904 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:28,908 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:29,912 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:30,917 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:31,921 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 50.77 sec -2013-09-11 19:31:32,928 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 81.31 sec -2013-09-11 19:31:33,934 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.9 sec -2013-09-11 19:31:34,939 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.9 sec -2013-09-11 19:31:35,944 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.9 sec -2013-09-11 19:31:36,949 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.9 sec -2013-09-11 19:31:37,954 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.9 sec -2013-09-11 19:31:38,960 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 112.9 sec -2013-09-11 19:31:39,965 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:40,970 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:41,975 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:42,980 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:43,985 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:44,990 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:45,994 Stage-1 map = 54%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:47,031 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:48,036 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:49,041 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:50,046 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:51,052 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:52,081 Stage-1 map = 61%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:53,086 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:54,091 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:55,096 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:56,101 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:57,115 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:58,119 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:31:59,128 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:32:00,132 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:32:01,137 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:32:02,165 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:32:03,170 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:32:04,175 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 112.9 sec -2013-09-11 19:32:05,181 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:06,186 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:07,191 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:08,196 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:09,201 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:10,206 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:11,210 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:12,215 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:13,220 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:14,226 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:15,230 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:16,236 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:17,241 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:18,246 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:19,251 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 181.27 sec -2013-09-11 19:32:20,256 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 200.03 sec -2013-09-11 19:32:21,260 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 200.03 sec -2013-09-11 19:32:22,268 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 200.03 sec -2013-09-11 19:32:23,273 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 200.03 sec -2013-09-11 19:32:24,278 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 200.03 sec -2013-09-11 19:32:25,282 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 221.55 sec -2013-09-11 19:32:26,287 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 221.55 sec -2013-09-11 19:32:27,291 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 221.55 sec -2013-09-11 19:32:28,296 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 221.55 sec -2013-09-11 19:32:29,300 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 221.55 sec -2013-09-11 19:32:30,304 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 221.55 sec -2013-09-11 19:32:31,311 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 224.51 sec -2013-09-11 19:32:32,315 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 224.51 sec -2013-09-11 19:32:33,320 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 224.51 sec -MapReduce Total cumulative CPU time: 3 minutes 44 seconds 510 msec -Ended Job = job_201309101627_0346 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 224.51 sec HDFS Read: 7797536 HDFS Write: 1080 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 44 seconds 510 msec -OK -Time taken: 121.982 seconds, Fetched: 1 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6746@mturlrep13_201309111932_688661171.txt -hive> ; -hive> quit; - -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 hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7179@mturlrep13_201309111932_74538691.txt -hive> 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;; -Total MapReduce jobs = 1 -Launching Job 1 out of 1 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0347 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 1 -2013-09-11 19:32:47,696 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:32:58,740 Stage-1 map = 4%, reduce = 0% -2013-09-11 19:33:01,753 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:33:07,785 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:08,792 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:09,798 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:10,803 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:11,809 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:12,814 Stage-1 map = 14%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:13,819 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:14,824 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:15,829 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:16,834 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:17,840 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:18,845 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:19,850 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:20,855 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:21,860 Stage-1 map = 22%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:22,865 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:23,870 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:24,876 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:25,881 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:26,886 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:27,891 Stage-1 map = 29%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:28,896 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:29,901 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:30,906 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:31,911 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:32,915 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:33,920 Stage-1 map = 36%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:34,925 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:35,930 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:36,935 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:37,939 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:38,944 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:39,952 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 39.05 sec -2013-09-11 19:33:40,959 Stage-1 map = 47%, reduce = 0%, Cumulative CPU 76.73 sec -2013-09-11 19:33:41,965 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 114.74 sec -2013-09-11 19:33:42,970 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 114.74 sec -2013-09-11 19:33:43,975 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 114.74 sec -2013-09-11 19:33:44,981 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 114.74 sec -2013-09-11 19:33:45,986 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 114.74 sec -2013-09-11 19:33:46,991 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 114.74 sec -2013-09-11 19:33:47,996 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 114.74 sec -2013-09-11 19:33:49,020 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:33:50,025 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:33:51,030 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:33:52,035 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:33:53,040 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:33:54,045 Stage-1 map = 50%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:33:55,050 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:33:56,055 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:33:57,060 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:33:58,064 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:33:59,113 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:34:00,118 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:34:01,123 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:34:02,128 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:34:03,132 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:34:04,146 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 114.74 sec -2013-09-11 19:34:05,152 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:06,161 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:07,166 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:08,171 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:09,196 Stage-1 map = 69%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:10,201 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:11,206 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:12,211 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:13,217 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:14,237 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:15,242 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:16,247 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:17,252 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:18,257 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:19,262 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:20,267 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:21,272 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:22,277 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:23,282 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:24,287 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:25,292 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:26,297 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:27,302 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:28,307 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:29,311 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 167.26 sec -2013-09-11 19:34:30,316 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 193.91 sec -2013-09-11 19:34:31,320 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 193.91 sec -2013-09-11 19:34:32,325 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 193.91 sec -2013-09-11 19:34:33,330 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 193.91 sec -2013-09-11 19:34:34,335 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 193.91 sec -2013-09-11 19:34:35,339 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 193.91 sec -2013-09-11 19:34:36,344 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 226.92 sec -2013-09-11 19:34:37,348 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 226.92 sec -2013-09-11 19:34:38,353 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 226.92 sec -2013-09-11 19:34:39,358 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 226.92 sec -2013-09-11 19:34:40,364 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 230.01 sec -2013-09-11 19:34:41,369 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 230.01 sec -MapReduce Total cumulative CPU time: 3 minutes 50 seconds 10 msec -Ended Job = job_201309101627_0347 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 1 Cumulative CPU: 230.01 sec HDFS Read: 7797536 HDFS Write: 1080 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 50 seconds 10 msec -OK -Time taken: 122.089 seconds, Fetched: 1 row(s) -hive> quit; --- много тупых агрегатных функций.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9487@mturlrep13_201309111934_1498253935.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9958@mturlrep13_201309111934_990229488.txt -hive> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0348 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:35:01,929 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:35:08,962 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:35:11,984 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.29 sec -2013-09-11 19:35:12,991 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.29 sec -2013-09-11 19:35:13,999 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.29 sec -2013-09-11 19:35:15,006 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.29 sec -2013-09-11 19:35:16,013 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.29 sec -2013-09-11 19:35:17,021 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.29 sec -2013-09-11 19:35:18,028 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 25.29 sec -2013-09-11 19:35:19,035 Stage-1 map = 80%, reduce = 8%, Cumulative CPU 25.29 sec -2013-09-11 19:35:20,041 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 25.29 sec -2013-09-11 19:35:21,048 Stage-1 map = 89%, reduce = 17%, Cumulative CPU 36.02 sec -2013-09-11 19:35:22,054 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.81 sec -2013-09-11 19:35:23,061 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.81 sec -2013-09-11 19:35:24,068 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.81 sec -2013-09-11 19:35:25,074 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.81 sec -2013-09-11 19:35:26,080 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.81 sec -2013-09-11 19:35:27,087 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.81 sec -2013-09-11 19:35:28,094 Stage-1 map = 100%, reduce = 54%, Cumulative CPU 47.81 sec -2013-09-11 19:35:29,102 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.62 sec -2013-09-11 19:35:30,109 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.62 sec -2013-09-11 19:35:31,116 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.62 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 620 msec -Ended Job = job_201309101627_0348 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0349 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:35:34,611 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:35:41,634 Stage-2 map = 52%, reduce = 0% -2013-09-11 19:35:43,642 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.27 sec -2013-09-11 19:35:44,647 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.27 sec -2013-09-11 19:35:45,653 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.27 sec -2013-09-11 19:35:46,657 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.27 sec -2013-09-11 19:35:47,662 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.27 sec -2013-09-11 19:35:48,667 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.27 sec -2013-09-11 19:35:49,672 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.27 sec -2013-09-11 19:35:50,678 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 11.27 sec -2013-09-11 19:35:51,683 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 13.93 sec -2013-09-11 19:35:52,688 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 13.93 sec -2013-09-11 19:35:53,694 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 13.93 sec -MapReduce Total cumulative CPU time: 13 seconds 930 msec -Ended Job = job_201309101627_0349 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.62 sec HDFS Read: 69312553 HDFS Write: 31841963 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 13.93 sec HDFS Read: 31842732 HDFS Write: 372 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 16 seconds 550 msec -OK -Time taken: 61.8 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13220@mturlrep13_201309111935_1450466202.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13656@mturlrep13_201309111935_882257175.txt -hive> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0350 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:36:06,591 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:36:14,621 Stage-1 map = 39%, reduce = 0% -2013-09-11 19:36:15,633 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-11 19:36:16,640 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-11 19:36:17,650 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-11 19:36:18,657 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-11 19:36:19,663 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-11 19:36:20,670 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-11 19:36:21,677 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-11 19:36:22,683 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 23.79 sec -2013-09-11 19:36:23,689 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 23.79 sec -2013-09-11 19:36:24,694 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.4 sec -2013-09-11 19:36:25,700 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.4 sec -2013-09-11 19:36:26,706 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.4 sec -2013-09-11 19:36:27,711 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.4 sec -2013-09-11 19:36:28,717 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.4 sec -2013-09-11 19:36:29,723 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.4 sec -2013-09-11 19:36:30,728 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.4 sec -2013-09-11 19:36:31,734 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 47.4 sec -2013-09-11 19:36:32,740 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 47.4 sec -2013-09-11 19:36:33,748 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.39 sec -2013-09-11 19:36:34,754 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.39 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 390 msec -Ended Job = job_201309101627_0350 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0351 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:36:38,242 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:36:45,265 Stage-2 map = 52%, reduce = 0% -2013-09-11 19:36:47,275 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.17 sec -2013-09-11 19:36:48,280 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.17 sec -2013-09-11 19:36:49,285 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.17 sec -2013-09-11 19:36:50,289 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.17 sec -2013-09-11 19:36:51,294 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.17 sec -2013-09-11 19:36:52,299 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.17 sec -2013-09-11 19:36:53,303 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 12.17 sec -2013-09-11 19:36:54,308 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 12.17 sec -2013-09-11 19:36:55,313 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 12.17 sec -2013-09-11 19:36:56,319 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.91 sec -2013-09-11 19:36:57,324 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.91 sec -MapReduce Total cumulative CPU time: 14 seconds 910 msec -Ended Job = job_201309101627_0351 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.39 sec HDFS Read: 69312553 HDFS Write: 31841963 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 14.91 sec HDFS Read: 31842732 HDFS Write: 372 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 17 seconds 300 msec -OK -Time taken: 58.141 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16233@mturlrep13_201309111936_495916780.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16705@mturlrep13_201309111937_103662092.txt -hive> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0352 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:37:11,503 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:37:18,531 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:37:20,547 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.6 sec -2013-09-11 19:37:21,555 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.6 sec -2013-09-11 19:37:22,563 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.6 sec -2013-09-11 19:37:23,569 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.6 sec -2013-09-11 19:37:24,576 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.6 sec -2013-09-11 19:37:25,582 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.6 sec -2013-09-11 19:37:26,589 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.6 sec -2013-09-11 19:37:27,596 Stage-1 map = 88%, reduce = 8%, Cumulative CPU 24.6 sec -2013-09-11 19:37:28,602 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 24.6 sec -2013-09-11 19:37:29,608 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.04 sec -2013-09-11 19:37:30,613 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.04 sec -2013-09-11 19:37:31,618 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.04 sec -2013-09-11 19:37:32,624 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.04 sec -2013-09-11 19:37:33,630 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.04 sec -2013-09-11 19:37:34,635 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.04 sec -2013-09-11 19:37:35,640 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.04 sec -2013-09-11 19:37:36,645 Stage-1 map = 100%, reduce = 55%, Cumulative CPU 48.04 sec -2013-09-11 19:37:37,652 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.72 sec -2013-09-11 19:37:38,658 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.72 sec -2013-09-11 19:37:39,664 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 62.72 sec -MapReduce Total cumulative CPU time: 1 minutes 2 seconds 720 msec -Ended Job = job_201309101627_0352 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0353 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:37:42,202 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:37:50,230 Stage-2 map = 52%, reduce = 0% -2013-09-11 19:37:51,236 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.46 sec -2013-09-11 19:37:52,243 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.46 sec -2013-09-11 19:37:53,248 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.46 sec -2013-09-11 19:37:54,253 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.46 sec -2013-09-11 19:37:55,258 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.46 sec -2013-09-11 19:37:56,263 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.46 sec -2013-09-11 19:37:57,268 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 11.46 sec -2013-09-11 19:37:58,272 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 11.46 sec -2013-09-11 19:37:59,277 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 11.46 sec -2013-09-11 19:38:00,283 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.2 sec -2013-09-11 19:38:01,289 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 14.2 sec -MapReduce Total cumulative CPU time: 14 seconds 200 msec -Ended Job = job_201309101627_0353 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 62.72 sec HDFS Read: 69312553 HDFS Write: 31841963 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 14.2 sec HDFS Read: 31842732 HDFS Write: 372 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 16 seconds 920 msec -OK -Time taken: 58.209 seconds, Fetched: 10 row(s) -hive> quit; --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19487@mturlrep13_201309111938_449325071.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19952@mturlrep13_201309111938_524378054.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0354 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:38:23,549 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:38:30,587 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:38:33,607 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.07 sec -2013-09-11 19:38:34,614 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.07 sec -2013-09-11 19:38:35,622 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.07 sec -2013-09-11 19:38:36,628 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.07 sec -2013-09-11 19:38:37,635 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.07 sec -2013-09-11 19:38:38,642 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.07 sec -2013-09-11 19:38:39,649 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.07 sec -2013-09-11 19:38:40,655 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.07 sec -2013-09-11 19:38:41,660 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 24.07 sec -2013-09-11 19:38:42,665 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 24.07 sec -2013-09-11 19:38:43,671 Stage-1 map = 89%, reduce = 17%, Cumulative CPU 36.3 sec -2013-09-11 19:38:44,676 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.95 sec -2013-09-11 19:38:45,681 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.95 sec -2013-09-11 19:38:46,686 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.95 sec -2013-09-11 19:38:47,691 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.95 sec -2013-09-11 19:38:48,696 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.95 sec -2013-09-11 19:38:49,702 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.95 sec -2013-09-11 19:38:50,707 Stage-1 map = 100%, reduce = 86%, Cumulative CPU 48.95 sec -2013-09-11 19:38:51,715 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 57.98 sec -2013-09-11 19:38:52,721 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 66.7 sec -2013-09-11 19:38:53,727 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 66.7 sec -MapReduce Total cumulative CPU time: 1 minutes 6 seconds 700 msec -Ended Job = job_201309101627_0354 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0355 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:38:57,273 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:39:07,307 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 10.01 sec -2013-09-11 19:39:08,312 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 10.01 sec -2013-09-11 19:39:09,316 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 10.01 sec -2013-09-11 19:39:10,321 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.48 sec -2013-09-11 19:39:11,326 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.48 sec -2013-09-11 19:39:12,331 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.48 sec -2013-09-11 19:39:13,336 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.48 sec -2013-09-11 19:39:14,341 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.48 sec -2013-09-11 19:39:15,346 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.48 sec -2013-09-11 19:39:16,350 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.48 sec -2013-09-11 19:39:17,356 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.48 sec -2013-09-11 19:39:18,361 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.48 sec -2013-09-11 19:39:19,366 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.34 sec -2013-09-11 19:39:20,372 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.34 sec -2013-09-11 19:39:21,377 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.34 sec -MapReduce Total cumulative CPU time: 22 seconds 340 msec -Ended Job = job_201309101627_0355 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 66.7 sec HDFS Read: 112931901 HDFS Write: 72725701 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 22.34 sec HDFS Read: 72726470 HDFS Write: 417 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 29 seconds 40 msec -OK -Time taken: 67.908 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_22641@mturlrep13_201309111939_2035435508.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_23093@mturlrep13_201309111939_526667313.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0356 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:39:35,562 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:39:42,588 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:39:44,605 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.84 sec -2013-09-11 19:39:45,612 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.84 sec -2013-09-11 19:39:46,620 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.84 sec -2013-09-11 19:39:47,626 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.84 sec -2013-09-11 19:39:48,632 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.84 sec -2013-09-11 19:39:49,638 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.84 sec -2013-09-11 19:39:50,644 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.84 sec -2013-09-11 19:39:51,650 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.84 sec -2013-09-11 19:39:52,655 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 24.84 sec -2013-09-11 19:39:53,660 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 36.18 sec -2013-09-11 19:39:54,665 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.13 sec -2013-09-11 19:39:55,671 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.13 sec -2013-09-11 19:39:56,676 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.13 sec -2013-09-11 19:39:57,682 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.13 sec -2013-09-11 19:39:58,688 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.13 sec -2013-09-11 19:39:59,693 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.13 sec -2013-09-11 19:40:00,698 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.13 sec -2013-09-11 19:40:01,705 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 48.13 sec -2013-09-11 19:40:02,710 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 48.13 sec -2013-09-11 19:40:03,717 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 66.55 sec -2013-09-11 19:40:04,723 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 66.55 sec -MapReduce Total cumulative CPU time: 1 minutes 6 seconds 550 msec -Ended Job = job_201309101627_0356 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0357 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:40:08,243 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:40:18,275 Stage-2 map = 50%, reduce = 0% -2013-09-11 19:40:21,285 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.04 sec -2013-09-11 19:40:22,290 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.04 sec -2013-09-11 19:40:23,295 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.04 sec -2013-09-11 19:40:24,299 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.04 sec -2013-09-11 19:40:25,303 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.04 sec -2013-09-11 19:40:26,308 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.04 sec -2013-09-11 19:40:27,312 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 18.04 sec -2013-09-11 19:40:28,319 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.04 sec -2013-09-11 19:40:29,325 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 18.04 sec -2013-09-11 19:40:30,331 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.87 sec -2013-09-11 19:40:31,336 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.87 sec -2013-09-11 19:40:32,341 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 22.87 sec -MapReduce Total cumulative CPU time: 22 seconds 870 msec -Ended Job = job_201309101627_0357 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 66.55 sec HDFS Read: 112931901 HDFS Write: 72725701 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 22.87 sec HDFS Read: 72726470 HDFS Write: 417 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 29 seconds 420 msec -OK -Time taken: 65.342 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26473@mturlrep13_201309111940_1279166240.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_26931@mturlrep13_201309111940_790078242.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0358 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:40:46,365 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:40:53,395 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:40:55,412 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.12 sec -2013-09-11 19:40:56,420 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.12 sec -2013-09-11 19:40:57,429 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.12 sec -2013-09-11 19:40:58,435 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.12 sec -2013-09-11 19:40:59,442 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.12 sec -2013-09-11 19:41:00,449 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.12 sec -2013-09-11 19:41:01,457 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 24.12 sec -2013-09-11 19:41:02,463 Stage-1 map = 68%, reduce = 8%, Cumulative CPU 24.12 sec -2013-09-11 19:41:03,469 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 24.12 sec -2013-09-11 19:41:04,475 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 46.93 sec -2013-09-11 19:41:05,481 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.42 sec -2013-09-11 19:41:06,487 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.42 sec -2013-09-11 19:41:07,493 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.42 sec -2013-09-11 19:41:08,499 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.42 sec -2013-09-11 19:41:09,505 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.42 sec -2013-09-11 19:41:10,511 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 48.42 sec -2013-09-11 19:41:11,518 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 48.42 sec -2013-09-11 19:41:12,524 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 48.42 sec -2013-09-11 19:41:13,531 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 56.93 sec -2013-09-11 19:41:14,537 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 65.67 sec -2013-09-11 19:41:15,544 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 65.67 sec -MapReduce Total cumulative CPU time: 1 minutes 5 seconds 670 msec -Ended Job = job_201309101627_0358 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0359 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 19:41:19,228 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:41:29,262 Stage-2 map = 50%, reduce = 0% -2013-09-11 19:41:32,273 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.08 sec -2013-09-11 19:41:33,278 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.08 sec -2013-09-11 19:41:34,283 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.08 sec -2013-09-11 19:41:35,287 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.08 sec -2013-09-11 19:41:36,291 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.08 sec -2013-09-11 19:41:37,296 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.08 sec -2013-09-11 19:41:38,301 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 17.08 sec -2013-09-11 19:41:39,307 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.08 sec -2013-09-11 19:41:40,311 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 17.08 sec -2013-09-11 19:41:41,317 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 21.69 sec -2013-09-11 19:41:42,323 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 21.69 sec -2013-09-11 19:41:43,328 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 21.69 sec -MapReduce Total cumulative CPU time: 21 seconds 690 msec -Ended Job = job_201309101627_0359 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 65.67 sec HDFS Read: 112931901 HDFS Write: 72725701 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 21.69 sec HDFS Read: 72726470 HDFS Write: 417 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 27 seconds 360 msec -OK -Time taken: 65.41 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_29665@mturlrep13_201309111941_1737478499.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30133@mturlrep13_201309111941_1994654944.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0360 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:42:06,244 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:42:13,269 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:42:16,281 Stage-1 map = 18%, reduce = 0% -2013-09-11 19:42:19,293 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:42:22,306 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:42:25,316 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:42:27,330 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 25.54 sec -2013-09-11 19:42:28,336 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.11 sec -2013-09-11 19:42:29,343 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.11 sec -2013-09-11 19:42:30,349 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.11 sec -2013-09-11 19:42:31,354 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.11 sec -2013-09-11 19:42:32,359 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.11 sec -2013-09-11 19:42:33,365 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.11 sec -2013-09-11 19:42:34,370 Stage-1 map = 54%, reduce = 13%, Cumulative CPU 53.11 sec -2013-09-11 19:42:35,375 Stage-1 map = 57%, reduce = 13%, Cumulative CPU 53.11 sec -2013-09-11 19:42:36,381 Stage-1 map = 57%, reduce = 13%, Cumulative CPU 53.11 sec -2013-09-11 19:42:37,386 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 53.11 sec -2013-09-11 19:42:38,391 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 53.11 sec -2013-09-11 19:42:39,397 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 53.11 sec -2013-09-11 19:42:40,403 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 53.11 sec -2013-09-11 19:42:41,408 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 53.11 sec -2013-09-11 19:42:42,423 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 53.11 sec -2013-09-11 19:42:43,428 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 53.11 sec -2013-09-11 19:42:44,433 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 53.11 sec -2013-09-11 19:42:45,439 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 53.11 sec -2013-09-11 19:42:46,444 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 76.02 sec -2013-09-11 19:42:47,449 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 76.02 sec -2013-09-11 19:42:48,454 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 102.55 sec -2013-09-11 19:42:49,458 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 102.55 sec -2013-09-11 19:42:50,462 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 102.55 sec -2013-09-11 19:42:51,466 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 102.55 sec -2013-09-11 19:42:52,471 Stage-1 map = 100%, reduce = 46%, Cumulative CPU 102.55 sec -2013-09-11 19:42:53,475 Stage-1 map = 100%, reduce = 46%, Cumulative CPU 102.55 sec -2013-09-11 19:42:54,479 Stage-1 map = 100%, reduce = 46%, Cumulative CPU 102.55 sec -2013-09-11 19:42:55,484 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 102.55 sec -2013-09-11 19:42:57,407 Stage-1 map = 100%, reduce = 52%, Cumulative CPU 102.55 sec -2013-09-11 19:42:58,412 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 102.55 sec -2013-09-11 19:42:59,417 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 102.55 sec -2013-09-11 19:43:00,422 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 102.55 sec -2013-09-11 19:43:01,427 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 102.55 sec -2013-09-11 19:43:02,432 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 102.55 sec -2013-09-11 19:43:03,438 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 102.55 sec -2013-09-11 19:43:04,443 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 102.55 sec -2013-09-11 19:43:05,450 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 139.29 sec -2013-09-11 19:43:06,456 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 139.29 sec -2013-09-11 19:43:07,461 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 139.29 sec -2013-09-11 19:43:08,467 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 139.29 sec -2013-09-11 19:43:09,472 Stage-1 map = 100%, reduce = 83%, Cumulative CPU 139.29 sec -2013-09-11 19:43:10,477 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 139.29 sec -2013-09-11 19:43:11,487 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 139.29 sec -2013-09-11 19:43:12,493 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 139.29 sec -2013-09-11 19:43:13,498 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 139.29 sec -2013-09-11 19:43:14,503 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 139.29 sec -2013-09-11 19:43:15,508 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 139.29 sec -2013-09-11 19:43:16,514 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 151.55 sec -2013-09-11 19:43:17,625 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 151.55 sec -2013-09-11 19:43:18,631 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 151.55 sec -2013-09-11 19:43:19,637 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 151.55 sec -2013-09-11 19:43:20,642 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 151.55 sec -2013-09-11 19:43:21,646 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 151.55 sec -2013-09-11 19:43:22,850 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.64 sec -2013-09-11 19:43:23,855 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.64 sec -MapReduce Total cumulative CPU time: 2 minutes 49 seconds 640 msec -Ended Job = job_201309101627_0360 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0361 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-11 19:43:27,476 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:43:40,514 Stage-2 map = 36%, reduce = 0% -2013-09-11 19:43:49,541 Stage-2 map = 72%, reduce = 0% -2013-09-11 19:43:51,548 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 29.17 sec -2013-09-11 19:43:52,553 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 29.17 sec -2013-09-11 19:43:53,557 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 29.17 sec -2013-09-11 19:43:54,561 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 29.17 sec -2013-09-11 19:43:55,565 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 29.17 sec -2013-09-11 19:43:56,817 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 29.17 sec -2013-09-11 19:43:57,821 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 29.17 sec -2013-09-11 19:43:58,825 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.17 sec -2013-09-11 19:43:59,829 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.17 sec -2013-09-11 19:44:00,834 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.17 sec -2013-09-11 19:44:01,838 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.17 sec -2013-09-11 19:44:02,842 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.17 sec -2013-09-11 19:44:03,846 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.17 sec -2013-09-11 19:44:04,850 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.17 sec -2013-09-11 19:44:05,855 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 75.56 sec -2013-09-11 19:44:06,860 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 75.56 sec -2013-09-11 19:44:07,865 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 75.56 sec -2013-09-11 19:44:08,869 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.0 sec -2013-09-11 19:44:09,874 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.0 sec -2013-09-11 19:44:10,878 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.0 sec -2013-09-11 19:44:11,882 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.0 sec -2013-09-11 19:44:12,886 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.0 sec -2013-09-11 19:44:13,891 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.0 sec -2013-09-11 19:44:14,896 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.0 sec -2013-09-11 19:44:15,900 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.0 sec -2013-09-11 19:44:16,904 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.0 sec -2013-09-11 19:44:17,908 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 79.0 sec -2013-09-11 19:44:18,913 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 79.0 sec -2013-09-11 19:44:19,917 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 79.0 sec -2013-09-11 19:44:20,921 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 79.0 sec -2013-09-11 19:44:21,925 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 79.0 sec -2013-09-11 19:44:23,426 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 88.07 sec -2013-09-11 19:44:24,431 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 88.07 sec -2013-09-11 19:44:25,436 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 88.07 sec -MapReduce Total cumulative CPU time: 1 minutes 28 seconds 70 msec -Ended Job = job_201309101627_0361 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 169.64 sec HDFS Read: 85707829 HDFS Write: 413932232 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 88.07 sec HDFS Read: 413942944 HDFS Write: 420 SUCCESS -Total MapReduce CPU Time Spent: 4 minutes 17 seconds 710 msec -OK -Time taken: 149.238 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1184@mturlrep13_201309111944_629860670.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1658@mturlrep13_201309111944_819122527.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0362 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:44:39,567 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:44:49,606 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:44:52,619 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:44:55,633 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:44:58,646 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:45:01,658 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:45:03,673 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 25.75 sec -2013-09-11 19:45:04,680 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.01 sec -2013-09-11 19:45:05,689 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.01 sec -2013-09-11 19:45:06,695 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.01 sec -2013-09-11 19:45:07,702 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.01 sec -2013-09-11 19:45:08,708 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.01 sec -2013-09-11 19:45:09,714 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 53.01 sec -2013-09-11 19:45:10,720 Stage-1 map = 54%, reduce = 8%, Cumulative CPU 53.01 sec -2013-09-11 19:45:11,726 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 53.01 sec -2013-09-11 19:45:12,732 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 53.01 sec -2013-09-11 19:45:13,738 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 53.01 sec -2013-09-11 19:45:14,744 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 53.01 sec -2013-09-11 19:45:15,750 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 53.01 sec -2013-09-11 19:45:16,756 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 53.01 sec -2013-09-11 19:45:17,762 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 53.01 sec -2013-09-11 19:45:18,768 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 53.01 sec -2013-09-11 19:45:19,774 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 53.01 sec -2013-09-11 19:45:20,780 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 53.01 sec -2013-09-11 19:45:21,786 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 76.34 sec -2013-09-11 19:45:22,792 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 76.34 sec -2013-09-11 19:45:23,798 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 102.7 sec -2013-09-11 19:45:24,803 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 102.7 sec -2013-09-11 19:45:25,808 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 102.7 sec -2013-09-11 19:45:26,814 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 102.7 sec -2013-09-11 19:45:27,820 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 102.7 sec -2013-09-11 19:45:28,826 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 102.7 sec -2013-09-11 19:45:29,831 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 102.7 sec -2013-09-11 19:45:30,837 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 102.7 sec -2013-09-11 19:45:31,842 Stage-1 map = 100%, reduce = 68%, Cumulative CPU 102.7 sec -2013-09-11 19:45:33,451 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 102.7 sec -2013-09-11 19:45:34,456 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 102.7 sec -2013-09-11 19:45:35,462 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 102.7 sec -2013-09-11 19:45:36,468 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 102.7 sec -2013-09-11 19:45:37,473 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 102.7 sec -2013-09-11 19:45:38,479 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 102.7 sec -2013-09-11 19:45:39,485 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 102.7 sec -2013-09-11 19:45:40,490 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 102.7 sec -2013-09-11 19:45:41,496 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 102.7 sec -2013-09-11 19:45:42,502 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 102.7 sec -2013-09-11 19:45:43,508 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 102.7 sec -2013-09-11 19:45:44,514 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 102.7 sec -2013-09-11 19:45:45,519 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 102.7 sec -2013-09-11 19:45:46,525 Stage-1 map = 100%, reduce = 89%, Cumulative CPU 102.7 sec -2013-09-11 19:45:47,537 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 102.7 sec -2013-09-11 19:45:48,542 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 102.7 sec -2013-09-11 19:45:49,548 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 102.7 sec -2013-09-11 19:45:50,554 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 102.7 sec -2013-09-11 19:45:51,561 Stage-1 map = 100%, reduce = 99%, Cumulative CPU 134.99 sec -2013-09-11 19:45:52,567 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 168.39 sec -2013-09-11 19:45:53,572 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 168.39 sec -MapReduce Total cumulative CPU time: 2 minutes 48 seconds 390 msec -Ended Job = job_201309101627_0362 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0363 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-11 19:45:57,372 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:46:10,035 Stage-2 map = 36%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:11,039 Stage-2 map = 36%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:12,044 Stage-2 map = 36%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:13,049 Stage-2 map = 36%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:14,053 Stage-2 map = 36%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:15,058 Stage-2 map = 36%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:16,062 Stage-2 map = 36%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:17,067 Stage-2 map = 36%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:18,071 Stage-2 map = 36%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:19,076 Stage-2 map = 48%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:20,081 Stage-2 map = 72%, reduce = 0%, Cumulative CPU 19.18 sec -2013-09-11 19:46:21,086 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 37.8 sec -2013-09-11 19:46:22,091 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 37.8 sec -2013-09-11 19:46:23,095 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 37.8 sec -2013-09-11 19:46:24,100 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 37.8 sec -2013-09-11 19:46:25,104 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 37.8 sec -2013-09-11 19:46:26,109 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 37.8 sec -2013-09-11 19:46:27,115 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 37.8 sec -2013-09-11 19:46:28,120 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:29,124 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:30,128 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:31,133 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:32,138 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:33,142 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:34,147 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:35,152 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:36,157 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:37,161 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:38,166 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:39,171 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 37.8 sec -2013-09-11 19:46:40,175 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.37 sec -2013-09-11 19:46:41,180 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.37 sec -2013-09-11 19:46:42,184 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.37 sec -2013-09-11 19:46:43,189 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.37 sec -2013-09-11 19:46:44,194 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.37 sec -2013-09-11 19:46:45,198 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.37 sec -2013-09-11 19:46:46,202 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.37 sec -2013-09-11 19:46:47,207 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.37 sec -2013-09-11 19:46:48,211 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 77.37 sec -2013-09-11 19:46:49,216 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 77.37 sec -2013-09-11 19:46:50,222 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 77.37 sec -2013-09-11 19:46:51,227 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 87.68 sec -2013-09-11 19:46:52,232 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 87.68 sec -MapReduce Total cumulative CPU time: 1 minutes 27 seconds 680 msec -Ended Job = job_201309101627_0363 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 168.39 sec HDFS Read: 85707829 HDFS Write: 413932232 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 87.68 sec HDFS Read: 413942944 HDFS Write: 420 SUCCESS -Total MapReduce CPU Time Spent: 4 minutes 16 seconds 70 msec -OK -Time taken: 141.031 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5940@mturlrep13_201309111946_577038948.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6372@mturlrep13_201309111946_455901052.txt -hive> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0364 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:47:05,315 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:47:13,350 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:47:16,363 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:47:19,377 Stage-1 map = 32%, reduce = 0% -2013-09-11 19:47:22,391 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:47:25,410 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.14 sec -2013-09-11 19:47:26,416 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.14 sec -2013-09-11 19:47:27,424 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.14 sec -2013-09-11 19:47:28,430 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.14 sec -2013-09-11 19:47:29,436 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.14 sec -2013-09-11 19:47:30,442 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.14 sec -2013-09-11 19:47:31,448 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 52.14 sec -2013-09-11 19:47:32,453 Stage-1 map = 54%, reduce = 8%, Cumulative CPU 52.14 sec -2013-09-11 19:47:33,459 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-11 19:47:34,465 Stage-1 map = 57%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-11 19:47:35,470 Stage-1 map = 65%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-11 19:47:36,476 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-11 19:47:37,482 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-11 19:47:38,488 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-11 19:47:39,493 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-11 19:47:40,499 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-11 19:47:41,505 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-11 19:47:42,511 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 52.14 sec -2013-09-11 19:47:43,517 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 75.85 sec -2013-09-11 19:47:44,523 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 75.85 sec -2013-09-11 19:47:45,529 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 103.18 sec -2013-09-11 19:47:46,534 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 103.18 sec -2013-09-11 19:47:47,539 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 103.18 sec -2013-09-11 19:47:48,544 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 103.18 sec -2013-09-11 19:47:49,550 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 103.18 sec -2013-09-11 19:47:50,555 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 103.18 sec -2013-09-11 19:47:51,561 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 103.18 sec -2013-09-11 19:47:52,566 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 103.18 sec -2013-09-11 19:47:53,571 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 103.18 sec -2013-09-11 19:47:54,577 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 103.18 sec -2013-09-11 19:47:55,582 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 103.18 sec -2013-09-11 19:47:56,587 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 103.18 sec -2013-09-11 19:47:57,592 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 103.18 sec -2013-09-11 19:47:59,138 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 103.18 sec -2013-09-11 19:48:00,143 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 103.18 sec -2013-09-11 19:48:01,149 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 103.18 sec -2013-09-11 19:48:02,154 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 103.18 sec -2013-09-11 19:48:03,159 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 103.18 sec -2013-09-11 19:48:04,164 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 103.18 sec -2013-09-11 19:48:05,169 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 103.18 sec -2013-09-11 19:48:06,177 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 151.44 sec -2013-09-11 19:48:07,182 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 151.44 sec -2013-09-11 19:48:08,188 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 151.44 sec -2013-09-11 19:48:09,193 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 151.44 sec -2013-09-11 19:48:10,199 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 151.44 sec -2013-09-11 19:48:11,210 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 151.44 sec -2013-09-11 19:48:12,216 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 151.44 sec -2013-09-11 19:48:13,222 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 151.44 sec -2013-09-11 19:48:14,228 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.64 sec -2013-09-11 19:48:15,234 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.64 sec -2013-09-11 19:48:16,239 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.64 sec -MapReduce Total cumulative CPU time: 2 minutes 49 seconds 640 msec -Ended Job = job_201309101627_0364 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0365 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-11 19:48:18,890 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:48:32,931 Stage-2 map = 36%, reduce = 0% -2013-09-11 19:48:41,958 Stage-2 map = 72%, reduce = 0% -2013-09-11 19:48:43,966 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 29.01 sec -2013-09-11 19:48:44,971 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 29.01 sec -2013-09-11 19:48:45,976 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 29.01 sec -2013-09-11 19:48:46,981 Stage-2 map = 74%, reduce = 0%, Cumulative CPU 29.01 sec -2013-09-11 19:48:47,986 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 29.01 sec -2013-09-11 19:48:48,991 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 29.01 sec -2013-09-11 19:48:49,996 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 29.01 sec -2013-09-11 19:48:51,001 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.01 sec -2013-09-11 19:48:52,006 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.01 sec -2013-09-11 19:48:53,011 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.01 sec -2013-09-11 19:48:54,028 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.01 sec -2013-09-11 19:48:55,033 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.01 sec -2013-09-11 19:48:56,038 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.01 sec -2013-09-11 19:48:57,043 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.01 sec -2013-09-11 19:48:58,048 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.01 sec -2013-09-11 19:48:59,053 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.01 sec -2013-09-11 19:49:00,058 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 29.01 sec -2013-09-11 19:49:01,063 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.48 sec -2013-09-11 19:49:02,068 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.48 sec -2013-09-11 19:49:03,072 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.48 sec -2013-09-11 19:49:04,077 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.48 sec -2013-09-11 19:49:05,081 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 77.48 sec -2013-09-11 19:49:06,086 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.28 sec -2013-09-11 19:49:07,091 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.28 sec -2013-09-11 19:49:08,095 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.28 sec -2013-09-11 19:49:09,100 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 79.28 sec -2013-09-11 19:49:10,104 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 79.28 sec -2013-09-11 19:49:11,109 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 79.28 sec -2013-09-11 19:49:12,113 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 79.28 sec -2013-09-11 19:49:13,117 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 79.28 sec -2013-09-11 19:49:14,122 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 79.28 sec -2013-09-11 19:49:15,556 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 87.92 sec -2013-09-11 19:49:16,561 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 87.92 sec -MapReduce Total cumulative CPU time: 1 minutes 27 seconds 920 msec -Ended Job = job_201309101627_0365 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 169.64 sec HDFS Read: 85707829 HDFS Write: 413932232 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 87.92 sec HDFS Read: 413942944 HDFS Write: 420 SUCCESS -Total MapReduce CPU Time Spent: 4 minutes 17 seconds 560 msec -OK -Time taken: 138.703 seconds, Fetched: 10 row(s) -hive> quit; --- то же самое, но ещё и без фильтрации.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9642@mturlrep13_201309111949_27347921.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10116@mturlrep13_201309111949_204794215.txt -hive> SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0366 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:49:39,677 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:49:46,707 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:49:49,719 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:49:52,731 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:49:55,744 Stage-1 map = 39%, reduce = 0% -2013-09-11 19:49:58,755 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:50:00,769 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 27.34 sec -2013-09-11 19:50:01,775 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.01 sec -2013-09-11 19:50:02,783 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.01 sec -2013-09-11 19:50:03,789 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.01 sec -2013-09-11 19:50:04,794 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.01 sec -2013-09-11 19:50:05,800 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.01 sec -2013-09-11 19:50:06,806 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.01 sec -2013-09-11 19:50:07,812 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.01 sec -2013-09-11 19:50:08,818 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 55.01 sec -2013-09-11 19:50:09,824 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 55.01 sec -2013-09-11 19:50:10,830 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 55.01 sec -2013-09-11 19:50:11,836 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-11 19:50:12,841 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-11 19:50:13,847 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-11 19:50:14,853 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-11 19:50:15,859 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-11 19:50:16,864 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-11 19:50:17,871 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-11 19:50:18,877 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-11 19:50:19,882 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-11 19:50:20,888 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 55.01 sec -2013-09-11 19:50:21,893 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 79.56 sec -2013-09-11 19:50:22,898 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.88 sec -2013-09-11 19:50:23,903 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.88 sec -2013-09-11 19:50:24,909 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.88 sec -2013-09-11 19:50:25,914 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 106.88 sec -2013-09-11 19:50:26,919 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 106.88 sec -2013-09-11 19:50:27,924 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 106.88 sec -2013-09-11 19:50:28,929 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 106.88 sec -2013-09-11 19:50:29,935 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 106.88 sec -2013-09-11 19:50:32,380 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 106.88 sec -2013-09-11 19:50:34,489 Stage-1 map = 100%, reduce = 72%, Cumulative CPU 106.88 sec -2013-09-11 19:50:35,495 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 106.88 sec -2013-09-11 19:50:36,500 Stage-1 map = 100%, reduce = 77%, Cumulative CPU 106.88 sec -2013-09-11 19:50:37,506 Stage-1 map = 100%, reduce = 82%, Cumulative CPU 106.88 sec -2013-09-11 19:50:38,511 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 106.88 sec -2013-09-11 19:50:39,517 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 106.88 sec -2013-09-11 19:50:40,522 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 106.88 sec -2013-09-11 19:50:41,531 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 132.09 sec -2013-09-11 19:50:42,536 Stage-1 map = 100%, reduce = 96%, Cumulative CPU 132.09 sec -2013-09-11 19:50:43,542 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 158.78 sec -2013-09-11 19:50:44,569 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 158.78 sec -MapReduce Total cumulative CPU time: 2 minutes 38 seconds 780 msec -Ended Job = job_201309101627_0366 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0367 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-11 19:50:48,203 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:50:55,225 Stage-2 map = 25%, reduce = 0% -2013-09-11 19:50:57,233 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-11 19:50:58,238 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-11 19:50:59,243 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-11 19:51:00,248 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-11 19:51:02,193 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-11 19:51:03,198 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.37 sec -2013-09-11 19:51:04,203 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 13.37 sec -2013-09-11 19:51:05,207 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 13.37 sec -2013-09-11 19:51:06,213 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.58 sec -2013-09-11 19:51:07,218 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.58 sec -2013-09-11 19:51:08,223 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.58 sec -2013-09-11 19:51:09,227 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.58 sec -2013-09-11 19:51:10,232 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.58 sec -2013-09-11 19:51:11,237 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.58 sec -2013-09-11 19:51:12,242 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.58 sec -2013-09-11 19:51:13,591 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 37.58 sec -2013-09-11 19:51:14,596 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 37.58 sec -2013-09-11 19:51:15,601 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 43.87 sec -2013-09-11 19:51:16,606 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 43.87 sec -MapReduce Total cumulative CPU time: 43 seconds 870 msec -Ended Job = job_201309101627_0367 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 158.78 sec HDFS Read: 109451651 HDFS Write: 399298510 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 43.87 sec HDFS Read: 399308173 HDFS Write: 445 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 22 seconds 650 msec -OK -Time taken: 107.308 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13554@mturlrep13_201309111951_64701143.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13986@mturlrep13_201309111951_819836063.txt -hive> SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0368 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:51:31,116 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:51:38,143 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:51:41,156 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:51:44,168 Stage-1 map = 32%, reduce = 0% -2013-09-11 19:51:47,181 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:51:51,201 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 26.66 sec -2013-09-11 19:51:52,207 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.03 sec -2013-09-11 19:51:53,215 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.03 sec -2013-09-11 19:51:54,221 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.03 sec -2013-09-11 19:51:55,227 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.03 sec -2013-09-11 19:51:56,233 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.03 sec -2013-09-11 19:51:57,239 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.03 sec -2013-09-11 19:51:58,244 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 54.03 sec -2013-09-11 19:51:59,249 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 54.03 sec -2013-09-11 19:52:00,255 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 54.03 sec -2013-09-11 19:52:01,259 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 54.03 sec -2013-09-11 19:52:02,265 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 54.03 sec -2013-09-11 19:52:03,271 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 54.03 sec -2013-09-11 19:52:04,276 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 54.03 sec -2013-09-11 19:52:05,281 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 54.03 sec -2013-09-11 19:52:06,287 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 96.31 sec -2013-09-11 19:52:07,292 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 96.31 sec -2013-09-11 19:52:08,298 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 96.31 sec -2013-09-11 19:52:09,303 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 96.31 sec -2013-09-11 19:52:10,309 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 96.31 sec -2013-09-11 19:52:11,314 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 96.31 sec -2013-09-11 19:52:12,318 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 104.46 sec -2013-09-11 19:52:13,323 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 114.6 sec -2013-09-11 19:52:14,328 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 114.6 sec -2013-09-11 19:52:15,333 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 114.6 sec -2013-09-11 19:52:16,338 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 114.6 sec -2013-09-11 19:52:17,343 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 114.6 sec -2013-09-11 19:52:18,349 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 114.6 sec -2013-09-11 19:52:19,354 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 114.6 sec -2013-09-11 19:52:20,359 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 114.6 sec -2013-09-11 19:52:21,364 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 114.6 sec -2013-09-11 19:52:23,754 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 114.6 sec -2013-09-11 19:52:25,932 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 114.6 sec -2013-09-11 19:52:26,938 Stage-1 map = 100%, reduce = 75%, Cumulative CPU 114.6 sec -2013-09-11 19:52:27,944 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 114.6 sec -2013-09-11 19:52:28,949 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 114.6 sec -2013-09-11 19:52:29,955 Stage-1 map = 100%, reduce = 85%, Cumulative CPU 114.6 sec -2013-09-11 19:52:30,961 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 114.6 sec -2013-09-11 19:52:31,966 Stage-1 map = 100%, reduce = 94%, Cumulative CPU 114.6 sec -2013-09-11 19:52:32,974 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 137.85 sec -2013-09-11 19:52:33,979 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 159.23 sec -2013-09-11 19:52:34,984 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 159.23 sec -MapReduce Total cumulative CPU time: 2 minutes 39 seconds 230 msec -Ended Job = job_201309101627_0368 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0369 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-11 19:52:38,543 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:52:45,564 Stage-2 map = 25%, reduce = 0% -2013-09-11 19:52:48,575 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 19:52:49,579 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 19:52:50,583 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 19:52:51,588 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 19:52:52,593 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 19:52:53,597 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 19:52:54,602 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.71 sec -2013-09-11 19:52:55,607 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 13.71 sec -2013-09-11 19:52:56,611 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.61 sec -2013-09-11 19:52:57,615 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.61 sec -2013-09-11 19:52:58,619 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.61 sec -2013-09-11 19:52:59,623 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.61 sec -2013-09-11 19:53:00,627 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.61 sec -2013-09-11 19:53:01,631 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.61 sec -2013-09-11 19:53:02,635 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.61 sec -2013-09-11 19:53:03,639 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.61 sec -2013-09-11 19:53:04,644 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 37.61 sec -2013-09-11 19:53:05,647 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 37.61 sec -2013-09-11 19:53:06,652 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 44.99 sec -2013-09-11 19:53:07,656 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 44.99 sec -MapReduce Total cumulative CPU time: 44 seconds 990 msec -Ended Job = job_201309101627_0369 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 159.23 sec HDFS Read: 109451651 HDFS Write: 399298510 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 44.99 sec HDFS Read: 399308173 HDFS Write: 445 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 24 seconds 220 msec -OK -Time taken: 104.847 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16749@mturlrep13_201309111953_313353407.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_17196@mturlrep13_201309111953_1275155665.txt -hive> SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0370 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:53:21,695 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:53:28,723 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:53:31,736 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:53:34,749 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:53:37,762 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:53:42,786 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 56.41 sec -2013-09-11 19:53:43,792 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 56.41 sec -2013-09-11 19:53:44,801 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 56.41 sec -2013-09-11 19:53:45,807 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 56.41 sec -2013-09-11 19:53:46,812 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 56.41 sec -2013-09-11 19:53:47,817 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 56.41 sec -2013-09-11 19:53:48,823 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 56.41 sec -2013-09-11 19:53:49,829 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 56.41 sec -2013-09-11 19:53:50,834 Stage-1 map = 61%, reduce = 8%, Cumulative CPU 56.41 sec -2013-09-11 19:53:51,840 Stage-1 map = 61%, reduce = 8%, Cumulative CPU 56.41 sec -2013-09-11 19:53:52,846 Stage-1 map = 61%, reduce = 8%, Cumulative CPU 56.41 sec -2013-09-11 19:53:53,852 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 56.41 sec -2013-09-11 19:53:54,857 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 56.41 sec -2013-09-11 19:53:55,863 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 56.41 sec -2013-09-11 19:53:56,869 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 56.41 sec -2013-09-11 19:53:57,875 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 56.41 sec -2013-09-11 19:53:58,881 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 56.41 sec -2013-09-11 19:53:59,887 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 56.41 sec -2013-09-11 19:54:00,893 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 56.41 sec -2013-09-11 19:54:01,898 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 80.95 sec -2013-09-11 19:54:02,903 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 80.95 sec -2013-09-11 19:54:03,908 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 107.94 sec -2013-09-11 19:54:04,913 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 107.94 sec -2013-09-11 19:54:05,918 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 107.94 sec -2013-09-11 19:54:06,924 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 116.14 sec -2013-09-11 19:54:07,929 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 116.14 sec -2013-09-11 19:54:08,935 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 116.14 sec -2013-09-11 19:54:09,940 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 116.14 sec -2013-09-11 19:54:10,945 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 116.14 sec -2013-09-11 19:54:11,951 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 116.14 sec -2013-09-11 19:54:12,956 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 116.14 sec -2013-09-11 19:54:15,279 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 116.14 sec -2013-09-11 19:54:16,285 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 116.14 sec -2013-09-11 19:54:17,533 Stage-1 map = 100%, reduce = 73%, Cumulative CPU 116.14 sec -2013-09-11 19:54:18,539 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 116.14 sec -2013-09-11 19:54:19,545 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 116.14 sec -2013-09-11 19:54:20,551 Stage-1 map = 100%, reduce = 80%, Cumulative CPU 116.14 sec -2013-09-11 19:54:21,556 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 116.14 sec -2013-09-11 19:54:22,562 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 116.14 sec -2013-09-11 19:54:23,568 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 116.14 sec -2013-09-11 19:54:24,573 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 116.14 sec -2013-09-11 19:54:25,581 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 161.84 sec -2013-09-11 19:54:26,586 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 161.84 sec -MapReduce Total cumulative CPU time: 2 minutes 41 seconds 840 msec -Ended Job = job_201309101627_0370 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0371 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-11 19:54:29,028 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:54:37,052 Stage-2 map = 25%, reduce = 0% -2013-09-11 19:54:39,060 Stage-2 map = 50%, reduce = 0%, Cumulative CPU 13.75 sec -2013-09-11 19:54:40,065 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.75 sec -2013-09-11 19:54:41,070 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.75 sec -2013-09-11 19:54:42,074 Stage-2 map = 75%, reduce = 0%, Cumulative CPU 13.75 sec -2013-09-11 19:54:43,079 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.75 sec -2013-09-11 19:54:44,667 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.75 sec -2013-09-11 19:54:45,672 Stage-2 map = 87%, reduce = 0%, Cumulative CPU 13.75 sec -2013-09-11 19:54:46,677 Stage-2 map = 87%, reduce = 17%, Cumulative CPU 13.75 sec -2013-09-11 19:54:47,682 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.64 sec -2013-09-11 19:54:48,687 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.64 sec -2013-09-11 19:54:49,692 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.64 sec -2013-09-11 19:54:50,697 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.64 sec -2013-09-11 19:54:51,702 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.64 sec -2013-09-11 19:54:52,707 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.64 sec -2013-09-11 19:54:53,712 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.64 sec -2013-09-11 19:54:54,717 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.64 sec -2013-09-11 19:54:55,781 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 37.64 sec -2013-09-11 19:54:56,785 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 37.64 sec -2013-09-11 19:54:57,791 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 45.18 sec -2013-09-11 19:54:58,796 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 45.18 sec -2013-09-11 19:54:59,801 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 45.18 sec -MapReduce Total cumulative CPU time: 45 seconds 180 msec -Ended Job = job_201309101627_0371 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 161.84 sec HDFS Read: 109451651 HDFS Write: 399298510 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 45.18 sec HDFS Read: 399308173 HDFS Write: 445 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 27 seconds 20 msec -OK -Time taken: 106.335 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по URL.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20260@mturlrep13_201309111955_930446055.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_20919@mturlrep13_201309111955_1684376997.txt -hive> SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0372 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:55:24,288 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:55:31,317 Stage-1 map = 7%, reduce = 0% -2013-09-11 19:55:34,328 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:55:37,340 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:55:40,353 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:55:43,364 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:55:48,389 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 59.66 sec -2013-09-11 19:55:49,396 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 59.66 sec -2013-09-11 19:55:50,403 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 59.66 sec -2013-09-11 19:55:51,409 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 59.66 sec -2013-09-11 19:55:52,414 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 59.66 sec -2013-09-11 19:55:53,420 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 59.66 sec -2013-09-11 19:55:54,425 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 59.66 sec -2013-09-11 19:55:55,431 Stage-1 map = 57%, reduce = 4%, Cumulative CPU 59.66 sec -2013-09-11 19:55:56,436 Stage-1 map = 57%, reduce = 4%, Cumulative CPU 59.66 sec -2013-09-11 19:55:57,443 Stage-1 map = 57%, reduce = 4%, Cumulative CPU 59.66 sec -2013-09-11 19:55:58,448 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 59.66 sec -2013-09-11 19:55:59,454 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 59.66 sec -2013-09-11 19:56:00,459 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 59.66 sec -2013-09-11 19:56:01,464 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 59.66 sec -2013-09-11 19:56:02,470 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 59.66 sec -2013-09-11 19:56:03,475 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 59.66 sec -2013-09-11 19:56:04,479 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 59.66 sec -2013-09-11 19:56:05,484 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 59.66 sec -2013-09-11 19:56:06,489 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 59.66 sec -2013-09-11 19:56:07,494 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 111.58 sec -2013-09-11 19:56:08,499 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 116.3 sec -2013-09-11 19:56:09,503 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 116.3 sec -2013-09-11 19:56:10,509 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 123.6 sec -2013-09-11 19:56:11,514 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 123.6 sec -2013-09-11 19:56:12,518 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 123.6 sec -2013-09-11 19:56:13,523 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 123.6 sec -2013-09-11 19:56:14,529 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 123.6 sec -2013-09-11 19:56:15,534 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 123.6 sec -2013-09-11 19:56:16,538 Stage-1 map = 100%, reduce = 29%, Cumulative CPU 123.6 sec -2013-09-11 19:56:18,600 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 123.6 sec -2013-09-11 19:56:19,605 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 123.6 sec -2013-09-11 19:56:20,610 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 123.6 sec -2013-09-11 19:56:21,616 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 123.6 sec -2013-09-11 19:56:22,621 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 123.6 sec -2013-09-11 19:56:23,627 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 123.6 sec -2013-09-11 19:56:24,633 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 123.6 sec -2013-09-11 19:56:25,638 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 123.6 sec -2013-09-11 19:56:26,644 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 123.6 sec -2013-09-11 19:56:27,649 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 123.6 sec -2013-09-11 19:56:28,655 Stage-1 map = 100%, reduce = 79%, Cumulative CPU 123.6 sec -2013-09-11 19:56:29,661 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 123.6 sec -2013-09-11 19:56:30,666 Stage-1 map = 100%, reduce = 84%, Cumulative CPU 123.6 sec -2013-09-11 19:56:31,672 Stage-1 map = 100%, reduce = 88%, Cumulative CPU 123.6 sec -2013-09-11 19:56:32,677 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 123.6 sec -2013-09-11 19:56:33,683 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 123.6 sec -2013-09-11 19:56:34,692 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.4 sec -2013-09-11 19:56:35,701 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.4 sec -2013-09-11 19:56:36,706 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 169.4 sec -MapReduce Total cumulative CPU time: 2 minutes 49 seconds 400 msec -Ended Job = job_201309101627_0372 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0373 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-11 19:56:39,223 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:56:47,249 Stage-2 map = 25%, reduce = 0% -2013-09-11 19:56:50,259 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 14.04 sec -2013-09-11 19:56:51,263 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 14.04 sec -2013-09-11 19:56:52,268 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 14.04 sec -2013-09-11 19:56:53,273 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.04 sec -2013-09-11 19:56:54,278 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.04 sec -2013-09-11 19:56:55,283 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.04 sec -2013-09-11 19:56:56,288 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.04 sec -2013-09-11 19:56:57,293 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 14.04 sec -2013-09-11 19:56:58,297 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 14.04 sec -2013-09-11 19:56:59,302 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.58 sec -2013-09-11 19:57:00,306 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.58 sec -2013-09-11 19:57:01,310 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.58 sec -2013-09-11 19:57:02,315 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.58 sec -2013-09-11 19:57:03,320 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.58 sec -2013-09-11 19:57:04,324 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.58 sec -2013-09-11 19:57:05,328 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.58 sec -2013-09-11 19:57:06,333 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.58 sec -2013-09-11 19:57:07,343 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 44.0 sec -2013-09-11 19:57:08,348 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.0 sec -2013-09-11 19:57:09,352 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.0 sec -2013-09-11 19:57:10,357 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.0 sec -MapReduce Total cumulative CPU time: 46 seconds 0 msec -Ended Job = job_201309101627_0373 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 169.4 sec HDFS Read: 109451651 HDFS Write: 402873759 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 46.0 sec HDFS Read: 402889658 HDFS Write: 465 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 35 seconds 400 msec -OK -Time taken: 116.435 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24131@mturlrep13_201309111957_2140771388.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24572@mturlrep13_201309111957_461782804.txt -hive> SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0374 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:57:25,327 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:57:32,354 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:57:35,367 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:57:38,379 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:57:41,393 Stage-1 map = 36%, reduce = 0% -2013-09-11 19:57:44,406 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:57:47,423 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 57.83 sec -2013-09-11 19:57:48,429 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 57.83 sec -2013-09-11 19:57:49,437 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 57.83 sec -2013-09-11 19:57:50,443 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 57.83 sec -2013-09-11 19:57:51,449 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 57.83 sec -2013-09-11 19:57:52,455 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 57.83 sec -2013-09-11 19:57:53,460 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 57.83 sec -2013-09-11 19:57:54,466 Stage-1 map = 57%, reduce = 4%, Cumulative CPU 57.83 sec -2013-09-11 19:57:55,472 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 57.83 sec -2013-09-11 19:57:56,477 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 57.83 sec -2013-09-11 19:57:57,488 Stage-1 map = 73%, reduce = 13%, Cumulative CPU 57.83 sec -2013-09-11 19:57:58,494 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 19:57:59,500 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 19:58:00,505 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 19:58:01,511 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 19:58:02,516 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 19:58:03,522 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 19:58:04,528 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 19:58:05,534 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 19:58:06,539 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 57.83 sec -2013-09-11 19:58:07,545 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 117.68 sec -2013-09-11 19:58:08,550 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 117.68 sec -2013-09-11 19:58:09,555 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 120.18 sec -2013-09-11 19:58:10,561 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 120.18 sec -2013-09-11 19:58:11,569 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 120.18 sec -2013-09-11 19:58:12,577 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 120.18 sec -2013-09-11 19:58:13,582 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 120.18 sec -2013-09-11 19:58:14,587 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 120.18 sec -2013-09-11 19:58:15,593 Stage-1 map = 100%, reduce = 50%, Cumulative CPU 120.18 sec -2013-09-11 19:58:17,436 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 120.18 sec -2013-09-11 19:58:18,441 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 120.18 sec -2013-09-11 19:58:20,040 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 120.18 sec -2013-09-11 19:58:21,044 Stage-1 map = 100%, reduce = 74%, Cumulative CPU 120.18 sec -2013-09-11 19:58:22,050 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 120.18 sec -2013-09-11 19:58:23,055 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 120.18 sec -2013-09-11 19:58:24,060 Stage-1 map = 100%, reduce = 81%, Cumulative CPU 120.18 sec -2013-09-11 19:58:25,065 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 120.18 sec -2013-09-11 19:58:26,071 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 120.18 sec -2013-09-11 19:58:27,076 Stage-1 map = 100%, reduce = 90%, Cumulative CPU 120.18 sec -2013-09-11 19:58:28,082 Stage-1 map = 100%, reduce = 98%, Cumulative CPU 120.18 sec -2013-09-11 19:58:29,089 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 166.94 sec -2013-09-11 19:58:30,095 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 166.94 sec -MapReduce Total cumulative CPU time: 2 minutes 46 seconds 940 msec -Ended Job = job_201309101627_0374 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0375 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-11 19:58:32,596 Stage-2 map = 0%, reduce = 0% -2013-09-11 19:58:40,620 Stage-2 map = 25%, reduce = 0% -2013-09-11 19:58:43,629 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 14.59 sec -2013-09-11 19:58:44,634 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 14.59 sec -2013-09-11 19:58:45,638 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 14.59 sec -2013-09-11 19:58:47,175 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.59 sec -2013-09-11 19:58:48,180 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.59 sec -2013-09-11 19:58:49,185 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.59 sec -2013-09-11 19:58:50,189 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 14.59 sec -2013-09-11 19:58:51,194 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 14.59 sec -2013-09-11 19:58:52,199 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.13 sec -2013-09-11 19:58:53,203 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.13 sec -2013-09-11 19:58:54,208 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.13 sec -2013-09-11 19:58:55,213 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.13 sec -2013-09-11 19:58:56,217 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.13 sec -2013-09-11 19:58:57,222 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.13 sec -2013-09-11 19:58:58,227 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.13 sec -2013-09-11 19:58:59,232 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 39.13 sec -2013-09-11 19:59:00,237 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 39.13 sec -2013-09-11 19:59:01,242 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.59 sec -2013-09-11 19:59:02,248 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.59 sec -2013-09-11 19:59:03,253 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 46.59 sec -MapReduce Total cumulative CPU time: 46 seconds 590 msec -Ended Job = job_201309101627_0375 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 166.94 sec HDFS Read: 109451651 HDFS Write: 402873759 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 46.59 sec HDFS Read: 402889658 HDFS Write: 465 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 33 seconds 530 msec -OK -Time taken: 106.259 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27337@mturlrep13_201309111959_812215985.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27769@mturlrep13_201309111959_1007582094.txt -hive> SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0376 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 19:59:17,040 Stage-1 map = 0%, reduce = 0% -2013-09-11 19:59:25,072 Stage-1 map = 14%, reduce = 0% -2013-09-11 19:59:28,085 Stage-1 map = 22%, reduce = 0% -2013-09-11 19:59:31,097 Stage-1 map = 29%, reduce = 0% -2013-09-11 19:59:34,111 Stage-1 map = 43%, reduce = 0% -2013-09-11 19:59:39,135 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.82 sec -2013-09-11 19:59:40,142 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.82 sec -2013-09-11 19:59:41,150 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.82 sec -2013-09-11 19:59:42,156 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.82 sec -2013-09-11 19:59:43,161 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.82 sec -2013-09-11 19:59:44,167 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.82 sec -2013-09-11 19:59:45,174 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 55.82 sec -2013-09-11 19:59:46,180 Stage-1 map = 54%, reduce = 4%, Cumulative CPU 55.82 sec -2013-09-11 19:59:47,185 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 55.82 sec -2013-09-11 19:59:48,191 Stage-1 map = 57%, reduce = 8%, Cumulative CPU 55.82 sec -2013-09-11 19:59:49,196 Stage-1 map = 65%, reduce = 13%, Cumulative CPU 55.82 sec -2013-09-11 19:59:50,202 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 19:59:51,208 Stage-1 map = 73%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 19:59:52,214 Stage-1 map = 76%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 19:59:53,219 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 19:59:54,225 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 19:59:55,230 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 19:59:56,246 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 19:59:57,252 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 19:59:58,257 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 55.82 sec -2013-09-11 19:59:59,263 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 81.61 sec -2013-09-11 20:00:00,269 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 81.61 sec -2013-09-11 20:00:01,274 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 109.62 sec -2013-09-11 20:00:02,280 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 109.62 sec -2013-09-11 20:00:03,287 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 109.62 sec -2013-09-11 20:00:04,292 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 109.62 sec -2013-09-11 20:00:05,298 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 109.62 sec -2013-09-11 20:00:06,303 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 109.62 sec -2013-09-11 20:00:08,494 Stage-1 map = 100%, reduce = 33%, Cumulative CPU 129.4 sec -2013-09-11 20:00:09,500 Stage-1 map = 100%, reduce = 67%, Cumulative CPU 129.4 sec -2013-09-11 20:00:10,505 Stage-1 map = 100%, reduce = 69%, Cumulative CPU 129.4 sec -2013-09-11 20:00:11,511 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 129.4 sec -2013-09-11 20:00:12,517 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 129.4 sec -2013-09-11 20:00:13,522 Stage-1 map = 100%, reduce = 70%, Cumulative CPU 129.4 sec -2013-09-11 20:00:14,528 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 129.4 sec -2013-09-11 20:00:15,533 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 129.4 sec -2013-09-11 20:00:16,539 Stage-1 map = 100%, reduce = 78%, Cumulative CPU 129.4 sec -2013-09-11 20:00:17,545 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 129.4 sec -2013-09-11 20:00:18,551 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 129.4 sec -2013-09-11 20:00:19,559 Stage-1 map = 100%, reduce = 87%, Cumulative CPU 129.4 sec -2013-09-11 20:00:20,564 Stage-1 map = 100%, reduce = 95%, Cumulative CPU 129.4 sec -2013-09-11 20:00:21,572 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 146.62 sec -2013-09-11 20:00:22,578 Stage-1 map = 100%, reduce = 97%, Cumulative CPU 146.62 sec -2013-09-11 20:00:23,583 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 164.63 sec -2013-09-11 20:00:24,588 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 164.63 sec -MapReduce Total cumulative CPU time: 2 minutes 44 seconds 630 msec -Ended Job = job_201309101627_0376 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0377 -Hadoop job information for Stage-2: number of mappers: 2; number of reducers: 1 -2013-09-11 20:00:27,378 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:00:35,403 Stage-2 map = 25%, reduce = 0% -2013-09-11 20:00:38,414 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-11 20:00:39,419 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-11 20:00:40,424 Stage-2 map = 62%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-11 20:00:41,430 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-11 20:00:42,587 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-11 20:00:43,592 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-11 20:00:44,598 Stage-2 map = 88%, reduce = 0%, Cumulative CPU 13.92 sec -2013-09-11 20:00:45,602 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 13.92 sec -2013-09-11 20:00:46,607 Stage-2 map = 88%, reduce = 17%, Cumulative CPU 13.92 sec -2013-09-11 20:00:47,612 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.24 sec -2013-09-11 20:00:48,616 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.24 sec -2013-09-11 20:00:49,621 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.24 sec -2013-09-11 20:00:50,626 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.24 sec -2013-09-11 20:00:51,631 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.24 sec -2013-09-11 20:00:52,636 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.24 sec -2013-09-11 20:00:53,641 Stage-2 map = 100%, reduce = 17%, Cumulative CPU 38.24 sec -2013-09-11 20:00:54,646 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 38.24 sec -2013-09-11 20:00:55,651 Stage-2 map = 100%, reduce = 67%, Cumulative CPU 38.24 sec -2013-09-11 20:00:56,656 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 45.8 sec -2013-09-11 20:00:57,662 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 45.8 sec -MapReduce Total cumulative CPU time: 45 seconds 800 msec -Ended Job = job_201309101627_0377 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 164.63 sec HDFS Read: 109451651 HDFS Write: 402873759 SUCCESS -Job 1: Map: 2 Reduce: 1 Cumulative CPU: 45.8 sec HDFS Read: 402889658 HDFS Write: 465 SUCCESS -Total MapReduce CPU Time Spent: 3 minutes 30 seconds 430 msec -OK -Time taken: 108.254 seconds, Fetched: 10 row(s) -hive> quit; --- агрегация по URL и числу.; - -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31307@mturlrep13_201309112001_502707874.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_31761@mturlrep13_201309112001_1366550388.txt -hive> SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0378 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:01:22,869 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:01:29,898 Stage-1 map = 29%, reduce = 0% -2013-09-11 20:01:32,911 Stage-1 map = 43%, reduce = 0% -2013-09-11 20:01:33,921 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 15.13 sec -2013-09-11 20:01:34,929 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.89 sec -2013-09-11 20:01:35,937 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.89 sec -2013-09-11 20:01:36,944 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.89 sec -2013-09-11 20:01:37,951 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.89 sec -2013-09-11 20:01:38,957 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.89 sec -2013-09-11 20:01:39,962 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.89 sec -2013-09-11 20:01:40,968 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 29.89 sec -2013-09-11 20:01:41,974 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 29.89 sec -2013-09-11 20:01:42,980 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 29.89 sec -2013-09-11 20:01:43,986 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 29.89 sec -2013-09-11 20:01:44,992 Stage-1 map = 96%, reduce = 17%, Cumulative CPU 29.89 sec -2013-09-11 20:01:45,998 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 60.94 sec -2013-09-11 20:01:47,004 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 60.94 sec -2013-09-11 20:01:48,009 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 60.94 sec -2013-09-11 20:01:49,015 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 60.94 sec -2013-09-11 20:01:50,020 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 60.94 sec -2013-09-11 20:01:51,026 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 60.94 sec -2013-09-11 20:01:52,032 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 60.94 sec -2013-09-11 20:01:53,040 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 62.99 sec -2013-09-11 20:01:54,046 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 62.99 sec -2013-09-11 20:01:55,052 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 62.99 sec -2013-09-11 20:01:56,066 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 62.99 sec -2013-09-11 20:01:57,072 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 62.99 sec -2013-09-11 20:01:58,078 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 62.99 sec -2013-09-11 20:01:59,084 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 73.53 sec -2013-09-11 20:02:00,090 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 73.53 sec -MapReduce Total cumulative CPU time: 1 minutes 13 seconds 530 msec -Ended Job = job_201309101627_0378 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0379 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:02:02,569 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:02:14,605 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.88 sec -2013-09-11 20:02:15,610 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.88 sec -2013-09-11 20:02:16,615 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.88 sec -2013-09-11 20:02:17,620 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.88 sec -2013-09-11 20:02:18,625 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.88 sec -2013-09-11 20:02:19,630 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.88 sec -2013-09-11 20:02:20,635 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.88 sec -2013-09-11 20:02:21,641 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.88 sec -2013-09-11 20:02:22,646 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.88 sec -2013-09-11 20:02:23,651 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.04 sec -2013-09-11 20:02:24,656 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.04 sec -2013-09-11 20:02:25,661 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 20.04 sec -MapReduce Total cumulative CPU time: 20 seconds 40 msec -Ended Job = job_201309101627_0379 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 73.53 sec HDFS Read: 31344843 HDFS Write: 51717050 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 20.04 sec HDFS Read: 51717819 HDFS Write: 490 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 33 seconds 570 msec -OK -Time taken: 72.762 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1702@mturlrep13_201309112002_1101587188.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_2174@mturlrep13_201309112002_314206404.txt -hive> SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0380 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:02:38,506 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:02:46,538 Stage-1 map = 32%, reduce = 0% -2013-09-11 20:02:49,550 Stage-1 map = 43%, reduce = 0% -2013-09-11 20:02:50,562 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.29 sec -2013-09-11 20:02:51,569 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.29 sec -2013-09-11 20:02:52,577 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.29 sec -2013-09-11 20:02:53,583 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.29 sec -2013-09-11 20:02:54,590 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.29 sec -2013-09-11 20:02:55,596 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.29 sec -2013-09-11 20:02:56,602 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 30.29 sec -2013-09-11 20:02:57,608 Stage-1 map = 68%, reduce = 8%, Cumulative CPU 30.29 sec -2013-09-11 20:02:58,613 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 30.29 sec -2013-09-11 20:02:59,618 Stage-1 map = 88%, reduce = 17%, Cumulative CPU 30.29 sec -2013-09-11 20:03:00,623 Stage-1 map = 92%, reduce = 17%, Cumulative CPU 30.29 sec -2013-09-11 20:03:01,628 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 61.31 sec -2013-09-11 20:03:02,633 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 61.31 sec -2013-09-11 20:03:03,638 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 61.31 sec -2013-09-11 20:03:04,644 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 61.31 sec -2013-09-11 20:03:05,649 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 61.31 sec -2013-09-11 20:03:06,654 Stage-1 map = 100%, reduce = 21%, Cumulative CPU 61.31 sec -2013-09-11 20:03:07,661 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 63.28 sec -2013-09-11 20:03:08,669 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 64.61 sec -2013-09-11 20:03:09,675 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 64.61 sec -2013-09-11 20:03:10,680 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 64.61 sec -2013-09-11 20:03:11,685 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 64.61 sec -2013-09-11 20:03:12,691 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 64.61 sec -2013-09-11 20:03:13,696 Stage-1 map = 100%, reduce = 93%, Cumulative CPU 64.61 sec -2013-09-11 20:03:14,702 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 74.03 sec -2013-09-11 20:03:15,708 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 74.03 sec -2013-09-11 20:03:16,713 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 74.03 sec -MapReduce Total cumulative CPU time: 1 minutes 14 seconds 30 msec -Ended Job = job_201309101627_0380 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0381 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:03:20,309 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:03:31,341 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.54 sec -2013-09-11 20:03:32,346 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.54 sec -2013-09-11 20:03:33,351 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.54 sec -2013-09-11 20:03:34,355 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.54 sec -2013-09-11 20:03:35,361 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.54 sec -2013-09-11 20:03:36,365 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.54 sec -2013-09-11 20:03:37,370 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.54 sec -2013-09-11 20:03:38,375 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.54 sec -2013-09-11 20:03:39,380 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.54 sec -2013-09-11 20:03:40,385 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.84 sec -2013-09-11 20:03:41,392 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.84 sec -2013-09-11 20:03:42,397 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.84 sec -MapReduce Total cumulative CPU time: 19 seconds 840 msec -Ended Job = job_201309101627_0381 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 74.03 sec HDFS Read: 31344843 HDFS Write: 51717050 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 19.84 sec HDFS Read: 51717819 HDFS Write: 490 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 33 seconds 870 msec -OK -Time taken: 71.2 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4511@mturlrep13_201309112003_47105365.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5089@mturlrep13_201309112003_1512209035.txt -hive> SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0382 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:03:55,705 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:04:03,735 Stage-1 map = 29%, reduce = 0% -2013-09-11 20:04:06,747 Stage-1 map = 43%, reduce = 0% -2013-09-11 20:04:07,758 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.07 sec -2013-09-11 20:04:08,765 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.07 sec -2013-09-11 20:04:09,773 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.07 sec -2013-09-11 20:04:10,780 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.07 sec -2013-09-11 20:04:11,786 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.07 sec -2013-09-11 20:04:12,792 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.07 sec -2013-09-11 20:04:13,797 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.07 sec -2013-09-11 20:04:14,802 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 31.07 sec -2013-09-11 20:04:15,808 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 31.07 sec -2013-09-11 20:04:16,813 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 31.07 sec -2013-09-11 20:04:17,818 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 31.07 sec -2013-09-11 20:04:18,824 Stage-1 map = 97%, reduce = 17%, Cumulative CPU 46.19 sec -2013-09-11 20:04:19,829 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.28 sec -2013-09-11 20:04:20,835 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.28 sec -2013-09-11 20:04:21,840 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.28 sec -2013-09-11 20:04:22,845 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.28 sec -2013-09-11 20:04:23,851 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 63.28 sec -2013-09-11 20:04:24,857 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 63.28 sec -2013-09-11 20:04:25,863 Stage-1 map = 100%, reduce = 25%, Cumulative CPU 63.28 sec -2013-09-11 20:04:26,870 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 65.42 sec -2013-09-11 20:04:27,876 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 65.42 sec -2013-09-11 20:04:28,880 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 65.42 sec -2013-09-11 20:04:29,886 Stage-1 map = 100%, reduce = 63%, Cumulative CPU 65.42 sec -2013-09-11 20:04:30,891 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 65.42 sec -2013-09-11 20:04:31,896 Stage-1 map = 100%, reduce = 92%, Cumulative CPU 65.42 sec -2013-09-11 20:04:32,902 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 76.37 sec -2013-09-11 20:04:33,908 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 76.37 sec -MapReduce Total cumulative CPU time: 1 minutes 16 seconds 370 msec -Ended Job = job_201309101627_0382 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0383 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:04:36,402 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:04:48,438 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.09 sec -2013-09-11 20:04:49,443 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.09 sec -2013-09-11 20:04:50,448 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.09 sec -2013-09-11 20:04:51,452 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.09 sec -2013-09-11 20:04:52,456 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.09 sec -2013-09-11 20:04:53,461 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.09 sec -2013-09-11 20:04:54,466 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 15.09 sec -2013-09-11 20:04:55,471 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.09 sec -2013-09-11 20:04:56,476 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 15.09 sec -2013-09-11 20:04:57,481 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.29 sec -2013-09-11 20:04:58,486 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 19.29 sec -MapReduce Total cumulative CPU time: 19 seconds 290 msec -Ended Job = job_201309101627_0383 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 76.37 sec HDFS Read: 31344843 HDFS Write: 51717050 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 19.29 sec HDFS Read: 51717819 HDFS Write: 490 SUCCESS -Total MapReduce CPU Time Spent: 1 minutes 35 seconds 660 msec -OK -Time taken: 70.36 seconds, Fetched: 10 row(s) -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_7537@mturlrep13_201309112005_898269574.txt -hive> ; -hive> quit; - -times: 1 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8303@mturlrep13_201309112005_242909153.txt -hive> ; -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8520@mturlrep13_201309112005_740096707.txt -hive> ; -hive> quit; - -times: 2 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9324@mturlrep13_201309112005_1365009510.txt -hive> ; -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9538@mturlrep13_201309112005_81537852.txt -hive> ; -hive> quit; - -times: 3 -query: -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9966@mturlrep13_201309112005_1125410388.txt -hive> ; -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10208@mturlrep13_201309112005_1554289038.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_10658@mturlrep13_201309112005_874745706.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0384 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:05:43,792 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:05:50,819 Stage-1 map = 32%, reduce = 0% -2013-09-11 20:05:52,837 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.84 sec -2013-09-11 20:05:53,845 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.84 sec -2013-09-11 20:05:54,853 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.84 sec -2013-09-11 20:05:55,859 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.84 sec -2013-09-11 20:05:56,866 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.84 sec -2013-09-11 20:05:57,873 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.84 sec -2013-09-11 20:05:58,880 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.84 sec -2013-09-11 20:05:59,885 Stage-1 map = 80%, reduce = 17%, Cumulative CPU 18.84 sec -2013-09-11 20:06:00,891 Stage-1 map = 89%, reduce = 17%, Cumulative CPU 27.75 sec -2013-09-11 20:06:01,896 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.3 sec -2013-09-11 20:06:02,902 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.3 sec -2013-09-11 20:06:03,907 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.3 sec -2013-09-11 20:06:04,913 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.3 sec -2013-09-11 20:06:05,920 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.53 sec -2013-09-11 20:06:06,926 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.53 sec -2013-09-11 20:06:07,932 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.53 sec -MapReduce Total cumulative CPU time: 41 seconds 530 msec -Ended Job = job_201309101627_0384 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0385 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:06:10,591 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:06:12,599 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-11 20:06:13,604 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-11 20:06:14,609 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-11 20:06:15,614 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-11 20:06:16,618 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-11 20:06:17,622 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-11 20:06:18,627 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-11 20:06:19,633 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.74 sec -2013-09-11 20:06:20,638 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -2013-09-11 20:06:21,644 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.08 sec -MapReduce Total cumulative CPU time: 2 seconds 80 msec -Ended Job = job_201309101627_0385 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.53 sec HDFS Read: 118784021 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.08 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 610 msec -OK -Time taken: 47.898 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13025@mturlrep13_201309112006_366484138.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_13471@mturlrep13_201309112006_1061969476.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0386 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:06:36,447 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:06:43,482 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 9.43 sec -2013-09-11 20:06:44,489 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.35 sec -2013-09-11 20:06:45,496 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.35 sec -2013-09-11 20:06:46,503 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.35 sec -2013-09-11 20:06:47,508 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.35 sec -2013-09-11 20:06:48,514 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.35 sec -2013-09-11 20:06:49,520 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.35 sec -2013-09-11 20:06:50,526 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.35 sec -2013-09-11 20:06:51,533 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.7 sec -2013-09-11 20:06:52,538 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.7 sec -2013-09-11 20:06:53,543 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.7 sec -2013-09-11 20:06:54,548 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.7 sec -2013-09-11 20:06:55,554 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.7 sec -2013-09-11 20:06:56,560 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.7 sec -2013-09-11 20:06:57,568 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.93 sec -2013-09-11 20:06:58,810 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.93 sec -MapReduce Total cumulative CPU time: 41 seconds 930 msec -Ended Job = job_201309101627_0386 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0387 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:07:01,348 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:07:03,357 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-11 20:07:04,362 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-11 20:07:05,367 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-11 20:07:06,371 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-11 20:07:07,376 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-11 20:07:08,381 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-11 20:07:09,387 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.71 sec -2013-09-11 20:07:10,392 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.71 sec -2013-09-11 20:07:11,397 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -2013-09-11 20:07:12,402 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.1 sec -MapReduce Total cumulative CPU time: 2 seconds 100 msec -Ended Job = job_201309101627_0387 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.93 sec HDFS Read: 118784021 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.1 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 30 msec -OK -Time taken: 44.248 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_15787@mturlrep13_201309112007_1440494394.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_16251@mturlrep13_201309112007_1706863363.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0388 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:07:26,087 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:07:34,130 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 9.01 sec -2013-09-11 20:07:35,137 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.94 sec -2013-09-11 20:07:36,145 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.94 sec -2013-09-11 20:07:37,152 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.94 sec -2013-09-11 20:07:38,158 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.94 sec -2013-09-11 20:07:39,165 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.94 sec -2013-09-11 20:07:40,172 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.94 sec -2013-09-11 20:07:41,179 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.94 sec -2013-09-11 20:07:42,185 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.34 sec -2013-09-11 20:07:43,190 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.34 sec -2013-09-11 20:07:44,195 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.34 sec -2013-09-11 20:07:45,201 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.34 sec -2013-09-11 20:07:46,206 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.34 sec -2013-09-11 20:07:47,214 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 39.41 sec -2013-09-11 20:07:48,220 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.49 sec -2013-09-11 20:07:49,226 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.49 sec -MapReduce Total cumulative CPU time: 41 seconds 490 msec -Ended Job = job_201309101627_0388 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0389 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:07:51,717 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:07:53,726 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-11 20:07:54,731 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-11 20:07:55,737 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-11 20:07:56,742 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-11 20:07:57,747 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-11 20:07:58,752 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-11 20:07:59,757 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.91 sec -2013-09-11 20:08:00,763 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.91 sec -2013-09-11 20:08:01,768 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.27 sec -2013-09-11 20:08:02,774 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.27 sec -2013-09-11 20:08:03,780 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.27 sec -MapReduce Total cumulative CPU time: 2 seconds 270 msec -Ended Job = job_201309101627_0389 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.49 sec HDFS Read: 118784021 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.27 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 760 msec -OK -Time taken: 45.022 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_18797@mturlrep13_201309112008_775683828.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_19297@mturlrep13_201309112008_761964500.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0390 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:08:25,097 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:08:32,126 Stage-1 map = 36%, reduce = 0% -2013-09-11 20:08:34,143 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 20:08:35,151 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 20:08:36,159 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 20:08:37,166 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 20:08:38,172 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 20:08:39,181 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 20:08:40,191 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.7 sec -2013-09-11 20:08:41,197 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 18.7 sec -2013-09-11 20:08:42,203 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 27.69 sec -2013-09-11 20:08:43,208 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.14 sec -2013-09-11 20:08:44,214 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.14 sec -2013-09-11 20:08:45,219 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.14 sec -2013-09-11 20:08:46,227 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.14 sec -2013-09-11 20:08:47,235 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.23 sec -2013-09-11 20:08:48,242 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.23 sec -2013-09-11 20:08:49,248 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.23 sec -MapReduce Total cumulative CPU time: 41 seconds 230 msec -Ended Job = job_201309101627_0390 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0391 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:08:51,794 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:08:53,820 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-11 20:08:54,826 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-11 20:08:55,832 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-11 20:08:56,836 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-11 20:08:57,842 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-11 20:08:58,848 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-11 20:08:59,853 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.72 sec -2013-09-11 20:09:00,858 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.72 sec -2013-09-11 20:09:01,864 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.02 sec -2013-09-11 20:09:02,870 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.02 sec -MapReduce Total cumulative CPU time: 2 seconds 20 msec -Ended Job = job_201309101627_0391 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.23 sec HDFS Read: 115339269 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.02 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 250 msec -OK -Time taken: 47.69 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21543@mturlrep13_201309112009_592955581.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_21971@mturlrep13_201309112009_2025172247.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0392 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:09:16,753 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:09:23,786 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 9.28 sec -2013-09-11 20:09:24,794 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.17 sec -2013-09-11 20:09:25,802 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.17 sec -2013-09-11 20:09:26,809 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.17 sec -2013-09-11 20:09:27,816 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.17 sec -2013-09-11 20:09:28,823 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.17 sec -2013-09-11 20:09:29,829 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.17 sec -2013-09-11 20:09:30,835 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.17 sec -2013-09-11 20:09:31,842 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 20:09:32,847 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 20:09:33,853 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 20:09:34,858 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 20:09:35,864 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 20:09:36,870 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.06 sec -2013-09-11 20:09:37,877 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.27 sec -2013-09-11 20:09:38,884 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.27 sec -MapReduce Total cumulative CPU time: 41 seconds 270 msec -Ended Job = job_201309101627_0392 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0393 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:09:42,378 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:09:43,384 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-11 20:09:44,390 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-11 20:09:45,395 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-11 20:09:46,400 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-11 20:09:47,405 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-11 20:09:48,410 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-11 20:09:49,415 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-11 20:09:50,420 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.83 sec -2013-09-11 20:09:51,426 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -2013-09-11 20:09:52,432 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -2013-09-11 20:09:53,437 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -MapReduce Total cumulative CPU time: 2 seconds 200 msec -Ended Job = job_201309101627_0393 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.27 sec HDFS Read: 115339269 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.2 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 470 msec -OK -Time taken: 44.993 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24204@mturlrep13_201309112009_1407745219.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_24634@mturlrep13_201309112009_1617977624.txt -hive> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0394 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:10:07,429 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:10:14,464 Stage-1 map = 46%, reduce = 0%, Cumulative CPU 9.2 sec -2013-09-11 20:10:15,472 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 20:10:16,480 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 20:10:17,486 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 20:10:18,492 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 20:10:19,499 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 20:10:20,505 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 20:10:21,511 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.91 sec -2013-09-11 20:10:22,517 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.24 sec -2013-09-11 20:10:23,523 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.24 sec -2013-09-11 20:10:24,528 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.24 sec -2013-09-11 20:10:25,533 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.24 sec -2013-09-11 20:10:26,539 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.24 sec -2013-09-11 20:10:27,544 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.24 sec -2013-09-11 20:10:28,552 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.33 sec -2013-09-11 20:10:29,557 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.33 sec -MapReduce Total cumulative CPU time: 41 seconds 330 msec -Ended Job = job_201309101627_0394 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0395 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:10:33,096 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:10:34,102 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-11 20:10:35,108 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-11 20:10:36,113 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-11 20:10:37,118 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-11 20:10:38,123 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-11 20:10:39,127 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-11 20:10:40,132 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-11 20:10:41,137 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.69 sec -2013-09-11 20:10:42,143 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.09 sec -2013-09-11 20:10:43,148 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.09 sec -2013-09-11 20:10:44,154 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.09 sec -MapReduce Total cumulative CPU time: 2 seconds 90 msec -Ended Job = job_201309101627_0395 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.33 sec HDFS Read: 115339269 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.09 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 420 msec -OK -Time taken: 45.13 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_27651@mturlrep13_201309112010_413283612.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_28105@mturlrep13_201309112010_1811330810.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0396 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:11:08,317 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:11:15,347 Stage-1 map = 32%, reduce = 0% -2013-09-11 20:11:17,363 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.33 sec -2013-09-11 20:11:18,371 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.33 sec -2013-09-11 20:11:19,380 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.33 sec -2013-09-11 20:11:20,386 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.33 sec -2013-09-11 20:11:21,393 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.33 sec -2013-09-11 20:11:22,399 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.33 sec -2013-09-11 20:11:23,406 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.33 sec -2013-09-11 20:11:24,413 Stage-1 map = 68%, reduce = 8%, Cumulative CPU 19.33 sec -2013-09-11 20:11:25,420 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 19.33 sec -2013-09-11 20:11:26,427 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.65 sec -2013-09-11 20:11:27,432 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.65 sec -2013-09-11 20:11:28,438 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.65 sec -2013-09-11 20:11:29,443 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 37.65 sec -2013-09-11 20:11:30,451 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.61 sec -2013-09-11 20:11:31,458 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.61 sec -2013-09-11 20:11:32,464 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 41.61 sec -MapReduce Total cumulative CPU time: 41 seconds 610 msec -Ended Job = job_201309101627_0396 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0397 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:11:34,967 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:11:36,976 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 20:11:37,981 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 20:11:38,986 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 20:11:39,990 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 20:11:40,995 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 20:11:42,000 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 20:11:43,038 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.77 sec -2013-09-11 20:11:44,044 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.77 sec -2013-09-11 20:11:45,050 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.18 sec -2013-09-11 20:11:46,056 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.18 sec -2013-09-11 20:11:47,062 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.18 sec -MapReduce Total cumulative CPU time: 2 seconds 180 msec -Ended Job = job_201309101627_0397 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 41.61 sec HDFS Read: 118662691 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.18 sec HDFS Read: 959 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 43 seconds 790 msec -OK -Time taken: 48.915 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30480@mturlrep13_201309112011_933995601.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_30907@mturlrep13_201309112011_251050369.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0398 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:12:01,259 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:12:08,296 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 20:12:09,304 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 20:12:10,313 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 20:12:11,320 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 20:12:12,326 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 20:12:13,333 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 20:12:14,339 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 20:12:15,346 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.56 sec -2013-09-11 20:12:16,352 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.26 sec -2013-09-11 20:12:17,358 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.26 sec -2013-09-11 20:12:18,363 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.26 sec -2013-09-11 20:12:19,368 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.26 sec -2013-09-11 20:12:20,373 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.26 sec -2013-09-11 20:12:21,379 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 36.26 sec -2013-09-11 20:12:22,387 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.33 sec -2013-09-11 20:12:23,393 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.33 sec -MapReduce Total cumulative CPU time: 40 seconds 330 msec -Ended Job = job_201309101627_0398 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0399 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:12:26,898 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:12:27,903 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:12:28,909 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:12:29,915 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:12:30,920 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:12:31,925 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:12:32,929 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:12:33,934 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:12:34,939 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:12:35,944 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -2013-09-11 20:12:36,950 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -2013-09-11 20:12:37,956 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.25 sec -MapReduce Total cumulative CPU time: 2 seconds 250 msec -Ended Job = job_201309101627_0399 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 40.33 sec HDFS Read: 118662691 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.25 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 42 seconds 580 msec -OK -Time taken: 45.122 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_782@mturlrep13_201309112012_979888029.txt -hive> ; -hive> quit; - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_1252@mturlrep13_201309112012_1568189503.txt -hive> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000;; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0400 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:12:51,075 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:12:59,115 Stage-1 map = 43%, reduce = 0%, Cumulative CPU 9.84 sec -2013-09-11 20:13:00,123 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.99 sec -2013-09-11 20:13:01,130 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.99 sec -2013-09-11 20:13:02,136 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.99 sec -2013-09-11 20:13:03,142 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.99 sec -2013-09-11 20:13:04,149 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.99 sec -2013-09-11 20:13:05,156 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.99 sec -2013-09-11 20:13:06,163 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.99 sec -2013-09-11 20:13:07,169 Stage-1 map = 93%, reduce = 17%, Cumulative CPU 28.64 sec -2013-09-11 20:13:08,176 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 20:13:09,181 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 20:13:10,188 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 20:13:11,193 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 39.72 sec -2013-09-11 20:13:12,201 Stage-1 map = 100%, reduce = 58%, Cumulative CPU 41.17 sec -2013-09-11 20:13:13,208 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.56 sec -2013-09-11 20:13:14,214 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.56 sec -MapReduce Total cumulative CPU time: 42 seconds 560 msec -Ended Job = job_201309101627_0400 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0401 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:13:16,674 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:13:18,684 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.8 sec -2013-09-11 20:13:19,689 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.8 sec -2013-09-11 20:13:20,695 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.8 sec -2013-09-11 20:13:21,700 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.8 sec -2013-09-11 20:13:22,705 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.8 sec -2013-09-11 20:13:23,711 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.8 sec -2013-09-11 20:13:24,716 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.8 sec -2013-09-11 20:13:25,721 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.8 sec -2013-09-11 20:13:26,727 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -2013-09-11 20:13:27,734 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -2013-09-11 20:13:28,740 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.2 sec -MapReduce Total cumulative CPU time: 2 seconds 200 msec -Ended Job = job_201309101627_0401 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.56 sec HDFS Read: 118662691 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.2 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 760 msec -OK -Time taken: 44.996 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_3657@mturlrep13_201309112013_152749800.txt -hive> ; -hive> quit; - -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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4123@mturlrep13_201309112013_250715100.txt -hive> 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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; ; -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4384@mturlrep13_201309112013_11229958.txt -hive> ; -hive> quit; - -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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_4967@mturlrep13_201309112013_1609437791.txt -hive> 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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; ; -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5327@mturlrep13_201309112013_984749990.txt -hive> ; -hive> quit; - -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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_5791@mturlrep13_201309112014_528239556.txt -hive> 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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; ; -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6065@mturlrep13_201309112014_1043898379.txt -hive> ; -hive> quit; - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_6531@mturlrep13_201309112014_1491442849.txt -hive> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0402 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:14:25,167 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:14:32,197 Stage-1 map = 36%, reduce = 0% -2013-09-11 20:14:33,208 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.37 sec -2013-09-11 20:14:34,216 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.37 sec -2013-09-11 20:14:35,224 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.37 sec -2013-09-11 20:14:36,231 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.37 sec -2013-09-11 20:14:37,237 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.37 sec -2013-09-11 20:14:38,243 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.37 sec -2013-09-11 20:14:39,250 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.37 sec -2013-09-11 20:14:40,257 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 19.37 sec -2013-09-11 20:14:41,263 Stage-1 map = 84%, reduce = 17%, Cumulative CPU 19.37 sec -2013-09-11 20:14:42,271 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.01 sec -2013-09-11 20:14:43,276 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.01 sec -2013-09-11 20:14:44,282 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.01 sec -2013-09-11 20:14:45,287 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.01 sec -2013-09-11 20:14:46,293 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 38.01 sec -2013-09-11 20:14:47,301 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.21 sec -2013-09-11 20:14:48,308 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 42.21 sec -MapReduce Total cumulative CPU time: 42 seconds 210 msec -Ended Job = job_201309101627_0402 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0403 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:14:50,774 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:14:52,782 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:14:53,788 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:14:54,792 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:14:55,797 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:14:56,802 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:14:57,807 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:14:58,813 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.75 sec -2013-09-11 20:14:59,819 Stage-2 map = 100%, reduce = 33%, Cumulative CPU 0.75 sec -2013-09-11 20:15:00,824 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -2013-09-11 20:15:01,831 Stage-2 map = 100%, reduce = 100%, Cumulative CPU 2.13 sec -MapReduce Total cumulative CPU time: 2 seconds 130 msec -Ended Job = job_201309101627_0403 -MapReduce Jobs Launched: -Job 0: Map: 4 Reduce: 2 Cumulative CPU: 42.21 sec HDFS Read: 148406904 HDFS Write: 192 SUCCESS -Job 1: Map: 1 Reduce: 1 Cumulative CPU: 2.13 sec HDFS Read: 961 HDFS Write: 0 SUCCESS -Total MapReduce CPU Time Spent: 44 seconds 340 msec -OK -Time taken: 46.841 seconds -hive> quit; -status -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_8933@mturlrep13_201309112015_1862976498.txt -hive> ; -hive> quit; - -times: 2 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn hive - -Logging initialized using configuration in file:/opt/hive/conf/hive-log4j.properties -Hive history file=/tmp/kartavyy/hive_job_log_kartavyy_9402@mturlrep13_201309112015_389918397.txt -hive> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; ; -Total MapReduce jobs = 2 -Launching Job 1 out of 2 -Number of reduce tasks not specified. Estimated from input data size: 2 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0404 -Hadoop job information for Stage-1: number of mappers: 4; number of reducers: 2 -2013-09-11 20:15:15,126 Stage-1 map = 0%, reduce = 0% -2013-09-11 20:15:23,165 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.36 sec -2013-09-11 20:15:24,174 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.36 sec -2013-09-11 20:15:25,181 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.36 sec -2013-09-11 20:15:26,188 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.36 sec -2013-09-11 20:15:27,195 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.36 sec -2013-09-11 20:15:28,201 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.36 sec -2013-09-11 20:15:29,208 Stage-1 map = 50%, reduce = 0%, Cumulative CPU 18.36 sec -2013-09-11 20:15:30,214 Stage-1 map = 97%, reduce = 8%, Cumulative CPU 26.7 sec -2013-09-11 20:15:31,220 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.89 sec -2013-09-11 20:15:32,226 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.89 sec -2013-09-11 20:15:33,231 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.89 sec -2013-09-11 20:15:34,237 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.89 sec -2013-09-11 20:15:35,243 Stage-1 map = 100%, reduce = 17%, Cumulative CPU 35.89 sec -2013-09-11 20:15:36,251 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.1 sec -2013-09-11 20:15:37,257 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.1 sec -2013-09-11 20:15:38,263 Stage-1 map = 100%, reduce = 100%, Cumulative CPU 40.1 sec -MapReduce Total cumulative CPU time: 40 seconds 100 msec -Ended Job = job_201309101627_0404 -Launching Job 2 out of 2 -Number of reduce tasks determined at compile time: 1 -In order to change the average load for a reducer (in bytes): - set hive.exec.reducers.bytes.per.reducer= -In order to limit the maximum number of reducers: - set hive.exec.reducers.max= -In order to set a constant number of reducers: - set mapred.reduce.tasks= -Kill Command = /usr/libexec/../bin/hadoop job -kill job_201309101627_0405 -Hadoop job information for Stage-2: number of mappers: 1; number of reducers: 1 -2013-09-11 20:15:40,779 Stage-2 map = 0%, reduce = 0% -2013-09-11 20:15:42,789 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec -2013-09-11 20:15:43,794 Stage-2 map = 100%, reduce = 0%, Cumulative CPU 0.74 sec diff --git a/benchmark/hive/queries.sql b/benchmark/hive/queries.sql deleted file mode 100644 index e48aa7057d8..00000000000 --- a/benchmark/hive/queries.sql +++ /dev/null @@ -1,109 +0,0 @@ -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(*) AS c FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY c 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 c 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(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c 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(*) AS c FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; --- агрегация по числу и строке, большое количество ключей.; - -SELECT UserID, count(*) AS c FROM hits_10m GROUP BY UserID ORDER BY c DESC LIMIT 10; --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; --- ещё более сложная агрегация.; - -SELECT UserID, SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; --- то же самое, но без сортировки.; - -SELECT UserID, minute(EventTime), SearchPhrase, count(*) AS c FROM hits_10m GROUP BY UserID, minute(EventTime), SearchPhrase ORDER BY c DESC LIMIT 10; --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; --- мощная фильтрация по столбцу типа UInt64.; - -SELECT count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%'; --- фильтрация по поиску подстроки в строке.; - -SELECT SearchPhrase, MAX(URL), count(*) AS c FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; --- вынимаем большие столбцы, фильтрация по строке.; - -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 != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; --- чуть больше столбцы.; - -SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; --- плохой запрос - вынимаем все столбцы.; - -SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; --- большая сортировка.; - -SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; --- большая сортировка по строкам.; - -SELECT SearchPhrase, EventTime FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; --- большая сортировка по кортежу.; - -SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_10m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; --- считаем средние длины URL для крупных счётчиков.; - -SELECT SUBSTRING(SUBSTRING(Referer, instr(Referer, '//') + 2), 1, if(0 < instr(SUBSTRING(Referer, instr(Referer, '//') + 2), '/') - 1, instr(SUBSTRING(Referer, instr(Referer, '//') + 2), '/' ) - 1, 0)), avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer != '' GROUP BY SUBSTRING(SUBSTRING(Referer, instr(Referer, '//') + 2), 1, if(0 < instr(SUBSTRING(Referer, instr(Referer, '//') + 2), '/') - 1, instr(SUBSTRING(Referer, instr(Referer, '//') + 2), '/' ) - 1, 0)) 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 c DESC LIMIT 10; --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; --- то же самое, но ещё и без фильтрации.; - -SELECT URL, count(*) AS c FROM hits_10m GROUP BY URL ORDER BY c DESC LIMIT 10; --- агрегация по URL.; - -SELECT 1, URL, count(*) AS c FROM hits_10m GROUP BY 1, URL ORDER BY c DESC LIMIT 10; --- агрегация по URL и числу.; - -SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; - -SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; - - -SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT DontCountHits != 0 AND NOT Refresh != 0 AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; - - -SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND IsLink != 0 AND NOT IsDownload != 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; - -SELECT TraficSourceID, SearchEngineID, AdvEngineID, URL, count(*) as c, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') as src FROM hits_100m WHERE CounterID = 62 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, URL, if(SearchEngineID = 0 AND AdvEngineID = 0 , Referer, '') ORDER BY c DESC LIMIT 1000; - -SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 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 >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-31') AND NOT Refresh != 0 AND NOT DontCountHits != 0 AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; - -SELECT unix_timestamp(EventTime) - SECOND(EventTime) AS m, count(*) FROM hits_10m WHERE CounterID = 62 AND EventDate >= TIMESTAMP('2013-07-01') AND EventDate <= TIMESTAMP('2013-07-02') AND NOT Refresh != 0 AND NOT DontCountHits != 0 GROUP BY unix_timestamp(EventTime) - SECOND(EventTime) ORDER BY m; diff --git a/benchmark/hive/run_hive.sh b/benchmark/hive/run_hive.sh deleted file mode 100755 index 48604cd8aa6..00000000000 --- a/benchmark/hive/run_hive.sh +++ /dev/null @@ -1,2 +0,0 @@ -cd /home/kartavyy/benchmark -./benchmark.sh -c hive/conf.sh -n $1 > hive/log/log_$1 diff --git a/benchmark/infinidb/conf.sh b/benchmark/infinidb/conf.sh deleted file mode 100644 index cd201658c9e..00000000000 --- a/benchmark/infinidb/conf.sh +++ /dev/null @@ -1,4 +0,0 @@ -CONF_DIR=/home/kartavyy/benchmark/mysql -expect_file=$CONF_DIR/expect.tcl -test_file=$CONF_DIR/queries.sql -etc_init_d_service=/etc/init.d/mysql diff --git a/benchmark/infinidb/define_schema.sql b/benchmark/infinidb/define_schema.sql deleted file mode 100644 index 6348cda5f6f..00000000000 --- a/benchmark/infinidb/define_schema.sql +++ /dev/null @@ -1,7 +0,0 @@ -create table hits_10m( WatchID BIGINT, JavaEnable TINYINT UNSIGNED, Title VARCHAR(1024), GoodEvent SMALLINT, EventTime DATETIME, EventDate DATE, CounterID INTEGER UNSIGNED, ClientIP INTEGER UNSIGNED, RegionID INTEGER UNSIGNED, UserID BIGINT, CounterClass TINYINT, OS SMALLINT, UserAgent SMALLINT, URL VARCHAR(6072), Referer VARCHAR(2048), Refresh TINYINT, RefererCategoryID SMALLINT UNSIGNED, RefererRegionID INTEGER UNSIGNED, URLCategoryID SMALLINT UNSIGNED, URLRegionID INTEGER UNSIGNED, ResolutionWidth SMALLINT UNSIGNED, ResolutionHeight SMALLINT UNSIGNED, ResolutionDepth TINYINT UNSIGNED, FlashMajor TINYINT UNSIGNED, FlashMinor TINYINT UNSIGNED, FlashMinor2 VARCHAR(256), NetMajor TINYINT UNSIGNED, NetMinor TINYINT UNSIGNED, UserAgentMajor SMALLINT UNSIGNED, UserAgentMinor CHAR(2), CookieEnable TINYINT UNSIGNED, JavascriptEnable TINYINT UNSIGNED, IsMobile TINYINT UNSIGNED, MobilePhone TINYINT UNSIGNED, MobilePhoneModel VARCHAR(80), Params VARCHAR(2048), IPNetworkID INT UNSIGNED, TraficSourceID SMALLINT, SearchEngineID SMALLINT UNSIGNED, SearchPhrase VARCHAR(1024), AdvEngineID TINYINT UNSIGNED, IsArtifical TINYINT UNSIGNED, WindowClientWidth SMALLINT UNSIGNED, WindowClientHeight SMALLINT UNSIGNED, ClientTimeZone INTEGER, ClientEventTime DATETIME, SilverlightVersion1 TINYINT UNSIGNED, SilverlightVersion2 TINYINT UNSIGNED, SilverlightVersion3 INT UNSIGNED, SilverlightVersion4 SMALLINT UNSIGNED, PageCharset VARCHAR(80), CodeVersion INT UNSIGNED, IsLink TINYINT UNSIGNED, IsDownload TINYINT UNSIGNED, IsNotBounce TINYINT UNSIGNED, FUniqID BIGINT, OriginalURL VARCHAR(6072), HID INT UNSIGNED, IsOldCounter TINYINT UNSIGNED, IsEvent TINYINT UNSIGNED, IsParameter TINYINT UNSIGNED, DontCountHits TINYINT UNSIGNED, WithHash TINYINT UNSIGNED, HitColor CHAR(1), LocalEventTime DATETIME, Age TINYINT UNSIGNED, Sex TINYINT UNSIGNED, Income TINYINT UNSIGNED, Interests SMALLINT UNSIGNED, Robotness TINYINT UNSIGNED, RemoteIP INT UNSIGNED, WindowName INT, OpenerName INT, HistoryLength SMALLINT, BrowserLanguage CHAR(2), BrowserCountry CHAR(2), SocialNetwork VARCHAR(128), SocialAction VARCHAR(128), HTTPError SMALLINT UNSIGNED, SendTiming INT UNSIGNED, DNSTiming INT UNSIGNED, ConnectTiming INTEGER UNSIGNED, ResponseStartTiming INTEGER UNSIGNED, ResponseEndTiming INTEGER UNSIGNED, FetchTiming INTEGER UNSIGNED, SocialSourceNetworkID TINYINT UNSIGNED, SocialSourcePage VARCHAR(128), ParamPrice BIGINT, ParamOrderID VARCHAR(80), ParamCurrency CHAR(3), ParamCurrencyID SMALLINT UNSIGNED, OpenstatServiceName VARCHAR(80), OpenstatCampaignID VARCHAR(80), OpenstatAdID VARCHAR(80), OpenstatSourceID VARCHAR(80), UTMSource VARCHAR(256), UTMMedium VARCHAR(256), UTMCampaign VARCHAR(256), UTMContent VARCHAR(256), UTMTerm VARCHAR(256), FromTag VARCHAR(256), HasGCLID TINYINT UNSIGNED, RefererHash BIGINT, URLHash BIGINT, CLID INTEGER UNSIGNED, UserIDHash BIGINT UNSIGNED) ENGINE=MYISAM; - -CREATE INDEX hits_10m_ind on hits_10m (CounterID, EventDate, UserIDHash, EventTime) using BTREE; - -load data infile '/opt/dump/dump_0.3/dump_hits_10m_meshed_utf8.tsv' into table hits_10m FIELDS TERMINATED BY '\t' ESCAPED BY '\\' ; - - diff --git a/benchmark/infinidb/expect.tcl b/benchmark/infinidb/expect.tcl deleted file mode 100644 index 6b3a86c070a..00000000000 --- a/benchmark/infinidb/expect.tcl +++ /dev/null @@ -1,23 +0,0 @@ -#!/usr/bin/env bash -#!/bin/expect - -# Set timeout -set timeout 600 - -# Get arguments -set query [lindex $argv 0] - -spawn mysql -u root - -expect "mysql>" -send "use hits\r" - -expect "mysql>" - -send "$query\r" - -expect "mysql>" - -send "quit\r" - -expect eof \ No newline at end of file diff --git a/benchmark/infinidb/log/log_100m b/benchmark/infinidb/log/log_100m deleted file mode 100644 index e797ac2d06d..00000000000 --- a/benchmark/infinidb/log/log_100m +++ /dev/null @@ -1,5540 +0,0 @@ -start time: Сб. сент. 21 01:16:53 MSK 2013 -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 956 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 958 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m; -+-----------+ -+-----------+ -+-----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 959 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 961 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m; -+-----------+ -+-----------+ -+-----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 962 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 964 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m; -+-----------+ -+-----------+ -+-----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 965 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 967 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (3 min 40.39 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 968 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 970 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (3 min 54.32 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 971 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 973 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (5 min 5.28 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 974 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 976 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -1 row in set (3 min 40.45 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 977 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 979 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -1 row in set (3 min 18.31 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 980 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 982 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -1 row in set (3 min 18.37 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 983 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 985 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -+-----------------------------+ -+-----------------------------+ -+-----------------------------+ -1 row in set (3 min 27.60 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 986 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 988 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -+-----------------------------+ -+-----------------------------+ -+-----------------------------+ -1 row in set (3 min 10.59 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 989 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 991 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -+-----------------------------+ -+-----------------------------+ -+-----------------------------+ -1 row in set (3 min 8.35 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 992 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 994 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (4 min 35.96 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 995 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 997 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (4 min 10.84 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 998 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1000 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (4 min 6.93 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1001 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1003 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (4 min 52.17 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1004 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1006 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (4 min 14.14 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1007 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1009 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (4 min 11.06 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1010 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1012 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (1 min 15.51 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1013 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1015 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (1 min 16.11 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1016 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1018 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (1 min 14.98 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1019 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1021 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -18 rows in set (3 min 23.94 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1022 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1024 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -18 rows in set (3 min 4.14 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1025 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1027 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -18 rows in set (3 min 0.82 sec) - -mysql> quit -Bye --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1028 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1030 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+---------+ -+----------+---------+ -+----------+---------+ -10 rows in set (4 min 47.28 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1031 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1033 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+---------+ -+----------+---------+ -+----------+---------+ -10 rows in set (4 min 12.52 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1034 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1036 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+---------+ -+----------+---------+ -+----------+---------+ -10 rows in set (4 min 9.48 sec) - -mysql> quit -Bye --- агрегация, среднее количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1037 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1039 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DES C LIMIT 10; -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -10 rows in set (4 min 59.44 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1040 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1042 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DES C LIMIT 10; -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -10 rows in set (4 min 42.02 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1043 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1045 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DES C LIMIT 10; -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -10 rows in set (4 min 31.33 sec) - -mysql> quit -Bye --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1046 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1048 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+--------+ -+------------------+--------+ -+------------------+--------+ -10 rows in set (3 min 38.71 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1049 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1051 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+--------+ -+------------------+--------+ -+------------------+--------+ -10 rows in set (3 min 17.51 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1052 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1054 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+--------+ -+------------------+--------+ -+------------------+--------+ -10 rows in set (3 min 15.94 sec) - -mysql> quit -Bye --- мощная фильтрация по строкам, затем агрегация по строкам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1055 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1057 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -+-------------+------------------+--------+ -+-------------+------------------+--------+ -+-------------+------------------+--------+ -10 rows in set (3 min 40.10 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1058 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1060 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -+-------------+------------------+--------+ -+-------------+------------------+--------+ -+-------------+------------------+--------+ -10 rows in set (3 min 17.17 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1061 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1063 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -+-------------+------------------+--------+ -+-------------+------------------+--------+ -+-------------+------------------+--------+ -10 rows in set (3 min 19.88 sec) - -mysql> quit -Bye --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1064 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1066 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -+-----------------------------------------------------------+----------+ -+-----------------------------------------------------------+----------+ -+-----------------------------------------------------------+----------+ -10 rows in set (15 min 29.45 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1067 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1069 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -+-----------------------------------------------------------+----------+ -+-----------------------------------------------------------+----------+ -+-----------------------------------------------------------+----------+ -10 rows in set (14 min 29.74 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1070 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1072 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -+-----------------------------------------------------------+----------+ -+-----------------------------------------------------------+----------+ -+-----------------------------------------------------------+----------+ -10 rows in set (12 min 19.53 sec) - -mysql> quit -Bye --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1073 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1075 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1076 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1078 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1079 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1081 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -quit --- агрегация чуть сложнее.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1082 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1084 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LI MIT 10; -quit -+----------------+-----------------------------------------------------------+----------+ -+----------------+-----------------------------------------------------------+----------+ -+----------------+-----------------------------------------------------------+----------+ -10 rows in set (19 min 56.42 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1085 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1087 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LI MIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1088 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1090 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LI MIT 10; -quit --- агрегация по числу и строке, большое количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1091 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1093 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1094 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1096 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1097 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1099 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -quit --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1100 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1102 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1103 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1105 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1108 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1110 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit --- ещё более сложная агрегация.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1111 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1113 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1114 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1116 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1117 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1119 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -quit --- то же самое, но без сортировки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1120 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1122 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1123 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1125 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1126 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1128 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1129 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1131 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -Empty set (7 min 27.72 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1132 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1134 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -Empty set (3 min 19.09 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1135 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1137 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -Empty set (3 min 5.82 sec) - -mysql> quit -Bye --- мощная фильтрация по столбцу типа UInt64.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1138 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1140 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (9 min 42.73 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1141 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1143 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (3 min 16.73 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1144 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1146 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (3 min 15.80 sec) - -mysql> quit -Bye --- фильтрация по поиску подстроки в строке.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1147 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1149 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (9 min 42.54 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1150 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1152 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (3 min 25.89 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1153 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1155 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (3 min 19.15 sec) - -mysql> quit -Bye --- вынимаем большие столбцы, фильтрация по строке.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1156 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1158 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yande x.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------------------------------------------+-----------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+------------------------+ -+---------------------------------------------------------+-----------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+------------------------+ -+---------------------------------------------------------+-----------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+------------------------+ -10 rows in set (9 min 28.75 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1159 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1161 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yande x.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------------------------------------------+-----------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+------------------------+ -+---------------------------------------------------------+-----------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+------------------------+ -+---------------------------------------------------------+-----------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+------------------------+ -10 rows in set (3 min 37.15 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1162 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1164 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yande x.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------------------------------------------+-----------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+------------------------+ -+---------------------------------------------------------+-----------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+------------------------+ -+---------------------------------------------------------+-----------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+------------------------+ -10 rows in set (3 min 29.65 sec) - -mysql> quit -Bye --- чуть больше столбцы.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1165 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1167 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -10 rows in set (9 min 22.72 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1168 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1170 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -10 rows in set (3 min 26.77 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1171 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1173 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -10 rows in set (3 min 23.19 sec) - -mysql> quit -Bye --- плохой запрос - вынимаем все столбцы.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1174 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1176 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -quit -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -10 rows in set (10 min 2.47 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1177 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1179 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -10 rows in set (3 min 6.80 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1180 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1182 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -10 rows in set (3 min 6.62 sec) - -mysql> quit -Bye --- большая сортировка.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1183 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1185 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+----------------------------------------------------------------------------------------------------------------------------------+ -+----------------------------------------------------------------------------------------------------------------------------------+ -+----------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (9 min 25.26 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1186 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1188 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+----------------------------------------------------------------------------------------------------------------------------------+ -+----------------------------------------------------------------------------------------------------------------------------------+ -+----------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (3 min 19.44 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1189 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1191 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+----------------------------------------------------------------------------------------------------------------------------------+ -+----------------------------------------------------------------------------------------------------------------------------------+ -+----------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (3 min 19.24 sec) - -mysql> quit -Bye --- большая сортировка по строкам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1192 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1194 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -quit -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -10 rows in set (10 min 57.78 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1195 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1197 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -10 rows in set (3 min 22.53 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1198 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1200 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------+ -10 rows in set (3 min 16.95 sec) - -mysql> quit -Bye --- большая сортировка по кортежу.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1201 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1203 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -quit -+-----------+----------+----------+ -+-----------+----------+----------+ -+-----------+----------+----------+ -25 rows in set (11 min 15.84 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1204 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1206 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -+-----------+----------+----------+ -+-----------+----------+----------+ -+-----------+----------+----------+ -25 rows in set (4 min 10.11 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1207 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1209 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -+-----------+----------+----------+ -+-----------+----------+----------+ -+-----------+----------+----------+ -25 rows in set (4 min 8.90 sec) - -mysql> quit -Bye --- считаем средние длины URL для крупных счётчиков.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1210 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1212 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1213 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1215 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1216 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1218 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -quit --- то же самое, но с разбивкой по доменам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1219 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1221 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1222 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1224 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -quit -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (17 min 41.89 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1225 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1227 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -quit -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (17 min 34.60 sec) - -mysql> quit -Bye --- много тупых агрегатных функций.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1228 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1230 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, C lientIP ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1231 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1233 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, C lientIP ORDER BY count(*) DESC LIMIT 10; -quit -+----------------+------------+------+--------------+----------------------+ -+----------------+------------+------+--------------+----------------------+ -+----------------+------------+------+--------------+----------------------+ -10 rows in set (16 min 33.89 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1234 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1236 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, C lientIP ORDER BY count(*) DESC LIMIT 10; -quit -+----------------+------------+------+--------------+----------------------+ -+----------------+------------+------+--------------+----------------------+ -+----------------+------------+------+--------------+----------------------+ -10 rows in set (15 min 18.67 sec) - -mysql> quit -Bye --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1237 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1239 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1240 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1242 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -quit -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (10 min 4.48 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1243 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1245 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 rows in set (9 min 13.33 sec) - -mysql> quit -Bye --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1246 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1248 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10 ; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1249 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1251 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10 ; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1252 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1254 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10 ; -quit --- то же самое, но ещё и без фильтрации.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1255 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1257 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1258 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1260 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1261 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1263 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit --- агрегация по URL.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1264 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1266 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1267 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1269 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1270 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1272 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit --- агрегация по URL и числу.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1273 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1275 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_100m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1276 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1278 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_100m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1279 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1281 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_100m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1282 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1284 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1285 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1287 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1288 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1290 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1291 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1293 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1294 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1296 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1297 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1299 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1300 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1302 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND N OT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1303 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1305 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND N OT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1306 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1308 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND N OT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -quit -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -10 rows in set (14 min 33.29 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1309 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1311 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1312 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1314 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1315 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1317 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1318 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1320 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1321 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1323 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1324 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1326 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1329 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1331 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013- 07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1332 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1334 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013- 07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1335 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1337 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013- 07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1338 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1340 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND Event Date <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DES C LIMIT 10000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1341 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1343 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND Event Date <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DES C LIMIT 10000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1344 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1346 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND Event Date <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DES C LIMIT 10000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1347 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1349 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013 -07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -Empty set (0.67 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1350 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1352 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013 -07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -Empty set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1353 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 1355 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013 -07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -Empty set (0.00 sec) - -mysql> quit -Bye -stop time: Вс. сент. 22 13:02:57 MSK 2013 diff --git a/benchmark/infinidb/log/log_100m_tuned b/benchmark/infinidb/log/log_100m_tuned deleted file mode 100644 index ce0cffcc6e1..00000000000 --- a/benchmark/infinidb/log/log_100m_tuned +++ /dev/null @@ -1,5383 +0,0 @@ -start time: Вт. сент. 24 01:22:51 MSK 2013 -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 436 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 438 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m; -+-----------+ -+-----------+ -+-----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 439 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 441 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m; -+-----------+ -+-----------+ -+-----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 442 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 444 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m; -+-----------+ -+-----------+ -+-----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 445 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 447 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (4 min 9.62 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 448 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 450 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (3 min 19.97 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 451 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 453 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (3 min 17.75 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 454 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 456 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -1 row in set (4 min 19.81 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 457 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 459 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -1 row in set (3 min 25.92 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 460 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 462 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -+------------------+-----------+----------------------+ -1 row in set (3 min 29.53 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 463 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 465 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -+-----------------------------+ -+-----------------------------+ -+-----------------------------+ -1 row in set (4 min 16.53 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 466 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 468 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -+-----------------------------+ -+-----------------------------+ -+-----------------------------+ -1 row in set (3 min 20.29 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 469 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 471 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_100m; -+-----------------------------+ -+-----------------------------+ -+-----------------------------+ -1 row in set (3 min 18.65 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 472 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 474 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (5 min 22.25 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 475 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 477 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (4 min 21.56 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 478 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 480 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_100m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (4 min 31.99 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 481 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 483 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (5 min 56.15 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 484 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 486 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (5 min 1.56 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 487 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 489 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_100m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (4 min 38.27 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 490 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 492 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (1 min 16.32 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 493 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 495 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (1 min 8.47 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 496 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 498 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_100m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (1 min 8.16 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 499 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 501 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -18 rows in set (4 min 9.69 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 502 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 504 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -18 rows in set (3 min 14.80 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 505 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 507 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -18 rows in set (3 min 17.54 sec) - -mysql> quit -Bye --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 508 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 510 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+---------+ -+----------+---------+ -+----------+---------+ -10 rows in set (5 min 25.68 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 511 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 513 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+---------+ -+----------+---------+ -+----------+---------+ -10 rows in set (4 min 30.04 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 514 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 516 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+---------+ -+----------+---------+ -+----------+---------+ -10 rows in set (4 min 26.70 sec) - -mysql> quit -Bye --- агрегация, среднее количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 517 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 519 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DES C LIMIT 10; -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -10 rows in set (5 min 49.32 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 520 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 522 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DES C LIMIT 10; -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -10 rows in set (4 min 52.83 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 523 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 525 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DES C LIMIT 10; -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -+----------+------------------+----------+----------------------+------------------------+ -10 rows in set (4 min 40.82 sec) - -mysql> quit -Bye --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 526 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 528 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+--------+ -+------------------+--------+ -+------------------+--------+ -10 rows in set (4 min 31.19 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 529 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 531 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+--------+ -+------------------+--------+ -+------------------+--------+ -10 rows in set (3 min 30.03 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 532 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 534 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+--------+ -+------------------+--------+ -+------------------+--------+ -10 rows in set (3 min 28.72 sec) - -mysql> quit -Bye --- мощная фильтрация по строкам, затем агрегация по строкам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 535 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 537 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -+-------------+------------------+--------+ -+-------------+------------------+--------+ -+-------------+------------------+--------+ -10 rows in set (4 min 29.82 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 538 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 540 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -+-------------+------------------+--------+ -+-------------+------------------+--------+ -+-------------+------------------+--------+ -10 rows in set (3 min 32.21 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 541 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 543 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -+-------------+------------------+--------+ -+-------------+------------------+--------+ -+-------------+------------------+--------+ -10 rows in set (3 min 30.17 sec) - -mysql> quit -Bye --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 544 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 546 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -+-----------------------------------------------------------------------------------------------------------------+----------+ -+-----------------------------------------------------------------------------------------------------------------+----------+ -+-----------------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (17 min 7.93 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 547 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 549 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -+-----------------------------------------------------------------------------------------------------------------+----------+ -+-----------------------------------------------------------------------------------------------------------------+----------+ -+-----------------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (13 min 50.35 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 550 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 552 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -+-----------------------------------------------------------------------------------------------------------------+----------+ -+-----------------------------------------------------------------------------------------------------------------+----------+ -+-----------------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (12 min 2.12 sec) - -mysql> quit -Bye --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 553 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 555 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 556 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 558 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 559 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 561 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -quit --- агрегация чуть сложнее.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 562 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 564 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LI MIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 565 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 567 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LI MIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 568 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 570 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LI MIT 10; -quit --- агрегация по числу и строке, большое количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 571 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 573 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 574 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 576 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 577 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 579 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -quit --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 580 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 582 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 585 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 587 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 588 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 590 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit --- ещё более сложная агрегация.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 591 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 593 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 594 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 596 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 597 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 599 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -quit --- то же самое, но без сортировки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 600 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 602 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 603 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 605 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 606 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 608 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 609 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 611 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -quit -Empty set (12 min 50.82 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 612 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 614 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -Empty set (6 min 15.34 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 615 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 617 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_100m WHERE UserID = 12345678901234567890; -quit -Empty set (10 min 15.52 sec) - -mysql> quit -Bye --- мощная фильтрация по столбцу типа UInt64.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 618 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 620 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -quit -+----------+ -+----------+ -+----------+ -1 row in set (15 min 5.81 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 621 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 623 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (8 min 56.63 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 624 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 626 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -quit -+----------+ -+----------+ -+----------+ -1 row in set (16 min 21.78 sec) - -mysql> quit -Bye --- фильтрация по поиску подстроки в строке.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 627 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 629 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -quit -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (15 min 23.48 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 630 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 632 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (3 min 46.57 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 633 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 635 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (3 min 47.04 sec) - -mysql> quit -Bye --- вынимаем большие столбцы, фильтрация по строке.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 636 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 638 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yande x.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -ERROR 1267 (HY000): Illegal mix of collations (latin1_swedish_ci,IMPLICIT) and (utf8_general_ci,COERCIBLE) for operation 'like' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 639 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 641 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yande x.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -ERROR 1267 (HY000): Illegal mix of collations (latin1_swedish_ci,IMPLICIT) and (utf8_general_ci,COERCIBLE) for operation 'like' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 642 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 644 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yande x.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -ERROR 1267 (HY000): Illegal mix of collations (latin1_swedish_ci,IMPLICIT) and (utf8_general_ci,COERCIBLE) for operation 'like' -mysql> quit -Bye --- чуть больше столбцы.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 645 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 647 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -quit -+---------------------+------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -10 rows in set (14 min 40.39 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 648 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 650 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -10 rows in set (3 min 28.27 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 651 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 653 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+---------------------+------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -10 rows in set (3 min 25.72 sec) - -mysql> quit -Bye --- плохой запрос - вынимаем все столбцы.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 654 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 656 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -quit -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (14 min 48.76 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 657 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 659 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (3 min 36.37 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 660 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 662 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (3 min 28.34 sec) - -mysql> quit -Bye --- большая сортировка.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 663 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 665 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -quit -+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (16 min 11.61 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 666 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 668 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (3 min 44.06 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 669 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 671 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (3 min 44.03 sec) - -mysql> quit -Bye --- большая сортировка по строкам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 672 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 674 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -quit -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (16 min 15.34 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 675 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 677 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (3 min 36.92 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 678 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 680 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_100m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (3 min 34.24 sec) - -mysql> quit -Bye --- большая сортировка по кортежу.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 681 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 683 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -quit -+-----------+----------+----------+ -+-----------+----------+----------+ -+-----------+----------+----------+ -25 rows in set (17 min 7.30 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 684 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 686 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -+-----------+----------+----------+ -+-----------+----------+----------+ -+-----------+----------+----------+ -25 rows in set (4 min 25.74 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 687 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 689 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -+-----------+----------+----------+ -+-----------+----------+----------+ -+-----------+----------+----------+ -25 rows in set (4 min 38.87 sec) - -mysql> quit -Bye --- считаем средние длины URL для крупных счётчиков.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 690 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 692 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 693 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 695 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 696 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 698 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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_100m WHERE Referer != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25 ; -quit --- то же самое, но с разбивкой по доменам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 699 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 701 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 702 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 704 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -quit -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (19 min 5.15 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 705 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 707 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -quit -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (18 min 24.86 sec) - -mysql> quit -Bye --- много тупых агрегатных функций.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 708 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 710 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, C lientIP ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 711 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 713 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, C lientIP ORDER BY count(*) DESC LIMIT 10; -quit -+----------------+------------+------+--------------+----------------------+ -+----------------+------------+------+--------------+----------------------+ -+----------------+------------+------+--------------+----------------------+ -10 rows in set (11 min 4.21 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 714 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 716 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY SearchEngineID, C lientIP ORDER BY count(*) DESC LIMIT 10; -quit -+----------------+------------+------+--------------+----------------------+ -+----------------+------------+------+--------------+----------------------+ -+----------------+------------+------+--------------+----------------------+ -10 rows in set (11 min 41.50 sec) - -mysql> quit -Bye --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 717 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 719 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -quit -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (18 min 40.79 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 720 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 722 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 rows in set (6 min 13.66 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 723 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 725 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 rows in set (6 min 0.99 sec) - -mysql> quit -Bye --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 726 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 728 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10 ; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 729 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 731 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10 ; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 732 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 734 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10 ; -quit --- то же самое, но ещё и без фильтрации.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 735 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 737 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 738 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 740 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 741 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 743 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit --- агрегация по URL.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 744 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 746 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 747 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 749 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 750 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 752 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit --- агрегация по URL и числу.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 753 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 755 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_100m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 756 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 758 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_100m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 759 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 761 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_100m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 762 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 764 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 765 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 767 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 768 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 770 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 771 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 773 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 774 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 776 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 777 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 779 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 780 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 782 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND N OT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 783 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 785 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND N OT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 786 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 788 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND N OT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 789 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 791 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 792 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 794 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 795 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 797 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 800 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 802 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 803 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 805 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 806 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 808 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 809 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 811 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013- 07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 812 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 814 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013- 07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 815 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 817 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013- 07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 818 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 820 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND Event Date <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DES C LIMIT 10000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 821 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 823 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND Event Date <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DES C LIMIT 10000; -ERROR 1317 (70100): Query execution was interrupted -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql stop/killed, process 22087 -spawn mysql -u root -ERROR 2002 (HY000): Can't connect to local MySQL server through socket '/var/run/mysqld/mysqld.sock' (2) -restart server: /etc/init.d/mysql restart -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql restart - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the stop(8) and then start(8) utilities, -e.g. stop mysql ; start mysql. The restart(8) utility is also available. -waiting diff --git a/benchmark/infinidb/log/log_10m b/benchmark/infinidb/log/log_10m deleted file mode 100644 index f35f2460898..00000000000 --- a/benchmark/infinidb/log/log_10m +++ /dev/null @@ -1,5818 +0,0 @@ -start time: Пт. сент. 20 20:04:14 MSK 2013 -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 560 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 562 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m; -+----------+ -+----------+ -+----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 563 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 565 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m; -+----------+ -+----------+ -+----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 566 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 568 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m; -+----------+ -+----------+ -+----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 569 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 571 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (21.55 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 572 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 574 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (18.91 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 575 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 577 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (18.28 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 578 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 580 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -1 row in set (22.71 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 581 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 583 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -1 row in set (19.86 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 584 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 586 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -1 row in set (20.53 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 587 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 589 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -+----------------------------+ -+----------------------------+ -+----------------------------+ -1 row in set (21.30 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 590 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 592 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -+----------------------------+ -+----------------------------+ -+----------------------------+ -1 row in set (18.93 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 593 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 595 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -+----------------------------+ -+----------------------------+ -+----------------------------+ -1 row in set (19.25 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 596 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 598 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (26.77 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 599 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 601 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (25.74 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 602 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 604 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (25.65 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 605 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 607 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (29.14 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 608 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 610 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (26.92 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 611 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 613 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (26.53 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 614 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 616 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (7.47 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 617 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 619 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (7.38 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 620 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 622 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (7.23 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 623 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 625 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -9 rows in set (20.56 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 626 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 628 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -9 rows in set (18.84 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 629 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 631 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -9 rows in set (18.41 sec) - -mysql> quit -Bye --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 632 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 634 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+----------+--------+ -+----------+--------+ -10 rows in set (27.53 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 635 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 637 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+----------+--------+ -+----------+--------+ -10 rows in set (25.14 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 638 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 640 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+----------+--------+ -+----------+--------+ -10 rows in set (24.45 sec) - -mysql> quit -Bye --- агрегация, среднее количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 641 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 643 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -10 rows in set (30.08 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 644 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 646 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -10 rows in set (26.07 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 647 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 649 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -10 rows in set (26.75 sec) - -mysql> quit -Bye --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 650 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 652 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+------------------+-------+ -+------------------+-------+ -10 rows in set (22.93 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 653 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 655 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+------------------+-------+ -+------------------+-------+ -10 rows in set (19.82 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 656 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 658 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+------------------+-------+ -+------------------+-------+ -10 rows in set (20.23 sec) - -mysql> quit -Bye --- мощная фильтрация по строкам, затем агрегация по строкам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 659 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 661 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel O RDER BY u DESC LIMIT 10; -+-------------+------------------+-------+ -+-------------+------------------+-------+ -+-------------+------------------+-------+ -10 rows in set (21.79 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 662 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 664 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel O RDER BY u DESC LIMIT 10; -+-------------+------------------+-------+ -+-------------+------------------+-------+ -+-------------+------------------+-------+ -10 rows in set (19.75 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 665 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 667 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel O RDER BY u DESC LIMIT 10; -+-------------+------------------+-------+ -+-------------+------------------+-------+ -+-------------+------------------+-------+ -10 rows in set (19.64 sec) - -mysql> quit -Bye --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 668 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 670 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------------+----------+ -+-----------------------------+----------+ -+-----------------------------+----------+ -10 rows in set (51.30 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 671 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 673 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------------+----------+ -+-----------------------------+----------+ -+-----------------------------+----------+ -10 rows in set (42.27 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 674 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 676 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------------+----------+ -+-----------------------------+----------+ -+-----------------------------+----------+ -10 rows in set (46.45 sec) - -mysql> quit -Bye --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 677 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 679 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-------------------------------------------+------+ -+-------------------------------------------+------+ -+-------------------------------------------+------+ -10 rows in set (43.75 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 680 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 682 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-------------------------------------------+------+ -+-------------------------------------------+------+ -+-------------------------------------------+------+ -10 rows in set (42.38 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 683 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 685 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-------------------------------------------+------+ -+-------------------------------------------+------+ -+-------------------------------------------+------+ -10 rows in set (42.36 sec) - -mysql> quit -Bye --- агрегация чуть сложнее.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 686 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 688 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIM IT 10; -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -10 rows in set (43.76 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 689 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 691 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIM IT 10; -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -10 rows in set (48.66 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 692 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 694 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIM IT 10; -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -10 rows in set (46.80 sec) - -mysql> quit -Bye --- агрегация по числу и строке, большое количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 695 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 697 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+----------+ -+---------------------+----------+ -+---------------------+----------+ -10 rows in set (1 min 46.76 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 698 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 700 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+----------+ -+---------------------+----------+ -+---------------------+----------+ -10 rows in set (1 min 46.53 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 701 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 703 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+----------+ -+---------------------+----------+ -+---------------------+----------+ -10 rows in set (1 min 45.32 sec) - -mysql> quit -Bye --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 704 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 706 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -10 rows in set (2 min 52.51 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 707 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 709 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -10 rows in set (3 min 1.59 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 710 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 712 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -10 rows in set (2 min 57.70 sec) - -mysql> quit -Bye --- ещё более сложная агрегация.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 713 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 715 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+----------------------+---------------------------------------------------------------+----------+ -+----------------------+---------------------------------------------------------------+----------+ -+----------------------+---------------------------------------------------------------+----------+ -10 rows in set (3 min 9.92 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 716 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 718 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+----------------------+---------------------------------------------------------------+----------+ -+----------------------+---------------------------------------------------------------+----------+ -+----------------------+---------------------------------------------------------------+----------+ -10 rows in set (2 min 46.09 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 719 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 721 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+----------------------+---------------------------------------------------------------+----------+ -+----------------------+---------------------------------------------------------------+----------+ -+----------------------+---------------------------------------------------------------+----------+ -10 rows in set (2 min 52.03 sec) - -mysql> quit -Bye --- то же самое, но без сортировки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 722 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 724 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -10 rows in set (3 min 5.61 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 725 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 727 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -10 rows in set (3 min 1.38 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 728 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 730 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -10 rows in set (3 min 26.92 sec) - -mysql> quit -Bye --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 731 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 733 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -Empty set (20.30 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 734 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 736 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -Empty set (19.24 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 737 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 739 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -Empty set (18.49 sec) - -mysql> quit -Bye --- мощная фильтрация по столбцу типа UInt64.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 740 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 742 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (21.43 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 743 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 745 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (20.03 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 746 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 748 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (19.99 sec) - -mysql> quit -Bye --- фильтрация по поиску подстроки в строке.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 749 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 751 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -10 rows in set (21.88 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 752 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 754 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -10 rows in set (20.06 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 755 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 757 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -10 rows in set (20.30 sec) - -mysql> quit -Bye --- вынимаем большие столбцы, фильтрация по строке.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 758 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 760 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex .%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+---------------------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+---------------------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -10 rows in set (25.51 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 761 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 763 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex .%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+---------------------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+---------------------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -10 rows in set (20.72 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 764 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 766 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex .%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+---------------------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+---------------------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -10 rows in set (20.58 sec) - -mysql> quit -Bye --- чуть больше столбцы.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 767 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 769 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -10 rows in set (24.14 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 770 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 772 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -10 rows in set (20.04 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 773 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 775 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -10 rows in set (19.29 sec) - -mysql> quit -Bye --- плохой запрос - вынимаем все столбцы.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 776 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 778 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (21.65 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 779 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 781 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (19.11 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 782 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 784 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (19.00 sec) - -mysql> quit -Bye --- большая сортировка.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 785 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 787 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (22.44 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 788 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 790 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (20.35 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 791 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 793 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (20.02 sec) - -mysql> quit -Bye --- большая сортировка по строкам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 794 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 796 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (21.41 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 797 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 799 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (19.96 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 800 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 802 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (19.91 sec) - -mysql> quit -Bye --- большая сортировка по кортежу.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 803 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 805 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 rows in set (27.15 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 806 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 808 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 rows in set (26.28 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 809 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 811 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 rows in set (25.32 sec) - -mysql> quit -Bye --- считаем средние длины URL для крупных счётчиков.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 812 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 814 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; ; -+------------------+----------+---------+-----------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+-----------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+-----------------------------------------------------------------------------------------------------------------------------+ -11 rows in set (2 min 15.45 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 815 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 817 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; ; -+------------------+----------+---------+-----------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+-----------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+-----------------------------------------------------------------------------------------------------------------------------+ -11 rows in set (2 min 14.31 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 818 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 820 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; ; -+------------------+----------+---------+-----------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+-----------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+-----------------------------------------------------------------------------------------------------------------------------+ -11 rows in set (2 min 13.12 sec) - -mysql> quit -Bye --- то же самое, но с разбивкой по доменам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 821 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 823 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (1 min 47.44 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 824 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 826 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (1 min 46.26 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 827 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 829 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (1 min 46.16 sec) - -mysql> quit -Bye --- много тупых агрегатных функций.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 830 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 832 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, Cl ientIP ORDER BY count(*) DESC LIMIT 10; -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -10 rows in set (40.47 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 833 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 835 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, Cl ientIP ORDER BY count(*) DESC LIMIT 10; -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -10 rows in set (36.67 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 836 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 838 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, Cl ientIP ORDER BY count(*) DESC LIMIT 10; -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -10 rows in set (37.07 sec) - -mysql> quit -Bye --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 839 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 841 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER B Y count(*) DESC LIMIT 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (39.20 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 842 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 844 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER B Y count(*) DESC LIMIT 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (36.86 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 845 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 847 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER B Y count(*) DESC LIMIT 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (37.22 sec) - -mysql> quit -Bye --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 848 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 850 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; ; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (2 min 14.50 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 851 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 853 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; ; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (2 min 10.75 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 854 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 856 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; ; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (2 min 13.55 sec) - -mysql> quit -Bye --- то же самое, но ещё и без фильтрации.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 857 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 859 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -10 rows in set (17 min 37.45 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 860 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 862 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -10 rows in set (17 min 55.29 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 863 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 865 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -10 rows in set (15 min 28.38 sec) - -mysql> quit -Bye --- агрегация по URL.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 866 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 868 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -10 rows in set (14 min 27.64 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 869 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 871 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -10 rows in set (17 min 3.33 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 872 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 874 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -10 rows in set (17 min 43.40 sec) - -mysql> quit -Bye --- агрегация по URL и числу.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 875 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 877 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER B Y count(*) DESC LIMIT 10; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_10m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 878 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 880 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER B Y count(*) DESC LIMIT 10; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_10m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 881 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 883 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER B Y count(*) DESC LIMIT 10; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_10m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 884 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 886 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 887 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 889 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 890 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 892 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 893 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 895 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -quit -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -10 rows in set (19 min 20.03 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 896 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 898 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -10 rows in set (23.32 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 899 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 901 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -10 rows in set (23.08 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 902 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 904 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NO T DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -quit -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -10 rows in set (18 min 29.20 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 905 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 907 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NO T DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -10 rows in set (14.24 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 908 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 910 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NO T DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -10 rows in set (14.38 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 911 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 913 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -quit -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (18 min 6.92 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 914 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 916 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (7.29 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 917 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 919 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (6.63 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 920 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 922 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, S earchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -quit -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 923 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 925 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, S earchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (31.74 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 926 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 928 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, S earchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (31.79 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 929 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 931 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-0 7-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -quit -Empty set (17 min 54.29 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 932 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 934 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-0 7-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -Empty set (5.51 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 935 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 937 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-0 7-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -Empty set (5.43 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 938 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 940 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventD ate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -quit -Empty set (17 min 49.16 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 941 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 943 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventD ate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Empty set (5.60 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 944 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 946 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventD ate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Empty set (5.30 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 947 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 949 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013- 07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -quit -+---------------------+-----------+ -+---------------------+-----------+ -+---------------------+-----------+ -652 rows in set (10 min 52.84 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 950 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 952 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013- 07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -+---------------------+-----------+ -+---------------------+-----------+ -+---------------------+-----------+ -652 rows in set (4.84 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 19577 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 953 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 955 -Server version: 5.5.32-0ubuntu0.12.04.1 (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013- 07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -+---------------------+-----------+ -+---------------------+-----------+ -+---------------------+-----------+ -652 rows in set (4.57 sec) - -mysql> quit -Bye -stop time: Сб. сент. 21 01:16:53 MSK 2013 diff --git a/benchmark/infinidb/log/log_10m_tuned b/benchmark/infinidb/log/log_10m_tuned deleted file mode 100644 index 7a3685446eb..00000000000 --- a/benchmark/infinidb/log/log_10m_tuned +++ /dev/null @@ -1,5812 +0,0 @@ -start time: Пн. сент. 23 21:13:28 MSK 2013 -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 40 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 42 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m; -+----------+ -+----------+ -+----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 43 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 45 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m; -+----------+ -+----------+ -+----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 46 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 48 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m; -+----------+ -+----------+ -+----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 49 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 51 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (24.67 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 52 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 54 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (19.89 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 55 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 57 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (19.40 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 58 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 60 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -1 row in set (28.22 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 61 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 63 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -1 row in set (21.02 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 64 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 66 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -1 row in set (20.72 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 67 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 69 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -+----------------------------+ -+----------------------------+ -+----------------------------+ -1 row in set (28.05 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 70 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 72 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -+----------------------------+ -+----------------------------+ -+----------------------------+ -1 row in set (20.23 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 73 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 75 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -+----------------------------+ -+----------------------------+ -+----------------------------+ -1 row in set (20.17 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 76 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 78 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (31.56 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 79 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 81 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (25.77 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 82 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 84 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (25.94 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 85 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 87 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (36.92 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 88 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 90 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (27.81 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 91 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 93 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (27.69 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 94 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 96 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (7.44 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 97 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 99 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (6.31 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 100 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 102 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (6.45 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 103 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 105 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -9 rows in set (24.57 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 106 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 108 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -9 rows in set (19.90 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 109 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 111 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -9 rows in set (19.73 sec) - -mysql> quit -Bye --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 112 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 114 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+----------+--------+ -+----------+--------+ -10 rows in set (31.77 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 115 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 117 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+----------+--------+ -+----------+--------+ -10 rows in set (25.16 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 118 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 120 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+----------+--------+ -+----------+--------+ -10 rows in set (25.15 sec) - -mysql> quit -Bye --- агрегация, среднее количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 121 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 123 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -10 rows in set (32.71 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 124 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 126 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -10 rows in set (27.17 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 127 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 129 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -+----------+------------------+---------+----------------------+------------------------+ -10 rows in set (27.17 sec) - -mysql> quit -Bye --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 130 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 132 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+------------------+-------+ -+------------------+-------+ -10 rows in set (28.13 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 133 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 135 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+------------------+-------+ -+------------------+-------+ -10 rows in set (21.12 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 136 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 138 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+------------------+-------+ -+------------------+-------+ -10 rows in set (20.82 sec) - -mysql> quit -Bye --- мощная фильтрация по строкам, затем агрегация по строкам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 139 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 141 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel O RDER BY u DESC LIMIT 10; -+-------------+------------------+-------+ -+-------------+------------------+-------+ -+-------------+------------------+-------+ -10 rows in set (28.32 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 142 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 144 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel O RDER BY u DESC LIMIT 10; -+-------------+------------------+-------+ -+-------------+------------------+-------+ -+-------------+------------------+-------+ -10 rows in set (20.89 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 145 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 147 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel O RDER BY u DESC LIMIT 10; -+-------------+------------------+-------+ -+-------------+------------------+-------+ -+-------------+------------------+-------+ -10 rows in set (20.75 sec) - -mysql> quit -Bye --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 148 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 150 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------------------------------------------+----------+ -+---------------------------------------------------------+----------+ -+---------------------------------------------------------+----------+ -10 rows in set (49.31 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 151 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 153 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------------------------------------------+----------+ -+---------------------------------------------------------+----------+ -+---------------------------------------------------------+----------+ -10 rows in set (46.13 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 154 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 156 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------------------------------------------+----------+ -+---------------------------------------------------------+----------+ -+---------------------------------------------------------+----------+ -10 rows in set (47.95 sec) - -mysql> quit -Bye --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 157 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 159 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+----------------------------------------------------------------------------------------+------+ -+----------------------------------------------------------------------------------------+------+ -+----------------------------------------------------------------------------------------+------+ -10 rows in set (1 min 4.20 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 160 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 162 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+----------------------------------------------------------------------------------------+------+ -+----------------------------------------------------------------------------------------+------+ -+----------------------------------------------------------------------------------------+------+ -10 rows in set (42.81 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 163 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 165 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+----------------------------------------------------------------------------------------+------+ -+----------------------------------------------------------------------------------------+------+ -+----------------------------------------------------------------------------------------+------+ -10 rows in set (43.65 sec) - -mysql> quit -Bye --- агрегация чуть сложнее.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 166 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 168 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIM IT 10; -+----------------+---------------------------------------------------------+----------+ -+----------------+---------------------------------------------------------+----------+ -+----------------+---------------------------------------------------------+----------+ -10 rows in set (48.47 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 169 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 171 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIM IT 10; -+----------------+---------------------------------------------------------+----------+ -+----------------+---------------------------------------------------------+----------+ -+----------------+---------------------------------------------------------+----------+ -10 rows in set (48.79 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 172 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 174 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIM IT 10; -+----------------+---------------------------------------------------------+----------+ -+----------------+---------------------------------------------------------+----------+ -+----------------+---------------------------------------------------------+----------+ -10 rows in set (43.92 sec) - -mysql> quit -Bye --- агрегация по числу и строке, большое количество ключей.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 175 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 177 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+----------+ -+---------------------+----------+ -+---------------------+----------+ -10 rows in set (1 min 37.97 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 178 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 180 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+----------+ -+---------------------+----------+ -+---------------------+----------+ -10 rows in set (1 min 29.48 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 181 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 183 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+----------+ -+---------------------+----------+ -+---------------------+----------+ -10 rows in set (1 min 29.29 sec) - -mysql> quit -Bye --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 184 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 186 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -10 rows in set (2 min 51.31 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 187 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 189 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -10 rows in set (2 min 49.48 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 190 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 192 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -10 rows in set (2 min 49.25 sec) - -mysql> quit -Bye --- ещё более сложная агрегация.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 193 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 195 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+----------------------+------------------------------------------------------------------------------------------------------------------------------+----------+ -+----------------------+------------------------------------------------------------------------------------------------------------------------------+----------+ -+----------------------+------------------------------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (2 min 51.33 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 196 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 198 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+----------------------+------------------------------------------------------------------------------------------------------------------------------+----------+ -+----------------------+------------------------------------------------------------------------------------------------------------------------------+----------+ -+----------------------+------------------------------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (2 min 30.93 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 199 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 201 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+----------------------+------------------------------------------------------------------------------------------------------------------------------+----------+ -+----------------------+------------------------------------------------------------------------------------------------------------------------------+----------+ -+----------------------+------------------------------------------------------------------------------------------------------------------------------+----------+ -10 rows in set (2 min 36.59 sec) - -mysql> quit -Bye --- то же самое, но без сортировки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 202 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 204 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -10 rows in set (3 min 20.41 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 205 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 207 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -10 rows in set (2 min 52.73 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 208 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 210 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -10 rows in set (2 min 42.49 sec) - -mysql> quit -Bye --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 211 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 213 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -Empty set (28.61 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 214 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 216 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -Empty set (19.75 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 217 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 219 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; -Empty set (19.64 sec) - -mysql> quit -Bye --- мощная фильтрация по столбцу типа UInt64.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 220 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 222 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (26.78 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 223 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 225 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (20.81 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 226 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 228 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (20.72 sec) - -mysql> quit -Bye --- фильтрация по поиску подстроки в строке.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 229 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 231 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -10 rows in set (26.31 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 232 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 234 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -10 rows in set (21.54 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 235 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 237 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -10 rows in set (21.33 sec) - -mysql> quit -Bye --- вынимаем большие столбцы, фильтрация по строке.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 238 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 240 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex .%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -ERROR 1267 (HY000): Illegal mix of collations (latin1_swedish_ci,IMPLICIT) and (utf8_general_ci,COERCIBLE) for operation 'like' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 241 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 243 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex .%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -ERROR 1267 (HY000): Illegal mix of collations (latin1_swedish_ci,IMPLICIT) and (utf8_general_ci,COERCIBLE) for operation 'like' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 244 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 246 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex .%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -ERROR 1267 (HY000): Illegal mix of collations (latin1_swedish_ci,IMPLICIT) and (utf8_general_ci,COERCIBLE) for operation 'like' -mysql> quit -Bye --- чуть больше столбцы.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 247 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 249 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-----------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-----------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-----------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -10 rows in set (25.20 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 250 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 252 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-----------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-----------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-----------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -10 rows in set (20.57 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 253 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 255 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-----------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-----------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-----------------------------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -10 rows in set (20.37 sec) - -mysql> quit -Bye --- плохой запрос - вынимаем все столбцы.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 256 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 258 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (28.62 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 259 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 261 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (19.96 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 262 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 264 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (19.73 sec) - -mysql> quit -Bye --- большая сортировка.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 265 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 267 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (28.33 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 268 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 270 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (21.23 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 271 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 273 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (21.03 sec) - -mysql> quit -Bye --- большая сортировка по строкам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 274 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 276 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (25.94 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 277 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 279 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (21.08 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 280 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 282 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (20.88 sec) - -mysql> quit -Bye --- большая сортировка по кортежу.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 283 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 285 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 rows in set (31.29 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 286 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 288 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 rows in set (26.67 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 289 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 291 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 rows in set (26.13 sec) - -mysql> quit -Bye --- считаем средние длины URL для крупных счётчиков.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 292 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 294 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; ; -+------------------+----------+---------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -11 rows in set (2 min 28.78 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 295 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 297 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; ; -+------------------+----------+---------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -11 rows in set (2 min 16.61 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 298 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 300 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; ; -+------------------+----------+---------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -11 rows in set (2 min 17.29 sec) - -mysql> quit -Bye --- то же самое, но с разбивкой по доменам.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 301 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 303 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (1 min 59.84 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 304 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 306 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (1 min 48.96 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 307 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 309 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWi dth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), s um(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(Reso lutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionW idth + 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), su m(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(Resol utionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWi dth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 5 5), 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(Resolu tionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWid th + 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; -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (1 min 47.56 sec) - -mysql> quit -Bye --- много тупых агрегатных функций.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 310 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 312 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, Cl ientIP ORDER BY count(*) DESC LIMIT 10; -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -10 rows in set (44.77 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 313 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 315 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, Cl ientIP ORDER BY count(*) DESC LIMIT 10; -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -10 rows in set (36.73 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 316 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 318 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, Cl ientIP ORDER BY count(*) DESC LIMIT 10; -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -10 rows in set (37.20 sec) - -mysql> quit -Bye --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 319 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 321 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER B Y count(*) DESC LIMIT 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (45.03 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 322 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 324 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER B Y count(*) DESC LIMIT 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (37.40 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 325 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 327 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER B Y count(*) DESC LIMIT 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (37.66 sec) - -mysql> quit -Bye --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 328 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 330 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; ; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (2 min 19.49 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 331 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 333 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; ; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (2 min 7.79 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 334 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 336 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; ; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (2 min 6.97 sec) - -mysql> quit -Bye --- то же самое, но ещё и без фильтрации.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 337 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 339 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -10 rows in set (14 min 15.58 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 340 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 342 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -10 rows in set (15 min 30.25 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 343 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 345 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -quit -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -10 rows in set (16 min 21.86 sec) - -mysql> quit -Bye --- агрегация по URL.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 346 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 348 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -10 rows in set (13 min 39.56 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 349 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 351 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -10 rows in set (16 min 1.08 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 352 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 354 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -quit -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -10 rows in set (11 min 4.87 sec) - -mysql> quit -Bye --- агрегация по URL и числу.; - -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 355 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 357 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER B Y count(*) DESC LIMIT 10; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_10m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 358 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 360 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER B Y count(*) DESC LIMIT 10; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_10m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 361 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 363 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER B Y count(*) DESC LIMIT 10; -ERROR 1690 (22003): BIGINT UNSIGNED value is out of range in '(`hits`.`hits_10m`.`ClientIP` - 1)' -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 364 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 366 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 367 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 369 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 370 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 372 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 373 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 375 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -quit -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -10 rows in set (11 min 31.67 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 376 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 378 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -10 rows in set (24.08 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 379 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 381 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -10 rows in set (23.66 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 382 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 384 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NO T DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -quit -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -10 rows in set (10 min 55.00 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 385 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 387 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NO T DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -10 rows in set (14.56 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 388 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 390 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NO T DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -10 rows in set (14.07 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 391 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 393 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -quit -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (10 min 51.40 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 394 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 396 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (6.89 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 397 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 399 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (6.77 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 400 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 402 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, S earchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -quit -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (11 min 48.71 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 403 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 405 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, S earchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (32.55 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 406 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 408 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, co unt(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, S earchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (54.01 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 409 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 411 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-0 7-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -quit -Empty set (10 min 39.92 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 412 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 414 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-0 7-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -Empty set (5.32 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 415 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 417 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-0 7-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -Empty set (5.14 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 418 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 420 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventD ate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -quit -Empty set (10 min 39.47 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 421 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 423 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventD ate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Empty set (5.32 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 424 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 426 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventD ate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Empty set (5.09 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 427 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 1 -query: SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 429 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013- 07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -+---------------------+-----------+ -+---------------------+-----------+ -+---------------------+-----------+ -652 rows in set (6 min 40.66 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 430 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 2 -query: SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 432 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013- 07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -+---------------------+-----------+ -+---------------------+-----------+ -+---------------------+-----------+ -652 rows in set (5.15 sec) - -mysql> quit -Bye -Rather than invoking init scripts through /etc/init.d, use the service(8) -utility, e.g. service mysql status - -Since the script you are attempting to invoke has been converted to an -Upstart job, you may also use the status(8) utility, e.g. status mysql -mysql start/running, process 22087 -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 433 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> -mysql> quit -Bye - -times: 3 -query: SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql -u root -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 435 -Server version: 5.5.32-0ubuntu0.12.04.1-log (Ubuntu) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> use hits -Reading table information for completion of table and column names -You can turn off this feature to get a quicker startup with -A - -Database changed -mysql> SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013- 07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -+---------------------+-----------+ -+---------------------+-----------+ -+---------------------+-----------+ -652 rows in set (5.02 sec) - -mysql> quit -Bye -stop time: Вт. сент. 24 01:22:51 MSK 2013 diff --git a/benchmark/infinidb/queries.sql b/benchmark/infinidb/queries.sql deleted file mode 100644 index b4da23ac8c8..00000000000 --- a/benchmark/infinidb/queries.sql +++ /dev/null @@ -1,111 +0,0 @@ -SELECT SQL_NO_CACHE count(*) FROM hits_10m; -SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE AdvEngineID != 0; -SELECT SQL_NO_CACHE sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -SELECT SQL_NO_CACHE sum(UserID) FROM hits_10m; -SELECT SQL_NO_CACHE count(DISTINCT UserID) FROM hits_10m; -SELECT SQL_NO_CACHE count(DISTINCT SearchPhrase) FROM hits_10m; -SELECT SQL_NO_CACHE min(EventDate), max(EventDate) FROM hits_10m; - -SELECT SQL_NO_CACHE AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -SELECT SQL_NO_CACHE RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; --- агрегация, среднее количество ключей.; - -SELECT SQL_NO_CACHE RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -SELECT SQL_NO_CACHE MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; --- мощная фильтрация по строкам, затем агрегация по строкам.; - -SELECT SQL_NO_CACHE MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -SELECT SQL_NO_CACHE SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -SELECT SQL_NO_CACHE SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; --- агрегация чуть сложнее.; - -SELECT SQL_NO_CACHE SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- агрегация по числу и строке, большое количество ключей.; - -SELECT SQL_NO_CACHE UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- ещё более сложная агрегация.; - -SELECT SQL_NO_CACHE UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; --- то же самое, но без сортировки.; - -SELECT SQL_NO_CACHE UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -SELECT SQL_NO_CACHE UserID FROM hits_10m WHERE UserID = 12345678901234567890; --- мощная фильтрация по столбцу типа UInt64.; - -SELECT SQL_NO_CACHE count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; --- фильтрация по поиску подстроки в строке.; - -SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- вынимаем большие столбцы, фильтрация по строке.; - -SELECT SQL_NO_CACHE SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- чуть больше столбцы.; - -SELECT SQL_NO_CACHE * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; --- плохой запрос - вынимаем все столбцы.; - -SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; --- большая сортировка.; - -SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; --- большая сортировка по строкам.; - -SELECT SQL_NO_CACHE SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; --- большая сортировка по кортежу.; - -SELECT SQL_NO_CACHE 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 SQL_NO_CACHE 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; --- то же самое, но с разбивкой по доменам.; - -SELECT SQL_NO_CACHE 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 SQL_NO_CACHE 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 SQL_NO_CACHE 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 SQL_NO_CACHE WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; --- то же самое, но ещё и без фильтрации.; - -SELECT SQL_NO_CACHE URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; --- агрегация по URL.; - -SELECT SQL_NO_CACHE 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; --- агрегация по URL и числу.; - -SELECT SQL_NO_CACHE 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 SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; - - -SELECT SQL_NO_CACHE Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; - - -SELECT SQL_NO_CACHE URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; - - -SELECT SQL_NO_CACHE 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; - - -SELECT SQL_NO_CACHE URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 SQL_NO_CACHE WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; - -SELECT SQL_NO_CACHE EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; \ No newline at end of file diff --git a/benchmark/compatible/infobright/benchmark.sh b/benchmark/infobright/benchmark.sh similarity index 100% rename from benchmark/compatible/infobright/benchmark.sh rename to benchmark/infobright/benchmark.sh diff --git a/benchmark/infobright/conf.sh b/benchmark/infobright/conf.sh deleted file mode 100644 index 48cd341208e..00000000000 --- a/benchmark/infobright/conf.sh +++ /dev/null @@ -1,5 +0,0 @@ -CONF_DIR=/home/kartavyy/benchmark/infobright -expect_file=$CONF_DIR/expect.tcl -test_file=$CONF_DIR/queries.sql -etc_init_d_service=/etc/init.d/mysqld-ib - diff --git a/benchmark/compatible/infobright/create.sql b/benchmark/infobright/create.sql similarity index 100% rename from benchmark/compatible/infobright/create.sql rename to benchmark/infobright/create.sql diff --git a/benchmark/infobright/define_schema.sql b/benchmark/infobright/define_schema.sql deleted file mode 100644 index e2390d5ae75..00000000000 --- a/benchmark/infobright/define_schema.sql +++ /dev/null @@ -1,111 +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 -); - -LOAD DATA INFILE '/opt/dump/dump_0.3/dump_hits_10m_meshed_utf8.tsv' INTO TABLE hits_10m FIELDS TERMINATED BY '\t' ESCAPED BY '\\' ENCLOSED BY "NULL"; \ No newline at end of file diff --git a/benchmark/infobright/expect.tcl b/benchmark/infobright/expect.tcl deleted file mode 100644 index d07ab257260..00000000000 --- a/benchmark/infobright/expect.tcl +++ /dev/null @@ -1,18 +0,0 @@ -#!/usr/bin/env bash -#!/bin/expect - -# Set timeout -set timeout 600 - -# Get arguments -set query [lindex $argv 0] - -spawn mysql-ib -u root -D hits - -expect "mysql>" -send "$query\r" - -expect "mysql>" -send "quit\r" - -expect eof \ No newline at end of file diff --git a/benchmark/infobright/log-community/log_10m b/benchmark/infobright/log-community/log_10m deleted file mode 100644 index 817404241e5..00000000000 --- a/benchmark/infobright/log-community/log_10m +++ /dev/null @@ -1,2511 +0,0 @@ -start time: Вт. сент. 3 15:44:00 MSK 2013 --- set GLOBAL max_length_for_sort_data = 8388608; - - -times: 1 -query: SELECT count(*) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 11 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(*) FROM hits_10m; -+----------+ -+----------+ -+----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT count(*) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 12 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(*) FROM hits_10m; -+----------+ -+----------+ -+----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT count(*) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 13 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(*) FROM hits_10m; -+----------+ -+----------+ -+----------+ -1 row in set (0.00 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 14 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (0.40 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 15 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (0.38 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 16 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(*) FROM hits_10m WHERE AdvEngineID != 0; -+----------+ -+----------+ -+----------+ -1 row in set (0.39 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 17 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -1 row in set (0.10 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 18 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -1 row in set (0.00 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 19 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -+------------------+----------+----------------------+ -1 row in set (0.00 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT sum(UserID) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 20 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT sum(UserID) FROM hits_10m; -ERROR 5 (HY000): The query includes syntax that is not supported by the Infobright Optimizer. Either restructure the query with supported syntax, or enable the MySQL Query Path in the brighthouse.ini file to execute the query with reduced performance. -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT sum(UserID) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 21 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT sum(UserID) FROM hits_10m; -ERROR 5 (HY000): The query includes syntax that is not supported by the Infobright Optimizer. Either restructure the query with supported syntax, or enable the MySQL Query Path in the brighthouse.ini file to execute the query with reduced performance. -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT sum(UserID) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 22 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT sum(UserID) FROM hits_10m; -ERROR 5 (HY000): The query includes syntax that is not supported by the Infobright Optimizer. Either restructure the query with supported syntax, or enable the MySQL Query Path in the brighthouse.ini file to execute the query with reduced performance. -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 23 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(DISTINCT UserID) FROM hits_10m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (2.83 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 24 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(DISTINCT UserID) FROM hits_10m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (1.91 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 25 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(DISTINCT UserID) FROM hits_10m; -+------------------------+ -+------------------------+ -+------------------------+ -1 row in set (1.95 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 26 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (9.16 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 27 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (1.65 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 28 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+------------------------------+ -+------------------------------+ -+------------------------------+ -1 row in set (1.70 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 29 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT min(EventDate), max(EventDate) FROM hits_10m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (0.03 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 30 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT min(EventDate), max(EventDate) FROM hits_10m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (0.00 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 31 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT min(EventDate), max(EventDate) FROM hits_10m; -+----------------+----------------+ -+----------------+----------------+ -+----------------+----------------+ -1 row in set (0.00 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 32 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -9 rows in set (0.46 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 33 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -9 rows in set (0.40 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 34 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+----------+ -+-------------+----------+ -+-------------+----------+ -9 rows in set (0.41 sec) - -mysql> quit -Bye - * MySQL running (21735) --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 35 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+----------+--------+ -+----------+--------+ -10 rows in set (4.13 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 36 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+----------+--------+ -+----------+--------+ -10 rows in set (2.97 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 37 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+----------+--------+ -+----------+--------+ -10 rows in set (3.43 sec) - -mysql> quit -Bye - * MySQL running (21735) --- агрегация, среднее количество ключей.; - - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 38 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 rows in set (5.12 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 39 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 rows in set (4.46 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 40 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 rows in set (4.15 sec) - -mysql> quit -Bye - * MySQL running (21735) --- агрегация, среднее количество ключей, несколько агрегатных функций.; - - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 41 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+------------------+-------+ -+------------------+-------+ -10 rows in set (1.98 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 42 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+------------------+-------+ -+------------------+-------+ -10 rows in set (1.24 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 43 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+------------------+-------+ -+------------------+-------+ -10 rows in set (1.36 sec) - -mysql> quit -Bye - * MySQL running (21735) --- мощная фильтрация по строкам, затем агрегация по строкам.; - - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 44 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DES C LIMIT 10; -+-------------+------------------+-------+ -+-------------+------------------+-------+ -+-------------+------------------+-------+ -10 rows in set (1.58 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 45 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DES C LIMIT 10; -+-------------+------------------+-------+ -+-------------+------------------+-------+ -+-------------+------------------+-------+ -10 rows in set (1.26 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 46 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DES C LIMIT 10; -+-------------+------------------+-------+ -+-------------+------------------+-------+ -+-------------+------------------+-------+ -10 rows in set (1.27 sec) - -mysql> quit -Bye - * MySQL running (21735) --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - - -times: 1 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 47 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------------+----------+ -+-----------------------------+----------+ -+-----------------------------+----------+ -10 rows in set (13.37 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 48 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------------+----------+ -+-----------------------------+----------+ -+-----------------------------+----------+ -10 rows in set (30.81 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 49 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------------+----------+ -+-----------------------------+----------+ -+-----------------------------+----------+ -10 rows in set (29.76 sec) - -mysql> quit -Bye - * MySQL running (21735) --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 50 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-------------------------------------------+------+ -+-------------------------------------------+------+ -+-------------------------------------------+------+ -10 rows in set (32.59 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 51 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-------------------------------------------+------+ -+-------------------------------------------+------+ -+-------------------------------------------+------+ -10 rows in set (30.22 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 52 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-------------------------------------------+------+ -+-------------------------------------------+------+ -+-------------------------------------------+------+ -10 rows in set (13.00 sec) - -mysql> quit -Bye - * MySQL running (21735) --- агрегация чуть сложнее.; - - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 53 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -10 rows in set (12.93 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 54 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -10 rows in set (11.39 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 55 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -+----------------+-----------------------------+----------+ -10 rows in set (30.46 sec) - -mysql> quit -Bye - * MySQL running (21735) --- агрегация по числу и строке, большое количество ключей.; - - -times: 1 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 56 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+----------+ -+---------------------+----------+ -+---------------------+----------+ -10 rows in set (2.98 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 57 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+----------+ -+---------------------+----------+ -+---------------------+----------+ -10 rows in set (3.05 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 58 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+----------+ -+---------------------+----------+ -+---------------------+----------+ -10 rows in set (2.96 sec) - -mysql> quit -Bye - * MySQL running (21735) --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 59 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -10 rows in set (1 min 9.90 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 60 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -10 rows in set (1 min 20.86 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 61 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -+---------------------+--------------+----------+ -10 rows in set (1 min 25.17 sec) - -mysql> quit -Bye - * MySQL running (21735) --- ещё более сложная агрегация.; - - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 62 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+---------------------+-----------------------------------------------------------------+----------+ -+---------------------+-----------------------------------------------------------------+----------+ -+---------------------+-----------------------------------------------------------------+----------+ -10 rows in set (10.38 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 63 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+---------------------+-----------------------------------------------------------------+----------+ -+---------------------+-----------------------------------------------------------------+----------+ -+---------------------+-----------------------------------------------------------------+----------+ -10 rows in set (10.19 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 64 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+---------------------+-----------------------------------------------------------------+----------+ -+---------------------+-----------------------------------------------------------------+----------+ -+---------------------+-----------------------------------------------------------------+----------+ -10 rows in set (10.28 sec) - -mysql> quit -Bye - * MySQL running (21735) --- то же самое, но без сортировки.; - - -times: 1 -query: SELECT UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 65 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -10 rows in set (2 min 42.43 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 66 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -10 rows in set (2 min 44.35 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 67 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID, Minute(EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -+---------------------+------+--------------+----------+ -10 rows in set (2 min 49.28 sec) - -mysql> quit -Bye - * MySQL running (21735) --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - - -times: 1 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 68 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -ERROR 5 (HY000): The query includes syntax that is not supported by the Infobright Optimizer. Either restructure the query with supported syntax, or enable the MySQL Query Path in the brighthouse.ini file to execute the query with reduced performance. -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 69 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -ERROR 5 (HY000): The query includes syntax that is not supported by the Infobright Optimizer. Either restructure the query with supported syntax, or enable the MySQL Query Path in the brighthouse.ini file to execute the query with reduced performance. -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 70 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT UserID FROM hits_10m WHERE UserID = 12345678901234567890; -ERROR 5 (HY000): The query includes syntax that is not supported by the Infobright Optimizer. Either restructure the query with supported syntax, or enable the MySQL Query Path in the brighthouse.ini file to execute the query with reduced performance. -mysql> quit -Bye - * MySQL running (21735) --- мощная фильтрация по столбцу типа UInt64.; - - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 71 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (47.80 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 72 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (4.40 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 73 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+----------+ -+----------+ -+----------+ -1 row in set (4.47 sec) - -mysql> quit -Bye - * MySQL running (21735) --- фильтрация по поиску подстроки в строке.; - - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 74 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -10 rows in set (4.83 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 75 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -10 rows in set (1.15 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 76 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -+--------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+----------+ -10 rows in set (1.14 sec) - -mysql> quit -Bye - * MySQL running (21735) --- вынимаем большие столбцы, фильтрация по строке.; - - -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 != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 77 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+-----------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+-----------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -10 rows in set (43.82 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 78 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+-----------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+-----------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -10 rows in set (2.14 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 79 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 != '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+-----------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -+-----------------------------------------------------------------------------------+----------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+---+------------------------+ -10 rows in set (2.16 sec) - -mysql> quit -Bye - * MySQL running (21735) --- чуть больше столбцы.; - - -times: 1 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 80 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -10 rows in set (6.14 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 81 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -10 rows in set (4.39 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 82 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+---------------------+------------+-------------------------+-----------+---------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+---------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+---------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -10 rows in set (4.35 sec) - -mysql> quit -Bye - * MySQL running (21735) --- плохой запрос - вынимаем все столбцы.; - - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 83 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (0.41 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 84 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (0.41 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 85 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (0.41 sec) - -mysql> quit -Bye - * MySQL running (21735) --- большая сортировка.; - - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 86 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (0.80 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 87 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (0.80 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 88 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -+------------------------------------------------------------------------------------------------------------------------------------+ -10 rows in set (0.80 sec) - -mysql> quit -Bye - * MySQL running (21735) --- большая сортировка по строкам.; - - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 89 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (0.41 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 90 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (0.41 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 91 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -+--------------------------------------------------------------------+ -10 rows in set (0.40 sec) - -mysql> quit -Bye - * MySQL running (21735) --- большая сортировка по кортежу.; - - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 92 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 rows in set (10.39 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 93 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 rows in set (10.12 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 94 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 rows in set (9.88 sec) - -mysql> quit -Bye - * MySQL running (21735) --- считаем средние длины 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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 95 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SUBSTRING(SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, GREATEST(0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1)) AS k, a vg(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; -+------------------+----------+---------+--------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+--------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+--------------------------------------------------------------------------------------------------+ -11 rows in set (2 min 19.25 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 96 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SUBSTRING(SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, GREATEST(0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1)) AS k, a vg(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; -+------------------+----------+---------+--------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+--------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+--------------------------------------------------------------------------------------------------+ -11 rows in set (1 min 19.33 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 != '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 97 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SUBSTRING(SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, GREATEST(0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1)) AS k, a vg(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; -+------------------+----------+---------+--------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+--------------------------------------------------------------------------------------------------+ -+------------------+----------+---------+--------------------------------------------------------------------------------------------------+ -11 rows in set (1 min 18.92 sec) - -mysql> quit -Bye - * MySQL running (21735) --- то же самое, но с разбивкой по доменам.; - - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 98 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Resolu tionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), s um(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionW idth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), su m(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWi dth + 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(ResolutionWid th + 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(ResolutionWidt h + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(R esolutionWidth + 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(Re solutionWidth + 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; -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (2 min 25.80 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 99 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Resolu tionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), s um(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionW idth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), su m(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWi dth + 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(ResolutionWid th + 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(ResolutionWidt h + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(R esolutionWidth + 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(Re solutionWidth + 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; -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (2 min 26.20 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 100 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Resolu tionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), s um(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionW idth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), su m(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWi dth + 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(ResolutionWid th + 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(ResolutionWidt h + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(R esolutionWidth + 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(Re solutionWidth + 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; -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -+----------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+--------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+---------------------------+ -1 row in set (2 min 24.97 sec) - -mysql> quit -Bye - * MySQL running (21735) --- много тупых агрегатных функций.; - - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 101 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY coun t(*) DESC LIMIT 10; -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -10 rows in set (3.34 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 102 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY coun t(*) DESC LIMIT 10; -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -10 rows in set (2.02 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 103 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY coun t(*) DESC LIMIT 10; -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -+----------------+------------+-----+--------------+----------------------+ -10 rows in set (2.06 sec) - -mysql> quit -Bye - * MySQL running (21735) --- сложная агрегация, для больших таблиц может не хватить оперативки.; - - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 104 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMI T 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (3.68 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 105 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMI T 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (3.05 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 106 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMI T 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (3.02 sec) - -mysql> quit -Bye - * MySQL running (21735) --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 107 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (14.76 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 108 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (14.82 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 109 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -+---------------------+------------+---+--------------+----------------------+ -10 rows in set (14.76 sec) - -mysql> quit -Bye - * MySQL running (21735) --- то же самое, но ещё и без фильтрации.; - - -times: 1 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 110 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -10 rows in set (1 min 17.49 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 111 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -10 rows in set (1 min 31.40 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 112 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -+--------------------------------------------------------------+----------+ -10 rows in set (1 min 30.90 sec) - -mysql> quit -Bye - * MySQL running (21735) --- агрегация по URL.; - - -times: 1 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 113 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -10 rows in set (1 min 15.37 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 114 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -10 rows in set (1 min 23.55 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 115 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -+---+--------------------------------------------------------------+----------+ -10 rows in set (1 min 3.55 sec) - -mysql> quit -Bye - * MySQL running (21735) --- агрегация по 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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 116 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 LIMI T 10; -+------------+--------------+--------------+--------------+----------+ -+------------+--------------+--------------+--------------+----------+ -+------------+--------------+--------------+--------------+----------+ -10 rows in set (10.80 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 117 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 LIMI T 10; -+------------+--------------+--------------+--------------+----------+ -+------------+--------------+--------------+--------------+----------+ -+------------+--------------+--------------+--------------+----------+ -10 rows in set (10.52 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 118 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 LIMI T 10; -+------------+--------------+--------------+--------------+----------+ -+------------+--------------+--------------+--------------+----------+ -+------------+--------------+--------------+--------------+----------+ -10 rows in set (10.67 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 119 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 120 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 121 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 122 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AN D NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -10 rows in set (23.77 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 123 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AN D NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -10 rows in set (9.47 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 124 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AN D NOT Refresh AND URL != '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------+-----------+ -10 rows in set (9.30 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 125 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -10 rows in set (50.21 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 126 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -10 rows in set (3.37 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 127 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND Title != '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -+---------------------------------------------------------------------------------------+-----------+ -10 rows in set (3.36 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 128 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink A ND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (1.26 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 129 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink A ND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (0.77 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 130 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND IsLink A ND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (0.76 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 131 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (1 min 38.25 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 132 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (1 min 44.60 sec) - -mysql> quit -Bye - * MySQL running (21735) - -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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 133 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> 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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -1000 rows in set (1 min 34.29 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 134 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refre sh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -Empty set (2.04 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 135 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refre sh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -Empty set (0.47 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 136 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refre sh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -Empty set (0.54 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 137 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07 -31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Empty set (1.36 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 138 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07 -31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Empty set (0.42 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 139 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07 -31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -Empty set (0.41 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 1 -query: SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 140 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDat e <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -+---------------------+-----------+ -+---------------------+-----------+ -+---------------------+-----------+ -652 rows in set (4.58 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 2 -query: SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 141 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDat e <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -+---------------------+-----------+ -+---------------------+-----------+ -+---------------------+-----------+ -652 rows in set (4.01 sec) - -mysql> quit -Bye - * MySQL running (21735) - -times: 3 -query: SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mysql-ib -u root -D hits -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 142 -Server version: 5.1.40 build number (revision)=IB_4.0.7_r16961_17249(ice) (static) - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= DATE('2013-07-01') AND EventDat e <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -+---------------------+-----------+ -+---------------------+-----------+ -+---------------------+-----------+ -652 rows in set (3.98 sec) - -mysql> quit -Bye - * MySQL running (21735) -stop time: Вт. сент. 3 16:34:22 MSK 2013 diff --git a/benchmark/infobright/queries.sql b/benchmark/infobright/queries.sql index f5a14c5908e..ea2bde47802 100644 --- a/benchmark/infobright/queries.sql +++ b/benchmark/infobright/queries.sql @@ -1,113 +1,43 @@ --- set GLOBAL max_length_for_sort_data = 8388608; - -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, Minute(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 = 123456789; --- мощная фильтрация по столбцу типа 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 NOT LIKE '%.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, 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 != '' 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 >= DATE('2013-07-01') AND EventDate <= DATE('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 >= DATE('2013-07-01') AND EventDate <= DATE('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 >= DATE('2013-07-01') AND EventDate <= DATE('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 >= DATE('2013-07-01') AND EventDate <= DATE('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 >= DATE('2013-07-01') AND EventDate <= DATE('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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; - -SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; \ No newline at end of file +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/infobright/results/c6a.4xlarge.txt b/benchmark/infobright/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/infobright/results/c6a.4xlarge.txt rename to benchmark/infobright/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/infobright/run.sh b/benchmark/infobright/run.sh similarity index 100% rename from benchmark/compatible/infobright/run.sh rename to benchmark/infobright/run.sh diff --git a/benchmark/compatible/locustdb/README.md b/benchmark/locustdb/README.md similarity index 100% rename from benchmark/compatible/locustdb/README.md rename to benchmark/locustdb/README.md diff --git a/benchmark/compatible/locustdb/benchmark.sh b/benchmark/locustdb/benchmark.sh similarity index 100% rename from benchmark/compatible/locustdb/benchmark.sh rename to benchmark/locustdb/benchmark.sh diff --git a/benchmark/compatible/mariadb-columnstore/README.md b/benchmark/mariadb-columnstore/README.md similarity index 100% rename from benchmark/compatible/mariadb-columnstore/README.md rename to benchmark/mariadb-columnstore/README.md diff --git a/benchmark/compatible/mariadb-columnstore/benchmark.sh b/benchmark/mariadb-columnstore/benchmark.sh similarity index 100% rename from benchmark/compatible/mariadb-columnstore/benchmark.sh rename to benchmark/mariadb-columnstore/benchmark.sh diff --git a/benchmark/compatible/mariadb-columnstore/create.sql b/benchmark/mariadb-columnstore/create.sql similarity index 100% rename from benchmark/compatible/mariadb-columnstore/create.sql rename to benchmark/mariadb-columnstore/create.sql diff --git a/benchmark/compatible/infobright/queries.sql b/benchmark/mariadb-columnstore/queries.sql similarity index 100% rename from benchmark/compatible/infobright/queries.sql rename to benchmark/mariadb-columnstore/queries.sql diff --git a/benchmark/compatible/mariadb-columnstore/results/c6a.4xlarge.txt b/benchmark/mariadb-columnstore/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/mariadb-columnstore/results/c6a.4xlarge.txt rename to benchmark/mariadb-columnstore/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/mariadb-columnstore/run.sh b/benchmark/mariadb-columnstore/run.sh similarity index 100% rename from benchmark/compatible/mariadb-columnstore/run.sh rename to benchmark/mariadb-columnstore/run.sh diff --git a/benchmark/compatible/mariadb/benchmark.sh b/benchmark/mariadb/benchmark.sh similarity index 100% rename from benchmark/compatible/mariadb/benchmark.sh rename to benchmark/mariadb/benchmark.sh diff --git a/benchmark/compatible/mariadb/create.sql b/benchmark/mariadb/create.sql similarity index 100% rename from benchmark/compatible/mariadb/create.sql rename to benchmark/mariadb/create.sql diff --git a/benchmark/compatible/duckdb/queries.sql b/benchmark/mariadb/queries.sql similarity index 100% rename from benchmark/compatible/duckdb/queries.sql rename to benchmark/mariadb/queries.sql diff --git a/benchmark/compatible/mariadb/run.sh b/benchmark/mariadb/run.sh similarity index 100% rename from benchmark/compatible/mariadb/run.sh rename to benchmark/mariadb/run.sh diff --git a/benchmark/memsql/benchmark.sh b/benchmark/memsql/benchmark.sh deleted file mode 100644 index c4285370d74..00000000000 --- a/benchmark/memsql/benchmark.sh +++ /dev/null @@ -1,20 +0,0 @@ -#!/usr/bin/env bash - -QUERIES_FILE="queries.sql" -TABLE=$1 -TRIES=3 - -cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - - echo -n "[" - for i in $(seq 1 $TRIES); do - - RES=$(mysql -u root -h 127.0.0.1 -P 3306 --database=test -t -vvv -e "$query" 2>&1 | grep ' set ' | grep -oP '\d+\.\d+') - - [[ "$?" == "0" ]] && echo -n "$RES" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - done - echo "]," -done diff --git a/benchmark/memsql/instructions.txt b/benchmark/memsql/instructions.txt deleted file mode 100644 index 6f167d62faf..00000000000 --- a/benchmark/memsql/instructions.txt +++ /dev/null @@ -1,141 +0,0 @@ -Note: column store in MemSQL was introduced in Feb 2014. - -http://www.memsql.com/download/ -http://docs.memsql.com/docs/latest/setup/setup_onprem.html -wget http://download.memsql.com/8d9f4c4d99a547baa40ba097b171bd15/memsql-3.2.x86_64.deb -scp memsql-3.2.x86_64.deb example05e:~ -ssh example05e -sudo dpkg -i memsql-3.2.x86_64.deb - -sudo mkdir /opt/memsql-data/ -sudo cp -r /var/lib/memsql/data/* /opt/memsql-data/ -sudo rm -rf /var/lib/memsql/data -sudo ln -s /opt/memsql-data /var/lib/memsql/data -sudo chown -R memsql /opt/memsql-data -sudo chown -R memsql /var/lib/memsql/data - -sudo service memsql start -mysql -u root -h 127.0.0.1 -P 3306 --prompt="memsql> " - -CREATE DATABASE test; -USE test; - -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, - INDEX ColumnStoreIndex USING CLUSTERED COLUMNSTORE (CounterID, EventDate, UserID, EventTime) -); - -Table creation takes about 15 seconds. - -LOAD DATA INFILE '/opt/dumps/hits_10m_corrected.tsv' INTO TABLE hits_10m; - -12 min 24.51 sec - -13422 rows/sec. - -data size: 1 613 773 528 bytes. diff --git a/benchmark/memsql/queries.sql b/benchmark/memsql/queries.sql deleted file mode 100644 index 6069bad1b44..00000000000 --- a/benchmark/memsql/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -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, Minute(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 = 123456789; -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 NOT LIKE '%.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; -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 != '' 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; -SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -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 >= DATE('2013-07-01') AND EventDate <= DATE('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 >= DATE('2013-07-01') AND EventDate <= DATE('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 >= DATE('2013-07-01') AND EventDate <= DATE('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 >= DATE('2013-07-01') AND EventDate <= DATE('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 >= DATE('2013-07-01') AND EventDate <= DATE('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 >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -SELECT EventTime - INTERVAL SECOND(EventTime) SECOND AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; diff --git a/benchmark/monetdb/aws.log b/benchmark/monetdb/aws.log deleted file mode 100644 index e5fa40018bf..00000000000 --- a/benchmark/monetdb/aws.log +++ /dev/null @@ -1,1774 +0,0 @@ -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 ". -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-▉▉▉:~$ diff --git a/benchmark/monetdb/benchmark.sh b/benchmark/monetdb/benchmark.sh index 3766f1a8eef..f0afb400802 100755 --- a/benchmark/monetdb/benchmark.sh +++ b/benchmark/monetdb/benchmark.sh @@ -1,10 +1,40 @@ #!/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 +# Install - echo "$query"; - for i in {1..3}; do - ./send-query "$query" 2>&1 | grep -P '\d+ tuple|clk: |unknown|overflow|error'; - done; -done; +echo "deb https://dev.monetdb.org/downloads/deb/ $(lsb_release -cs) monetdb" | sudo tee /etc/apt/sources.list.d/monetdb.list + +sudo wget --output-document=/etc/apt/trusted.gpg.d/monetdb.gpg https://www.monetdb.org/downloads/MonetDB-GPG-KEY.gpg +sudo apt-get update +sudo apt-get install -y monetdb5-sql monetdb-client dos2unix + +sudo systemctl enable monetdbd +sudo systemctl start monetdbd + +sudo monetdbd create /var/lib/monetdb +sudo monetdbd start /var/lib/monetdb +sudo usermod -a -G monetdb $USER + +sudo monetdb create test +sudo monetdb release test + +sudo apt-get install -y expect + +./query.expect "$(cat create.sql)" + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz +chmod 777 ~ hits.tsv + +./query.expect "COPY INTO hits FROM '$(pwd)/hits.tsv' USING DELIMITERS '\t'" + +# 99997497 affected rows +# clk: 15:39 min + +./run.sh 2>&1 | tee log.txt + +sudo du -bcs /var/monetdb5/ + +cat log.txt | dos2unix -f | grep -P 'clk|tuple' | + awk '/tuple/ { ok = 1 } /clk/ { if (ok) { if ($3 == "ms") { print $2 / 1000 } else { print $2 } } else { print "null" }; ok = 0 }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/monetdb/create.sql b/benchmark/monetdb/create.sql similarity index 100% rename from benchmark/compatible/monetdb/create.sql rename to benchmark/monetdb/create.sql diff --git a/benchmark/monetdb/instruction.md b/benchmark/monetdb/instruction.md deleted file mode 100644 index b48bb19ecc1..00000000000 --- a/benchmark/monetdb/instruction.md +++ /dev/null @@ -1,356 +0,0 @@ -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.com/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 | 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`. -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/log.txt b/benchmark/monetdb/log.txt deleted file mode 100644 index 40f480bf216..00000000000 --- a/benchmark/monetdb/log.txt +++ /dev/null @@ -1,341 +0,0 @@ -3 -SELECT count(*) FROM hits; -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.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: 597.512 ms -1 tuple -clk: 579.383 ms -1 tuple -clk: 598.220 ms -3 -SELECT sum("UserID") FROM hits; -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: 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.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: 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.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: 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.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.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.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.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.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.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: 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: 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: 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: 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: 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.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.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.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.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.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.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.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: 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: 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.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.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.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: 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: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: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.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.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.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.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.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.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.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.199 sec -0 tuples -clk: 4.211 sec -0 tuples -clk: 4.190 sec diff --git a/benchmark/monetdb/queries.sql b/benchmark/monetdb/queries.sql index 0b9eadb1011..31f65fc898d 100644 --- a/benchmark/monetdb/queries.sql +++ b/benchmark/monetdb/queries.sql @@ -1,43 +1,43 @@ -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"); +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/monetdb/query.expect b/benchmark/monetdb/query.expect similarity index 100% rename from benchmark/compatible/monetdb/query.expect rename to benchmark/monetdb/query.expect diff --git a/benchmark/compatible/monetdb/results/c6a.4xlarge.txt b/benchmark/monetdb/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/monetdb/results/c6a.4xlarge.txt rename to benchmark/monetdb/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/monetdb/run.sh b/benchmark/monetdb/run.sh similarity index 100% rename from benchmark/compatible/monetdb/run.sh rename to benchmark/monetdb/run.sh diff --git a/benchmark/monetdb/send-query b/benchmark/monetdb/send-query deleted file mode 100755 index 6817302e2d5..00000000000 --- a/benchmark/monetdb/send-query +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/expect - -# Set timeout -set timeout 600 - -# Get arguments -set query [lindex $argv 0] - -spawn mclient -u monetdb -d test --timer=clock -expect "password:" -send "monetdb\r" - -expect "sql>" -send "$query;\r" - -expect "sql>" -send "\\q\r" - -expect eof diff --git a/benchmark/monetdb/usability.md b/benchmark/monetdb/usability.md deleted file mode 100644 index 741079b5663..00000000000 --- a/benchmark/monetdb/usability.md +++ /dev/null @@ -1,1222 +0,0 @@ -Go to https://www.monetdb.org/ - -The graphical design of the website is a bit old-fashioned but I do not afraid. - -Download 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 successfully 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 - - -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 -``` - -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 -``` - -When I run: - -``` -sudo systemctl stop monetdbd -``` - -It takes a few minutes to complete. diff --git a/benchmark/compatible/mysql-myisam/benchmark.sh b/benchmark/mysql-myisam/benchmark.sh similarity index 100% rename from benchmark/compatible/mysql-myisam/benchmark.sh rename to benchmark/mysql-myisam/benchmark.sh diff --git a/benchmark/compatible/mysql-myisam/create.sql b/benchmark/mysql-myisam/create.sql similarity index 100% rename from benchmark/compatible/mysql-myisam/create.sql rename to benchmark/mysql-myisam/create.sql diff --git a/benchmark/compatible/mariadb-columnstore/queries.sql b/benchmark/mysql-myisam/queries.sql similarity index 100% rename from benchmark/compatible/mariadb-columnstore/queries.sql rename to benchmark/mysql-myisam/queries.sql diff --git a/benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt b/benchmark/mysql-myisam/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/mysql-myisam/results/c6a.4xlarge.txt rename to benchmark/mysql-myisam/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/mysql-myisam/run.sh b/benchmark/mysql-myisam/run.sh similarity index 100% rename from benchmark/compatible/mysql-myisam/run.sh rename to benchmark/mysql-myisam/run.sh diff --git a/benchmark/compatible/mysql/benchmark.sh b/benchmark/mysql/benchmark.sh similarity index 100% rename from benchmark/compatible/mysql/benchmark.sh rename to benchmark/mysql/benchmark.sh diff --git a/benchmark/compatible/mysql/create.sql b/benchmark/mysql/create.sql similarity index 100% rename from benchmark/compatible/mysql/create.sql rename to benchmark/mysql/create.sql diff --git a/benchmark/compatible/mysql-myisam/queries.sql b/benchmark/mysql/queries.sql similarity index 100% rename from benchmark/compatible/mysql-myisam/queries.sql rename to benchmark/mysql/queries.sql diff --git a/benchmark/compatible/mysql/results/c6a.4xlarge.txt b/benchmark/mysql/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/mysql/results/c6a.4xlarge.txt rename to benchmark/mysql/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/mysql/run.sh b/benchmark/mysql/run.sh similarity index 100% rename from benchmark/compatible/mysql/run.sh rename to benchmark/mysql/run.sh diff --git a/benchmark/omnisci/benchmark.sh b/benchmark/omnisci/benchmark.sh deleted file mode 100755 index 2e4b10fab73..00000000000 --- a/benchmark/omnisci/benchmark.sh +++ /dev/null @@ -1,17 +0,0 @@ -#!/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 - sudo systemctl restart omnisci_server - for i in {1..1000}; do - /opt/omnisci/bin/omnisql -t -p HyperInteractive <<< "SELECT 1;" 2>&1 | grep -q '1 rows returned' && break; - sleep 0.1; - done - sleep 10; - - echo "$query"; - for i in {1..3}; do - /opt/omnisci/bin/omnisql -t -p HyperInteractive <<< "$query" 2>&1 | grep -P 'Exception:|Execution time:'; - done; -done; diff --git a/benchmark/omnisci/instruction.md b/benchmark/omnisci/instruction.md deleted file mode 100644 index e81e3783e44..00000000000 --- a/benchmark/omnisci/instruction.md +++ /dev/null @@ -1,332 +0,0 @@ -# Instruction to run benchmark for OmniSci on web-analytics dataset - -OmniSci (former name "MapD") is open-source (open-core) in-memory analytical DBMS with support for GPU processing. -It can run on CPU without GPU as well. It can show competitive performance on simple queries (like - simple aggregation on a single column). - -# How to install - -https://docs.omnisci.com/installation-and-configuration/installation/installing-on-ubuntu - -# Caveats - -- Dataset (at least needed columns) must fit in memory. -- It does not support data compression (only dictionary encoding for strings). -- First query execution is very slow because uncompressed data is read from disk. -- It does not support index for quick range queries. -- It does not support NOT NULL for data types. -- It does not support BLOB. -- No support for UNSIGNED data type (it's Ok according to SQL standard). -- Lack of string processing functions. -- Strings are limited to 32767 bytes. -- GROUP BY on text data type is supported only if it has dictionary encoding. -`Exception: Cannot group by string columns which are not dictionary encoded` -- Some aggregate functions are not supported for strings at all. -`Aggregate on TEXT is not supported yet.` -- Sometimes I hit a bug when query is run in infinite loop and does not finish (after retry it's finished successfully). -- One query executed in hours even with retries. -- Sorting is slow and disabled with default settings for large resultsets. -`Exception: Sorting the result would be too slow` -`Cast from dictionary-encoded string to none-encoded would be slow` -- There is approximate count distinct function but the precision is not documented. - -To enable sorting of large resultsets, see: -https://stackoverflow.com/questions/62977734/omnissci-sorting-the-result-would-be-too-slow - -The list of known issues is here: -https://github.com/omnisci/omniscidb/issues?q=is%3Aissue+author%3Aalexey-milovidov - -# How to prepare data - -Download the 100 million rows dataset from here and insert into ClickHouse: -https://clickhouse.com/docs/en/getting-started/example-datasets/metrica/ - -Convert the CREATE TABLE query: - -``` -clickhouse-client --query "SHOW CREATE TABLE hits_100m" --format TSVRaw | - tr '`' '"' | - sed -r -e ' - s/U?Int64/BIGINT/; - s/U?Int32/INTEGER/; - s/U?Int16/SMALLINT/; - s/U?Int8/TINYINT/; - s/DateTime/TIMESTAMP ENCODING FIXED(32)/; - s/ Date/ DATE ENCODING DAYS(16)/; - s/FixedString\(2\)/TEXT ENCODING DICT(16)/; - s/FixedString\(3\)/TEXT ENCODING DICT/; - s/FixedString\(\d+\)/TEXT ENCODING DICT/; - s/String/TEXT ENCODING DICT/;' -``` -And cut `ENGINE` part. - -The resulting CREATE TABLE query: -``` -CREATE TABLE hits -( - "WatchID" BIGINT, - "JavaEnable" TINYINT, - "Title" TEXT ENCODING DICT, - "GoodEvent" SMALLINT, - "EventTime" TIMESTAMP ENCODING FIXED(32), - "EventDate" ENCODING DAYS(16) Date, - "CounterID" INTEGER, - "ClientIP" INTEGER, - "RegionID" INTEGER, - "UserID" BIGINT, - "CounterClass" TINYINT, - "OS" TINYINT, - "UserAgent" TINYINT, - "URL" TEXT ENCODING DICT, - "Referer" TEXT ENCODING DICT, - "Refresh" TINYINT, - "RefererCategoryID" SMALLINT, - "RefererRegionID" INTEGER, - "URLCategoryID" SMALLINT, - "URLRegionID" INTEGER, - "ResolutionWidth" SMALLINT, - "ResolutionHeight" SMALLINT, - "ResolutionDepth" TINYINT, - "FlashMajor" TINYINT, - "FlashMinor" TINYINT, - "FlashMinor2" TEXT ENCODING DICT, - "NetMajor" TINYINT, - "NetMinor" TINYINT, - "UserAgentMajor" SMALLINT, - "UserAgentMinor" TEXT ENCODING DICT(16), - "CookieEnable" TINYINT, - "JavascriptEnable" TINYINT, - "IsMobile" TINYINT, - "MobilePhone" TINYINT, - "MobilePhoneModel" TEXT ENCODING DICT, - "Params" TEXT ENCODING DICT, - "IPNetworkID" INTEGER, - "TraficSourceID" TINYINT, - "SearchEngineID" SMALLINT, - "SearchPhrase" TEXT ENCODING DICT, - "AdvEngineID" TINYINT, - "IsArtifical" TINYINT, - "WindowClientWidth" SMALLINT, - "WindowClientHeight" SMALLINT, - "ClientTimeZone" SMALLINT, - "ClientEventTime" TIMESTAMP ENCODING FIXED(32), - "SilverlightVersion1" TINYINT, - "SilverlightVersion2" TINYINT, - "SilverlightVersion3" INTEGER, - "SilverlightVersion4" SMALLINT, - "PageCharset" TEXT ENCODING DICT, - "CodeVersion" INTEGER, - "IsLink" TINYINT, - "IsDownload" TINYINT, - "IsNotBounce" TINYINT, - "FUniqID" BIGINT, - "OriginalURL" TEXT ENCODING DICT, - "HID" INTEGER, - "IsOldCounter" TINYINT, - "IsEvent" TINYINT, - "IsParameter" TINYINT, - "DontCountHits" TINYINT, - "WithHash" TINYINT, - "HitColor" TEXT ENCODING DICT(8), - "LocalEventTime" TIMESTAMP ENCODING FIXED(32), - "Age" TINYINT, - "Sex" TINYINT, - "Income" TINYINT, - "Interests" SMALLINT, - "Robotness" TINYINT, - "RemoteIP" INTEGER, - "WindowName" INTEGER, - "OpenerName" INTEGER, - "HistoryLength" SMALLINT, - "BrowserLanguage" TEXT ENCODING DICT(16), - "BrowserCountry" TEXT ENCODING DICT(16), - "SocialNetwork" TEXT ENCODING DICT, - "SocialAction" TEXT ENCODING DICT, - "HTTPError" SMALLINT, - "SendTiming" INTEGER, - "DNSTiming" INTEGER, - "ConnectTiming" INTEGER, - "ResponseStartTiming" INTEGER, - "ResponseEndTiming" INTEGER, - "FetchTiming" INTEGER, - "SocialSourceNetworkID" TINYINT, - "SocialSourcePage" TEXT ENCODING DICT, - "ParamPrice" BIGINT, - "ParamOrderID" TEXT ENCODING DICT, - "ParamCurrency" TEXT ENCODING DICT, - "ParamCurrencyID" SMALLINT, - "OpenstatServiceName" TEXT ENCODING DICT, - "OpenstatCampaignID" TEXT ENCODING DICT, - "OpenstatAdID" TEXT ENCODING DICT, - "OpenstatSourceID" TEXT ENCODING DICT, - "UTMSource" TEXT ENCODING DICT, - "UTMMedium" TEXT ENCODING DICT, - "UTMCampaign" TEXT ENCODING DICT, - "UTMContent" TEXT ENCODING DICT, - "UTMTerm" TEXT ENCODING DICT, - "FromTag" TEXT ENCODING DICT, - "HasGCLID" TINYINT, - "RefererHash" BIGINT, - "URLHash" BIGINT, - "CLID" INTEGER -); -``` - -Convert the dataset, prepare the list of fields for SELECT: - -``` -clickhouse-client --query "SHOW CREATE TABLE hits_100m" --format TSVRaw | - tr '`' '"' | - sed -r -e ' - s/"(\w+)" U?Int([0-9]+)/toInt\2(\1)/; - s/"(\w+)" (Fixed)?String(\([0-9]+\))?/toValidUTF8(toString(\1))/; - s/"(\w+)" \w+/\1/' -``` - -The resulting SELECT query for data preparation: - -``` -SELECT - toInt64(WatchID), - toInt8(JavaEnable), - toValidUTF8(toString(Title)), - toInt16(GoodEvent), - EventTime, - EventDate, - toInt32(CounterID), - toInt32(ClientIP), - toInt32(RegionID), - toInt64(UserID), - toInt8(CounterClass), - toInt8(OS), - toInt8(UserAgent), - toValidUTF8(toString(URL)), - toValidUTF8(toString(Referer)), - toInt8(Refresh), - toInt16(RefererCategoryID), - toInt32(RefererRegionID), - toInt16(URLCategoryID), - toInt32(URLRegionID), - toInt16(ResolutionWidth), - toInt16(ResolutionHeight), - toInt8(ResolutionDepth), - toInt8(FlashMajor), - toInt8(FlashMinor), - toValidUTF8(toString(FlashMinor2)), - toInt8(NetMajor), - toInt8(NetMinor), - toInt16(UserAgentMajor), - toValidUTF8(toString(UserAgentMinor)), - toInt8(CookieEnable), - toInt8(JavascriptEnable), - toInt8(IsMobile), - toInt8(MobilePhone), - toValidUTF8(toString(MobilePhoneModel)), - toValidUTF8(toString(Params)), - toInt32(IPNetworkID), - toInt8(TraficSourceID), - toInt16(SearchEngineID), - toValidUTF8(toString(SearchPhrase)), - toInt8(AdvEngineID), - toInt8(IsArtifical), - toInt16(WindowClientWidth), - toInt16(WindowClientHeight), - toInt16(ClientTimeZone), - ClientEventTime, - toInt8(SilverlightVersion1), - toInt8(SilverlightVersion2), - toInt32(SilverlightVersion3), - toInt16(SilverlightVersion4), - toValidUTF8(toString(PageCharset)), - toInt32(CodeVersion), - toInt8(IsLink), - toInt8(IsDownload), - toInt8(IsNotBounce), - toInt64(FUniqID), - toValidUTF8(toString(OriginalURL)), - toInt32(HID), - toInt8(IsOldCounter), - toInt8(IsEvent), - toInt8(IsParameter), - toInt8(DontCountHits), - toInt8(WithHash), - toValidUTF8(toString(HitColor)), - LocalEventTime, - toInt8(Age), - toInt8(Sex), - toInt8(Income), - toInt16(Interests), - toInt8(Robotness), - toInt32(RemoteIP), - toInt32(WindowName), - toInt32(OpenerName), - toInt16(HistoryLength), - toValidUTF8(toString(BrowserLanguage)), - toValidUTF8(toString(BrowserCountry)), - toValidUTF8(toString(SocialNetwork)), - toValidUTF8(toString(SocialAction)), - toInt16(HTTPError), - toInt32(SendTiming), - toInt32(DNSTiming), - toInt32(ConnectTiming), - toInt32(ResponseStartTiming), - toInt32(ResponseEndTiming), - toInt32(FetchTiming), - toInt8(SocialSourceNetworkID), - toValidUTF8(toString(SocialSourcePage)), - toInt64(ParamPrice), - toValidUTF8(toString(ParamOrderID)), - toValidUTF8(toString(ParamCurrency)), - 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), - toInt64(RefererHash), - toInt64(URLHash), - toInt32(CLID) -FROM hits_100m_obfuscated -INTO OUTFILE '/home/milovidov/example_datasets/hits_100m_obfuscated.csv' -FORMAT CSV; -``` - -Upload data to OmniSci: -``` -/opt/omnisci/bin/omnisql -t -p HyperInteractive -``` -Run CREATE TABLE statement, then run: -``` -COPY hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated.csv' WITH (HEADER = 'false'); -``` - -Data loading took -``` -336639 ms -``` -on a server (Linux Ubuntu, Xeon E5-2560v2, 32 logical CPU, 128 GiB RAM, 8xHDD RAID-5, 40 TB). - -Run benchmark: - -``` -./benchmark.sh -``` - -Prepare the result to paste into JSON: - -``` -grep -oP 'Total time: \d+' log.txt | - grep -oP '\d+' | - awk '{ - if (i % 3 == 0) { a = $1 } - else if (i % 3 == 1) { b = $1 } - else if (i % 3 == 2) { c = $1; print "[" a / 1000 ", " b / 1000 ", " c / 1000 "]," }; - ++i; }' -``` - -And fill out `[null, null, null]` for missing runs. diff --git a/benchmark/omnisci/log.txt b/benchmark/omnisci/log.txt deleted file mode 100644 index 986f00d2096..00000000000 --- a/benchmark/omnisci/log.txt +++ /dev/null @@ -1,210 +0,0 @@ -3 -SELECT count(*) FROM hits; -Execution time: 23471 ms, Total time: 23471 ms -Execution time: 42 ms, Total time: 43 ms -Execution time: 35 ms, Total time: 35 ms -3 -SELECT count(*) FROM hits WHERE AdvEngineID != 0; -Execution time: 17328 ms, Total time: 17329 ms -Execution time: 58 ms, Total time: 59 ms -Execution time: 57 ms, Total time: 59 ms -3 -SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits; -Execution time: 17309 ms, Total time: 17310 ms -Execution time: 115 ms, Total time: 115 ms -Execution time: 129 ms, Total time: 130 ms -3 -SELECT sum(UserID) FROM hits; -Execution time: 26091 ms, Total time: 26091 ms -Execution time: 88 ms, Total time: 89 ms -Execution time: 71 ms, Total time: 72 ms -3 -SELECT APPROX_COUNT_DISTINCT(UserID) FROM hits; -Execution time: 21720 ms, Total time: 21720 ms -Execution time: 364 ms, Total time: 364 ms -Execution time: 344 ms, Total time: 345 ms -3 -SELECT APPROX_COUNT_DISTINCT(SearchPhrase) FROM hits; -Execution time: 19314 ms, Total time: 19315 ms -Execution time: 385 ms, Total time: 386 ms -Execution time: 382 ms, Total time: 382 ms -3 -SELECT min(EventDate), max(EventDate) FROM hits; -Execution time: 19431 ms, Total time: 19432 ms -Execution time: 130 ms, Total time: 131 ms -Execution time: 147 ms, Total time: 148 ms -3 -SELECT AdvEngineID, count(*) FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -Execution time: 20660 ms, Total time: 20661 ms -Execution time: 63 ms, Total time: 64 ms -Execution time: 88 ms, Total time: 89 ms -3 -SELECT RegionID, APPROX_COUNT_DISTINCT(UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -Execution time: 21364 ms, Total time: 21472 ms -Execution time: 1387 ms, Total time: 1504 ms -Execution time: 1443 ms, Total time: 1505 ms -3 -SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), APPROX_COUNT_DISTINCT(UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -Execution time: 22205 ms, Total time: 22285 ms -Execution time: 1590 ms, Total time: 1655 ms -Execution time: 1591 ms, Total time: 1658 ms -3 -SELECT MobilePhoneModel, APPROX_COUNT_DISTINCT(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -Execution time: 22343 ms, Total time: 22344 ms -Execution time: 122 ms, Total time: 123 ms -Execution time: 117 ms, Total time: 118 ms -3 -SELECT MobilePhone, MobilePhoneModel, APPROX_COUNT_DISTINCT(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -Execution time: 21681 ms, Total time: 21695 ms -Execution time: 299 ms, Total time: 310 ms -Execution time: 275 ms, Total time: 292 ms -3 -SELECT SearchPhrase, count(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -Execution time: 23346 ms, Total time: 23360 ms -Execution time: 613 ms, Total time: 631 ms -Execution time: 606 ms, Total time: 624 ms -3 -SELECT SearchPhrase, APPROX_COUNT_DISTINCT(UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -Execution time: 66014 ms, Total time: 68618 ms -Execution time: 44309 ms, Total time: 47296 ms -Execution time: 44019 ms, Total time: 46866 ms -3 -SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -Execution time: 25853 ms, Total time: 25984 ms -Execution time: 2590 ms, Total time: 2728 ms -Execution time: 2652 ms, Total time: 2789 ms -3 -SELECT UserID, count(*) FROM hits GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -Execution time: 26581 ms, Total time: 26953 ms -Execution time: 5843 ms, Total time: 6158 ms -Execution time: 5970 ms, Total time: 6286 ms -3 -SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -Execution time: 33007 ms, Total time: 33581 ms -Execution time: 9943 ms, Total time: 10509 ms -Execution time: 9470 ms, Total time: 10047 ms -3 -SELECT UserID, SearchPhrase, count(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -Execution time: 39009 ms, Total time: 39575 ms -Execution time: 8151 ms, Total time: 8785 ms -Execution time: 8037 ms, Total time: 8665 ms -3 -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, count(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -Execution time: 56207 ms, Total time: 57764 ms -Execution time: 26653 ms, Total time: 28199 ms -Execution time: 25614 ms, Total time: 27336 ms -3 -SELECT UserID FROM hits WHERE UserID = -6101065172474983726; -Execution time: 18975 ms, Total time: 18976 ms -Execution time: 136 ms, Total time: 136 ms -Execution time: 136 ms, Total time: 136 ms -3 -SELECT count(*) FROM hits WHERE URL LIKE '%metrika%'; -Execution time: 32444 ms, Total time: 32445 ms -Execution time: 125 ms, Total time: 126 ms -Execution time: 134 ms, Total time: 136 ms -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; -Exception: Aggregate on TEXT is not supported yet. -Exception: Aggregate on TEXT is not supported yet. -Exception: Aggregate on TEXT is not supported yet. -3 -SELECT SearchPhrase, min(URL), min(Title), count(*) AS c, APPROX_COUNT_DISTINCT(UserID) FROM hits WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -Exception: Aggregate on TEXT is not supported yet. -Exception: Aggregate on TEXT is not supported yet. -Exception: Aggregate on TEXT is not supported yet. -3 -SELECT * FROM hits WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -Execution time: 96163 ms, Total time: 96166 ms -Execution time: 312 ms, Total time: 314 ms -Execution time: 303 ms, Total time: 305 ms -3 -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -Execution time: 27493 ms, Total time: 27494 ms -Execution time: 216 ms, Total time: 216 ms -Execution time: 221 ms, Total time: 222 ms -3 -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -Execution time: 38230 ms, Total time: 38308 ms -Execution time: 17175 ms, Total time: 17256 ms -Execution time: 17225 ms, Total time: 17310 ms -3 -SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -Execution time: 115614 ms, Total time: 115714 ms -Execution time: 95944 ms, Total time: 96041 ms -Execution time: 94274 ms, Total time: 94383 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; -Execution time: 31775 ms, Total time: 31779 ms -Execution time: 2643 ms, Total time: 2647 ms -Execution time: 2933 ms, Total time: 2937 ms -3 -SELECT domainWithoutWWW(Referer) AS key, avg(length(Referer)) AS l, count(*) AS c, min(Referer) FROM hits WHERE Referer != '' GROUP BY key HAVING c > 100000 ORDER BY l DESC LIMIT 25; -Exception: Exception occurred: org.apache.calcite.runtime.CalciteContextException: From line 1, column 8 to line 1, column 36: No match found for function signature domainWithoutWWW() -Exception: Exception occurred: org.apache.calcite.runtime.CalciteContextException: From line 1, column 8 to line 1, column 36: No match found for function signature domainWithoutWWW() -Exception: Exception occurred: org.apache.calcite.runtime.CalciteContextException: From line 1, column 8 to line 1, column 36: No match found for function signature domainWithoutWWW() -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; -Execution time: 28853 ms, Total time: 28854 ms -Execution time: 5654 ms, Total time: 5655 ms -Execution time: 5579 ms, Total time: 5581 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; -Execution time: 31694 ms, Total time: 31925 ms -Execution time: 3872 ms, Total time: 4142 ms -Execution time: 3928 ms, Total time: 4162 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; -Execution time: 43690 ms, Total time: 44297 ms -Execution time: 8221 ms, Total time: 8825 ms -Execution time: 8115 ms, Total time: 8711 ms -3 -SELECT URL, count(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -Execution time: 29669 ms, Total time: 29715 ms -Execution time: 1623 ms, Total time: 1669 ms -Execution time: 1534 ms, Total time: 1586 ms -3 -SELECT 1, URL, count(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -Execution time: 34860 ms, Total time: 35201 ms -Execution time: 7075 ms, Total time: 7414 ms -Execution time: 7164 ms, Total time: 7567 ms -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; -Execution time: 26467 ms, Total time: 26724 ms -Execution time: 5740 ms, Total time: 6026 ms -Execution time: 5667 ms, Total time: 5920 ms -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; -Execution time: 31899 ms, Total time: 31908 ms -Execution time: 1141 ms, Total time: 1154 ms -Execution time: 1155 ms, Total time: 1168 ms -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; -Execution time: 27991 ms, Total time: 27997 ms -Execution time: 719 ms, Total time: 724 ms -Execution time: 737 ms, Total time: 744 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; -Execution time: 34651 ms, Total time: 34661 ms -Execution time: 1182 ms, Total time: 1200 ms -Execution time: 1142 ms, Total time: 1159 ms -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, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -Execution time: 30130 ms, Total time: 30136 ms -Execution time: 461 ms, Total time: 467 ms -Execution time: 445 ms, Total time: 451 ms -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; -Execution time: 19989 ms, Total time: 19991 ms -Execution time: 326 ms, Total time: 327 ms -Execution time: 325 ms, Total time: 326 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; -Execution time: 18658 ms, Total time: 18660 ms -Execution time: 265 ms, Total time: 266 ms -Execution time: 254 ms, Total time: 255 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); -Execution time: 25225 ms, Total time: 25227 ms -Execution time: 210 ms, Total time: 212 ms -Execution time: 199 ms, Total time: 200 ms diff --git a/benchmark/omnisci/queries.sql b/benchmark/omnisci/queries.sql deleted file mode 100644 index 342de72db8a..00000000000 --- a/benchmark/omnisci/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -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 APPROX_COUNT_DISTINCT(UserID) FROM {table}; -SELECT APPROX_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, APPROX_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), APPROX_COUNT_DISTINCT(UserID) FROM {table} GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, APPROX_COUNT_DISTINCT(UserID) AS u FROM {table} WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, APPROX_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, APPROX_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, APPROX_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 c > 100000 ORDER BY l DESC LIMIT 25; -SELECT domainWithoutWWW(Referer) AS key, avg(length(Referer)) AS l, count(*) AS c, min(Referer) FROM {table} WHERE Referer != '' GROUP BY key HAVING c > 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, Src, Dst 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/postgresql/benchmark.sh b/benchmark/postgresql/benchmark.sh index 4a7139ec79d..13459abfe8a 100755 --- a/benchmark/postgresql/benchmark.sh +++ b/benchmark/postgresql/benchmark.sh @@ -1,12 +1,23 @@ #!/bin/bash -grep -v -P '^#' queries.sql | sed -e 's/{table}/hits_100m_pg/' | while read query; do +sudo apt-get update +sudo apt-get install -y postgresql-common +sudo apt-get install -y postgresql-14 - echo 3 | sudo tee /proc/sys/vm/drop_caches +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz +chmod 777 ~ hits.tsv - echo "$query"; - for i in {1..3}; do - # For some reason JIT does not work on my machine - sudo -u postgres psql tutorial -t -c 'set jit = off' -c '\timing' -c "$query" | grep 'Time' | tee --append log - done; -done; +sudo -u postgres psql -t -c 'CREATE DATABASE test' +sudo -u postgres psql test -t < create.sql +sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" + +# COPY 99997497 +# Time: 2341543.463 ms (39:01.543) + +./run.sh 2>&1 | tee log.txt + +sudo du -bcs /var/lib/postgresql/14/main/ + +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/postgresql/create.sql b/benchmark/postgresql/create.sql similarity index 100% rename from benchmark/compatible/postgresql/create.sql rename to benchmark/postgresql/create.sql diff --git a/benchmark/postgresql/instructions.md b/benchmark/postgresql/instructions.md deleted file mode 100644 index 296fa377bd1..00000000000 --- a/benchmark/postgresql/instructions.md +++ /dev/null @@ -1,142 +0,0 @@ -Create a table in PostgreSQL: - -``` -CREATE TABLE hits_100m_pg -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - Refresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor CHAR(2) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -); -``` - -Create a dump from ClickHouse: - -``` -SELECT WatchID::Int64, JavaEnable, replaceAll(replaceAll(replaceAll(toValidUTF8(Title), '\0', ''), '"', ''), '\\', ''), GoodEvent, EventTime, EventDate, CounterID::Int32, ClientIP::Int32, RegionID::Int32, - UserID::Int64, CounterClass, OS, UserAgent, replaceAll(replaceAll(replaceAll(toValidUTF8(URL), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(Referer), '\0', ''), '"', ''), '\\', ''), Refresh, RefererCategoryID::Int16, RefererRegionID::Int32, - URLCategoryID::Int16, URLRegionID::Int32, ResolutionWidth::Int16, ResolutionHeight::Int16, ResolutionDepth, FlashMajor, FlashMinor, - FlashMinor2, NetMajor, NetMinor, UserAgentMajor::Int16, replaceAll(replaceAll(replaceAll(toValidUTF8(UserAgentMinor::String), '\0', ''), '"', ''), '\\', ''), CookieEnable, JavascriptEnable, IsMobile, MobilePhone, - replaceAll(replaceAll(replaceAll(toValidUTF8(MobilePhoneModel), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(Params), '\0', ''), '"', ''), '\\', ''), IPNetworkID::Int32, TraficSourceID, SearchEngineID::Int16, replaceAll(replaceAll(replaceAll(toValidUTF8(SearchPhrase), '\0', ''), '"', ''), '\\', ''), - AdvEngineID, IsArtifical, WindowClientWidth::Int16, WindowClientHeight::Int16, ClientTimeZone, ClientEventTime, - SilverlightVersion1, SilverlightVersion2, SilverlightVersion3::Int32, SilverlightVersion4::Int16, replaceAll(replaceAll(replaceAll(toValidUTF8(PageCharset), '\0', ''), '"', ''), '\\', ''), - CodeVersion::Int32, IsLink, IsDownload, IsNotBounce, FUniqID::Int64, replaceAll(replaceAll(replaceAll(toValidUTF8(OriginalURL), '\0', ''), '"', ''), '\\', ''), HID::Int32, IsOldCounter, IsEvent, - IsParameter, DontCountHits, WithHash, replaceAll(replaceAll(replaceAll(toValidUTF8(HitColor::String), '\0', ''), '"', ''), '\\', ''), LocalEventTime, Age, Sex, Income, Interests::Int16, Robotness, RemoteIP::Int32, - WindowName, OpenerName, HistoryLength, replaceAll(replaceAll(replaceAll(toValidUTF8(BrowserLanguage::String), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(BrowserCountry::String), '\0', ''), '"', ''), '\\', ''), - replaceAll(replaceAll(replaceAll(toValidUTF8(SocialNetwork), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(SocialAction), '\0', ''), '"', ''), '\\', ''), - HTTPError, least(SendTiming, 30000), least(DNSTiming, 30000), least(ConnectTiming, 30000), least(ResponseStartTiming, 30000), - least(ResponseEndTiming, 30000), least(FetchTiming, 30000), SocialSourceNetworkID, - replaceAll(replaceAll(replaceAll(toValidUTF8(SocialSourcePage), '\0', ''), '"', ''), '\\', ''), ParamPrice, replaceAll(replaceAll(replaceAll(toValidUTF8(ParamOrderID), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(ParamCurrency::String), '\0', ''), '"', ''), '\\', ''), - ParamCurrencyID::Int16, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, - UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash::Int64, URLHash::Int64, CLID::Int32 -FROM hits_100m_obfuscated -INTO OUTFILE 'dump.tsv' -FORMAT TSV -``` - -Insert data into PostgreSQL: - -``` -\copy hits_100m_pg FROM 'dump.tsv'; -``` diff --git a/benchmark/postgresql/log b/benchmark/postgresql/log deleted file mode 100644 index 6a95561e6da..00000000000 --- a/benchmark/postgresql/log +++ /dev/null @@ -1,129 +0,0 @@ -Time: 122020.258 ms (02:02.020) -Time: 5060.281 ms (00:05.060) -Time: 5052.692 ms (00:05.053) -Time: 129594.172 ms (02:09.594) -Time: 8079.623 ms (00:08.080) -Time: 7866.964 ms (00:07.867) -Time: 129584.717 ms (02:09.585) -Time: 8276.161 ms (00:08.276) -Time: 8153.295 ms (00:08.153) -Time: 123707.890 ms (02:03.708) -Time: 6835.297 ms (00:06.835) -Time: 6607.039 ms (00:06.607) -Time: 166640.676 ms (02:46.641) -Time: 75401.239 ms (01:15.401) -Time: 73526.027 ms (01:13.526) -Time: 272715.750 ms (04:32.716) -Time: 182721.613 ms (03:02.722) -Time: 182880.525 ms (03:02.881) -Time: 127108.191 ms (02:07.108) -Time: 6542.913 ms (00:06.543) -Time: 6339.887 ms (00:06.340) -Time: 127339.314 ms (02:07.339) -Time: 8376.381 ms (00:08.376) -Time: 7831.872 ms (00:07.832) -Time: 179176.439 ms (02:59.176) -Time: 58559.297 ms (00:58.559) -Time: 58139.265 ms (00:58.139) -Time: 182019.101 ms (03:02.019) -Time: 58435.027 ms (00:58.435) -Time: 58130.994 ms (00:58.131) -Time: 132449.502 ms (02:12.450) -Time: 11203.104 ms (00:11.203) -Time: 11048.435 ms (00:11.048) -Time: 128445.641 ms (02:08.446) -Time: 11602.145 ms (00:11.602) -Time: 11418.356 ms (00:11.418) -Time: 162831.387 ms (02:42.831) -Time: 41510.710 ms (00:41.511) -Time: 41682.899 ms (00:41.683) -Time: 171898.965 ms (02:51.899) -Time: 47379.274 ms (00:47.379) -Time: 47429.908 ms (00:47.430) -Time: 161607.811 ms (02:41.608) -Time: 41674.409 ms (00:41.674) -Time: 40854.340 ms (00:40.854) -Time: 175247.929 ms (02:55.248) -Time: 46721.776 ms (00:46.722) -Time: 46507.631 ms (00:46.508) -Time: 335961.271 ms (05:35.961) -Time: 248535.866 ms (04:08.536) -Time: 247383.678 ms (04:07.384) -Time: 132852.983 ms (02:12.853) -Time: 14939.304 ms (00:14.939) -Time: 14607.525 ms (00:14.608) -Time: 243461.844 ms (04:03.462) -Time: 157307.904 ms (02:37.308) -Time: 155093.101 ms (02:35.093) -Time: 122090.761 ms (02:02.091) -Time: 6411.266 ms (00:06.411) -Time: 6308.178 ms (00:06.308) -Time: 126584.819 ms (02:06.585) -Time: 8836.471 ms (00:08.836) -Time: 8532.176 ms (00:08.532) -Time: 125225.097 ms (02:05.225) -Time: 10236.910 ms (00:10.237) -Time: 9849.757 ms (00:09.850) -Time: 139140.064 ms (02:19.140) -Time: 21797.859 ms (00:21.798) -Time: 21559.214 ms (00:21.559) -Time: 124757.485 ms (02:04.757) -Time: 8728.403 ms (00:08.728) -Time: 8714.130 ms (00:08.714) -Time: 120687.258 ms (02:00.687) -Time: 8366.245 ms (00:08.366) -Time: 8146.856 ms (00:08.147) -Time: 122327.148 ms (02:02.327) -Time: 8698.359 ms (00:08.698) -Time: 8480.807 ms (00:08.481) -Time: 123958.614 ms (02:03.959) -Time: 8595.931 ms (00:08.596) -Time: 8241.773 ms (00:08.242) -Time: 128982.905 ms (02:08.983) -Time: 11252.783 ms (00:11.253) -Time: 10957.931 ms (00:10.958) -Time: 208455.385 ms (03:28.455) -Time: 102530.897 ms (01:42.531) -Time: 102049.298 ms (01:42.049) -Time: 131268.420 ms (02:11.268) -Time: 21094.466 ms (00:21.094) -Time: 20934.610 ms (00:20.935) -Time: 164084.134 ms (02:44.084) -Time: 77418.547 ms (01:17.419) -Time: 75422.290 ms (01:15.422) -Time: 174800.022 ms (02:54.800) -Time: 87859.594 ms (01:27.860) -Time: 85733.954 ms (01:25.734) -Time: 419357.463 ms (06:59.357) -Time: 339047.269 ms (05:39.047) -Time: 334808.230 ms (05:34.808) -Time: 475011.901 ms (07:55.012) -Time: 344406.246 ms (05:44.406) -Time: 347197.731 ms (05:47.198) -Time: 464657.732 ms (07:44.658) -Time: 332084.079 ms (05:32.084) -Time: 330921.322 ms (05:30.921) -Time: 152490.615 ms (02:32.491) -Time: 30954.343 ms (00:30.954) -Time: 31379.062 ms (00:31.379) -Time: 128539.127 ms (02:08.539) -Time: 12802.672 ms (00:12.803) -Time: 12494.088 ms (00:12.494) -Time: 125850.120 ms (02:05.850) -Time: 10318.773 ms (00:10.319) -Time: 9953.030 ms (00:09.953) -Time: 126602.092 ms (02:06.602) -Time: 8935.571 ms (00:08.936) -Time: 8711.184 ms (00:08.711) -Time: 133222.456 ms (02:13.222) -Time: 11848.869 ms (00:11.849) -Time: 11752.640 ms (00:11.753) -Time: 126950.067 ms (02:06.950) -Time: 11260.892 ms (00:11.261) -Time: 10943.649 ms (00:10.944) -Time: 128451.171 ms (02:08.451) -Time: 10984.980 ms (00:10.985) -Time: 10770.609 ms (00:10.771) -Time: 124621.000 ms (02:04.621) -Time: 8885.466 ms (00:08.885) -Time: 8857.296 ms (00:08.857) diff --git a/benchmark/postgresql/queries.sql b/benchmark/postgresql/queries.sql index d7a2fe2d8bf..31f65fc898d 100644 --- a/benchmark/postgresql/queries.sql +++ b/benchmark/postgresql/queries.sql @@ -1,43 +1,43 @@ -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 REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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, Src, Dst 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); +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/postgresql/results/c6a.4xlarge.txt b/benchmark/postgresql/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/postgresql/results/c6a.4xlarge.txt rename to benchmark/postgresql/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/postgresql/run.sh b/benchmark/postgresql/run.sh similarity index 100% rename from benchmark/compatible/postgresql/run.sh rename to benchmark/postgresql/run.sh diff --git a/benchmark/compatible/questdb/benchmark.sh b/benchmark/questdb/benchmark.sh similarity index 100% rename from benchmark/compatible/questdb/benchmark.sh rename to benchmark/questdb/benchmark.sh diff --git a/benchmark/compatible/questdb/create.sql b/benchmark/questdb/create.sql similarity index 100% rename from benchmark/compatible/questdb/create.sql rename to benchmark/questdb/create.sql diff --git a/benchmark/compatible/questdb/queries.sql b/benchmark/questdb/queries.sql similarity index 100% rename from benchmark/compatible/questdb/queries.sql rename to benchmark/questdb/queries.sql diff --git a/benchmark/compatible/questdb/result/c6a.4xlarge.txt b/benchmark/questdb/result/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/questdb/result/c6a.4xlarge.txt rename to benchmark/questdb/result/c6a.4xlarge.txt diff --git a/benchmark/compatible/questdb/run.sh b/benchmark/questdb/run.sh similarity index 100% rename from benchmark/compatible/questdb/run.sh rename to benchmark/questdb/run.sh diff --git a/benchmark/compatible/redshift-serverless/README.md b/benchmark/redshift-serverless/README.md similarity index 100% rename from benchmark/compatible/redshift-serverless/README.md rename to benchmark/redshift-serverless/README.md diff --git a/benchmark/compatible/redshift-serverless/create.sql b/benchmark/redshift-serverless/create.sql similarity index 100% rename from benchmark/compatible/redshift-serverless/create.sql rename to benchmark/redshift-serverless/create.sql diff --git a/benchmark/compatible/redshift-serverless/queries.sql b/benchmark/redshift-serverless/queries.sql similarity index 100% rename from benchmark/compatible/redshift-serverless/queries.sql rename to benchmark/redshift-serverless/queries.sql diff --git a/benchmark/compatible/redshift-serverless/result/serverless.txt b/benchmark/redshift-serverless/result/serverless.txt similarity index 100% rename from benchmark/compatible/redshift-serverless/result/serverless.txt rename to benchmark/redshift-serverless/result/serverless.txt diff --git a/benchmark/compatible/redshift-serverless/run.sh b/benchmark/redshift-serverless/run.sh similarity index 100% rename from benchmark/compatible/redshift-serverless/run.sh rename to benchmark/redshift-serverless/run.sh diff --git a/benchmark/compatible/redshift/README.md b/benchmark/redshift/README.md similarity index 100% rename from benchmark/compatible/redshift/README.md rename to benchmark/redshift/README.md diff --git a/benchmark/compatible/redshift/create.sql b/benchmark/redshift/create.sql similarity index 100% rename from benchmark/compatible/redshift/create.sql rename to benchmark/redshift/create.sql diff --git a/benchmark/compatible/redshift/queries.sql b/benchmark/redshift/queries.sql similarity index 100% rename from benchmark/compatible/redshift/queries.sql rename to benchmark/redshift/queries.sql diff --git a/benchmark/compatible/redshift/result/4x.ra3.xplus.txt b/benchmark/redshift/result/4x.ra3.xplus.txt similarity index 100% rename from benchmark/compatible/redshift/result/4x.ra3.xplus.txt rename to benchmark/redshift/result/4x.ra3.xplus.txt diff --git a/benchmark/compatible/redshift/run.sh b/benchmark/redshift/run.sh similarity index 100% rename from benchmark/compatible/redshift/run.sh rename to benchmark/redshift/run.sh diff --git a/benchmark/compatible/singlestore/benchmark.sh b/benchmark/singlestore/benchmark.sh similarity index 100% rename from benchmark/compatible/singlestore/benchmark.sh rename to benchmark/singlestore/benchmark.sh diff --git a/benchmark/compatible/singlestore/create.sql b/benchmark/singlestore/create.sql similarity index 100% rename from benchmark/compatible/singlestore/create.sql rename to benchmark/singlestore/create.sql diff --git a/benchmark/compatible/greenplum/queries.sql b/benchmark/singlestore/queries.sql similarity index 100% rename from benchmark/compatible/greenplum/queries.sql rename to benchmark/singlestore/queries.sql diff --git a/benchmark/compatible/singlestore/results/c6a.4xlarge.txt b/benchmark/singlestore/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/singlestore/results/c6a.4xlarge.txt rename to benchmark/singlestore/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/singlestore/run.sh b/benchmark/singlestore/run.sh similarity index 100% rename from benchmark/compatible/singlestore/run.sh rename to benchmark/singlestore/run.sh diff --git a/benchmark/compatible/snowflake/NOTES.md b/benchmark/snowflake/NOTES.md similarity index 100% rename from benchmark/compatible/snowflake/NOTES.md rename to benchmark/snowflake/NOTES.md diff --git a/benchmark/compatible/snowflake/README.md b/benchmark/snowflake/README.md similarity index 100% rename from benchmark/compatible/snowflake/README.md rename to benchmark/snowflake/README.md diff --git a/benchmark/compatible/snowflake/create.sql b/benchmark/snowflake/create.sql similarity index 100% rename from benchmark/compatible/snowflake/create.sql rename to benchmark/snowflake/create.sql diff --git a/benchmark/compatible/snowflake/queries.sql b/benchmark/snowflake/queries.sql similarity index 100% rename from benchmark/compatible/snowflake/queries.sql rename to benchmark/snowflake/queries.sql diff --git a/benchmark/compatible/snowflake/results/2xl.txt b/benchmark/snowflake/results/2xl.txt similarity index 100% rename from benchmark/compatible/snowflake/results/2xl.txt rename to benchmark/snowflake/results/2xl.txt diff --git a/benchmark/compatible/snowflake/results/3xl.txt b/benchmark/snowflake/results/3xl.txt similarity index 100% rename from benchmark/compatible/snowflake/results/3xl.txt rename to benchmark/snowflake/results/3xl.txt diff --git a/benchmark/compatible/snowflake/results/4xl.txt b/benchmark/snowflake/results/4xl.txt similarity index 100% rename from benchmark/compatible/snowflake/results/4xl.txt rename to benchmark/snowflake/results/4xl.txt diff --git a/benchmark/compatible/snowflake/results/l.txt b/benchmark/snowflake/results/l.txt similarity index 100% rename from benchmark/compatible/snowflake/results/l.txt rename to benchmark/snowflake/results/l.txt diff --git a/benchmark/compatible/snowflake/results/m.txt b/benchmark/snowflake/results/m.txt similarity index 100% rename from benchmark/compatible/snowflake/results/m.txt rename to benchmark/snowflake/results/m.txt diff --git a/benchmark/compatible/snowflake/results/s.txt b/benchmark/snowflake/results/s.txt similarity index 100% rename from benchmark/compatible/snowflake/results/s.txt rename to benchmark/snowflake/results/s.txt diff --git a/benchmark/compatible/snowflake/results/xl.txt b/benchmark/snowflake/results/xl.txt similarity index 100% rename from benchmark/compatible/snowflake/results/xl.txt rename to benchmark/snowflake/results/xl.txt diff --git a/benchmark/compatible/snowflake/results/xs.txt b/benchmark/snowflake/results/xs.txt similarity index 100% rename from benchmark/compatible/snowflake/results/xs.txt rename to benchmark/snowflake/results/xs.txt diff --git a/benchmark/compatible/snowflake/run.sh b/benchmark/snowflake/run.sh similarity index 100% rename from benchmark/compatible/snowflake/run.sh rename to benchmark/snowflake/run.sh diff --git a/benchmark/compatible/sqlite/benchmark.sh b/benchmark/sqlite/benchmark.sh similarity index 100% rename from benchmark/compatible/sqlite/benchmark.sh rename to benchmark/sqlite/benchmark.sh diff --git a/benchmark/compatible/sqlite/create.sql b/benchmark/sqlite/create.sql similarity index 100% rename from benchmark/compatible/sqlite/create.sql rename to benchmark/sqlite/create.sql diff --git a/benchmark/compatible/sqlite/queries.sql b/benchmark/sqlite/queries.sql similarity index 100% rename from benchmark/compatible/sqlite/queries.sql rename to benchmark/sqlite/queries.sql diff --git a/benchmark/compatible/sqlite/results/c6a.4xlarge.txt b/benchmark/sqlite/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/sqlite/results/c6a.4xlarge.txt rename to benchmark/sqlite/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/sqlite/run.sh b/benchmark/sqlite/run.sh similarity index 100% rename from benchmark/compatible/sqlite/run.sh rename to benchmark/sqlite/run.sh diff --git a/benchmark/compatible/starrocks/README.md b/benchmark/starrocks/README.md similarity index 100% rename from benchmark/compatible/starrocks/README.md rename to benchmark/starrocks/README.md diff --git a/benchmark/compatible/starrocks/benchmark.sh b/benchmark/starrocks/benchmark.sh similarity index 100% rename from benchmark/compatible/starrocks/benchmark.sh rename to benchmark/starrocks/benchmark.sh diff --git a/benchmark/timescaledb/benchmark.sh b/benchmark/timescaledb/benchmark.sh index 691f4c42bfa..2330deaf1b4 100755 --- a/benchmark/timescaledb/benchmark.sh +++ b/benchmark/timescaledb/benchmark.sh @@ -1,11 +1,47 @@ #!/bin/bash -grep -v -P '^#' queries.sql | sed -e 's/{table}/hits_100m_obfuscated/' | while read query; do +# Install - echo 3 | sudo tee /proc/sys/vm/drop_caches +sudo apt-get update +sudo apt-get install -y gnupg postgresql-common apt-transport-https lsb-release wget +sudo /usr/share/postgresql-common/pgdg/apt.postgresql.org.sh +sudo bash -c 'echo "deb https://packagecloud.io/timescale/timescaledb/ubuntu/ $(lsb_release -c -s) main" > /etc/apt/sources.list.d/timescaledb.list' +wget --quiet -O - https://packagecloud.io/timescale/timescaledb/gpgkey | sudo apt-key add - +sudo apt-get update +sudo apt-get install -y timescaledb-2-postgresql-14 +sudo bash -c "echo \"shared_preload_libraries = 'timescaledb'\" >> /etc/postgresql/14/main/postgresql.conf" +sudo systemctl restart postgresql - echo "$query"; - for i in {1..3}; do - sudo -u postgres psql tutorial -t -c 'set jit = off' -c '\timing' -c "$query" | grep 'Time' | tee --append log - done; -done; +sudo -u postgres psql -c "CREATE DATABASE test" +sudo -u postgres psql test -c "CREATE EXTENSION IF NOT EXISTS timescaledb" + +# Import the data + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz +chmod 777 ~ hits.tsv + +sudo -u postgres psql test < create.sql +sudo -u postgres psql test -c "SELECT create_hypertable('hits', 'eventtime')" +sudo -u postgres psql test -c "CREATE INDEX ix_counterid ON hits (counterid)" +sudo -u postgres psql test -c "ALTER TABLE hits SET (timescaledb.compress, timescaledb.compress_orderby = 'counterid, eventdate, userid, eventtime')" +sudo -u postgres psql test -c "SELECT add_compression_policy('hits', INTERVAL '1s')" + +sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" + +# 1619875.288 ms (26:59.875) + +# See https://github.com/timescale/timescaledb/issues/4473#issuecomment-1167095245 +# https://docs.timescale.com/timescaledb/latest/how-to-guides/compression/manually-compress-chunks/#compress-chunks-manually +# Omit this step to proceed without compression. + +time sudo -u postgres psql test -c "SELECT compress_chunk(i, if_not_compressed => true) FROM show_chunks('hits') i" + +# 49m45.120s + +./run.sh 2>&1 | tee log.txt + +sudo du -bcs /var/lib/postgresql/14/main/ + +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/timescaledb/create.sql b/benchmark/timescaledb/create.sql similarity index 100% rename from benchmark/compatible/timescaledb/create.sql rename to benchmark/timescaledb/create.sql diff --git a/benchmark/timescaledb/log b/benchmark/timescaledb/log deleted file mode 100644 index 9c98da60612..00000000000 --- a/benchmark/timescaledb/log +++ /dev/null @@ -1,215 +0,0 @@ -3 -SELECT count(*) FROM hits_100m_obfuscated; -Time: 3259.733 ms (00:03.260) -Time: 3135.484 ms (00:03.135) -Time: 3135.579 ms (00:03.136) -3 -SELECT count(*) FROM hits_100m_obfuscated WHERE AdvEngineID != 0; -Time: 146854.557 ms (02:26.855) -Time: 6921.736 ms (00:06.922) -Time: 6619.892 ms (00:06.620) -3 -SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m_obfuscated; -Time: 146568.297 ms (02:26.568) -Time: 7481.610 ms (00:07.482) -Time: 7258.209 ms (00:07.258) -3 -SELECT sum(UserID) FROM hits_100m_obfuscated; -Time: 146864.106 ms (02:26.864) -Time: 5690.024 ms (00:05.690) -Time: 5381.820 ms (00:05.382) -3 -SELECT COUNT(DISTINCT UserID) FROM hits_100m_obfuscated; -Time: 227507.331 ms (03:47.507) -Time: 69165.471 ms (01:09.165) -Time: 72216.950 ms (01:12.217) -3 -SELECT COUNT(DISTINCT SearchPhrase) FROM hits_100m_obfuscated; -Time: 323644.397 ms (05:23.644) -Time: 177578.740 ms (02:57.579) -Time: 175055.738 ms (02:55.056) -3 -SELECT min(EventDate), max(EventDate) FROM hits_100m_obfuscated; -Time: 146147.843 ms (02:26.148) -Time: 5735.128 ms (00:05.735) -Time: 5428.638 ms (00:05.429) -3 -SELECT AdvEngineID, count(*) FROM hits_100m_obfuscated WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -Time: 148658.450 ms (02:28.658) -Time: 7014.882 ms (00:07.015) -Time: 6599.736 ms (00:06.600) -3 -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits_100m_obfuscated GROUP BY RegionID ORDER BY u DESC LIMIT 10; -Time: 202423.122 ms (03:22.423) -Time: 54439.047 ms (00:54.439) -Time: 54800.354 ms (00:54.800) -3 -SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits_100m_obfuscated GROUP BY RegionID ORDER BY c DESC LIMIT 10; -Time: 201152.491 ms (03:21.152) -Time: 55875.854 ms (00:55.876) -Time: 55200.330 ms (00:55.200) -3 -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits_100m_obfuscated WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -Time: 146042.603 ms (02:26.043) -Time: 9931.633 ms (00:09.932) -Time: 10037.032 ms (00:10.037) -3 -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits_100m_obfuscated WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -Time: 150811.952 ms (02:30.812) -Time: 10320.230 ms (00:10.320) -Time: 9993.232 ms (00:09.993) -3 -SELECT SearchPhrase, count(*) AS c FROM hits_100m_obfuscated WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -Time: 173071.218 ms (02:53.071) -Time: 34314.835 ms (00:34.315) -Time: 34420.919 ms (00:34.421) -3 -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits_100m_obfuscated WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -Time: 172874.155 ms (02:52.874) -Time: 43704.494 ms (00:43.704) -Time: 43918.380 ms (00:43.918) -3 -SELECT SearchEngineID, SearchPhrase, count(*) AS c FROM hits_100m_obfuscated WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -Time: 178484.822 ms (02:58.485) -Time: 36850.436 ms (00:36.850) -Time: 35789.029 ms (00:35.789) -3 -SELECT UserID, count(*) FROM hits_100m_obfuscated GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -Time: 169720.759 ms (02:49.721) -Time: 24125.730 ms (00:24.126) -Time: 23782.745 ms (00:23.783) -3 -SELECT UserID, SearchPhrase, count(*) FROM hits_100m_obfuscated GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -Time: 182335.631 ms (03:02.336) -Time: 37324.563 ms (00:37.325) -Time: 37124.250 ms (00:37.124) -3 -SELECT UserID, SearchPhrase, count(*) FROM hits_100m_obfuscated GROUP BY UserID, SearchPhrase LIMIT 10; -Time: 163799.714 ms (02:43.800) -Time: 18514.031 ms (00:18.514) -Time: 18968.524 ms (00:18.969) -3 -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, count(*) FROM hits_100m_obfuscated GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -Time: 294799.480 ms (04:54.799) -Time: 149592.992 ms (02:29.593) -Time: 149466.291 ms (02:29.466) -3 -SELECT UserID FROM hits_100m_obfuscated WHERE UserID = -6101065172474983726; -Time: 140797.496 ms (02:20.797) -Time: 5312.321 ms (00:05.312) -Time: 5020.502 ms (00:05.021) -3 -SELECT count(*) FROM hits_100m_obfuscated WHERE URL LIKE '%metrika%'; -Time: 143092.287 ms (02:23.092) -Time: 7893.874 ms (00:07.894) -Time: 7661.326 ms (00:07.661) -3 -SELECT SearchPhrase, min(URL), count(*) AS c FROM hits_100m_obfuscated WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -Time: 143682.424 ms (02:23.682) -Time: 9249.962 ms (00:09.250) -Time: 9073.876 ms (00:09.074) -3 -SELECT SearchPhrase, min(URL), min(Title), count(*) AS c, COUNT(DISTINCT UserID) FROM hits_100m_obfuscated WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -Time: 150965.884 ms (02:30.966) -Time: 20350.812 ms (00:20.351) -Time: 20074.939 ms (00:20.075) -3 -SELECT * FROM hits_100m_obfuscated WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -Time: 4674.669 ms (00:04.675) -Time: 4532.389 ms (00:04.532) -Time: 4555.457 ms (00:04.555) -3 -SELECT SearchPhrase FROM hits_100m_obfuscated WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; -Time: 5.177 ms -Time: 5.031 ms -Time: 4.419 ms -3 -SELECT SearchPhrase FROM hits_100m_obfuscated WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; -Time: 141152.210 ms (02:21.152) -Time: 7492.968 ms (00:07.493) -Time: 7300.428 ms (00:07.300) -3 -SELECT SearchPhrase FROM hits_100m_obfuscated WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; -Time: 30.736 ms -Time: 5.018 ms -Time: 5.132 ms -3 -SELECT CounterID, avg(length(URL)) AS l, count(*) AS c FROM hits_100m_obfuscated WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -Time: 144034.016 ms (02:24.034) -Time: 10701.672 ms (00:10.702) -Time: 10348.565 ms (00:10.349) -3 -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www.)?([^/]+)/.*$', '1') AS key, avg(length(Referer)) AS l, count(*) AS c, min(Referer) FROM hits_100m_obfuscated WHERE Referer != '' GROUP BY key HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -Time: 191575.080 ms (03:11.575) -Time: 97836.706 ms (01:37.837) -Time: 97673.219 ms (01:37.673) -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_100m_obfuscated; -Time: 143652.317 ms (02:23.652) -Time: 22185.656 ms (00:22.186) -Time: 21887.411 ms (00:21.887) -3 -SELECT SearchEngineID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits_100m_obfuscated WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -Time: 153481.944 ms (02:33.482) -Time: 17748.628 ms (00:17.749) -Time: 17551.116 ms (00:17.551) -3 -SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits_100m_obfuscated WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -Time: 167448.684 ms (02:47.449) -Time: 25902.961 ms (00:25.903) -Time: 25592.018 ms (00:25.592) -3 -SELECT WatchID, ClientIP, count(*) AS c, sum("refresh"), avg(ResolutionWidth) FROM hits_100m_obfuscated GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -Time: 299183.443 ms (04:59.183) -Time: 145349.772 ms (02:25.350) -Time: 143214.688 ms (02:23.215) -3 -SELECT URL, count(*) AS c FROM hits_100m_obfuscated GROUP BY URL ORDER BY c DESC LIMIT 10; -Time: 389851.369 ms (06:29.851) -Time: 228158.639 ms (03:48.159) -Time: 231811.118 ms (03:51.811) -3 -SELECT 1, URL, count(*) AS c FROM hits_100m_obfuscated GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -Time: 407458.343 ms (06:47.458) -Time: 230125.530 ms (03:50.126) -Time: 230764.511 ms (03:50.765) -3 -SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) AS c FROM hits_100m_obfuscated GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -Time: 174098.556 ms (02:54.099) -Time: 23503.975 ms (00:23.504) -Time: 24322.856 ms (00:24.323) -3 -SELECT URL, count(*) AS PageViews FROM hits_100m_obfuscated 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; -Time: 145906.025 ms (02:25.906) -Time: 10824.695 ms (00:10.825) -Time: 10484.885 ms (00:10.485) -3 -SELECT Title, count(*) AS PageViews FROM hits_100m_obfuscated 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; -Time: 144063.711 ms (02:24.064) -Time: 8947.980 ms (00:08.948) -Time: 8608.434 ms (00:08.608) -3 -SELECT URL, count(*) AS PageViews FROM hits_100m_obfuscated 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; -Time: 141883.596 ms (02:21.884) -Time: 7977.257 ms (00:07.977) -Time: 7673.547 ms (00:07.674) -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_100m_obfuscated WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND "refresh" = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -Time: 147100.084 ms (02:27.100) -Time: 9527.812 ms (00:09.528) -Time: 9457.663 ms (00:09.458) -3 -SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m_obfuscated 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; -Time: 144585.669 ms (02:24.586) -Time: 10815.223 ms (00:10.815) -Time: 10594.707 ms (00:10.595) -3 -SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m_obfuscated 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; -Time: 145738.341 ms (02:25.738) -Time: 10592.979 ms (00:10.593) -Time: 10181.477 ms (00:10.181) -3 -SELECT DATE_TRUNC('minute', EventTime) AS "Minute", count(*) AS PageViews FROM hits_100m_obfuscated 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); -Time: 145023.796 ms (02:25.024) -Time: 8035.337 ms (00:08.035) -Time: 7865.698 ms (00:07.866) diff --git a/benchmark/timescaledb/log_compressed b/benchmark/timescaledb/log_compressed deleted file mode 100644 index 235f659791f..00000000000 --- a/benchmark/timescaledb/log_compressed +++ /dev/null @@ -1,129 +0,0 @@ -Time: 1784.299 ms (00:01.784) -Time: 1223.461 ms (00:01.223) -Time: 1200.665 ms (00:01.201) -Time: 22730.141 ms (00:22.730) -Time: 1379.227 ms (00:01.379) -Time: 1361.595 ms (00:01.362) -Time: 29888.235 ms (00:29.888) -Time: 3160.611 ms (00:03.161) -Time: 3207.363 ms (00:03.207) -Time: 53922.569 ms (00:53.923) -Time: 2301.456 ms (00:02.301) -Time: 2277.009 ms (00:02.277) -Time: 45363.999 ms (00:45.364) -Time: 43765.848 ms (00:43.766) -Time: 44066.621 ms (00:44.067) -Time: 172945.633 ms (02:52.946) -Time: 136944.098 ms (02:16.944) -Time: 138268.413 ms (02:18.268) -Time: 16764.579 ms (00:16.765) -Time: 2579.907 ms (00:02.580) -Time: 2590.390 ms (00:02.590) -Time: 1498.034 ms (00:01.498) -Time: 1434.534 ms (00:01.435) -Time: 1448.123 ms (00:01.448) -Time: 113533.016 ms (01:53.533) -Time: 78465.335 ms (01:18.465) -Time: 80778.839 ms (01:20.779) -Time: 90456.388 ms (01:30.456) -Time: 87050.166 ms (01:27.050) -Time: 88426.851 ms (01:28.427) -Time: 45021.632 ms (00:45.022) -Time: 12486.342 ms (00:12.486) -Time: 12222.489 ms (00:12.222) -Time: 44246.843 ms (00:44.247) -Time: 15606.856 ms (00:15.607) -Time: 15251.554 ms (00:15.252) -Time: 29654.719 ms (00:29.655) -Time: 29441.858 ms (00:29.442) -Time: 29608.141 ms (00:29.608) -Time: 103547.383 ms (01:43.547) -Time: 104733.648 ms (01:44.734) -Time: 105779.016 ms (01:45.779) -Time: 29695.834 ms (00:29.696) -Time: 15395.447 ms (00:15.395) -Time: 15819.650 ms (00:15.820) -Time: 27841.552 ms (00:27.842) -Time: 29521.849 ms (00:29.522) -Time: 27508.521 ms (00:27.509) -Time: 56665.709 ms (00:56.666) -Time: 56459.321 ms (00:56.459) -Time: 56407.620 ms (00:56.408) -Time: 27488.888 ms (00:27.489) -Time: 25557.427 ms (00:25.557) -Time: 25634.140 ms (00:25.634) -Time: 97376.463 ms (01:37.376) -Time: 96047.902 ms (01:36.048) -Time: 99918.341 ms (01:39.918) -Time: 6294.887 ms (00:06.295) -Time: 6407.262 ms (00:06.407) -Time: 6376.369 ms (00:06.376) -Time: 40787.808 ms (00:40.788) -Time: 11206.256 ms (00:11.206) -Time: 11219.871 ms (00:11.220) -Time: 12420.227 ms (00:12.420) -Time: 12548.301 ms (00:12.548) -Time: 12468.458 ms (00:12.468) -Time: 57679.878 ms (00:57.680) -Time: 35466.123 ms (00:35.466) -Time: 35562.064 ms (00:35.562) -Time: 13551.276 ms (00:13.551) -Time: 13417.313 ms (00:13.417) -Time: 13645.287 ms (00:13.645) -Time: 150.297 ms -Time: 55.995 ms -Time: 55.796 ms -Time: 3059.796 ms (00:03.060) -Time: 3038.246 ms (00:03.038) -Time: 3041.210 ms (00:03.041) -Time: 4461.720 ms (00:04.462) -Time: 4446.691 ms (00:04.447) -Time: 4424.526 ms (00:04.425) -Time: 29275.463 ms (00:29.275) -Time: 17558.747 ms (00:17.559) -Time: 17438.621 ms (00:17.439) -Time: 203316.184 ms (03:23.316) -Time: 190037.946 ms (03:10.038) -Time: 189276.624 ms (03:09.277) -Time: 36921.542 ms (00:36.922) -Time: 36963.771 ms (00:36.964) -Time: 36660.406 ms (00:36.660) -Time: 38307.345 ms (00:38.307) -Time: 17597.355 ms (00:17.597) -Time: 17324.776 ms (00:17.325) -Time: 39857.567 ms (00:39.858) -Time: 26776.411 ms (00:26.776) -Time: 26592.819 ms (00:26.593) -Time: 162782.290 ms (02:42.782) -Time: 160722.582 ms (02:40.723) -Time: 162487.263 ms (02:42.487) -Time: 261494.290 ms (04:21.494) -Time: 263594.014 ms (04:23.594) -Time: 260436.201 ms (04:20.436) -Time: 265758.455 ms (04:25.758) -Time: 270087.523 ms (04:30.088) -Time: 266617.218 ms (04:26.617) -Time: 30677.159 ms (00:30.677) -Time: 28933.542 ms (00:28.934) -Time: 29815.271 ms (00:29.815) -Time: 19754.932 ms (00:19.755) -Time: 16851.157 ms (00:16.851) -Time: 16703.289 ms (00:16.703) -Time: 10379.500 ms (00:10.379) -Time: 10267.336 ms (00:10.267) -Time: 10287.944 ms (00:10.288) -Time: 17320.582 ms (00:17.321) -Time: 9786.410 ms (00:09.786) -Time: 9760.578 ms (00:09.761) -Time: 33487.352 ms (00:33.487) -Time: 26056.528 ms (00:26.057) -Time: 25958.258 ms (00:25.958) -Time: 28020.227 ms (00:28.020) -Time: 5609.725 ms (00:05.610) -Time: 5538.744 ms (00:05.539) -Time: 15119.473 ms (00:15.119) -Time: 5057.455 ms (00:05.057) -Time: 5063.154 ms (00:05.063) -Time: 3627.703 ms (00:03.628) -Time: 3645.232 ms (00:03.645) -Time: 3546.855 ms (00:03.547) diff --git a/benchmark/timescaledb/queries.sql b/benchmark/timescaledb/queries.sql index d7a2fe2d8bf..31f65fc898d 100644 --- a/benchmark/timescaledb/queries.sql +++ b/benchmark/timescaledb/queries.sql @@ -1,43 +1,43 @@ -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 REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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, Src, Dst 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); +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/timescaledb/results/c6a.4xlarge.compression.txt b/benchmark/timescaledb/results/c6a.4xlarge.compression.txt similarity index 100% rename from benchmark/compatible/timescaledb/results/c6a.4xlarge.compression.txt rename to benchmark/timescaledb/results/c6a.4xlarge.compression.txt diff --git a/benchmark/compatible/timescaledb/results/c6a.4xlarge.txt b/benchmark/timescaledb/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/compatible/timescaledb/results/c6a.4xlarge.txt rename to benchmark/timescaledb/results/c6a.4xlarge.txt diff --git a/benchmark/compatible/timescaledb/run.sh b/benchmark/timescaledb/run.sh similarity index 100% rename from benchmark/compatible/timescaledb/run.sh rename to benchmark/timescaledb/run.sh diff --git a/benchmark/timescaledb/usability.md b/benchmark/timescaledb/usability.md deleted file mode 100644 index 6914dc24118..00000000000 --- a/benchmark/timescaledb/usability.md +++ /dev/null @@ -1,1663 +0,0 @@ -This is a "usability testing" of TimescaleDB. I did not use TimescaleDB before. I will try to install it, load the data and conduct benchmarks. And record every obstacle that I will face. -Usability testing need to be conducted by the most clueless person in the room. Doing this "usability testing" requires a bit of patience and courage (to publish all the struggles as is). - -Note: insted of using clear VM, I have to run benchmark on exactly the same baremetal server where all other benchmarks were run. - - -## Installation - -Install as following: -https://docs.timescale.com/timescaledb/latest/how-to-guides/install-timescaledb/self-hosted/ubuntu/installation-apt-ubuntu/#installation-apt-ubuntu - -I've noticed that TimescaleDB documentation website does not have favicon in contrast to the main page. -In other means, it is quite neat. - -``` -sudo apt install postgresql-common -sudo sh /usr/share/postgresql-common/pgdg/apt.postgresql.org.sh -sudo sh -c "echo 'deb [signed-by=/usr/share/keyrings/timescale.keyring] https://packagecloud.io/timescale/timescaledb/ubuntu/ $(lsb_release -c -s) main' > /etc/apt/sources.list.d/timescaledb.list" -wget --quiet -O - https://packagecloud.io/timescale/timescaledb/gpgkey | sudo gpg --dearmor -o /usr/share/keyrings/timescale.keyring -sudo apt-get update -sudo apt install timescaledb-2-postgresql-13 -``` - -It recommends to tune it: - -``` -sudo apt install timescaledb-tune - -sudo timescaledb-tune --quiet --yes -Using postgresql.conf at this path: -/etc/postgresql/13/main/postgresql.conf - -Writing backup to: -/tmp/timescaledb_tune.backup202110292328 - -Recommendations based on 125.88 GB of available memory and 32 CPUs for PostgreSQL 13 -shared_preload_libraries = 'timescaledb' # (change requires restart) -shared_buffers = 32226MB -effective_cache_size = 96678MB -maintenance_work_mem = 2047MB -work_mem = 10312kB -timescaledb.max_background_workers = 8 -max_worker_processes = 43 -max_parallel_workers_per_gather = 16 -max_parallel_workers = 32 -wal_buffers = 16MB -min_wal_size = 512MB -default_statistics_target = 500 -random_page_cost = 1.1 -checkpoint_completion_target = 0.9 -max_locks_per_transaction = 512 -autovacuum_max_workers = 10 -autovacuum_naptime = 10 -effective_io_concurrency = 256 -timescaledb.last_tuned = '2021-10-29T23:28:49+03:00' -timescaledb.last_tuned_version = '0.12.0' -Saving changes to: /etc/postgresql/13/main/postgresql.conf -``` - -``` -sudo service postgresql restart -``` - -Post-install setup: -https://docs.timescale.com/timescaledb/latest/how-to-guides/install-timescaledb/post-install-setup/ - -``` -$ psql -U postgres -h localhost -Password for user postgres: -psql: error: connection to server at "localhost" (::1), port 5432 failed: fe_sendauth: no password supplied -``` - -How to set up password? - -``` -milovidov@mtlog-perftest03j:~$ psql -U postgres -h localhost -Password for user postgres: -psql: error: connection to server at "localhost" (::1), port 5432 failed: fe_sendauth: no password supplied -milovidov@mtlog-perftest03j:~$ psql -psql: error: connection to server on socket "/var/run/postgresql/.s.PGSQL.5432" failed: FATAL: role "milovidov" does not exist -milovidov@mtlog-perftest03j:~$ sudo psql -psql: error: connection to server on socket "/var/run/postgresql/.s.PGSQL.5432" failed: FATAL: role "root" does not exist -milovidov@mtlog-perftest03j:~$ psql -U postgres -psql: error: connection to server on socket "/var/run/postgresql/.s.PGSQL.5432" failed: FATAL: Peer authentication failed for user "postgres" -milovidov@mtlog-perftest03j:~$ psql -U postgres -h localost -psql: error: could not translate host name "localost" to address: Name or service not known -milovidov@mtlog-perftest03j:~$ sudo psql -U postgres -h localost -psql: error: could not translate host name "localost" to address: Name or service not known -milovidov@mtlog-perftest03j:~$ sudo psql -U postgres -h localhost -Password for user postgres: -psql: error: connection to server at "localhost" (::1), port 5432 failed: fe_sendauth: no password supplied -milovidov@mtlog-perftest03j:~$ sudo -u postgres psql -h localhost -Password for user postgres: -psql: error: connection to server at "localhost" (::1), port 5432 failed: fe_sendauth: no password supplied -``` - -I found an answer here: https://stackoverflow.com/questions/12720967/how-to-change-postgresql-user-password - -``` -$ sudo -u postgres psql -psql (13.4 (Ubuntu 13.4-4.pgdg18.04+1), server 9.5.25) -Type "help" for help. - -postgres=# \password postgres -Enter new password: -Enter it again: -postgres=# - -CREATE database tutorial; - -postgres=# CREATE EXTENSION IF NOT EXISTS timescaledb; -ERROR: could not open extension control file "/usr/share/postgresql/9.5/extension/timescaledb.control": No such file or directory -``` - -Looks like I have old PostgreSQL. - -``` -$ ls -l /usr/share/postgresql/ -10/ 11/ 13/ 9.5/ -``` - -But there is also newer PostgreSQL. - -``` -$ psql --version -psql (PostgreSQL) 13.4 (Ubuntu 13.4-4.pgdg18.04+1) - -psql is new, so what is wrong? -``` - -Looks like I have all versions running simultaneously? - -https://askubuntu.com/questions/17823/how-to-list-all-installed-packages - -``` -$ ps auxw | grep postgres -postgres 718818 0.0 0.5 33991600 730184 ? Ss 23:29 0:00 /usr/lib/postgresql/13/bin/postgres -D /var/lib/postgresql/13/main -c config_file=/etc/postgresql/13/main/postgresql.conf -postgres 718825 0.0 0.0 320356 27660 ? S 23:29 0:00 /usr/lib/postgresql/10/bin/postgres -D /var/lib/postgresql/10/main -c config_file=/etc/postgresql/10/main/postgresql.conf -postgres 718826 0.0 0.0 320712 27900 ? S 23:29 0:00 /usr/lib/postgresql/11/bin/postgres -D /var/lib/postgresql/11/main -c config_file=/etc/postgresql/11/main/postgresql.conf -postgres 718829 0.0 0.0 320468 7092 ? Ss 23:29 0:00 postgres: 10/main: checkpointer process -postgres 718830 0.0 0.0 320356 4300 ? Ss 23:29 0:00 postgres: 10/main: writer process -postgres 718831 0.0 0.0 320356 9204 ? Ss 23:29 0:00 postgres: 10/main: wal writer process -postgres 718832 0.0 0.0 320776 6964 ? Ss 23:29 0:00 postgres: 10/main: autovacuum launcher process -postgres 718833 0.0 0.0 175404 3596 ? Ss 23:29 0:00 postgres: 10/main: stats collector process -postgres 718834 0.0 0.0 320640 5052 ? Ss 23:29 0:00 postgres: 10/main: bgworker: logical replication launcher -postgres 718835 0.0 0.0 320820 5592 ? Ss 23:29 0:00 postgres: 11/main: checkpointer -postgres 718836 0.0 0.0 320712 4164 ? Ss 23:29 0:00 postgres: 11/main: background writer -postgres 718837 0.0 0.0 320712 9040 ? Ss 23:29 0:00 postgres: 11/main: walwriter -postgres 718838 0.0 0.0 321116 6824 ? Ss 23:29 0:00 postgres: 11/main: autovacuum launcher -postgres 718839 0.0 0.0 175752 3652 ? Ss 23:29 0:00 postgres: 11/main: stats collector -postgres 718840 0.0 0.0 321120 6640 ? Ss 23:29 0:00 postgres: 11/main: logical replication launcher -postgres 718842 0.0 0.1 33991700 263860 ? Ss 23:29 0:00 postgres: 13/main: checkpointer -postgres 718843 0.0 0.2 33991600 264096 ? Ss 23:29 0:00 postgres: 13/main: background writer -postgres 718844 0.0 0.0 33991600 22044 ? Ss 23:29 0:00 postgres: 13/main: walwriter -postgres 718845 0.0 0.0 33992284 7040 ? Ss 23:29 0:00 postgres: 13/main: autovacuum launcher -postgres 718846 0.0 0.0 177920 4320 ? Ss 23:29 0:00 postgres: 13/main: stats collector -postgres 718847 0.0 0.0 33992136 7972 ? Ss 23:29 0:00 postgres: 13/main: TimescaleDB Background Worker Launcher -postgres 718848 0.0 0.0 33992164 7248 ? Ss 23:29 0:00 postgres: 13/main: logical replication launcher -postgres 718857 0.0 0.0 304492 26284 ? S 23:29 0:00 /usr/lib/postgresql/9.5/bin/postgres -D /var/lib/postgresql/9.5/main -c config_file=/etc/postgresql/9.5/main/postgresql.conf -postgres 718859 0.0 0.0 304592 6480 ? Ss 23:29 0:00 postgres: checkpointer process -postgres 718860 0.0 0.0 304492 5656 ? Ss 23:29 0:00 postgres: writer process -postgres 718861 0.0 0.0 304492 4144 ? Ss 23:29 0:00 postgres: wal writer process -postgres 718862 0.0 0.0 304928 6896 ? Ss 23:29 0:00 postgres: autovacuum launcher process -postgres 718863 0.0 0.0 159744 4156 ? Ss 23:29 0:00 postgres: stats collector process -milovid+ 724277 0.0 0.0 14364 1024 pts/17 S+ 23:41 0:00 grep --color=auto postgres - -$ apt list --installed | grep postgres - -WARNING: apt does not have a stable CLI interface. Use with caution in scripts. - -postgresql-10/now 10.16-1.pgdg18.04+1 amd64 [installed,upgradable to: 10.18-1.pgdg18.04+1] -postgresql-11/now 11.11-1.pgdg18.04+1 amd64 [installed,upgradable to: 11.13-1.pgdg18.04+1] -postgresql-11-postgis-3/now 3.1.1+dfsg-1.pgdg18.04+1 amd64 [installed,upgradable to: 3.1.4+dfsg-1.pgdg18.04+1] -postgresql-11-postgis-3-scripts/now 3.1.1+dfsg-1.pgdg18.04+1 all [installed,upgradable to: 3.1.4+dfsg-1.pgdg18.04+1] -postgresql-13/bionic-pgdg,now 13.4-4.pgdg18.04+1 amd64 [installed,automatic] -postgresql-9.5/bionic-pgdg,now 9.5.25-1.pgdg18.04+1 amd64 [installed] -postgresql-9.5-postgis-2.2-scripts/now 2.2.2+dfsg-4.pgdg14.04+1.yandex all [installed,local] -postgresql-client-10/now 10.16-1.pgdg18.04+1 amd64 [installed,upgradable to: 10.18-1.pgdg18.04+1] -postgresql-client-11/now 11.11-1.pgdg18.04+1 amd64 [installed,upgradable to: 11.13-1.pgdg18.04+1] -postgresql-client-13/bionic-pgdg,now 13.4-4.pgdg18.04+1 amd64 [installed,automatic] -postgresql-client-9.5/bionic-pgdg,now 9.5.25-1.pgdg18.04+1 amd64 [installed] -postgresql-client-common/bionic-pgdg,now 231.pgdg18.04+1 all [installed] -postgresql-common/bionic-pgdg,now 231.pgdg18.04+1 all [installed] -timescaledb-2-loader-postgresql-13/bionic,now 2.5.0~ubuntu18.04 amd64 [installed,automatic] -timescaledb-2-postgresql-13/bionic,now 2.5.0~ubuntu18.04 amd64 [installed] -``` - -Let's remove all older packages. - -``` -sudo apt remove postgresql-10 postgresql-11 postgresql-9.5 postgresql-client-10 postgresql-client-11 postgresql-client-9.5 -``` - -Just in case: - -``` -sudo service postgresql restart -``` - -Now it stopped to work: - -``` -$ sudo -u postgres psql -psql: error: connection to server on socket "/var/run/postgresql/.s.PGSQL.5432" failed: No such file or directory - Is the server running locally and accepting connections on that socket? - -$ sudo -u postgres psql -h localhost -psql: error: connection to server at "localhost" (::1), port 5432 failed: Connection refused - Is the server running on that host and accepting TCP/IP connections? -connection to server at "localhost" (127.0.0.1), port 5432 failed: Connection refused - Is the server running on that host and accepting TCP/IP connections? -``` - -But it's running: - -``` -$ ps auxw | grep postgres -postgres 726158 0.5 0.5 33991600 730084 ? Ss 23:45 0:00 /usr/lib/postgresql/13/bin/postgres -D /var/lib/postgresql/13/main -c config_file=/etc/postgresql/13/main/postgresql.conf -postgres 726160 0.0 0.0 33991600 4256 ? Ss 23:45 0:00 postgres: 13/main: checkpointer -postgres 726161 0.1 0.1 33991600 150048 ? Ss 23:45 0:00 postgres: 13/main: background writer -postgres 726162 0.0 0.0 33991600 22044 ? Ss 23:45 0:00 postgres: 13/main: walwriter -postgres 726163 0.0 0.0 33992284 6976 ? Ss 23:45 0:00 postgres: 13/main: autovacuum launcher -postgres 726164 0.0 0.0 177920 4384 ? Ss 23:45 0:00 postgres: 13/main: stats collector -postgres 726165 0.0 0.0 33992136 7840 ? Ss 23:45 0:00 postgres: 13/main: TimescaleDB Background Worker Launcher -postgres 726166 0.0 0.0 33992164 7244 ? Ss 23:45 0:00 postgres: 13/main: logical replication launcher -milovid+ 726578 0.0 0.0 14364 1100 pts/17 S+ 23:46 0:00 grep --color=auto postgres -``` - -But it does not listen 5432: - -``` -$ netstat -n | grep 5432 -``` - -Let's look at the config: - -``` -sudo mcedit /etc/postgresql/13/main/postgresql.conf -``` - -``` -# - Connection Settings - - -#listen_addresses = 'localhost' -``` - -Looks like I need to uncomment it. - -``` -sudo service postgresql restart -``` - -But it did not help: - -``` -$ sudo -u postgres psql -h localhost -psql: error: connection to server at "localhost" (::1), port 5432 failed: Connection refused - Is the server running on that host and accepting TCP/IP connections? -connection to server at "localhost" (127.0.0.1), port 5432 failed: Connection refused - Is the server running on that host and accepting TCP/IP connections? -``` - -Let's consult https://stackoverflow.com/questions/31091748/postgres-server-not-listening - -It is mentioning some pg_hba.conf. BTW what is HBA*? Let's find this file... - -``` -sudo mcedit /etc/postgresql/13/main/pg_hba.conf -``` - -\* host based authentication rules - it is explained inside this file. - -Nothing wrong in this file... - -``` -$ sudo service postgresql status -● postgresql.service - PostgreSQL RDBMS - Loaded: loaded (/lib/systemd/system/postgresql.service; enabled; vendor preset: enabled) - Active: active (exited) since Fri 2021-10-29 23:50:14 MSK; 6min ago - Process: 728545 ExecStart=/bin/true (code=exited, status=0/SUCCESS) - Main PID: 728545 (code=exited, status=0/SUCCESS) - -Oct 29 23:50:14 mtlog-perftest03j systemd[1]: postgresql.service: Changed dead -> start -Oct 29 23:50:14 mtlog-perftest03j systemd[1]: Starting PostgreSQL RDBMS... -Oct 29 23:50:14 mtlog-perftest03j systemd[728545]: postgresql.service: Executing: /bin/true -Oct 29 23:50:14 mtlog-perftest03j systemd[1]: postgresql.service: Child 728545 belongs to postgresql.service. -Oct 29 23:50:14 mtlog-perftest03j systemd[1]: postgresql.service: Main process exited, code=exited, status=0/SUCCESS -Oct 29 23:50:14 mtlog-perftest03j systemd[1]: postgresql.service: Changed start -> exited -Oct 29 23:50:14 mtlog-perftest03j systemd[1]: postgresql.service: Job postgresql.service/start finished, result=done -Oct 29 23:50:14 mtlog-perftest03j systemd[1]: Started PostgreSQL RDBMS. -Oct 29 23:50:14 mtlog-perftest03j systemd[1]: postgresql.service: Failed to send unit change signal for postgresql.service: Connection reset by peer -``` - -It's quite cryptic. What does it mean "Failed to send unit change signal"? Is it good or bad? -What is the "unit"? Maybe it is "SystemD Unit" - the phrase that I've heard many times but don't really understand. - -Almost gave up... Wow, I found the culprit! In `/etc/postgresql/13/main/postgresql.conf`: - -``` -port = 5435 -``` - -Most likely this has happened, because multiple versions of PostgreSQL were installed. - -Let's change to 5432. - -``` -sudo mcedit /etc/postgresql/13/main/postgresql.conf -sudo service postgresql restart -``` - -But now it does not accept password: - -``` -milovidov@mtlog-perftest03j:~$ sudo -u postgres psql -h 127.0.0.1 -Password for user postgres: -psql: error: connection to server at "127.0.0.1", port 5432 failed: fe_sendauth: no password supplied -milovidov@mtlog-perftest03j:~$ sudo -u postgres psql -h 127.0.0.1 --password '' -Password: -psql: error: connection to server at "127.0.0.1", port 5432 failed: fe_sendauth: no password supplied -milovidov@mtlog-perftest03j:~$ sudo -u postgres psql -h 127.0.0.1 -Password for user postgres: -psql: error: connection to server at "127.0.0.1", port 5432 failed: fe_sendauth: no password supplied -``` - -Works this way: - -``` -$ sudo -u postgres psql -psql (13.4 (Ubuntu 13.4-4.pgdg18.04+1)) -Type "help" for help. - -postgres=# \password -Enter new password: -Enter it again: -``` - -It works with fine ASCII arc: - -``` -postgres=# CREATE database tutorial; -CREATE DATABASE -postgres=# \c tutorial -You are now connected to database "tutorial" as user "postgres". -tutorial=# CREATE EXTENSION IF NOT EXISTS timescaledb; -WARNING: -WELCOME TO - _____ _ _ ____________ -|_ _(_) | | | _ \ ___ \ - | | _ _ __ ___ ___ ___ ___ __ _| | ___| | | | |_/ / - | | | | _ ` _ \ / _ \/ __|/ __/ _` | |/ _ \ | | | ___ \ - | | | | | | | | | __/\__ \ (_| (_| | | __/ |/ /| |_/ / - |_| |_|_| |_| |_|\___||___/\___\__,_|_|\___|___/ \____/ - Running version 2.5.0 -For more information on TimescaleDB, please visit the following links: - - 1. Getting started: https://docs.timescale.com/timescaledb/latest/getting-started - 2. API reference documentation: https://docs.timescale.com/api/latest - 3. How TimescaleDB is designed: https://docs.timescale.com/timescaledb/latest/overview/core-concepts - -Note: TimescaleDB collects anonymous reports to better understand and assist our users. -For more information and how to disable, please see our docs https://docs.timescale.com/timescaledb/latest/how-to-guides/configuration/telemetry. - -CREATE EXTENSION -``` - - -## Creating Table - -Continuing to https://docs.timescale.com/timescaledb/latest/how-to-guides/hypertables/create/ - -Create table: - -``` -CREATE TABLE hits_100m_obfuscated ( -WatchID BIGINT, -JavaEnable SMALLINT, -Title TEXT, -GoodEvent SMALLINT, -EventTime TIMESTAMP, -EventDate Date, -CounterID INTEGER, -ClientIP INTEGER, -RegionID INTEGER, -UserID BIGINT, -CounterClass SMALLINT, -OS SMALLINT, -UserAgent SMALLINT, -URL TEXT, -Referer TEXT, -Refresh SMALLINT, -RefererCategoryID SMALLINT, -RefererRegionID INTEGER, -URLCategoryID SMALLINT, -URLRegionID INTEGER, -ResolutionWidth SMALLINT, -ResolutionHeight SMALLINT, -ResolutionDepth SMALLINT, -FlashMajor SMALLINT, -FlashMinor SMALLINT, -FlashMinor2 TEXT, -NetMajor SMALLINT, -NetMinor SMALLINT, -UserAgentMajor SMALLINT, -UserAgentMinor CHAR(2), -CookieEnable SMALLINT, -JavascriptEnable SMALLINT, -IsMobile SMALLINT, -MobilePhone SMALLINT, -MobilePhoneModel TEXT, -Params TEXT, -IPNetworkID INTEGER, -TraficSourceID SMALLINT, -SearchEngineID SMALLINT, -SearchPhrase TEXT, -AdvEngineID SMALLINT, -IsArtifical SMALLINT, -WindowClientWidth SMALLINT, -WindowClientHeight SMALLINT, -ClientTimeZone SMALLINT, -ClientEventTime TIMESTAMP, -SilverlightVersion1 SMALLINT, -SilverlightVersion2 SMALLINT, -SilverlightVersion3 INTEGER, -SilverlightVersion4 SMALLINT, -PageCharset TEXT, -CodeVersion INTEGER, -IsLink SMALLINT, -IsDownload SMALLINT, -IsNotBounce SMALLINT, -FUniqID BIGINT, -OriginalURL TEXT, -HID INTEGER, -IsOldCounter SMALLINT, -IsEvent SMALLINT, -IsParameter SMALLINT, -DontCountHits SMALLINT, -WithHash SMALLINT, -HitColor CHAR, -LocalEventTime TIMESTAMP, -Age SMALLINT, -Sex SMALLINT, -Income SMALLINT, -Interests SMALLINT, -Robotness SMALLINT, -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 SMALLINT, -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 SMALLINT, -RefererHash BIGINT, -URLHash BIGINT, -CLID INTEGER -); -``` - -I remember PostgreSQL does not support unsigned integers. It also does not support TINYINT. -And it does not support zero bytes in TEXT fields. We will deal with it... - -``` -tutorial=# SELECT create_hypertable('hits_100m_obfuscated', 'EventTime'); -ERROR: column "EventTime" does not exist -``` - -WTF? - -Maybe it because column names are lowercased? - -``` -tutorial=# SELECT create_hypertable('hits_100m_obfuscated', 'eventtime'); -NOTICE: adding not-null constraint to column "eventtime" -DETAIL: Time dimensions cannot have NULL values. - create_hypertable ------------------------------------ - (1,public,hits_100m_obfuscated,t) -(1 row) -``` - -Looks like I forgot to specify NOT NULL for every column. -Let's repeat... - -``` -tutorial=# DROP TABLE hits_100m_obfuscated -tutorial-# ; -DROP TABLE -tutorial=# CREATE TABLE hits_100m_obfuscated ( -tutorial(# WatchID BIGINT NOT NULL, -tutorial(# JavaEnable SMALLINT NOT NULL, -tutorial(# Title TEXT NOT NULL, -tutorial(# GoodEvent SMALLINT NOT NULL, -tutorial(# EventTime TIMESTAMP NOT NULL, -tutorial(# EventDate Date NOT NULL, -tutorial(# CounterID INTEGER NOT NULL, -tutorial(# ClientIP INTEGER NOT NULL, -tutorial(# RegionID INTEGER NOT NULL, -tutorial(# UserID BIGINT NOT NULL, -tutorial(# CounterClass SMALLINT NOT NULL, -tutorial(# OS SMALLINT NOT NULL, -tutorial(# UserAgent SMALLINT NOT NULL, -tutorial(# URL TEXT NOT NULL, -tutorial(# Referer TEXT NOT NULL, -tutorial(# Refresh SMALLINT NOT NULL, -tutorial(# RefererCategoryID SMALLINT NOT NULL, -tutorial(# RefererRegionID INTEGER NOT NULL, -tutorial(# URLCategoryID SMALLINT NOT NULL, -tutorial(# URLRegionID INTEGER NOT NULL, -tutorial(# ResolutionWidth SMALLINT NOT NULL, -tutorial(# ResolutionHeight SMALLINT NOT NULL, -tutorial(# ResolutionDepth SMALLINT NOT NULL, -tutorial(# FlashMajor SMALLINT NOT NULL, -tutorial(# FlashMinor SMALLINT NOT NULL, -tutorial(# FlashMinor2 TEXT NOT NULL, -tutorial(# NetMajor SMALLINT NOT NULL, -tutorial(# NetMinor SMALLINT NOT NULL, -tutorial(# UserAgentMajor SMALLINT NOT NULL, -tutorial(# UserAgentMinor CHAR(2) NOT NULL, -tutorial(# CookieEnable SMALLINT NOT NULL, -tutorial(# JavascriptEnable SMALLINT NOT NULL, -tutorial(# IsMobile SMALLINT NOT NULL, -tutorial(# MobilePhone SMALLINT NOT NULL, -tutorial(# MobilePhoneModel TEXT NOT NULL, -tutorial(# Params TEXT NOT NULL, -tutorial(# IPNetworkID INTEGER NOT NULL, -tutorial(# TraficSourceID SMALLINT NOT NULL, -tutorial(# SearchEngineID SMALLINT NOT NULL, -tutorial(# SearchPhrase TEXT NOT NULL, -tutorial(# AdvEngineID SMALLINT NOT NULL, -tutorial(# IsArtifical SMALLINT NOT NULL, -tutorial(# WindowClientWidth SMALLINT NOT NULL, -tutorial(# WindowClientHeight SMALLINT NOT NULL, -tutorial(# ClientTimeZone SMALLINT NOT NULL, -tutorial(# ClientEventTime TIMESTAMP NOT NULL, -tutorial(# SilverlightVersion1 SMALLINT NOT NULL, -tutorial(# SilverlightVersion2 SMALLINT NOT NULL, -tutorial(# SilverlightVersion3 INTEGER NOT NULL, -tutorial(# SilverlightVersion4 SMALLINT NOT NULL, -tutorial(# PageCharset TEXT NOT NULL, -tutorial(# CodeVersion INTEGER NOT NULL, -tutorial(# IsLink SMALLINT NOT NULL, -tutorial(# IsDownload SMALLINT NOT NULL, -tutorial(# IsNotBounce SMALLINT NOT NULL, -tutorial(# FUniqID BIGINT NOT NULL, -tutorial(# OriginalURL TEXT NOT NULL, -tutorial(# HID INTEGER NOT NULL, -tutorial(# IsOldCounter SMALLINT NOT NULL, -tutorial(# IsEvent SMALLINT NOT NULL, -tutorial(# IsParameter SMALLINT NOT NULL, -tutorial(# DontCountHits SMALLINT NOT NULL, -tutorial(# WithHash SMALLINT NOT NULL, -tutorial(# HitColor CHAR NOT NULL, -tutorial(# LocalEventTime TIMESTAMP NOT NULL, -tutorial(# Age SMALLINT NOT NULL, -tutorial(# Sex SMALLINT NOT NULL, -tutorial(# Income SMALLINT NOT NULL, -tutorial(# Interests SMALLINT NOT NULL, -tutorial(# Robotness SMALLINT NOT NULL, -tutorial(# RemoteIP INTEGER NOT NULL, -tutorial(# WindowName INTEGER NOT NULL, -tutorial(# OpenerName INTEGER NOT NULL, -tutorial(# HistoryLength SMALLINT NOT NULL, -tutorial(# BrowserLanguage TEXT NOT NULL, -tutorial(# BrowserCountry TEXT NOT NULL, -tutorial(# SocialNetwork TEXT NOT NULL, -tutorial(# SocialAction TEXT NOT NULL, -tutorial(# HTTPError SMALLINT NOT NULL, -tutorial(# SendTiming INTEGER NOT NULL, -tutorial(# DNSTiming INTEGER NOT NULL, -tutorial(# ConnectTiming INTEGER NOT NULL, -tutorial(# ResponseStartTiming INTEGER NOT NULL, -tutorial(# ResponseEndTiming INTEGER NOT NULL, -tutorial(# FetchTiming INTEGER NOT NULL, -tutorial(# SocialSourceNetworkID SMALLINT NOT NULL, -tutorial(# SocialSourcePage TEXT NOT NULL, -tutorial(# ParamPrice BIGINT NOT NULL, -tutorial(# ParamOrderID TEXT NOT NULL, -tutorial(# ParamCurrency TEXT NOT NULL, -tutorial(# ParamCurrencyID SMALLINT NOT NULL, -tutorial(# OpenstatServiceName TEXT NOT NULL, -tutorial(# OpenstatCampaignID TEXT NOT NULL, -tutorial(# OpenstatAdID TEXT NOT NULL, -tutorial(# OpenstatSourceID TEXT NOT NULL, -tutorial(# UTMSource TEXT NOT NULL, -tutorial(# UTMMedium TEXT NOT NULL, -tutorial(# UTMCampaign TEXT NOT NULL, -tutorial(# UTMContent TEXT NOT NULL, -tutorial(# UTMTerm TEXT NOT NULL, -tutorial(# FromTag TEXT NOT NULL, -tutorial(# HasGCLID SMALLINT NOT NULL, -tutorial(# RefererHash BIGINT NOT NULL, -tutorial(# URLHash BIGINT NOT NULL, -tutorial(# CLID INTEGER NOT NULL -tutorial(# ); -CREATE TABLE -tutorial=# SELECT create_hypertable('hits_100m_obfuscated', 'eventtime'); - create_hypertable ------------------------------------ - (2,public,hits_100m_obfuscated,t) -(1 row) - -tutorial=# -``` - -Now ok. - - -## Loading Data - -Next - importing data: -https://docs.timescale.com/timescaledb/latest/how-to-guides/migrate-data/import-csv/#csv-import - -``` -SELECT WatchID::Int64, JavaEnable, toValidUTF8(Title), GoodEvent, EventTime, EventDate, CounterID::Int32, ClientIP::Int32, RegionID::Int32, UserID::Int64, CounterClass, OS, UserAgent, toValidUTF8(URL), toValidUTF8(Referer), Refresh, RefererCategoryID::Int16, RefererRegionID::Int32, URLCategoryID::Int16, URLRegionID::Int32, ResolutionWidth::Int16, ResolutionHeight::Int16, ResolutionDepth, FlashMajor, FlashMinor, FlashMinor2, NetMajor, NetMinor, UserAgentMajor::Int16, UserAgentMinor, CookieEnable, JavascriptEnable, IsMobile, MobilePhone, toValidUTF8(MobilePhoneModel), toValidUTF8(Params), IPNetworkID::Int32, TraficSourceID, SearchEngineID::Int16, toValidUTF8(SearchPhrase), AdvEngineID, IsArtifical, WindowClientWidth::Int16, WindowClientHeight::Int16, ClientTimeZone, ClientEventTime, SilverlightVersion1, SilverlightVersion2, SilverlightVersion3::Int32, SilverlightVersion4::Int16, toValidUTF8(PageCharset), CodeVersion::Int32, IsLink, IsDownload, IsNotBounce, FUniqID::Int64, toValidUTF8(OriginalURL), HID::Int32, IsOldCounter, IsEvent, IsParameter, DontCountHits, WithHash, HitColor, LocalEventTime, Age, Sex, Income, Interests::Int16, Robotness, RemoteIP::Int32, WindowName, OpenerName, HistoryLength, BrowserLanguage, BrowserCountry, toValidUTF8(SocialNetwork), toValidUTF8(SocialAction), HTTPError, SendTiming, DNSTiming, ConnectTiming, ResponseStartTiming, ResponseEndTiming, FetchTiming, SocialSourceNetworkID, toValidUTF8(SocialSourcePage), ParamPrice, toValidUTF8(ParamOrderID), ParamCurrency, ParamCurrencyID::Int16, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash::Int64, URLHash::Int64, CLID::Int32 -FROM hits_100m_obfuscated -INTO OUTFILE 'dump.csv' -FORMAT CSV -``` - -https://github.com/ClickHouse/ClickHouse/issues/30872 -https://github.com/ClickHouse/ClickHouse/issues/30873 - -``` -$ wc -c dump.csv -80865718769 dump.csv -``` - -``` -milovidov@mtlog-perftest03j:~$ timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -panic: could not connect: pq: password authentication failed for user "postgres" - -goroutine 12 [running]: -main.processBatches(0xc00001e3c0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:238 +0x887 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb -milovidov@mtlog-perftest03j:~$ sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -panic: could not connect: pq: password authentication failed for user "postgres" - -goroutine 25 [running]: -main.processBatches(0xc00019a350, 0xc00019e660) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:238 +0x887 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - - -milovidov@mtlog-perftest03j:~$ sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" --host localhost -flag provided but not defined: -host -Usage of timescaledb-parallel-copy: - -batch-size int - Number of rows per insert (default 5000) - -columns string - Comma-separated columns present in CSV - -connection string - PostgreSQL connection url (default "host=localhost user=postgres sslmode=disable") - -copy-options string - Additional options to pass to COPY (e.g., NULL 'NULL') (default "CSV") - -db-name string - Database where the destination table exists - -file string - File to read from rather than stdin - -header-line-count int - Number of header lines (default 1) - -limit int - Number of rows to insert overall; 0 means to insert all - -log-batches - Whether to time individual batches. - -reporting-period duration - Period to report insert stats; if 0s, intermediate results will not be reported - -schema string - Destination table's schema (default "public") - -skip-header - Skip the first line of the input - -split string - Character to split by (default ",") - -table string - Destination table for insertions (default "test_table") - -token-size int - Maximum size to use for tokens. By default, this is 64KB, so any value less than that will be ignored (default 65536) - -truncate - Truncate the destination table before insert - -verbose - Print more information about copying statistics - -version - Show the version of this tool - -workers int - Number of parallel requests to make (default 1) - - -milovidov@mtlog-perftest03j:~$ sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -connection 'host=localhost' -panic: could not connect: pq: password authentication failed for user "postgres" - -goroutine 14 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:238 +0x887 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb -panic: could not connect: pq: password authentication failed for user "postgres" - -goroutine 13 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:238 +0x887 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb -panic: could not connect: pq: password authentication failed for user "postgres" - -goroutine 12 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:238 +0x887 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - - -milovidov@mtlog-perftest03j:~$ sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -connection 'host=localhost password 12345' -panic: could not connect: cannot parse `host=localhost password 12345`: failed to parse as DSN (invalid dsn) - -goroutine 13 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:238 +0x887 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - - -milovidov@mtlog-perftest03j:~$ sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -connection 'host=localhost password=12345' -panic: pq: invalid byte sequence for encoding "UTF8": 0xe0 0x22 0x2c - -goroutine 34 [running]: -main.processBatches(0xc000132350, 0xc000136660) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb -panic: pq: invalid byte sequence for encoding "UTF8": 0xe0 0x22 0x2c - -goroutine 30 [running]: -main.processBatches(0xc000132350, 0xc000136660) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb -``` - -Ok, now I've got something meaningful. -But it does not show, what line has error... - -``` -$ echo -e '\xe0\x22\x2c' -�", -``` - -Let's recreate the dump: - -``` -rm dump.csv - -SELECT WatchID::Int64, JavaEnable, toValidUTF8(Title), GoodEvent, EventTime, EventDate, CounterID::Int32, ClientIP::Int32, RegionID::Int32, - UserID::Int64, CounterClass, OS, UserAgent, toValidUTF8(URL), toValidUTF8(Referer), Refresh, RefererCategoryID::Int16, RefererRegionID::Int32, - URLCategoryID::Int16, URLRegionID::Int32, ResolutionWidth::Int16, ResolutionHeight::Int16, ResolutionDepth, FlashMajor, FlashMinor, - FlashMinor2, NetMajor, NetMinor, UserAgentMajor::Int16, toValidUTF8(UserAgentMinor::String), CookieEnable, JavascriptEnable, IsMobile, MobilePhone, - toValidUTF8(MobilePhoneModel), toValidUTF8(Params), IPNetworkID::Int32, TraficSourceID, SearchEngineID::Int16, toValidUTF8(SearchPhrase), - AdvEngineID, IsArtifical, WindowClientWidth::Int16, WindowClientHeight::Int16, ClientTimeZone, ClientEventTime, - SilverlightVersion1, SilverlightVersion2, SilverlightVersion3::Int32, SilverlightVersion4::Int16, toValidUTF8(PageCharset), - CodeVersion::Int32, IsLink, IsDownload, IsNotBounce, FUniqID::Int64, toValidUTF8(OriginalURL), HID::Int32, IsOldCounter, IsEvent, - IsParameter, DontCountHits, WithHash, toValidUTF8(HitColor::String), LocalEventTime, Age, Sex, Income, Interests::Int16, Robotness, RemoteIP::Int32, - WindowName, OpenerName, HistoryLength, toValidUTF8(BrowserLanguage::String), toValidUTF8(BrowserCountry::String), - toValidUTF8(SocialNetwork), toValidUTF8(SocialAction), - HTTPError, SendTiming, DNSTiming, ConnectTiming, ResponseStartTiming, ResponseEndTiming, FetchTiming, SocialSourceNetworkID, - toValidUTF8(SocialSourcePage), ParamPrice, toValidUTF8(ParamOrderID), toValidUTF8(ParamCurrency::String), - ParamCurrencyID::Int16, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, - UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash::Int64, URLHash::Int64, CLID::Int32 -FROM hits_100m_obfuscated -INTO OUTFILE 'dump.csv' -FORMAT CSV -``` - -``` -$ sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 1 --copy-options "CSV" -connection 'host=localhost password=12345' -panic: pq: value too long for type character(2) - -goroutine 6 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb -``` - -ALTER does not work: - -``` -tutorial=# ALTER TABLE hits_100m_obfuscated MODIFY COLUMN UserAgentMinor TEXT -tutorial-# ; -ERROR: syntax error at or near "MODIFY" -LINE 1: ALTER TABLE hits_100m_obfuscated MODIFY COLUMN UserAgentMino... - ^ -``` - -PostgreSQL is using unusual syntax for ALTER: - -``` -tutorial=# ALTER TABLE hits_100m_obfuscated ALTER COLUMN UserAgentMinor TYPE TEXT -; -ALTER TABLE -tutorial=# \q -``` - -https://github.com/ClickHouse/ClickHouse/issues/30874 - -Now something again: - -``` -$ sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 1 --copy-options "CSV" -connection 'host=localhost password=12345' -panic: pq: value "2149615427" is out of range for type integer - -goroutine 6 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb -``` - -``` -$ grep -F '2149615427' dump.csv -5607505572457935073,0,"Лазар автоматические пылесосы подробная школы. Когалерея — Курсы на Автория пище Сноудента новые устами",1,"2013-07-15 07:47:45","2013-07-15",38,-1194330980,229,-6649844357037090659,0,2,3,"https://produkty%2Fkategory_id=&auto-nexus.html?blockfesty-i-korroszhego","http://tambov.irr.ua/yandex.ru/saledParam=0&user/auto.ria",1,10282,995,15014,519,1996,1781,23,14,2,"800",0,0,7,"D�",1,1,0,0,"","",3392210,-1,0,"",0,0,1261,1007,135,"2013-07-15 21:54:13",0,0,0,0,"windows-1251;charset",1601,0,0,0,8184671896482443026,"",451733382,0,0,0,0,0,"5","2013-07-15 15:41:14",31,1,3,60,13,-1855237933,-1,-1,-1,"S0","h1","","",0,0,0,0,2149615427,36,3,0,"",0,"","NH",0,"","","","","","","","","","",0,-1103774879459415602,-2414747266057209563,0 -^C -``` - -Let's recreate the dump: - -``` -rm dump.csv - -SELECT WatchID::Int64, JavaEnable, toValidUTF8(Title), GoodEvent, EventTime, EventDate, CounterID::Int32, ClientIP::Int32, RegionID::Int32, - UserID::Int64, CounterClass, OS, UserAgent, toValidUTF8(URL), toValidUTF8(Referer), Refresh, RefererCategoryID::Int16, RefererRegionID::Int32, - URLCategoryID::Int16, URLRegionID::Int32, ResolutionWidth::Int16, ResolutionHeight::Int16, ResolutionDepth, FlashMajor, FlashMinor, - FlashMinor2, NetMajor, NetMinor, UserAgentMajor::Int16, toValidUTF8(UserAgentMinor::String), CookieEnable, JavascriptEnable, IsMobile, MobilePhone, - toValidUTF8(MobilePhoneModel), toValidUTF8(Params), IPNetworkID::Int32, TraficSourceID, SearchEngineID::Int16, toValidUTF8(SearchPhrase), - AdvEngineID, IsArtifical, WindowClientWidth::Int16, WindowClientHeight::Int16, ClientTimeZone, ClientEventTime, - SilverlightVersion1, SilverlightVersion2, SilverlightVersion3::Int32, SilverlightVersion4::Int16, toValidUTF8(PageCharset), - CodeVersion::Int32, IsLink, IsDownload, IsNotBounce, FUniqID::Int64, toValidUTF8(OriginalURL), HID::Int32, IsOldCounter, IsEvent, - IsParameter, DontCountHits, WithHash, toValidUTF8(HitColor::String), LocalEventTime, Age, Sex, Income, Interests::Int16, Robotness, RemoteIP::Int32, - WindowName, OpenerName, HistoryLength, toValidUTF8(BrowserLanguage::String), toValidUTF8(BrowserCountry::String), - toValidUTF8(SocialNetwork), toValidUTF8(SocialAction), - HTTPError, least(SendTiming, 30000), least(DNSTiming, 30000), least(ConnectTiming, 30000), least(ResponseStartTiming, 30000), - least(ResponseEndTiming, 30000), least(FetchTiming, 30000), SocialSourceNetworkID, - toValidUTF8(SocialSourcePage), ParamPrice, toValidUTF8(ParamOrderID), toValidUTF8(ParamCurrency::String), - ParamCurrencyID::Int16, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, - UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash::Int64, URLHash::Int64, CLID::Int32 -FROM hits_100m_obfuscated -INTO OUTFILE 'dump.csv' -FORMAT CSV -``` - -PostgreSQL does not support USE database. -But I remember, that I can write `\c` instead. I guess `\c` means "change" (the database). Or it is called "schema" or "catalog". - -``` -milovidov@mtlog-perftest03j:~$ sudo -u postgres psql -psql (13.4 (Ubuntu 13.4-4.pgdg18.04+1)) -Type "help" for help. - -postgres=# SELECT count(*) FROM hits_100m_obfuscated; -ERROR: relation "hits_100m_obfuscated" does not exist -LINE 1: SELECT count(*) FROM hits_100m_obfuscated; - ^ -postgres=# USE tutorial; -ERROR: syntax error at or near "USE" -LINE 1: USE tutorial; - ^ -postgres=# \c tutorial -You are now connected to database "tutorial" as user "postgres". -tutorial=# SELECT count(*) FROM hits_100m_obfuscated; - count -------- - 69996 -(1 row) -``` - -And parallel loader already loaded some part of data into my table (it is not transactional). -Let's truncate table: - -``` -tutorial=# TRUNCATE TABLE hits_100m_obfuscated; -TRUNCATE TABLE -``` - -Surprisingly, it works! - -Now it started loading data: -``` -$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -connection 'host=localhost password=12345' -``` - -But the loading is not using 16 CPU cores and it is not bottlenecked by IO. - -WTF: - -``` -$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -connection 'host=localhost password=12345' -panic: pq: could not extend file "base/16384/31264.1": wrote only 4096 of 8192 bytes at block 145407 - -goroutine 6 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 3m31.328s -user 0m35.016s -sys 0m6.964s -``` - -Looks like there is no space: - -``` -milovidov@mtlog-perftest03j:~$ df -h /var/lib/postgresql/13/main -Filesystem Size Used Avail Use% Mounted on -/dev/md1 35G 33G 1.4G 97% / -``` - -https://github.com/ClickHouse/ClickHouse/issues/30883 - -Let's move to another device. - -``` -milovidov@mtlog-perftest03j:~$ sudo mkdir /opt/postgresql -milovidov@mtlog-perftest03j:~$ sudo ls -l /var/lib/postgresql/13/main -total 88 -drwx------ 6 postgres postgres 4096 Oct 30 00:06 base -drwx------ 2 postgres postgres 4096 Oct 30 02:07 global -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_commit_ts -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_dynshmem -drwx------ 4 postgres postgres 4096 Oct 30 02:10 pg_logical -drwx------ 4 postgres postgres 4096 Oct 29 23:27 pg_multixact -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_notify -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_replslot -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_serial -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_snapshots -drwx------ 2 postgres postgres 4096 Oct 30 02:10 pg_stat -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_stat_tmp -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_subtrans -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_tblspc -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_twophase --rw------- 1 postgres postgres 3 Oct 29 23:27 PG_VERSION -drwx------ 3 postgres postgres 12288 Oct 30 02:10 pg_wal -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_xact --rw------- 1 postgres postgres 88 Oct 29 23:27 postgresql.auto.conf --rw------- 1 postgres postgres 130 Oct 30 00:03 postmaster.opts -milovidov@mtlog-perftest03j:~$ sudo chown postgres:postgres /opt/postgresql -milovidov@mtlog-perftest03j:~$ sudo mv /var/lib/postgresql/13/main/* /opt/postgresql -mv: cannot stat '/var/lib/postgresql/13/main/*': No such file or directory -milovidov@mtlog-perftest03j:~$ sudo bash -c 'mv /var/lib/postgresql/13/main/* /opt/postgresql' -sudo ln milovidov@mtlog-perftest03j:~$ #sudo ln -s /opt/postgresql /var/lib/postgresql/13/main -milovidov@mtlog-perftest03j:~$ sudo rm /var/lib/postgresql/13/main -rm: cannot remove '/var/lib/postgresql/13/main': Is a directory -milovidov@mtlog-perftest03j:~$ sudo rm -rf /var/lib/postgresql/13/main -milovidov@mtlog-perftest03j:~$ sudo ln -s /opt/postgresql /var/lib/postgresql/13/main -milovidov@mtlog-perftest03j:~$ sudo ls -l /var/lib/postgresql/13/main -lrwxrwxrwx 1 root root 15 Oct 30 02:12 /var/lib/postgresql/13/main -> /opt/postgresql -milovidov@mtlog-perftest03j:~$ sudo ls -l /opt/postgresql/ -total 80 -drwx------ 6 postgres postgres 4096 Oct 30 00:06 base -drwx------ 2 postgres postgres 4096 Oct 30 02:07 global -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_commit_ts -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_dynshmem -drwx------ 4 postgres postgres 4096 Oct 30 02:10 pg_logical -drwx------ 4 postgres postgres 4096 Oct 29 23:27 pg_multixact -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_notify -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_replslot -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_serial -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_snapshots -drwx------ 2 postgres postgres 4096 Oct 30 02:10 pg_stat -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_stat_tmp -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_subtrans -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_tblspc -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_twophase --rw------- 1 postgres postgres 3 Oct 29 23:27 PG_VERSION -drwx------ 3 postgres postgres 4096 Oct 30 02:10 pg_wal -drwx------ 2 postgres postgres 4096 Oct 29 23:27 pg_xact --rw------- 1 postgres postgres 88 Oct 29 23:27 postgresql.auto.conf --rw------- 1 postgres postgres 130 Oct 30 00:03 postmaster.opts - -sudo service postgresql start - -sudo less /var/log/postgresql/postgresql-13-main.log - -2021-10-30 02:13:41.284 MSK [791362] FATAL: data directory "/var/lib/postgresql/13/main" has invalid permissions -2021-10-30 02:13:41.284 MSK [791362] DETAIL: Permissions should be u=rwx (0700) or u=rwx,g=rx (0750). -pg_ctl: could not start server -Examine the log output. - -sudo chmod 0700 /var/lib/postgresql/13/main /opt/postgresql -sudo service postgresql start - -postgres=# \c tutorial -You are now connected to database "tutorial" as user "postgres". -tutorial=# TRUNCATE TABLE hits_100m_obfuscated; -TRUNCATE TABLE -``` - -``` -$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -connection 'host=localhost password=12345' -``` - -No success: - -``` -$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -connection 'host=localhost password=12345' -panic: pq: invalid byte sequence for encoding "UTF8": 0x00 - -goroutine 29 [running]: -main.processBatches(0xc000132350, 0xc000136660) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 11m47.879s -user 3m10.980s -sys 0m45.256s -``` - -The error message is false, because UTF-8 **does** support 0x00. It is just some PostgreSQL quirk. - -Let's recreate the dump: - -``` -rm dump.csv - -SELECT WatchID::Int64, JavaEnable, replaceAll(toValidUTF8(Title), '\0', ''), GoodEvent, EventTime, EventDate, CounterID::Int32, ClientIP::Int32, RegionID::Int32, - UserID::Int64, CounterClass, OS, UserAgent, replaceAll(toValidUTF8(URL), '\0', ''), replaceAll(toValidUTF8(Referer), '\0', ''), Refresh, RefererCategoryID::Int16, RefererRegionID::Int32, - URLCategoryID::Int16, URLRegionID::Int32, ResolutionWidth::Int16, ResolutionHeight::Int16, ResolutionDepth, FlashMajor, FlashMinor, - FlashMinor2, NetMajor, NetMinor, UserAgentMajor::Int16, replaceAll(toValidUTF8(UserAgentMinor::String), '\0', ''), CookieEnable, JavascriptEnable, IsMobile, MobilePhone, - replaceAll(toValidUTF8(MobilePhoneModel), '\0', ''), replaceAll(toValidUTF8(Params), '\0', ''), IPNetworkID::Int32, TraficSourceID, SearchEngineID::Int16, replaceAll(toValidUTF8(SearchPhrase), '\0', ''), - AdvEngineID, IsArtifical, WindowClientWidth::Int16, WindowClientHeight::Int16, ClientTimeZone, ClientEventTime, - SilverlightVersion1, SilverlightVersion2, SilverlightVersion3::Int32, SilverlightVersion4::Int16, replaceAll(toValidUTF8(PageCharset), '\0', ''), - CodeVersion::Int32, IsLink, IsDownload, IsNotBounce, FUniqID::Int64, replaceAll(toValidUTF8(OriginalURL), '\0', ''), HID::Int32, IsOldCounter, IsEvent, - IsParameter, DontCountHits, WithHash, replaceAll(toValidUTF8(HitColor::String), '\0', ''), LocalEventTime, Age, Sex, Income, Interests::Int16, Robotness, RemoteIP::Int32, - WindowName, OpenerName, HistoryLength, replaceAll(toValidUTF8(BrowserLanguage::String), '\0', ''), replaceAll(toValidUTF8(BrowserCountry::String), '\0', ''), - replaceAll(toValidUTF8(SocialNetwork), '\0', ''), replaceAll(toValidUTF8(SocialAction), '\0', ''), - HTTPError, least(SendTiming, 30000), least(DNSTiming, 30000), least(ConnectTiming, 30000), least(ResponseStartTiming, 30000), - least(ResponseEndTiming, 30000), least(FetchTiming, 30000), SocialSourceNetworkID, - replaceAll(toValidUTF8(SocialSourcePage), '\0', ''), ParamPrice, replaceAll(toValidUTF8(ParamOrderID), '\0', ''), replaceAll(toValidUTF8(ParamCurrency::String), '\0', ''), - ParamCurrencyID::Int16, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, - UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash::Int64, URLHash::Int64, CLID::Int32 -FROM hits_100m_obfuscated -INTO OUTFILE 'dump.csv' -FORMAT CSV -``` - -WTF: - -``` -tutorial=# SELECT count(*) FROM hits_100m_obfuscated; -ERROR: could not load library "/usr/lib/postgresql/13/lib/llvmjit.so": libLLVM-6.0.so.1: cannot open shared object file: No such file or directory -``` - -Maybe just install LLVM? - -``` -sudo apt install llvm -``` - -It does not help: - -``` -milovidov@mtlog-perftest03j:~$ sudo -u postgres psql -psql (13.4 (Ubuntu 13.4-4.pgdg18.04+1)) -Type "help" for help. - -postgres=# \c tutorial -You are now connected to database "tutorial" as user "postgres". -tutorial=# SELECT count(*) FROM hits_100m_obfuscated; -ERROR: could not load library "/usr/lib/postgresql/13/lib/llvmjit.so": libLLVM-6.0.so.1: cannot open shared object file: No such file or directory -tutorial=# -``` - -Dependency on system libraries is harmful. - -``` -milovidov@mtlog-perftest03j:~$ ls -l /usr/lib/x86_64-linux-gnu/libLLVM-6.0.so -lrwxrwxrwx 1 root root 16 Apr 6 2018 /usr/lib/x86_64-linux-gnu/libLLVM-6.0.so -> libLLVM-6.0.so.1 -milovidov@mtlog-perftest03j:~$ ls -l /usr/lib/x86_64-linux-gnu/libLLVM-6.0.so.1 -ls: cannot access '/usr/lib/x86_64-linux-gnu/libLLVM-6.0.so.1': No such file or directory -``` - -https://askubuntu.com/questions/481/how-do-i-find-the-package-that-provides-a-file - -``` -milovidov@mtlog-perftest03j:~$ dpkg -S libLLVM-6.0.so.1 -llvm-6.0-dev: /usr/lib/llvm-6.0/lib/libLLVM-6.0.so.1 -libllvm6.0:amd64: /usr/lib/x86_64-linux-gnu/libLLVM-6.0.so.1 -``` - -Wow, it's absolutely broken: - -``` -milovidov@mtlog-perftest03j:~$ sudo apt remove llvm-6.0-dev -Reading package lists... Done -Building dependency tree -Reading state information... Done -The following packages were automatically installed and are no longer required: - libcgal13 libgmpxx4ldbl liblldb-11 libprotobuf-c1 libsfcgal1 mysql-server-core-5.7 -Use 'sudo apt autoremove' to remove them. -The following packages will be REMOVED: - liblld-6.0-dev lld lld-6.0 llvm-6.0-dev -0 upgraded, 0 newly installed, 4 to remove and 293 not upgraded. -After this operation, 163 MB disk space will be freed. -Do you want to continue? [Y/n] -(Reading database ... 268641 files and directories currently installed.) -Removing liblld-6.0-dev (1:6.0-1ubuntu2) ... -Removing lld (1:6.0-41~exp5~ubuntu1) ... -Removing lld-6.0 (1:6.0-1ubuntu2) ... -Removing llvm-6.0-dev (1:6.0-1ubuntu2) ... -Processing triggers for man-db (2.8.3-2ubuntu0.1) ... -Processing triggers for libc-bin (2.27-3ubuntu1.4) ... -milovidov@mtlog-perftest03j:~$ sudo apt install llvm-6.0-dev -Reading package lists... Done -Building dependency tree -Reading state information... Done -The following packages were automatically installed and are no longer required: - libcgal13 libgmpxx4ldbl liblldb-11 libprotobuf-c1 libsfcgal1 mysql-server-core-5.7 -Use 'sudo apt autoremove' to remove them. -The following NEW packages will be installed: - llvm-6.0-dev -0 upgraded, 1 newly installed, 0 to remove and 293 not upgraded. -Need to get 23.0 MB of archives. -After this operation, 160 MB of additional disk space will be used. -Get:1 http://mirror.yandex.ru/ubuntu bionic/main amd64 llvm-6.0-dev amd64 1:6.0-1ubuntu2 [23.0 MB] -Fetched 23.0 MB in 1s (42.5 MB/s) -Selecting previously unselected package llvm-6.0-dev. -(Reading database ... 267150 files and directories currently installed.) -Preparing to unpack .../llvm-6.0-dev_1%3a6.0-1ubuntu2_amd64.deb ... -Unpacking llvm-6.0-dev (1:6.0-1ubuntu2) ... -Setting up llvm-6.0-dev (1:6.0-1ubuntu2) ... -Processing triggers for libc-bin (2.27-3ubuntu1.4) ... -milovidov@mtlog-perftest03j:~$ ls -l /usr/lib/x86_64-linux-gnu/libLLVM-6.0.so -lrwxrwxrwx 1 root root 16 Apr 6 2018 /usr/lib/x86_64-linux-gnu/libLLVM-6.0.so -> libLLVM-6.0.so.1 -milovidov@mtlog-perftest03j:~$ ls -l /usr/lib/x86_64-linux-gnu/libLLVM-6.0.so.1 -ls: cannot access '/usr/lib/x86_64-linux-gnu/libLLVM-6.0.so.1': No such file or directory -``` - -Let's remove just in case: - -``` -sudo apt remove llvm-6.0-dev -``` - -https://dba.stackexchange.com/questions/264955/handling-performance-problems-with-jit-in-postgres-12 - -JIT can be disabled by `set jit = off;` - -``` -tutorial=# set jit = off; -SET -tutorial=# -tutorial=# SELECT count(*) FROM hits_100m_obfuscated; -``` - -But now this SELECT query started and hanged for multiple minutes without any result. -And I see something strange in `top`: - -``` - 792553 postgres 20 0 32.418g 0.031t 0.031t D 2.4 25.3 3:43.84 postgres: 13/main: checkpointer - 814659 postgres 20 0 32.432g 0.023t 0.023t D 0.0 18.8 0:14.53 postgres: 13/main: parallel worker for PID 813980 - 813980 postgres 20 0 32.433g 0.023t 0.023t D 0.0 18.4 0:14.47 postgres: 13/main: postgres tutorial [local] SELECT - 814657 postgres 20 0 32.432g 0.016t 0.016t D 0.0 12.6 0:09.83 postgres: 13/main: parallel worker for PID 813980 - 814658 postgres 20 0 32.432g 0.015t 0.015t D 2.4 12.6 0:09.45 postgres: 13/main: parallel worker for PID 813980 - 814656 postgres 20 0 32.432g 0.015t 0.015t D 0.0 12.0 0:07.36 postgres: 13/main: parallel worker for PID 813980 - 792554 postgres 20 0 32.417g 5.394g 5.392g D 0.0 4.3 0:04.78 postgres: 13/main: background writer -``` - -The query did not finish in 30 minutes. How it can be so enormously slow? - - -Loading failed, again: - -``` -$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -connection 'host=localhost password=12345' -panic: pq: extra data after last expected column - -goroutine 14 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 20m57.936s -user 4m14.444s -sys 1m11.412s -``` - -Most likely PostgreSQL cannot recognize proper CSV escaping of quotes like `"Hello "" world"`. -Let's simply remove all double quotes from String values. - -``` -rm dump.csv - -SELECT WatchID::Int64, JavaEnable, replaceAll(replaceAll(toValidUTF8(Title), '\0', ''), '"', ''), GoodEvent, EventTime, EventDate, CounterID::Int32, ClientIP::Int32, RegionID::Int32, - UserID::Int64, CounterClass, OS, UserAgent, replaceAll(replaceAll(toValidUTF8(URL), '\0', ''), '"', ''), replaceAll(replaceAll(toValidUTF8(Referer), '\0', ''), '"', ''), Refresh, RefererCategoryID::Int16, RefererRegionID::Int32, - URLCategoryID::Int16, URLRegionID::Int32, ResolutionWidth::Int16, ResolutionHeight::Int16, ResolutionDepth, FlashMajor, FlashMinor, - FlashMinor2, NetMajor, NetMinor, UserAgentMajor::Int16, replaceAll(replaceAll(toValidUTF8(UserAgentMinor::String), '\0', ''), '"', ''), CookieEnable, JavascriptEnable, IsMobile, MobilePhone, - replaceAll(replaceAll(toValidUTF8(MobilePhoneModel), '\0', ''), '"', ''), replaceAll(replaceAll(toValidUTF8(Params), '\0', ''), '"', ''), IPNetworkID::Int32, TraficSourceID, SearchEngineID::Int16, replaceAll(replaceAll(toValidUTF8(SearchPhrase), '\0', ''), '"', ''), - AdvEngineID, IsArtifical, WindowClientWidth::Int16, WindowClientHeight::Int16, ClientTimeZone, ClientEventTime, - SilverlightVersion1, SilverlightVersion2, SilverlightVersion3::Int32, SilverlightVersion4::Int16, replaceAll(replaceAll(toValidUTF8(PageCharset), '\0', ''), '"', ''), - CodeVersion::Int32, IsLink, IsDownload, IsNotBounce, FUniqID::Int64, replaceAll(replaceAll(toValidUTF8(OriginalURL), '\0', ''), '"', ''), HID::Int32, IsOldCounter, IsEvent, - IsParameter, DontCountHits, WithHash, replaceAll(replaceAll(toValidUTF8(HitColor::String), '\0', ''), '"', ''), LocalEventTime, Age, Sex, Income, Interests::Int16, Robotness, RemoteIP::Int32, - WindowName, OpenerName, HistoryLength, replaceAll(replaceAll(toValidUTF8(BrowserLanguage::String), '\0', ''), '"', ''), replaceAll(replaceAll(toValidUTF8(BrowserCountry::String), '\0', ''), '"', ''), - replaceAll(replaceAll(toValidUTF8(SocialNetwork), '\0', ''), '"', ''), replaceAll(replaceAll(toValidUTF8(SocialAction), '\0', ''), '"', ''), - HTTPError, least(SendTiming, 30000), least(DNSTiming, 30000), least(ConnectTiming, 30000), least(ResponseStartTiming, 30000), - least(ResponseEndTiming, 30000), least(FetchTiming, 30000), SocialSourceNetworkID, - replaceAll(replaceAll(toValidUTF8(SocialSourcePage), '\0', ''), '"', ''), ParamPrice, replaceAll(replaceAll(toValidUTF8(ParamOrderID), '\0', ''), '"', ''), replaceAll(replaceAll(toValidUTF8(ParamCurrency::String), '\0', ''), '"', ''), - ParamCurrencyID::Int16, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, - UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash::Int64, URLHash::Int64, CLID::Int32 -FROM hits_100m_obfuscated -INTO OUTFILE 'dump.csv' -FORMAT CSV -``` - -Oops, another trouble: - -``` -$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -connection 'host=localhost password=12345' -panic: pq: unterminated CSV quoted field - -goroutine 19 [running]: -main.processBatches(0xc000132350, 0xc000136660) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m38.278s -user 0m13.544s -sys 0m3.552s -``` - -I have hypothesis, maybe it is interpreting both backslashes and quotes in CSV? -We need to check, what is CSV, exactly, from TimescaleDB's standpoint. - -https://www.postgresql.org/docs/9.2/sql-copy.html - -Yes, PostgreSQL is using "fake CSV": - -> This format option is used for importing and exporting the Comma Separated Value (CSV) file format used by many other programs, such as spreadsheets. Instead of the escaping rules used by PostgreSQL's standard text format, it produces and recognizes the common CSV escaping mechanism. - -> The values in each record are separated by the DELIMITER character. If the value contains the delimiter character, the QUOTE character, the NULL string, a carriage return, or line feed character, then the whole value is prefixed and suffixed by the QUOTE character, and any occurrence within the value of a QUOTE character or the ESCAPE character is preceded by the escape character. - -So, it looks like CSV but is using C-style backslash escapes inside values. -Let's remove both backslash and quote from our strings to make PostgreSQL happy. - -``` -rm dump.csv - -SELECT WatchID::Int64, JavaEnable, replaceAll(replaceAll(replaceAll(toValidUTF8(Title), '\0', ''), '"', ''), '\\', ''), GoodEvent, EventTime, EventDate, CounterID::Int32, ClientIP::Int32, RegionID::Int32, - UserID::Int64, CounterClass, OS, UserAgent, replaceAll(replaceAll(replaceAll(toValidUTF8(URL), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(Referer), '\0', ''), '"', ''), '\\', ''), Refresh, RefererCategoryID::Int16, RefererRegionID::Int32, - URLCategoryID::Int16, URLRegionID::Int32, ResolutionWidth::Int16, ResolutionHeight::Int16, ResolutionDepth, FlashMajor, FlashMinor, - FlashMinor2, NetMajor, NetMinor, UserAgentMajor::Int16, replaceAll(replaceAll(replaceAll(toValidUTF8(UserAgentMinor::String), '\0', ''), '"', ''), '\\', ''), CookieEnable, JavascriptEnable, IsMobile, MobilePhone, - replaceAll(replaceAll(replaceAll(toValidUTF8(MobilePhoneModel), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(Params), '\0', ''), '"', ''), '\\', ''), IPNetworkID::Int32, TraficSourceID, SearchEngineID::Int16, replaceAll(replaceAll(replaceAll(toValidUTF8(SearchPhrase), '\0', ''), '"', ''), '\\', ''), - AdvEngineID, IsArtifical, WindowClientWidth::Int16, WindowClientHeight::Int16, ClientTimeZone, ClientEventTime, - SilverlightVersion1, SilverlightVersion2, SilverlightVersion3::Int32, SilverlightVersion4::Int16, replaceAll(replaceAll(replaceAll(toValidUTF8(PageCharset), '\0', ''), '"', ''), '\\', ''), - CodeVersion::Int32, IsLink, IsDownload, IsNotBounce, FUniqID::Int64, replaceAll(replaceAll(replaceAll(toValidUTF8(OriginalURL), '\0', ''), '"', ''), '\\', ''), HID::Int32, IsOldCounter, IsEvent, - IsParameter, DontCountHits, WithHash, replaceAll(replaceAll(replaceAll(toValidUTF8(HitColor::String), '\0', ''), '"', ''), '\\', ''), LocalEventTime, Age, Sex, Income, Interests::Int16, Robotness, RemoteIP::Int32, - WindowName, OpenerName, HistoryLength, replaceAll(replaceAll(replaceAll(toValidUTF8(BrowserLanguage::String), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(BrowserCountry::String), '\0', ''), '"', ''), '\\', ''), - replaceAll(replaceAll(replaceAll(toValidUTF8(SocialNetwork), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(SocialAction), '\0', ''), '"', ''), '\\', ''), - HTTPError, least(SendTiming, 30000), least(DNSTiming, 30000), least(ConnectTiming, 30000), least(ResponseStartTiming, 30000), - least(ResponseEndTiming, 30000), least(FetchTiming, 30000), SocialSourceNetworkID, - replaceAll(replaceAll(replaceAll(toValidUTF8(SocialSourcePage), '\0', ''), '"', ''), '\\', ''), ParamPrice, replaceAll(replaceAll(replaceAll(toValidUTF8(ParamOrderID), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(ParamCurrency::String), '\0', ''), '"', ''), '\\', ''), - ParamCurrencyID::Int16, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, - UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash::Int64, URLHash::Int64, CLID::Int32 -FROM hits_100m_obfuscated -INTO OUTFILE 'dump.csv' -FORMAT CSV -``` - -It does not work at all: - -``` -$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.csv --workers 16 --copy-options "CSV" -connection 'host=localhost password=12345' -panic: pq: invalid input syntax for type bigint: " ПЕСНЮ ПРЕСТИВАРКЕ ДОЛЖНО ЛИ,1,306,31432,304,22796,1011,879,37,15,5,700.224,2,7,13,D�,1,1,0,0,",",3039109,-1,0,",0,0,779,292,135,2013-07-31 09:37:12,0,0,0,0,windows,1,0,0,0,6888403766694734958,http%3A//maps&sort_order_Kurzarm_DOB&sr=http%3A%2F%3Fpage=/ok.html?1=1&cid=577&oki=1&op_seo_entry=&op_uid=13225;IC" - -goroutine 20 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 1m47.915s -user 0m33.676s -sys 0m8.028s -``` - -Maybe let's switch from CSV to TSV that PostgreSQL seems to understand better. - -``` -SELECT WatchID::Int64, JavaEnable, replaceAll(replaceAll(replaceAll(toValidUTF8(Title), '\0', ''), '"', ''), '\\', ''), GoodEvent, EventTime, EventDate, CounterID::Int32, ClientIP::Int32, RegionID::Int32, - UserID::Int64, CounterClass, OS, UserAgent, replaceAll(replaceAll(replaceAll(toValidUTF8(URL), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(Referer), '\0', ''), '"', ''), '\\', ''), Refresh, RefererCategoryID::Int16, RefererRegionID::Int32, - URLCategoryID::Int16, URLRegionID::Int32, ResolutionWidth::Int16, ResolutionHeight::Int16, ResolutionDepth, FlashMajor, FlashMinor, - FlashMinor2, NetMajor, NetMinor, UserAgentMajor::Int16, replaceAll(replaceAll(replaceAll(toValidUTF8(UserAgentMinor::String), '\0', ''), '"', ''), '\\', ''), CookieEnable, JavascriptEnable, IsMobile, MobilePhone, - replaceAll(replaceAll(replaceAll(toValidUTF8(MobilePhoneModel), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(Params), '\0', ''), '"', ''), '\\', ''), IPNetworkID::Int32, TraficSourceID, SearchEngineID::Int16, replaceAll(replaceAll(replaceAll(toValidUTF8(SearchPhrase), '\0', ''), '"', ''), '\\', ''), - AdvEngineID, IsArtifical, WindowClientWidth::Int16, WindowClientHeight::Int16, ClientTimeZone, ClientEventTime, - SilverlightVersion1, SilverlightVersion2, SilverlightVersion3::Int32, SilverlightVersion4::Int16, replaceAll(replaceAll(replaceAll(toValidUTF8(PageCharset), '\0', ''), '"', ''), '\\', ''), - CodeVersion::Int32, IsLink, IsDownload, IsNotBounce, FUniqID::Int64, replaceAll(replaceAll(replaceAll(toValidUTF8(OriginalURL), '\0', ''), '"', ''), '\\', ''), HID::Int32, IsOldCounter, IsEvent, - IsParameter, DontCountHits, WithHash, replaceAll(replaceAll(replaceAll(toValidUTF8(HitColor::String), '\0', ''), '"', ''), '\\', ''), LocalEventTime, Age, Sex, Income, Interests::Int16, Robotness, RemoteIP::Int32, - WindowName, OpenerName, HistoryLength, replaceAll(replaceAll(replaceAll(toValidUTF8(BrowserLanguage::String), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(BrowserCountry::String), '\0', ''), '"', ''), '\\', ''), - replaceAll(replaceAll(replaceAll(toValidUTF8(SocialNetwork), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(SocialAction), '\0', ''), '"', ''), '\\', ''), - HTTPError, least(SendTiming, 30000), least(DNSTiming, 30000), least(ConnectTiming, 30000), least(ResponseStartTiming, 30000), - least(ResponseEndTiming, 30000), least(FetchTiming, 30000), SocialSourceNetworkID, - replaceAll(replaceAll(replaceAll(toValidUTF8(SocialSourcePage), '\0', ''), '"', ''), '\\', ''), ParamPrice, replaceAll(replaceAll(replaceAll(toValidUTF8(ParamOrderID), '\0', ''), '"', ''), '\\', ''), replaceAll(replaceAll(replaceAll(toValidUTF8(ParamCurrency::String), '\0', ''), '"', ''), '\\', ''), - ParamCurrencyID::Int16, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, - UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash::Int64, URLHash::Int64, CLID::Int32 -FROM hits_100m_obfuscated -INTO OUTFILE 'dump.tsv' -FORMAT TSV -``` - -But how to pass TSV to `timescaledb-parallel-copy` tool? - -``` -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.tsv --workers 16 -connection 'host=localhost password=12345' panic: pq: invalid input syntax for type bigint: "9076997425961590393\t0\tКино\t1\t2013-07-06 17:47:29\t2013-07-06\t225510\t-1056921538\t229\t3467937489264290637\t0\t2\t3\thttp://liver.ru/belgorod/page/1006.jки/доп_приборы\thttp://video.yandex.ru/1.561.540.000703/?order_Kurzarm_alia\t0\t16124\t20\t14328\t22\t1638\t1658\t23\t15\t7\t700\t0\t0\t17\tD�\t1\t1\t0\t0\t\t\t2095433\t-1\t0\t\t0\t1\t1369\t713\t135\t2013-07-06 16:25:42\t0\t0\t0\t0\twindows\t1601\t0\t0\t0\t5566829288329160346\t\t940752990\t0\t0\t0\t0\t0\t5\t2013-07-06 01:32:13\t55\t2\t3\t0\t2\t-1352932082\t-1\t-1\t-1\tS0\t�\\f\t\t\t0\t0\t0\t0\t0\t0\t0\t0\t\t0\t\tNH\t0\t\t\t\t\t\t\t\t\t\t\t0\t6811023348165660452\t7011450103338277684\t0" - -goroutine 20 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m0.304s -user 0m0.044s -sys 0m0.044s -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.tsv --copy-options "TEXT" --workers 16 -connection 'host=localhost password=12345' -panic: pq: syntax error at or near "TEXT" - -goroutine 18 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m0.044s -user 0m0.048s -sys 0m0.036s -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.tsv --copy-options "text" --workers 16 -connection 'host=localhost password=12345' -panic: pq: syntax error at or near "text" - -goroutine 18 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb -panic: pq: syntax error at or near "text" - -goroutine 19 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m0.057s -user 0m0.060s -sys 0m0.028s -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.tsv --copy-options "Text" --workers 16 -connection 'host=localhost password=12345' -panic: pq: syntax error at or near "Text" - -goroutine 11 [running]: -main.processBatches(0xc0000183d0, 0xc0000a66c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m0.041s -user 0m0.052s -sys 0m0.032s -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.tsv --copy-options "FORMAT text" --workers 16 -connection 'host=localhost password=12345' -panic: pq: syntax error at or near "FORMAT" - -goroutine 21 [running]: -main.processBatches(0xc00019a350, 0xc00019e660) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m0.045s -user 0m0.052s -sys 0m0.028s -``` - -Nothing works: - -``` -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --help -Usage of timescaledb-parallel-copy: - -batch-size int - Number of rows per insert (default 5000) - -columns string - Comma-separated columns present in CSV - -connection string - PostgreSQL connection url (default "host=localhost user=postgres sslmode=disable") - -copy-options string - Additional options to pass to COPY (e.g., NULL 'NULL') (default "CSV") - -db-name string - Database where the destination table exists - -file string - File to read from rather than stdin - -header-line-count int - Number of header lines (default 1) - -limit int - Number of rows to insert overall; 0 means to insert all - -log-batches - Whether to time individual batches. - -reporting-period duration - Period to report insert stats; if 0s, intermediate results will not be reported - -schema string - Destination table's schema (default "public") - -skip-header - Skip the first line of the input - -split string - Character to split by (default ",") - -table string - Destination table for insertions (default "test_table") - -token-size int - Maximum size to use for tokens. By default, this is 64KB, so any value less than that will be ignored (default 65536) - -truncate - Truncate the destination table before insert - -verbose - Print more information about copying statistics - -version - Show the version of this tool - -workers int - Number of parallel requests to make (default 1) - -real 0m0.009s -user 0m0.004s -sys 0m0.000s -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.tsv --truncate --copy-options "" --workers 16 -connection 'host=localhost password=12345' -panic: pq: invalid input syntax for type bigint: "9076997425961590393 0 Кино 1 2013-07-06 17:47:29 2013-07-06 225510 -1056921538 229 3467937489264290637 0 2 3http://liver.ru/belgorod/page/1006.jки/доп_приборы http://video.yandex.ru/1.561.540.000703/?order_Kurzarm_alia 0 16124 20 14328 22 1638 1658 23 15 7 700 0017 D� 1 1 0 0 2095433 -1 0 0 1 1369 713 135 2013-07-06 16:25:42 0 0 0 0 windows 1601 000 5566829288329160346 940752990 0 0 0 0 0 5 2013-07-06 01:32:13 55 2 3 0 2 -1352932082 -1 -1 -1 S0�\f 0 0 0 0 0 0 0 0 0 NH 0 06811023348165660452 7011450103338277684 0" - -goroutine 13 [running]: -main.processBatches(0xc000019140, 0xc0001eb080) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m0.191s -user 0m0.036s -sys 0m0.040s -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.tsv --truncate --copy-options "NULL AS '\N'" --workers 16 -connection 'host=localhost password=12345' -panic: pq: invalid input syntax for type bigint: "9076997425961590393 0 Кино 1 2013-07-06 17:47:29 2013-07-06 225510 -1056921538 229 3467937489264290637 0 2 3http://liver.ru/belgorod/page/1006.jки/доп_приборы http://video.yandex.ru/1.561.540.000703/?order_Kurzarm_alia 0 16124 20 14328 22 1638 1658 23 15 7 700 0017 D� 1 1 0 0 2095433 -1 0 0 1 1369 713 135 2013-07-06 16:25:42 0 0 0 0 windows 1601 000 5566829288329160346 940752990 0 0 0 0 0 5 2013-07-06 01:32:13 55 2 3 0 2 -1352932082 -1 -1 -1 S0�\f 0 0 0 0 0 0 0 0 0 NH 0 06811023348165660452 7011450103338277684 0" - -goroutine 11 [running]: -main.processBatches(0xc000018900, 0xc0002886c0) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m0.187s -user 0m0.020s -sys 0m0.048s -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.tsv --truncate --copy-options "DELIMITER AS '\t'" --workers 16 -connection 'host=localhost password=12345' -panic: pq: conflicting or redundant options - -goroutine 13 [running]: -main.processBatches(0xc000019140, 0xc0001e9080) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m0.196s -user 0m0.048s -sys 0m0.020s -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.tsv --truncate --copy-options "TEXT DELIMITER AS '\t'" --workers 16 -connection 'host=localhost password=12345' -panic: pq: syntax error at or near "TEXT" - -goroutine 22 [running]: -main.processBatches(0xc000019140, 0xc0001e9080) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb -panic: pq: syntax error at or near "TEXT" - -goroutine 11 [running]: -main.processBatches(0xc000019140, 0xc0001e9080) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m0.191s -user 0m0.032s -sys 0m0.036s -milovidov@mtlog-perftest03j:~$ time sudo -u postgres timescaledb-parallel-copy --db-name tutorial --table hits_100m_obfuscated --file dump.tsv --truncate --copy-options "DELIMITER AS e'\t'" --workers 16 -connection 'host=localhost password=12345' -panic: pq: conflicting or redundant options - -goroutine 26 [running]: -main.processBatches(0xc0001330d0, 0xc0001e3020) - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:262 +0x879 -created by main.main - /home/builder/go/src/github.com/timescale/timescaledb-parallel-copy/cmd/timescaledb-parallel-copy/main.go:148 +0x1bb - -real 0m0.169s -user 0m0.056s -sys 0m0.016s -``` - -I will try to avoid `timescaledb-parallel-copy` and use `psql` instead. - -``` -milovidov@mtlog-perftest03j:~$ sudo -u postgres psql -psql (13.4 (Ubuntu 13.4-4.pgdg18.04+1)) -Type "help" for help. - -postgres=# \c tutorial -You are now connected to database "tutorial" as user "postgres". -tutorial=# timing -tutorial-# COPY hits_100m_obfuscated FROM 'dump.tsv' -tutorial-# ; -ERROR: syntax error at or near "timing" -LINE 1: timing - ^ -tutorial=# \timing -Timing is on. -tutorial=# COPY hits_100m_obfuscated FROM 'dump.tsv'; -ERROR: could not open file "dump.tsv" for reading: No such file or directory -HINT: COPY FROM instructs the PostgreSQL server process to read a file. You may want a client-side facility such as psql's \copy. -Time: 4.348 ms -tutorial=# \copy hits_100m_obfuscated FROM 'dump.tsv'; -``` - -It started to do something... fairly slow with using less than one CPU core. - -Folks from TimescaleDB always recommend to enable compression, which is not by default. -Let's read about it: - -https://docs.timescale.com/timescaledb/latest/how-to-guides/compression/ - -> We strongly recommend that you understand how compression works before you start enabling it on your hypertables. - -The amount of hackery to overcome PostgreSQL limitations is overwhelming: - -> When compression is enabled, TimescaleDB converts data stored in many rows into an array. This means that instead of using lots of rows to store the data, it stores the same data in a single row. - -In the meantime, copy finished in "just" 1.5 hours, 19 245 rows/second. This is extremely slow, even for single core. - -``` -tutorial=# \copy hits_100m_obfuscated FROM 'dump.tsv'; -COPY 100000000 -Time: 5195909.154 ms (01:26:35.909) -``` - -## Running Benchmark - -Let's prepare for benchmark... -What is needed to execute single query in batch mode? - -`man psql` - -``` -sudo -u postgres psql tutorial -t -c '\timing' -c 'SELECT 1' | grep 'Time' -``` - -Now we are ready to run our benchmark. - -PostgreSQL does not have `SHOW PROCESSLIST`. -It has `select * from pg_stat_activity;` instead. - -https://ma.ttias.be/show-full-processlist-equivalent-of-mysql-for-postgresql/ - -But it does not show query progress. -The first query `SELECT count(*) FROM hits_100m_obfuscated` just hanged. It reads something from disk... - -Let's check the data volume: - -``` -$ sudo du -hcs /opt/postgresql/ -68G /opt/postgresql/ -``` - -Looks consistent for uncompressed data. - -``` -./benchmark.sh - -grep -oP 'Time: \d+' log | grep -oP '\d+' | awk '{ if (n % 3 == 0) { printf("[") }; ++n; printf("%g", $1 / 1000); if (n % 3 == 0) { printf("],\n") } else { printf(", ") } }' -``` - -Now let's enable compression. - -``` -ALTER TABLE hits_100m_obfuscated SET (timescaledb.compress); -SELECT add_compression_policy('hits_100m_obfuscated', INTERVAL '0 seconds'); -``` - -``` -milovidov@mtlog-perftest03j:~ClickHouse/benchmark/timescaledb$ sudo -u postgres psql tutorial -psql (13.4 (Ubuntu 13.4-4.pgdg18.04+1)) -Type "help" for help. - -tutorial=# ALTER TABLE hits_100m_obfuscated SET (timescaledb.compress); -ALTER TABLE -tutorial=# SELECT add_compression_policy('hits_100m_obfuscated', INTERVAL '0 seconds'); - add_compression_policy ------------------------- - 1000 -(1 row) -``` - -Ok, in `top` I see that it started compression with using single CPU core. - -``` -300464 postgres 20 0 32.456g 932044 911452 D 48.0 0.7 1:08.11 postgres: 13/main: Compression Policy [1000] -``` - -Let's also define better order of data: - -``` -ALTER TABLE hits_100m_obfuscated - SET (timescaledb.compress, - timescaledb.compress_orderby = 'counterid, userid, event_time'); -``` - -The query hanged. Maybe it's waiting for finish of previous compression? - -After several minutes it answered: - -``` -ERROR: cannot change configuration on already compressed chunks -DETAIL: There are compressed chunks that prevent changing the existing compression configuration. -``` - -Ok, at least some of the chunks will have the proper order. - -After a few hours looks like the compression finished. - -``` -sudo ncdu /var/lib/postgresql/13/main/ - -28.9 GiB [##########] /base -``` - -Yes, looks like it's compressed. About two times - not too much. - -Let's rerun the benchmark. - -Ok, it's slightly faster. diff --git a/benchmark/trino/README.md b/benchmark/trino/README.md new file mode 100644 index 00000000000..82b0b2ff959 --- /dev/null +++ b/benchmark/trino/README.md @@ -0,0 +1 @@ +Incomplete. diff --git a/benchmark/trino/benchmark.sh b/benchmark/trino/benchmark.sh new file mode 100644 index 00000000000..46a8ae06014 --- /dev/null +++ b/benchmark/trino/benchmark.sh @@ -0,0 +1,7 @@ +#!/bin/bash + +sudo apt-get update +sudo apt-get install -y docker.io +sudo docker run --network host -p 8080:8080 --name trino trinodb/trino + +sudo docker exec -it trino trino diff --git a/benchmark/trino/create_partitioned.sql b/benchmark/trino/create_partitioned.sql new file mode 100644 index 00000000000..b0f9e5d0be5 --- /dev/null +++ b/benchmark/trino/create_partitioned.sql @@ -0,0 +1,112 @@ +CREATE EXTERNAL TABLE IF NOT EXISTS `test`.`hits` ( +`watchid` bigint, +`javaenable` smallint, +`title` string, +`goodevent` smallint, +`eventtime` timestamp, +`eventdate` date, +`counterid` int, +`clientip` int, +`regionid` int, +`userid` bigint, +`counterclass` smallint, +`os` smallint, +`useragent` smallint, +`url` string, +`referer` string, +`isrefresh` smallint, +`referercategoryid` smallint, +`refererregionid` int, +`urlcategoryid` smallint, +`urlregionid` int, +`resolutionwidth` smallint, +`resolutionheight` smallint, +`resolutiondepth` smallint, +`flashmajor` smallint, +`flashminor` smallint, +`flashminor2` string, +`netmajor` smallint, +`netminor` smallint, +`useragentmajor` smallint, +`useragentminor` string, +`cookieenable` smallint, +`javascriptenable` smallint, +`ismobile` smallint, +`mobilephone` smallint, +`mobilephonemodel` string, +`params` string, +`ipnetworkid` int, +`traficsourceid` smallint, +`searchengineid` smallint, +`searchphrase` string, +`advengineid` smallint, +`isartifical` smallint, +`windowclientwidth` smallint, +`windowclientheight` smallint, +`clienttimezone` smallint, +`clienteventtime` timestamp, +`silverlightversion1` smallint, +`silverlightversion2` smallint, +`silverlightversion3` int, +`silverlightversion4` smallint, +`pagecharset` string, +`codeversion` int, +`islink` smallint, +`isdownload` smallint, +`isnotbounce` smallint, +`funiqid` bigint, +`originalurl` string, +`hid` int, +`isoldcounter` smallint, +`isevent` smallint, +`isparameter` smallint, +`dontcounthits` smallint, +`withhash` smallint, +`hitcolor` string, +`localeventtime` timestamp, +`age` smallint, +`sex` smallint, +`income` smallint, +`interests` smallint, +`robotness` smallint, +`remoteip` int, +`windowname` int, +`openername` int, +`historylength` smallint, +`browserlanguage` string, +`browsercountry` string, +`socialnetwork` string, +`socialaction` string, +`httperror` smallint, +`sendtiming` int, +`dnstiming` int, +`connecttiming` int, +`responsestarttiming` int, +`responseendtiming` int, +`fetchtiming` int, +`socialsourcenetworkid` smallint, +`socialsourcepage` string, +`paramprice` bigint, +`paramorderid` string, +`paramcurrency` string, +`paramcurrencyid` smallint, +`openstatservicename` string, +`openstatcampaignid` string, +`openstatadid` string, +`openstatsourceid` string, +`utmsource` string, +`utmmedium` string, +`utmcampaign` string, +`utmcontent` string, +`utmterm` string, +`fromtag` string, +`hasgclid` smallint, +`refererhash` bigint, +`urlhash` bigint, +`clid` int +) +ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' +WITH SERDEPROPERTIES ( +'serialization.format' = '1' +) LOCATION 's3://clickhouse-public-datasets/hits_compatible/athena_partitioned' +TBLPROPERTIES ('has_encrypted_data'='false'); diff --git a/benchmark/trino/create_single.sql b/benchmark/trino/create_single.sql new file mode 100644 index 00000000000..2dddf94d70e --- /dev/null +++ b/benchmark/trino/create_single.sql @@ -0,0 +1,108 @@ +CREATE TABLE IF NOT EXISTS "test"."test"."hits" ( +"watchid" bigint, +"javaenable" smallint, +"title" string, +"goodevent" smallint, +"eventtime" timestamp, +"eventdate" date, +"counterid" int, +"clientip" int, +"regionid" int, +"userid" bigint, +"counterclass" smallint, +"os" smallint, +"useragent" smallint, +"url" string, +"referer" string, +"isrefresh" smallint, +"referercategoryid" smallint, +"refererregionid" int, +"urlcategoryid" smallint, +"urlregionid" int, +"resolutionwidth" smallint, +"resolutionheight" smallint, +"resolutiondepth" smallint, +"flashmajor" smallint, +"flashminor" smallint, +"flashminor2" string, +"netmajor" smallint, +"netminor" smallint, +"useragentmajor" smallint, +"useragentminor" string, +"cookieenable" smallint, +"javascriptenable" smallint, +"ismobile" smallint, +"mobilephone" smallint, +"mobilephonemodel" string, +"params" string, +"ipnetworkid" int, +"traficsourceid" smallint, +"searchengineid" smallint, +"searchphrase" string, +"advengineid" smallint, +"isartifical" smallint, +"windowclientwidth" smallint, +"windowclientheight" smallint, +"clienttimezone" smallint, +"clienteventtime" timestamp, +"silverlightversion1" smallint, +"silverlightversion2" smallint, +"silverlightversion3" int, +"silverlightversion4" smallint, +"pagecharset" string, +"codeversion" int, +"islink" smallint, +"isdownload" smallint, +"isnotbounce" smallint, +"funiqid" bigint, +"originalurl" string, +"hid" int, +"isoldcounter" smallint, +"isevent" smallint, +"isparameter" smallint, +"dontcounthits" smallint, +"withhash" smallint, +"hitcolor" string, +"localeventtime" timestamp, +"age" smallint, +"sex" smallint, +"income" smallint, +"interests" smallint, +"robotness" smallint, +"remoteip" int, +"windowname" int, +"openername" int, +"historylength" smallint, +"browserlanguage" string, +"browsercountry" string, +"socialnetwork" string, +"socialaction" string, +"httperror" smallint, +"sendtiming" int, +"dnstiming" int, +"connecttiming" int, +"responsestarttiming" int, +"responseendtiming" int, +"fetchtiming" int, +"socialsourcenetworkid" smallint, +"socialsourcepage" string, +"paramprice" bigint, +"paramorderid" string, +"paramcurrency" string, +"paramcurrencyid" smallint, +"openstatservicename" string, +"openstatcampaignid" string, +"openstatadid" string, +"openstatsourceid" string, +"utmsource" string, +"utmmedium" string, +"utmcampaign" string, +"utmcontent" string, +"utmterm" string, +"fromtag" string, +"hasgclid" smallint, +"refererhash" bigint, +"urlhash" bigint, +"clid" int +) +WITH (FORMAT=Parquet); diff --git a/benchmark/compatible/monetdb/queries.sql b/benchmark/trino/queries.sql similarity index 78% rename from benchmark/compatible/monetdb/queries.sql rename to benchmark/trino/queries.sql index 31f65fc898d..ef8c727ead6 100644 --- a/benchmark/compatible/monetdb/queries.sql +++ b/benchmark/trino/queries.sql @@ -16,7 +16,7 @@ SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, extract(minute FROM EventTime), SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; SELECT UserID FROM hits WHERE UserID = 435090932899640449; SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; @@ -26,7 +26,7 @@ SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; 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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; @@ -34,10 +34,10 @@ SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FR SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; 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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC OFFSET 1000 LIMIT 10; +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 >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN (SearchEngineID = 0 AND AdvEngineID = 0) THEN Referer ELSE '' END, URL ORDER BY PageViews DESC OFFSET 1000 LIMIT 10; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC OFFSET 100 LIMIT 10; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC OFFSET 10000 LIMIT 10; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-14' AND EventDate <= DATE '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) OFFSET 1000 LIMIT 10; diff --git a/benchmark/compatible/vertica/.gitignore b/benchmark/vertica/.gitignore similarity index 100% rename from benchmark/compatible/vertica/.gitignore rename to benchmark/vertica/.gitignore diff --git a/benchmark/vertica/README b/benchmark/vertica/README deleted file mode 100644 index 91af3e41e27..00000000000 --- a/benchmark/vertica/README +++ /dev/null @@ -1,40 +0,0 @@ -Quick installation instructions -------------------------------- - -Register on my.vertica.com -https://my.vertica.com/download-community-edition/ -Download HP Vertica 7.1.1 Analytic Database Server, Debian or Ubuntu 14.04 version. - -sudo apt-get install sysstat pstack mcelog -sudo dpkg -i vertica_7.1.1-0_amd64.deb -sudo sh -c "echo 'export TZ=Europe/Moscow' >> /home/dbadmin/.bash_profile" -# Don't specify localhost due to poor support of IPv6. -sudo /opt/vertica/sbin/install_vertica --hosts=127.0.0.1 --failure-threshold=NONE -sudo mkdir /opt/vertica-data/ -sudo chown dbadmin /opt/vertica-data/ - -sudo su dbadmin -/opt/vertica/bin/adminTools - -configuration menu -create database -name: default -empty password -both directories: /opt/vertica-data/ -main menu -exit - -How to prepare data -------------------- - -Prepare dumps with script create_dump.sh for tables hits_10m, hits_100m, hits_1000m. It takes about 5 hours (1m41.882s, 25m11.103s, 276m36.388s). -Start vsql command line client. -/opt/vertica/bin/vsql -U dbadmin -Create tables with queries from hits_define_schema.sql. - -Time to insert data: -hits_10m: 91 sec. -hits_100m: 774 sec. -hits_1000m: 13769 sec. - -You need to validate number of rows with SELECT count(*). diff --git a/benchmark/compatible/vertica/README.md b/benchmark/vertica/README.md similarity index 100% rename from benchmark/compatible/vertica/README.md rename to benchmark/vertica/README.md diff --git a/benchmark/vertica/benchmark.sh b/benchmark/vertica/benchmark.sh index f3c6f22fde3..86312a3a438 100644 --- a/benchmark/vertica/benchmark.sh +++ b/benchmark/vertica/benchmark.sh @@ -1,24 +1,26 @@ -#!/usr/bin/env bash +#!/bin/bash -QUERIES_FILE="queries.sql" -TABLE=$1 -TRIES=3 +sudo apt-get update +sudo apt-get install -y docker.io -cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null +sudo docker run -p 5433:5433 -p 5444:5444 --volume $(pwd):/workdir --mount type=volume,source=vertica-data,target=/data --name vertica_ce vertica/vertica-ce - echo -n "[" - for i in $(seq 1 $TRIES); do +sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c "$(cat create.sql)" - RES=$((echo '\timing'; echo "$query") | - /opt/vertica/bin/vsql -U dbadmin | - grep -oP 'All rows formatted: [^ ]+ ms' | - ssed -R -e 's/^All rows formatted: ([\d,]+) ms$/\1/' | - tr ',' '.') +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz - [[ "$?" == "0" ]] && echo -n "$(perl -e "print ${RES} / 1000")" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - done - echo "]," -done +time sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM LOCAL '/workdir/hits.tsv' DELIMITER E'\\t' NULL E'\\001' DIRECT" + +sudo docker exec vertica_ce du -bcs /data/vertica/VMart + +./run.sh 2>&1 | tee log.txt + +# If you run the script on your own, you may get numbers like this: +# 200m00.000s +# 25000000000 + +# Note: the real numbers cannot be published. + +grep -F 'All rows formatted' logs.txt | sed -r -e 's/^.* ([0-9.]+) ms$/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/compatible/vertica/create.sql b/benchmark/vertica/create.sql similarity index 100% rename from benchmark/compatible/vertica/create.sql rename to benchmark/vertica/create.sql diff --git a/benchmark/vertica/hits_define_schema.sql b/benchmark/vertica/hits_define_schema.sql deleted file mode 100644 index 37c9d45ffca..00000000000 --- a/benchmark/vertica/hits_define_schema.sql +++ /dev/null @@ -1,339 +0,0 @@ -\timing - -create table hits_10m -( - WatchID INTEGER, - JavaEnable INTEGER, - Title VARCHAR(1024), - GoodEvent INTEGER, - EventTime DATETIME, - EventDate DATE, - CounterID INTEGER, - ClientIP INTEGER, - RegionID INTEGER, - UserID INTEGER, - CounterClass INTEGER, - OS INTEGER, - UserAgent INTEGER, - URL VARCHAR(6072), - Referer VARCHAR(2048), - Refresh INTEGER, - RefererCategoryID INTEGER, - RefererRegionID INTEGER, - URLCategoryID INTEGER, - URLRegionID INTEGER, - ResolutionWidth INTEGER, - ResolutionHeight INTEGER, - ResolutionDepth INTEGER, - FlashMajor INTEGER, - FlashMinor INTEGER, - FlashMinor2 VARCHAR(256), - NetMajor INTEGER, - NetMinor INTEGER, - UserAgentMajor INTEGER, - UserAgentMinor CHAR(2), - CookieEnable INTEGER, - JavascriptEnable INTEGER, - IsMobile INTEGER, - MobilePhone INTEGER, - MobilePhoneModel VARCHAR(80), - Params VARCHAR(2048), - IPNetworkID INTEGER, - TraficSourceID INTEGER, - SearchEngineID INTEGER, - SearchPhrase VARCHAR(1024), - AdvEngineID INTEGER, - IsArtifical INTEGER, - WindowClientWidth INTEGER, - WindowClientHeight INTEGER, - ClientTimeZone INTEGER, - ClientEventTime DATETIME, - SilverlightVersion1 INTEGER, - SilverlightVersion2 INTEGER, - SilverlightVersion3 INTEGER, - SilverlightVersion4 INTEGER, - PageCharset VARCHAR(80), - CodeVersion INTEGER, - IsLink INTEGER, - IsDownload INTEGER, - IsNotBounce INTEGER, - FUniqID INTEGER, - OriginalURL VARCHAR(6072), - HID INTEGER, - IsOldCounter INTEGER, - IsEvent INTEGER, - IsParameter INTEGER, - DontCountHits INTEGER, - WithHash INTEGER, - HitColor CHAR(1), - LocalEventTime DATETIME, - Age INTEGER, - Sex INTEGER, - Income INTEGER, - Interests INTEGER, - Robotness INTEGER, - RemoteIP INTEGER, - WindowName INTEGER, - OpenerName INTEGER, - HistoryLength INTEGER, - BrowserLanguage CHAR(2), - BrowserCountry CHAR(2), - SocialNetwork VARCHAR(128), - SocialAction VARCHAR(128), - HTTPError INTEGER, - SendTiming INTEGER, - DNSTiming INTEGER, - ConnectTiming INTEGER, - ResponseStartTiming INTEGER, - ResponseEndTiming INTEGER, - FetchTiming INTEGER, - SocialSourceNetworkID INTEGER, - SocialSourcePage VARCHAR(128), - ParamPrice INTEGER, - ParamOrderID VARCHAR(80), - ParamCurrency CHAR(3), - ParamCurrencyID INTEGER, - OpenstatServiceName VARCHAR(80), - OpenstatCampaignID VARCHAR(80), - OpenstatAdID VARCHAR(80), - OpenstatSourceID VARCHAR(80), - UTMSource VARCHAR(256), - UTMMedium VARCHAR(256), - UTMCampaign VARCHAR(256), - UTMContent VARCHAR(256), - UTMTerm VARCHAR(256), - FromTag VARCHAR(256), - HasGCLID INTEGER, - RefererHash INTEGER, - URLHash INTEGER, - CLID INTEGER -) ORDER BY CounterID, EventDate, UserID, EventTime; - -\set input_file '''/opt/dumps/hits_10m_corrected.tsv''' -COPY hits_10m FROM :input_file DELIMITER E'\t' DIRECT; - - -create table hits_100m -( - WatchID INTEGER, - JavaEnable INTEGER, - Title VARCHAR(1024), - GoodEvent INTEGER, - EventTime DATETIME, - EventDate DATE, - CounterID INTEGER, - ClientIP INTEGER, - RegionID INTEGER, - UserID INTEGER, - CounterClass INTEGER, - OS INTEGER, - UserAgent INTEGER, - URL VARCHAR(6072), - Referer VARCHAR(2048), - Refresh INTEGER, - RefererCategoryID INTEGER, - RefererRegionID INTEGER, - URLCategoryID INTEGER, - URLRegionID INTEGER, - ResolutionWidth INTEGER, - ResolutionHeight INTEGER, - ResolutionDepth INTEGER, - FlashMajor INTEGER, - FlashMinor INTEGER, - FlashMinor2 VARCHAR(256), - NetMajor INTEGER, - NetMinor INTEGER, - UserAgentMajor INTEGER, - UserAgentMinor CHAR(2), - CookieEnable INTEGER, - JavascriptEnable INTEGER, - IsMobile INTEGER, - MobilePhone INTEGER, - MobilePhoneModel VARCHAR(80), - Params VARCHAR(2048), - IPNetworkID INTEGER, - TraficSourceID INTEGER, - SearchEngineID INTEGER, - SearchPhrase VARCHAR(1024), - AdvEngineID INTEGER, - IsArtifical INTEGER, - WindowClientWidth INTEGER, - WindowClientHeight INTEGER, - ClientTimeZone INTEGER, - ClientEventTime DATETIME, - SilverlightVersion1 INTEGER, - SilverlightVersion2 INTEGER, - SilverlightVersion3 INTEGER, - SilverlightVersion4 INTEGER, - PageCharset VARCHAR(80), - CodeVersion INTEGER, - IsLink INTEGER, - IsDownload INTEGER, - IsNotBounce INTEGER, - FUniqID INTEGER, - OriginalURL VARCHAR(6072), - HID INTEGER, - IsOldCounter INTEGER, - IsEvent INTEGER, - IsParameter INTEGER, - DontCountHits INTEGER, - WithHash INTEGER, - HitColor CHAR(1), - LocalEventTime DATETIME, - Age INTEGER, - Sex INTEGER, - Income INTEGER, - Interests INTEGER, - Robotness INTEGER, - RemoteIP INTEGER, - WindowName INTEGER, - OpenerName INTEGER, - HistoryLength INTEGER, - BrowserLanguage CHAR(2), - BrowserCountry CHAR(2), - SocialNetwork VARCHAR(128), - SocialAction VARCHAR(128), - HTTPError INTEGER, - SendTiming INTEGER, - DNSTiming INTEGER, - ConnectTiming INTEGER, - ResponseStartTiming INTEGER, - ResponseEndTiming INTEGER, - FetchTiming INTEGER, - SocialSourceNetworkID INTEGER, - SocialSourcePage VARCHAR(128), - ParamPrice INTEGER, - ParamOrderID VARCHAR(80), - ParamCurrency CHAR(3), - ParamCurrencyID INTEGER, - OpenstatServiceName VARCHAR(80), - OpenstatCampaignID VARCHAR(80), - OpenstatAdID VARCHAR(80), - OpenstatSourceID VARCHAR(80), - UTMSource VARCHAR(256), - UTMMedium VARCHAR(256), - UTMCampaign VARCHAR(256), - UTMContent VARCHAR(256), - UTMTerm VARCHAR(256), - FromTag VARCHAR(256), - HasGCLID INTEGER, - RefererHash INTEGER, - URLHash INTEGER, - CLID INTEGER -) ORDER BY CounterID, EventDate, UserID, EventTime; - -\set input_file '''/opt/dumps/hits_100m_corrected.tsv''' -COPY hits_100m FROM :input_file DELIMITER E'\t' DIRECT; - - -create table hits_1000m -( - WatchID INTEGER, - JavaEnable INTEGER, - Title VARCHAR(1024), - GoodEvent INTEGER, - EventTime DATETIME, - EventDate DATE, - CounterID INTEGER, - ClientIP INTEGER, - RegionID INTEGER, - UserID INTEGER, - CounterClass INTEGER, - OS INTEGER, - UserAgent INTEGER, - URL VARCHAR(6072), - Referer VARCHAR(2048), - Refresh INTEGER, - RefererCategoryID INTEGER, - RefererRegionID INTEGER, - URLCategoryID INTEGER, - URLRegionID INTEGER, - ResolutionWidth INTEGER, - ResolutionHeight INTEGER, - ResolutionDepth INTEGER, - FlashMajor INTEGER, - FlashMinor INTEGER, - FlashMinor2 VARCHAR(256), - NetMajor INTEGER, - NetMinor INTEGER, - UserAgentMajor INTEGER, - UserAgentMinor CHAR(2), - CookieEnable INTEGER, - JavascriptEnable INTEGER, - IsMobile INTEGER, - MobilePhone INTEGER, - MobilePhoneModel VARCHAR(80), - Params VARCHAR(2048), - IPNetworkID INTEGER, - TraficSourceID INTEGER, - SearchEngineID INTEGER, - SearchPhrase VARCHAR(1024), - AdvEngineID INTEGER, - IsArtifical INTEGER, - WindowClientWidth INTEGER, - WindowClientHeight INTEGER, - ClientTimeZone INTEGER, - ClientEventTime DATETIME, - SilverlightVersion1 INTEGER, - SilverlightVersion2 INTEGER, - SilverlightVersion3 INTEGER, - SilverlightVersion4 INTEGER, - PageCharset VARCHAR(80), - CodeVersion INTEGER, - IsLink INTEGER, - IsDownload INTEGER, - IsNotBounce INTEGER, - FUniqID INTEGER, - OriginalURL VARCHAR(6072), - HID INTEGER, - IsOldCounter INTEGER, - IsEvent INTEGER, - IsParameter INTEGER, - DontCountHits INTEGER, - WithHash INTEGER, - HitColor CHAR(1), - LocalEventTime DATETIME, - Age INTEGER, - Sex INTEGER, - Income INTEGER, - Interests INTEGER, - Robotness INTEGER, - RemoteIP INTEGER, - WindowName INTEGER, - OpenerName INTEGER, - HistoryLength INTEGER, - BrowserLanguage CHAR(2), - BrowserCountry CHAR(2), - SocialNetwork VARCHAR(128), - SocialAction VARCHAR(128), - HTTPError INTEGER, - SendTiming INTEGER, - DNSTiming INTEGER, - ConnectTiming INTEGER, - ResponseStartTiming INTEGER, - ResponseEndTiming INTEGER, - FetchTiming INTEGER, - SocialSourceNetworkID INTEGER, - SocialSourcePage VARCHAR(128), - ParamPrice INTEGER, - ParamOrderID VARCHAR(80), - ParamCurrency CHAR(3), - ParamCurrencyID INTEGER, - OpenstatServiceName VARCHAR(80), - OpenstatCampaignID VARCHAR(80), - OpenstatAdID VARCHAR(80), - OpenstatSourceID VARCHAR(80), - UTMSource VARCHAR(256), - UTMMedium VARCHAR(256), - UTMCampaign VARCHAR(256), - UTMContent VARCHAR(256), - UTMTerm VARCHAR(256), - FromTag VARCHAR(256), - HasGCLID INTEGER, - RefererHash INTEGER, - URLHash INTEGER, - CLID INTEGER -) ORDER BY CounterID, EventDate, UserID, EventTime; - -\set input_file '''/opt/dumps/hits_1000m_corrected.tsv''' -COPY hits_1000m FROM :input_file DELIMITER E'\t' DIRECT; diff --git a/benchmark/vertica/queries.sql b/benchmark/vertica/queries.sql index e7906ce65e3..31f65fc898d 100644 --- a/benchmark/vertica/queries.sql +++ b/benchmark/vertica/queries.sql @@ -1,43 +1,43 @@ -SELECT count(*) FROM {table}; -SELECT count(*) FROM {table} WHERE AdvEngineID != 0; -SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM {table}; -SELECT sum_float(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 count(*) 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(*) FROM {table} WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) 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(*) FROM {table} WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) 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, Minute(EventTime) AS m, SearchPhrase, count(*) FROM {table} GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -SELECT UserID FROM {table} WHERE UserID = 12345678901234567890; -SELECT count(*) FROM {table} WHERE URL LIKE '%metrika%'; -SELECT SearchPhrase, MAX(URL), count(*) FROM {table} 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 {table} WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY count(*) 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(OCTET_LENGTH(URL)) AS l, count(*) FROM {table} WHERE URL != '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -SELECT SUBSTRB(SUBSTRB(Referer, POSITIONB(Referer, '//') + 2), 1, GREATEST(0, POSITIONB(SUBSTRB(Referer, POSITIONB(Referer, '//') + 2), '/') - 1)) AS key, avg(OCTET_LENGTH(Referer)) AS l, count(*) AS c, MAX(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 count(*) DESC LIMIT 10; -SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM {table} WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM {table} GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -SELECT URL, count(*) FROM {table} GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -SELECT 1, URL, count(*) FROM {table} GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM {table} GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -SELECT URL, count(*) AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 {table} WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 {table} WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 {table} WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 {table} WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('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 {table} WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -SELECT TIME_SLICE(EventTime, 1, 'MINUTE') AS Minute, count(*) AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate >= DATE('2013-07-01') AND EventDate <= DATE('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/compatible/vertica/run.sh b/benchmark/vertica/run.sh similarity index 100% rename from benchmark/compatible/vertica/run.sh rename to benchmark/vertica/run.sh diff --git a/docs/en/operations/performance-test.md b/docs/en/operations/performance-test.md index b0b9a30fe2c..5be317bbd6b 100644 --- a/docs/en/operations/performance-test.md +++ b/docs/en/operations/performance-test.md @@ -55,9 +55,9 @@ chmod a+x clickhouse ``` 2. Download benchmark files: ```bash -wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/benchmark-new.sh +wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/hardware/benchmark-new.sh chmod a+x benchmark-new.sh -wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/queries.sql +wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/hardware/queries.sql ``` 3. Download the [web analytics dataset](../getting-started/example-datasets/metrica.md) (“hits” table containing 100 million rows). ```bash diff --git a/docs/zh/operations/performance-test.md b/docs/zh/operations/performance-test.md index b731e11efc2..f7e79d606f7 100644 --- a/docs/zh/operations/performance-test.md +++ b/docs/zh/operations/performance-test.md @@ -54,9 +54,9 @@ chmod a+x clickhouse 2. 下载基准文件 ```bash -wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/benchmark-new.sh +wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/hardware/benchmark-new.sh chmod a+x benchmark-new.sh -wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/queries.sql +wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/hardware/queries.sql ``` 3. 根据 [Yandex.Metrica 数据集](../getting-started/example-datasets/metrica.md) 中的说明下载测试数据(“ hits ” 数据表包含 1 亿行记录)。 From d0a7be5565d371ba35381913243e57db56411ac2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 13:41:05 +0200 Subject: [PATCH 148/627] Update README --- benchmark/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmark/README.md b/benchmark/README.md index d23e720ae21..32321df42d1 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -18,7 +18,7 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [x] MonetDB - [x] mapD/Omnisci/HeavyAI - [x] Databend -- [ ] Doris +- [ ] Doris/PALO - [x] Druid - [ ] Pinot - [x] CrateDB From e1e9a23dd745bf8117e297f95abe8c23a0e40b7e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 13:46:51 +0200 Subject: [PATCH 149/627] Update README --- benchmark/README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/benchmark/README.md b/benchmark/README.md index 32321df42d1..c7fa9cf5adf 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -47,6 +47,8 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] VictoriaMetrics - [ ] TDEngine - [ ] MongoDB +- [ ] Cassandra +- [ ] ScyllaDB - [ ] Elasticsearch - [ ] Apache Ignite - [ ] Infobright From 4548937f02ff6a2df62c47e257cd0755096dcfe1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 13:55:46 +0200 Subject: [PATCH 150/627] Some uniformity --- benchmark/questdb/{result => results}/c6a.4xlarge.txt | 0 benchmark/redshift-serverless/{result => results}/serverless.txt | 0 benchmark/redshift/{result => results}/4x.ra3.xplus.txt | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename benchmark/questdb/{result => results}/c6a.4xlarge.txt (100%) rename benchmark/redshift-serverless/{result => results}/serverless.txt (100%) rename benchmark/redshift/{result => results}/4x.ra3.xplus.txt (100%) diff --git a/benchmark/questdb/result/c6a.4xlarge.txt b/benchmark/questdb/results/c6a.4xlarge.txt similarity index 100% rename from benchmark/questdb/result/c6a.4xlarge.txt rename to benchmark/questdb/results/c6a.4xlarge.txt diff --git a/benchmark/redshift-serverless/result/serverless.txt b/benchmark/redshift-serverless/results/serverless.txt similarity index 100% rename from benchmark/redshift-serverless/result/serverless.txt rename to benchmark/redshift-serverless/results/serverless.txt diff --git a/benchmark/redshift/result/4x.ra3.xplus.txt b/benchmark/redshift/results/4x.ra3.xplus.txt similarity index 100% rename from benchmark/redshift/result/4x.ra3.xplus.txt rename to benchmark/redshift/results/4x.ra3.xplus.txt From 6b37ce2f8e7d71aa1fb1a44127943c9cf29e568b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 4 Jul 2022 14:06:20 +0200 Subject: [PATCH 151/627] Drop tables in test --- tests/integration/test_storage_rabbitmq/test.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 023c8ff3000..2b6c2867b88 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2788,6 +2788,9 @@ def test_rabbitmq_msgpack(rabbitmq_cluster): time.sleep(1) assert result.strip() == "kek" + instance.query("drop table rabbit_in sync") + instance.query("drop table rabbit_out sync") + def test_rabbitmq_address(rabbitmq_cluster): @@ -2825,3 +2828,6 @@ def test_rabbitmq_address(rabbitmq_cluster): break time.sleep(1) assert result.strip() == "kek" + + instance.query("drop table rabbit_in sync") + instance.query("drop table rabbit_out sync") From 0f054774f5d93f8f70b836602a883514ebaf9b34 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 4 Jul 2022 14:14:15 +0200 Subject: [PATCH 152/627] Review fixes --- src/Core/PostgreSQL/ConnectionHolder.h | 1 - src/Core/PostgreSQL/PoolWithFailover.cpp | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Core/PostgreSQL/ConnectionHolder.h b/src/Core/PostgreSQL/ConnectionHolder.h index d608f705559..2fd8717c643 100644 --- a/src/Core/PostgreSQL/ConnectionHolder.h +++ b/src/Core/PostgreSQL/ConnectionHolder.h @@ -7,7 +7,6 @@ #include #include #include -#include #include "Connection.h" diff --git a/src/Core/PostgreSQL/PoolWithFailover.cpp b/src/Core/PostgreSQL/PoolWithFailover.cpp index b5e2903ef2b..1bac17de579 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.cpp +++ b/src/Core/PostgreSQL/PoolWithFailover.cpp @@ -103,9 +103,9 @@ ConnectionHolderPtr PoolWithFailover::get() catch (const pqxx::broken_connection & pqxx_error) { LOG_ERROR(log, "Connection error: {}", pqxx_error.what()); - error_message << "Try " << try_idx + 1 << ". " - << "Connection to " << DB::backQuote(replica.connection_info.host_port) - << " failed with error: " << pqxx_error.what() << "\n"; + error_message << fmt::format( + "Try {}. Connection to {} failed with error: {}\n", + try_idx + 1, DB::backQuote(replica.connection_info.host_port), pqxx_error.what()); replica.pool->returnObject(std::move(connection)); continue; From 17da24b584d299c93c5e4cc812d3c4f5b43ba513 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 4 Jul 2022 13:02:22 +0000 Subject: [PATCH 153/627] Try to improve backward compatibility check --- docker/test/stress/run.sh | 26 ++++++-- tests/ci/download_release_packets.py | 89 ++++++++++++++++++++++++++++ tests/ci/functional_test_check.py | 4 +- tests/ci/get_previous_release_tag.py | 83 ++++++++++++++++++++++++++ tests/ci/integration_test_check.py | 4 +- 5 files changed, 196 insertions(+), 10 deletions(-) create mode 100755 tests/ci/download_release_packets.py create mode 100755 tests/ci/get_previous_release_tag.py diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index b3ee4e62f20..168851d5060 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -47,7 +47,8 @@ function configure() # we mount tests folder from repo to /usr/share ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test - ln -s /usr/share/clickhouse-test/ci/download_previous_release.py /usr/bin/download_previous_release + ln -s /usr/share/clickhouse-test/ci/download_release_packets.py /usr/bin/download_release_packets + ln -s /usr/share/clickhouse-test/ci/get_previous_release_tag.py /usr/bin/get_previous_release_tag # avoid too slow startup sudo cat /etc/clickhouse-server/config.d/keeper_port.xml | sed "s|100000|10000|" > /etc/clickhouse-server/config.d/keeper_port.xml.tmp @@ -267,16 +268,31 @@ zgrep -Fa " received signal " /test_output/gdb.log > /dev/null \ echo -e "Backward compatibility check\n" +echo "Get previous release tag" +previous_release_tag=$(clickhouse-client --query="SELECT version()" | get_previous_release_tag) +echo $previous_release_tag + +echo "Clone previous release repository" +git clone git@github.com:ClickHouse/ClickHouse.git --no-tags --progress --branch=$previous_release_tag --no-recurse-submodules --depth=1 previous_release_repository + echo "Download previous release server" mkdir previous_release_package_folder -clickhouse-client --query="SELECT version()" | download_previous_release && echo -e 'Download script exit code\tOK' >> /test_output/test_results.tsv \ + +echo $previous_release_tag | download_release_packets && echo -e 'Download script exit code\tOK' >> /test_output/test_results.tsv \ || echo -e 'Download script failed\tFAIL' >> /test_output/test_results.tsv stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.clean.log -if [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ] +# Check if we cloned previous release repository successfully +if ! [ -z "$(ls -A previous_release_repository/tests/queries)" ] then + echo -e "Backward compatibility check: Failed to clone previous release tests\tFAIL" >> /test_output/test_results.tsv +elif ! [ -z "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ] +then + echo -e "Backward compatibility check: Failed to download previous release packets\tFAIL" >> /test_output/test_results.tsv +else + echo -e "Successfully cloned previous release tests\tOK" >> /test_output/test_results.tsv echo -e "Successfully downloaded previous release packets\tOK" >> /test_output/test_results.tsv # Uninstall current packages @@ -310,7 +326,7 @@ then mkdir tmp_stress_output - ./stress --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \ + ./stress --test-cmd="/usr/bin/clickhouse-test --queries=\"previous_release_repository/tests/queries\"" --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \ && echo -e 'Backward compatibility check: Test script exit code\tOK' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: Test script failed\tFAIL' >> /test_output/test_results.tsv rm -rf tmp_stress_output @@ -400,8 +416,6 @@ then # Remove file bc_check_fatal_messages.txt if it's empty [ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt -else - echo -e "Backward compatibility check: Failed to download previous release packets\tFAIL" >> /test_output/test_results.tsv fi tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||: diff --git a/tests/ci/download_release_packets.py b/tests/ci/download_release_packets.py new file mode 100755 index 00000000000..919b54e7a6b --- /dev/null +++ b/tests/ci/download_release_packets.py @@ -0,0 +1,89 @@ +#!/usr/bin/env python3 + +import re +import os +import logging + +import requests # type: ignore + +from requests.adapters import HTTPAdapter # type: ignore +from urllib3.util.retry import Retry # type: ignore + +from get_previous_release_tag import ReleaseInfo, get_previous_release + +CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags" + +DOWNLOAD_PREFIX = ( + "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/" +) +CLICKHOUSE_COMMON_STATIC_PACKET_NAME = "clickhouse-common-static_{version}_amd64.deb" +CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME = ( + "clickhouse-common-static-dbg_{version}_amd64.deb" +) +CLICKHOUSE_SERVER_PACKET_NAME = "clickhouse-server_{version}_amd64.deb" +CLICKHOUSE_SERVER_PACKET_FALLBACK = "clickhouse-server_{version}_all.deb" +CLICKHOUSE_CLIENT_PACKET_NAME = "clickhouse-client_{version}_amd64.deb" +CLICKHOUSE_CLIENT_PACKET_FALLBACK = "clickhouse-client_{version}_all.deb" + +PACKETS_DIR = "previous_release_package_folder/" +VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)" + + +def download_packet(url, out_path, retries=10, backoff_factor=0.3): + session = requests.Session() + retry = Retry( + total=retries, read=retries, connect=retries, backoff_factor=backoff_factor + ) + adapter = HTTPAdapter(max_retries=retry) + session.mount("http://", adapter) + session.mount("https://", adapter) + response = session.get(url) + response.raise_for_status() + print(f"Download {url} to {out_path}") + with open(out_path, "wb") as fd: + fd.write(response.content) + + +def download_packets(release, dest_path=PACKETS_DIR): + if not os.path.exists(dest_path): + os.makedirs(dest_path) + + logging.info("Will download %s", release) + + def get_dest_path(pkg_name): + return os.path.join(dest_path, pkg_name) + + for pkg in ( + CLICKHOUSE_COMMON_STATIC_PACKET_NAME, + CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME, + ): + url = (DOWNLOAD_PREFIX + pkg).format(version=release.version, type=release.type) + pkg_name = get_dest_path(pkg.format(version=release.version)) + download_packet(url, pkg_name) + + for pkg, fallback in ( + (CLICKHOUSE_SERVER_PACKET_NAME, CLICKHOUSE_SERVER_PACKET_FALLBACK), + (CLICKHOUSE_CLIENT_PACKET_NAME, CLICKHOUSE_CLIENT_PACKET_FALLBACK), + ): + url = (DOWNLOAD_PREFIX + pkg).format(version=release.version, type=release.type) + pkg_name = get_dest_path(pkg.format(version=release.version)) + try: + download_packet(url, pkg_name) + except Exception: + url = (DOWNLOAD_PREFIX + fallback).format( + version=release.version, type=release.type + ) + pkg_name = get_dest_path(fallback.format(version=release.version)) + download_packet(url, pkg_name) + + + +def download_last_release(dest_path): + current_release = get_previous_release(None) + download_packets(current_release, dest_path=dest_path) + + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + release = ReleaseInfo(input()) + download_packets(release) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 6516a67d333..b73e6f9d708 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -14,7 +14,7 @@ from s3_helper import S3Helper from get_robot_token import get_best_robot_token from pr_info import FORCE_TESTS_LABEL, PRInfo from build_download_helper import download_all_deb_packages -from download_previous_release import download_previous_release +from download_release_packets import download_last_release from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version from commit_status_helper import ( @@ -268,7 +268,7 @@ if __name__ == "__main__": os.makedirs(packages_path) if validate_bugix_check: - download_previous_release(packages_path) + download_last_release(packages_path) else: download_all_deb_packages(check_name, reports_path, packages_path) diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py new file mode 100755 index 00000000000..26e4bee8875 --- /dev/null +++ b/tests/ci/get_previous_release_tag.py @@ -0,0 +1,83 @@ +#!/usr/bin/env python3 + +import re +import logging + +import requests + +CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags" +VERSION_PATTERN = r"(v(?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)" + + +class Version: + def __init__(self, version): + self.version = version + + def __lt__(self, other): + return list(map(int, self.version.split("."))) < list( + map(int, other.version.split(".")) + ) + + def __str__(self): + return self.version + + +class ReleaseInfo: + def __init__(self, release_tag): + self.version = Version(release_tag[1:].split("-")[0]) + self.type = release_tag[1:].split("-")[1] + + def __str__(self): + return f"v{self.version}-{self.type}" + + def __repr__(self): + return f"ReleaseInfo: {self.version}-{self.type}" + + +def find_previous_release(server_version, releases): + releases.sort(key=lambda x: x.version, reverse=True) + + if server_version is None: + return True, releases[0] + + for release in releases: + if release.version < server_version: + return True, release + + return False, None + + +def get_previous_release(server_version): + page = 1 + found = False + while not found: + response = requests.get(CLICKHOUSE_TAGS_URL, {"page": page, "per_page": 100}) + if not response.ok: + raise Exception( + "Cannot load the list of tags from github: " + response.reason + ) + + releases_str = set(re.findall(VERSION_PATTERN, response.text)) + if len(releases_str) == 0: + raise Exception( + "Cannot find previous release for " + + str(server_version) + + " server version" + ) + + releases = list( + map( + lambda x: ReleaseInfo(x), + releases_str, + ) + ) + found, previous_release = find_previous_release(server_version, releases) + page += 1 + + return previous_release + + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + server_version = Version(input()) + print(get_previous_release(server_version)) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 9fda2d09ae6..1c53247c072 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -15,7 +15,7 @@ from s3_helper import S3Helper from get_robot_token import get_best_robot_token from pr_info import PRInfo from build_download_helper import download_all_deb_packages -from download_previous_release import download_previous_release +from download_release_packets import download_last_release from upload_result_helper import upload_results from docker_pull_helper import get_images_with_versions from commit_status_helper import ( @@ -202,7 +202,7 @@ if __name__ == "__main__": os.makedirs(build_path) if validate_bugix_check: - download_previous_release(build_path) + download_last_release(build_path) else: download_all_deb_packages(check_name, reports_path, build_path) From fb2f60c61349a4d51fc4bbe78c1290338aaf9f43 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 4 Jul 2022 13:03:55 +0000 Subject: [PATCH 154/627] Remove old script --- tests/ci/download_previous_release.py | 152 -------------------------- 1 file changed, 152 deletions(-) delete mode 100755 tests/ci/download_previous_release.py diff --git a/tests/ci/download_previous_release.py b/tests/ci/download_previous_release.py deleted file mode 100755 index 86beed35b5a..00000000000 --- a/tests/ci/download_previous_release.py +++ /dev/null @@ -1,152 +0,0 @@ -#!/usr/bin/env python3 - -import re -import os -import logging - -import requests # type: ignore - -from requests.adapters import HTTPAdapter # type: ignore -from urllib3.util.retry import Retry # type: ignore - -CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags" - -DOWNLOAD_PREFIX = ( - "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/" -) -CLICKHOUSE_COMMON_STATIC_PACKET_NAME = "clickhouse-common-static_{version}_amd64.deb" -CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME = ( - "clickhouse-common-static-dbg_{version}_amd64.deb" -) -CLICKHOUSE_SERVER_PACKET_NAME = "clickhouse-server_{version}_amd64.deb" -CLICKHOUSE_SERVER_PACKET_FALLBACK = "clickhouse-server_{version}_all.deb" -CLICKHOUSE_CLIENT_PACKET_NAME = "clickhouse-client_{version}_amd64.deb" -CLICKHOUSE_CLIENT_PACKET_FALLBACK = "clickhouse-client_{version}_all.deb" - -PACKETS_DIR = "previous_release_package_folder/" -VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)" - - -class Version: - def __init__(self, version): - self.version = version - - def __lt__(self, other): - return list(map(int, self.version.split("."))) < list( - map(int, other.version.split(".")) - ) - - def __str__(self): - return self.version - - -class ReleaseInfo: - def __init__(self, version, release_type): - self.version = version - self.type = release_type - - def __repr__(self): - return f"ReleaseInfo: {self.version}-{self.type}" - - -def find_previous_release(server_version, releases): - releases.sort(key=lambda x: x.version, reverse=True) - - if server_version is None: - return True, releases[0] - - for release in releases: - if release.version < server_version: - return True, release - - return False, None - - -def get_previous_release(server_version): - page = 1 - found = False - while not found: - response = requests.get(CLICKHOUSE_TAGS_URL, {"page": page, "per_page": 100}) - if not response.ok: - raise Exception( - "Cannot load the list of tags from github: " + response.reason - ) - - releases_str = set(re.findall(VERSION_PATTERN, response.text)) - if len(releases_str) == 0: - raise Exception( - "Cannot find previous release for " - + str(server_version) - + " server version" - ) - - releases = list( - map( - lambda x: ReleaseInfo(Version(x.split("-")[0]), x.split("-")[1]), - releases_str, - ) - ) - found, previous_release = find_previous_release(server_version, releases) - page += 1 - - return previous_release - - -def download_packet(url, out_path, retries=10, backoff_factor=0.3): - session = requests.Session() - retry = Retry( - total=retries, read=retries, connect=retries, backoff_factor=backoff_factor - ) - adapter = HTTPAdapter(max_retries=retry) - session.mount("http://", adapter) - session.mount("https://", adapter) - response = session.get(url) - response.raise_for_status() - print(f"Download {url} to {out_path}") - with open(out_path, "wb") as fd: - fd.write(response.content) - - -def download_packets(release, dest_path=PACKETS_DIR): - if not os.path.exists(dest_path): - os.makedirs(dest_path) - - logging.info("Will download %s", release) - - def get_dest_path(pkg_name): - return os.path.join(dest_path, pkg_name) - - for pkg in ( - CLICKHOUSE_COMMON_STATIC_PACKET_NAME, - CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME, - ): - url = (DOWNLOAD_PREFIX + pkg).format(version=release.version, type=release.type) - pkg_name = get_dest_path(pkg.format(version=release.version)) - download_packet(url, pkg_name) - - for pkg, fallback in ( - (CLICKHOUSE_SERVER_PACKET_NAME, CLICKHOUSE_SERVER_PACKET_FALLBACK), - (CLICKHOUSE_CLIENT_PACKET_NAME, CLICKHOUSE_CLIENT_PACKET_FALLBACK), - ): - url = (DOWNLOAD_PREFIX + pkg).format(version=release.version, type=release.type) - pkg_name = get_dest_path(pkg.format(version=release.version)) - try: - download_packet(url, pkg_name) - except Exception: - url = (DOWNLOAD_PREFIX + fallback).format( - version=release.version, type=release.type - ) - pkg_name = get_dest_path(fallback.format(version=release.version)) - download_packet(url, pkg_name) - - -def download_previous_release(dest_path): - current_release = get_previous_release(None) - download_packets(current_release, dest_path=dest_path) - - -if __name__ == "__main__": - logging.basicConfig(level=logging.INFO) - server_version = Version(input()) - previous_release = get_previous_release(server_version) - download_packets(previous_release) From f814985adfeaf66db40e435c8c7763e7e3d53523 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 4 Jul 2022 16:03:59 +0300 Subject: [PATCH 155/627] minor improvements --- src/Processors/Transforms/WindowTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 8a692fb97a5..d05c703a7cd 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2248,7 +2248,7 @@ struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction argument_types[ARGUMENT_TIMESTAMP]->getName()); } - if (typeid_cast(argument_types[ARGUMENT_TIMESTAMP].get())) + if (isDateTime64(argument_types[ARGUMENT_TIMESTAMP])) { const auto & datetime64_type = assert_cast(*argument_types[ARGUMENT_TIMESTAMP]); ts_scale_multiplier = DecimalUtils::scaleMultiplier(datetime64_type.getScale()); From e493c46fdd27bc2743b29d7a31cdd9c292181e01 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 15:12:12 +0200 Subject: [PATCH 156/627] Some results --- benchmark/athena/run.sh | 0 benchmark/aurora-mysql/README.md | 2 +- benchmark/aurora-mysql/results/16acu.txt | 3 +- benchmark/aurora-mysql/run.sh | 0 benchmark/bigquery/run.sh | 0 benchmark/databend/benchmark.sh | 0 benchmark/databend/results/c6a.4xlarge.txt | 84 ++++----- benchmark/druid/results/c6a.4xlarge.txt | 43 ----- benchmark/elasticsearch/benchmark.sh | 0 benchmark/generate-results.sh | 5 + benchmark/greenplum/benchmark.sh | 0 benchmark/greenplum/results/c6a.4xlarge.txt | 2 +- benchmark/greenplum/run.sh | 0 benchmark/locustdb/benchmark.sh | 0 benchmark/mariadb-columnstore/benchmark.sh | 2 +- .../results/c6a.4xlarge.txt | 3 +- benchmark/mariadb-columnstore/run.sh | 0 benchmark/monetdb/query.expect | 0 benchmark/monetdb/run.sh | 0 benchmark/mysql-myisam/benchmark.sh | 2 +- .../mysql-myisam/results/c6a.4xlarge.txt | 3 +- benchmark/mysql/benchmark.sh | 2 +- benchmark/questdb/run.sh | 0 benchmark/redshift-serverless/run.sh | 0 benchmark/redshift/run.sh | 0 benchmark/singlestore/benchmark.sh | 2 +- benchmark/snowflake/run.sh | 0 benchmark/sqlite/benchmark.sh | 0 benchmark/sqlite/run.sh | 0 benchmark/starrocks/benchmark.sh | 0 benchmark/trino/benchmark.sh | 0 benchmark/vertica/benchmark.sh | 0 benchmark/vertica/run.sh | 0 website/benchmark/dbms/queries.js | 130 +++++--------- .../dbms/results/001_clickhouse_19_1_6.json | 163 ------------------ .../dbms/results/002_vertica_7_1_1.json | 163 ------------------ .../dbms/results/003_vertica_7_0_0_x3.json | 109 ------------ .../dbms/results/004_infinidb_3_6_23.json | 109 ------------ .../benchmark/dbms/results/005_monetdb.json | 55 ------ .../dbms/results/006_infobright_4_0_7.json | 55 ------ .../benchmark/dbms/results/007_hive_0_11.json | 109 ------------ .../benchmark/dbms/results/008_mysql_5_5.json | 109 ------------ .../dbms/results/009_memsql_3_2.json | 55 ------ .../dbms/results/010_greenplum_4_3_9.json | 161 ----------------- .../dbms/results/011_greenplum_4_3_9_x2.json | 161 ----------------- .../benchmark/dbms/results/012_omnisci.json | 56 ------ .../dbms/results/013_timescaledb.json | 56 ------ .../dbms/results/014_timescaledb.json | 56 ------ .../dbms/results/015_postgresql.json | 56 ------ .../benchmark/dbms/results/016_duckdb.json | 57 ------ .../dbms/results/athena.partitioned.json | 45 +++++ .../benchmark/dbms/results/athena.single.json | 45 +++++ .../dbms/results/aurora-mysql.16acu.json | 45 +++++ .../dbms/results/aurora-postgresql.16acu.json | 45 +++++ .../dbms/results/bigquery.serverless.json | 45 +++++ .../dbms/results/citus.c6a.4xlarge.json | 45 +++++ ...ckhouse-local.c6a.4xlarge.partitioned.json | 45 +++++ .../clickhouse-local.c6a.4xlarge.single.json | 45 +++++ .../dbms/results/clickhouse.c6a.4xlarge.json | 45 +++++ .../dbms/results/clickhouse.c6a.metal.json | 45 +++++ .../dbms/results/cratedb.c6a.4xlarge.json | 45 +++++ .../dbms/results/databend.c6a.4xlarge.json | 45 +++++ .../dbms/results/duckdb.c6a.4xlarge.json | 45 +++++ .../dbms/results/greenplum.c6a.4xlarge.json | 45 +++++ .../dbms/results/heavyai.c6a.4xlarge.json | 45 +++++ .../mariadb-columnstore.c6a.4xlarge.json | 45 +++++ .../dbms/results/monetdb.c6a.4xlarge.json | 45 +++++ .../results/mysql-myisam.c6a.4xlarge.json | 45 +++++ .../dbms/results/mysql.c6a.4xlarge.json | 45 +++++ .../dbms/results/postgresql.c6a.4xlarge.json | 45 +++++ .../dbms/results/questdb.c6a.4xlarge.json | 45 +++++ .../redshift-serverless.serverless.json | 45 +++++ .../dbms/results/redshift.4x.ra3.xplus.json | 45 +++++ .../dbms/results/singlestore.c6a.4xlarge.json | 45 +++++ .../benchmark/dbms/results/snowflake.2xl.json | 45 +++++ .../benchmark/dbms/results/snowflake.3xl.json | 45 +++++ .../benchmark/dbms/results/snowflake.4xl.json | 45 +++++ .../benchmark/dbms/results/snowflake.l.json | 45 +++++ .../benchmark/dbms/results/snowflake.m.json | 45 +++++ .../benchmark/dbms/results/snowflake.s.json | 45 +++++ .../benchmark/dbms/results/snowflake.xl.json | 45 +++++ .../benchmark/dbms/results/snowflake.xs.json | 45 +++++ .../dbms/results/sqlite.c6a.4xlarge.json | 45 +++++ .../timescaledb.c6a.4xlarge.compression.json | 45 +++++ .../dbms/results/timescaledb.c6a.4xlarge.json | 45 +++++ .../dbms/results/vertica.c6a.4xlarge.json | 45 +++++ 86 files changed, 1722 insertions(+), 1711 deletions(-) mode change 100644 => 100755 benchmark/athena/run.sh mode change 100644 => 100755 benchmark/aurora-mysql/run.sh mode change 100644 => 100755 benchmark/bigquery/run.sh mode change 100644 => 100755 benchmark/databend/benchmark.sh delete mode 100644 benchmark/druid/results/c6a.4xlarge.txt mode change 100644 => 100755 benchmark/elasticsearch/benchmark.sh create mode 100755 benchmark/generate-results.sh mode change 100644 => 100755 benchmark/greenplum/benchmark.sh mode change 100644 => 100755 benchmark/greenplum/run.sh mode change 100644 => 100755 benchmark/locustdb/benchmark.sh mode change 100644 => 100755 benchmark/mariadb-columnstore/run.sh mode change 100644 => 100755 benchmark/monetdb/query.expect mode change 100644 => 100755 benchmark/monetdb/run.sh mode change 100644 => 100755 benchmark/questdb/run.sh mode change 100644 => 100755 benchmark/redshift-serverless/run.sh mode change 100644 => 100755 benchmark/redshift/run.sh mode change 100644 => 100755 benchmark/snowflake/run.sh mode change 100644 => 100755 benchmark/sqlite/benchmark.sh mode change 100644 => 100755 benchmark/sqlite/run.sh mode change 100644 => 100755 benchmark/starrocks/benchmark.sh mode change 100644 => 100755 benchmark/trino/benchmark.sh mode change 100644 => 100755 benchmark/vertica/benchmark.sh mode change 100644 => 100755 benchmark/vertica/run.sh delete mode 100644 website/benchmark/dbms/results/001_clickhouse_19_1_6.json delete mode 100644 website/benchmark/dbms/results/002_vertica_7_1_1.json delete mode 100644 website/benchmark/dbms/results/003_vertica_7_0_0_x3.json delete mode 100644 website/benchmark/dbms/results/004_infinidb_3_6_23.json delete mode 100644 website/benchmark/dbms/results/005_monetdb.json delete mode 100644 website/benchmark/dbms/results/006_infobright_4_0_7.json delete mode 100644 website/benchmark/dbms/results/007_hive_0_11.json delete mode 100644 website/benchmark/dbms/results/008_mysql_5_5.json delete mode 100644 website/benchmark/dbms/results/009_memsql_3_2.json delete mode 100644 website/benchmark/dbms/results/010_greenplum_4_3_9.json delete mode 100644 website/benchmark/dbms/results/011_greenplum_4_3_9_x2.json delete mode 100644 website/benchmark/dbms/results/012_omnisci.json delete mode 100644 website/benchmark/dbms/results/013_timescaledb.json delete mode 100644 website/benchmark/dbms/results/014_timescaledb.json delete mode 100644 website/benchmark/dbms/results/015_postgresql.json delete mode 100644 website/benchmark/dbms/results/016_duckdb.json create mode 100644 website/benchmark/dbms/results/athena.partitioned.json create mode 100644 website/benchmark/dbms/results/athena.single.json create mode 100644 website/benchmark/dbms/results/aurora-mysql.16acu.json create mode 100644 website/benchmark/dbms/results/aurora-postgresql.16acu.json create mode 100644 website/benchmark/dbms/results/bigquery.serverless.json create mode 100644 website/benchmark/dbms/results/citus.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.partitioned.json create mode 100644 website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.single.json create mode 100644 website/benchmark/dbms/results/clickhouse.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/clickhouse.c6a.metal.json create mode 100644 website/benchmark/dbms/results/cratedb.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/databend.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/duckdb.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/greenplum.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/heavyai.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/mariadb-columnstore.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/monetdb.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/mysql-myisam.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/mysql.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/postgresql.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/questdb.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/redshift-serverless.serverless.json create mode 100644 website/benchmark/dbms/results/redshift.4x.ra3.xplus.json create mode 100644 website/benchmark/dbms/results/singlestore.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/snowflake.2xl.json create mode 100644 website/benchmark/dbms/results/snowflake.3xl.json create mode 100644 website/benchmark/dbms/results/snowflake.4xl.json create mode 100644 website/benchmark/dbms/results/snowflake.l.json create mode 100644 website/benchmark/dbms/results/snowflake.m.json create mode 100644 website/benchmark/dbms/results/snowflake.s.json create mode 100644 website/benchmark/dbms/results/snowflake.xl.json create mode 100644 website/benchmark/dbms/results/snowflake.xs.json create mode 100644 website/benchmark/dbms/results/sqlite.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/timescaledb.c6a.4xlarge.compression.json create mode 100644 website/benchmark/dbms/results/timescaledb.c6a.4xlarge.json create mode 100644 website/benchmark/dbms/results/vertica.c6a.4xlarge.json diff --git a/benchmark/athena/run.sh b/benchmark/athena/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/aurora-mysql/README.md b/benchmark/aurora-mysql/README.md index 89aca73ebd8..48faddfa2b3 100644 --- a/benchmark/aurora-mysql/README.md +++ b/benchmark/aurora-mysql/README.md @@ -55,7 +55,7 @@ Go to "Monitoring", find "[Billed] Volume Bytes Used". ./run.sh 2>&1 | tee log.txt cat log.txt | - grep -P 'rows? in set|^ERROR' | + grep -P 'rows? in set|Empty set|^ERROR' | sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/aurora-mysql/results/16acu.txt b/benchmark/aurora-mysql/results/16acu.txt index 386234b161b..7a491ec2f6d 100644 --- a/benchmark/aurora-mysql/results/16acu.txt +++ b/benchmark/aurora-mysql/results/16acu.txt @@ -42,4 +42,5 @@ Data size: 83.46 GiB [1.50,1.52,1.59], [3.62,3.57,3.61], [0.95,0.94,0.94], -[0.90,0.92,0.91] +[0.90,0.92,0.91], +[1.69,1.72,1.69] diff --git a/benchmark/aurora-mysql/run.sh b/benchmark/aurora-mysql/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/bigquery/run.sh b/benchmark/bigquery/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/databend/benchmark.sh b/benchmark/databend/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/databend/results/c6a.4xlarge.txt b/benchmark/databend/results/c6a.4xlarge.txt index 3c1c9edd174..927f6717b59 100644 --- a/benchmark/databend/results/c6a.4xlarge.txt +++ b/benchmark/databend/results/c6a.4xlarge.txt @@ -1,46 +1,46 @@ Load time: 484 sec Data size: 43 016 643 271 bytes -[0.010087, 0.002961, 0.003271] -[0.127964, 0.080012, 0.075741] -[0.162388, 0.143967, 0.144762] -[0.252904, 0.217471, 0.217369] -[34.281026, 34.844158, 34.526942] -[25.290307, 25.793068, 25.620563] -[0.112484, 0.093867, 0.090891] -[0.086604, 0.07796, 0.076448] -[20.723203, 20.7483, 20.354869] -[20.81994, 20.72446, 20.696573] -[1.964378, 1.93559, 1.893824] -[1.846866, 1.789111, 1.763664] -[4.468158, 4.407959, 4.438036] -[19.947276, 19.8859, 19.853514] -[5.478573, 5.474461, 5.460604] -[5.509521, 5.513413, 5.363123] -[15.430359, 15.5406, 15.461211] -[14.905998, 15.029721, 15.019642] -[31.069663, 30.811763, 30.737336] -[0.281067, 0.220021, 0.217741] -[8.89374, 4.12692, 4.131689] -[10.38448, 4.603694, 4.571757] -[19.980572, 8.836322, 8.892694] -[59.786474, 52.452881, 39.941988] -[2.804019, 0.994794, 0.958224] -[0.765299, 0.730434, 0.723964] -[2.784648, 0.94665, 0.936684] -[8.905027, 5.418438, 5.386109] -[12.187652, 12.230066, 12.164123] -[3.35748, 3.395991, 3.319434] -[4.309389, 3.854977, 3.772506] -[9.958201, 7.027432, 6.888253] -[50.200569, 50.535126, 50.283066] -[24.469412, 21.222713, 21.010188] -[26.115852, 23.93507, 24.835342] -[7.511517, 7.296179, 7.324549] -[2.156784, 1.298258, 1.278441] -[2.155447, 1.314499, 1.331237] -[2.007053, 1.181676, 1.155612] -[null, null, null] -[0.485363, 0.420291, 0.416819] -[0.372131, 0.322068, 0.323578] +[0.010087, 0.002961, 0.003271], +[0.127964, 0.080012, 0.075741], +[0.162388, 0.143967, 0.144762], +[0.252904, 0.217471, 0.217369], +[34.281026, 34.844158, 34.526942], +[25.290307, 25.793068, 25.620563], +[0.112484, 0.093867, 0.090891], +[0.086604, 0.07796, 0.076448], +[20.723203, 20.7483, 20.354869], +[20.81994, 20.72446, 20.696573], +[1.964378, 1.93559, 1.893824], +[1.846866, 1.789111, 1.763664], +[4.468158, 4.407959, 4.438036], +[19.947276, 19.8859, 19.853514], +[5.478573, 5.474461, 5.460604], +[5.509521, 5.513413, 5.363123], +[15.430359, 15.5406, 15.461211], +[14.905998, 15.029721, 15.019642], +[31.069663, 30.811763, 30.737336], +[0.281067, 0.220021, 0.217741], +[8.89374, 4.12692, 4.131689], +[10.38448, 4.603694, 4.571757], +[19.980572, 8.836322, 8.892694], +[59.786474, 52.452881, 39.941988], +[2.804019, 0.994794, 0.958224], +[0.765299, 0.730434, 0.723964], +[2.784648, 0.94665, 0.936684], +[8.905027, 5.418438, 5.386109], +[12.187652, 12.230066, 12.164123], +[3.35748, 3.395991, 3.319434], +[4.309389, 3.854977, 3.772506], +[9.958201, 7.027432, 6.888253], +[50.200569, 50.535126, 50.283066], +[24.469412, 21.222713, 21.010188], +[26.115852, 23.93507, 24.835342], +[7.511517, 7.296179, 7.324549], +[2.156784, 1.298258, 1.278441], +[2.155447, 1.314499, 1.331237], +[2.007053, 1.181676, 1.155612], +[null, null, null], +[0.485363, 0.420291, 0.416819], +[0.372131, 0.322068, 0.323578], [null, null, null] diff --git a/benchmark/druid/results/c6a.4xlarge.txt b/benchmark/druid/results/c6a.4xlarge.txt deleted file mode 100644 index dd185c349f9..00000000000 --- a/benchmark/druid/results/c6a.4xlarge.txt +++ /dev/null @@ -1,43 +0,0 @@ -[0.045539, 0.028086, 0.024169], -[0.287153, 0.026538, 0.024661], -[0.171896, 0.023620, 0.023566], -[1.553365, 0.021595, 0.021120], -[1.432360, 0.023391, 0.025546], -[3.418428, 0.020131, 0.020552], -[0.051194, 0.018222, 0.011529], -[0.268591, 0.041415, 0.034791], -[3.194689, 0.370084, 0.325961], -[2.973067, 0.316630, 0.300042], -[0.362751, 0.033894, 0.034276], -[0.352240, 0.067931, 0.069787], -[0.829385, 0.077289, 0.053920], -[3.075808, 0.235023, 0.281975], -[3.595905, 1.883551, 1.893956], -[6.186890, 0.039829, 0.032672], -[4.399702, 4.404082, 3.892671], -[3.744337, 4.524327, 5.312231], -[0.024576, 0.003828, 0.003150], -[0.943878, 0.899737, 0.895774], -[20.332790, 0.017449, 0.015121], -[0.081357, 0.025420, 0.014019], -[0.032782, 0.018635, 0.018162], -[8.747672, 10.071709, 6.743290], -[0.137468, 0.018731, 0.016640], -[0.014924, 0.011045, 0.011923], -[0.012629, 0.010551, 0.010522], -[300.245240, 300.337436, 300.220133], -[0.010773, 0.004759, 0.003960], -[1.618904, 0.114704, 0.095059], -[2.409857, 1.982224, 1.915120], -[4.036227, 1.908413, 1.817666], -[8.306265, 7.861117, 8.076500], -[17.398126, 0.054853, 0.039628], -[300.044544, 300.081014, 300.144849], -[4.389508, 3.980315, 4.060862], -[0.690731, 0.041153, 0.038142], -[0.574323, 0.034436, 0.036309], -[0.501737, 0.145582, 0.154863], -[0.005135, 0.004237, 0.003943], -[0.472923, 0.339387, 0.286213], -[0.349483, 0.142984, 0.095204], -[0.245475, 0.057260, 0.060106] diff --git a/benchmark/elasticsearch/benchmark.sh b/benchmark/elasticsearch/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/generate-results.sh b/benchmark/generate-results.sh new file mode 100755 index 00000000000..c317c3691a4 --- /dev/null +++ b/benchmark/generate-results.sh @@ -0,0 +1,5 @@ +#!/bin/bash + +ls -1 */results/*.txt | while read file; do SYSTEM=$(echo "$file" | grep -oP '^[\w-]+'); SETUP=$(echo "$file" | sed -r -e 's/^.*\/([a-zA-Z0-9_.-]+)\.txt$/\1/'); echo "[{\"system\": \"${SYSTEM} (${SETUP})\", \"version\": \"\", \"data_size\": 100000000, \"time\": \"2022-07-01 00:00:00\", \"comments\": \"\", \"result\": [ +$(grep -P '^\[.+\]' $file) +]}]" > ../website/benchmark/dbms/results/${SYSTEM}.${SETUP}.json; done diff --git a/benchmark/greenplum/benchmark.sh b/benchmark/greenplum/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/greenplum/results/c6a.4xlarge.txt b/benchmark/greenplum/results/c6a.4xlarge.txt index 5fca032be36..110155aaad3 100644 --- a/benchmark/greenplum/results/c6a.4xlarge.txt +++ b/benchmark/greenplum/results/c6a.4xlarge.txt @@ -43,4 +43,4 @@ Data size: 42 Gb [0.334809,0.275365,0.265053], [0.154522,0.107654,0.10529], [0.158957,0.117284,0.119068], -[0.193756,0.144787,0.145485], +[0.193756,0.144787,0.145485] diff --git a/benchmark/greenplum/run.sh b/benchmark/greenplum/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/locustdb/benchmark.sh b/benchmark/locustdb/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/mariadb-columnstore/benchmark.sh b/benchmark/mariadb-columnstore/benchmark.sh index 884a624490d..450120af282 100755 --- a/benchmark/mariadb-columnstore/benchmark.sh +++ b/benchmark/mariadb-columnstore/benchmark.sh @@ -30,7 +30,7 @@ time mysql --password="${PASSWORD}" --host 127.0.0.1 test -e " sudo docker exec mcs_container du -bcs /var/lib/columnstore cat log.txt | - grep -P 'rows? in set|^ERROR' | + grep -P 'rows? in set|Empty set|^ERROR' | sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/mariadb-columnstore/results/c6a.4xlarge.txt b/benchmark/mariadb-columnstore/results/c6a.4xlarge.txt index cfaa356717c..382ae71232b 100644 --- a/benchmark/mariadb-columnstore/results/c6a.4xlarge.txt +++ b/benchmark/mariadb-columnstore/results/c6a.4xlarge.txt @@ -42,4 +42,5 @@ Data size: 19 712 857 022 [0.117,0.091,0.093], [1.217,1.133,1.133], [0.114,0.063,0.062], -[0.100,0.062,0.061] +[0.100,0.062,0.061], +[null,null,null] diff --git a/benchmark/mariadb-columnstore/run.sh b/benchmark/mariadb-columnstore/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/monetdb/query.expect b/benchmark/monetdb/query.expect old mode 100644 new mode 100755 diff --git a/benchmark/monetdb/run.sh b/benchmark/monetdb/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/mysql-myisam/benchmark.sh b/benchmark/mysql-myisam/benchmark.sh index 5b83fa30c42..fa948d86132 100755 --- a/benchmark/mysql-myisam/benchmark.sh +++ b/benchmark/mysql-myisam/benchmark.sh @@ -23,7 +23,7 @@ time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" sudo du -bcs /var/lib/mysql cat log.txt | - grep -P 'rows? in set|^ERROR' | + grep -P 'rows? in set|Empty set|^ERROR' | sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/mysql-myisam/results/c6a.4xlarge.txt b/benchmark/mysql-myisam/results/c6a.4xlarge.txt index 5dc7c642eb6..e4cf2895beb 100644 --- a/benchmark/mysql-myisam/results/c6a.4xlarge.txt +++ b/benchmark/mysql-myisam/results/c6a.4xlarge.txt @@ -42,4 +42,5 @@ Data size: 121 588 958 061 bytes [1.65,1.45,1.46], [6.33,5.14,6.15], [1.60,1.41,1.41], -[1.56,1.42,1.39] +[1.56,1.42,1.39], +[7.04,1.17,1.13] diff --git a/benchmark/mysql/benchmark.sh b/benchmark/mysql/benchmark.sh index c70ff807b1c..ab44a3ce5d7 100755 --- a/benchmark/mysql/benchmark.sh +++ b/benchmark/mysql/benchmark.sh @@ -23,7 +23,7 @@ time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" sudo du -bcs /var/lib/mysql cat log.txt | - grep -P 'rows? in set|^ERROR' | + grep -P 'rows? in set|Empty set|^ERROR' | sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/questdb/run.sh b/benchmark/questdb/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/redshift-serverless/run.sh b/benchmark/redshift-serverless/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/redshift/run.sh b/benchmark/redshift/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/singlestore/benchmark.sh b/benchmark/singlestore/benchmark.sh index 07fa597dedf..492a7aa1e86 100755 --- a/benchmark/singlestore/benchmark.sh +++ b/benchmark/singlestore/benchmark.sh @@ -38,7 +38,7 @@ sudo docker exec memsql-ciab du -bcs /var/lib/memsql # 29836263469 bytes cat log.txt | - grep -P 'rows? in set|^ERROR' | + grep -P 'rows? in set|Empty set|^ERROR' | sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/snowflake/run.sh b/benchmark/snowflake/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/sqlite/benchmark.sh b/benchmark/sqlite/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/sqlite/run.sh b/benchmark/sqlite/run.sh old mode 100644 new mode 100755 diff --git a/benchmark/starrocks/benchmark.sh b/benchmark/starrocks/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/trino/benchmark.sh b/benchmark/trino/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/vertica/benchmark.sh b/benchmark/vertica/benchmark.sh old mode 100644 new mode 100755 diff --git a/benchmark/vertica/run.sh b/benchmark/vertica/run.sh old mode 100644 new mode 100755 diff --git a/website/benchmark/dbms/queries.js b/website/benchmark/dbms/queries.js index f3cf25f2c8d..3f77544b5a2 100644 --- a/website/benchmark/dbms/queries.js +++ b/website/benchmark/dbms/queries.js @@ -1,179 +1,135 @@ var current_data_size = 100000000; - var current_systems = ["ClickHouse", "Vertica", "Greenplum"]; var queries = [ { - "query": "SELECT count() FROM hits", - "comment": "", + "query": "SELECT count() FROM hits" }, { - "query": "SELECT count() FROM hits WHERE AdvEngineID != 0", - "comment": "", + "query": "SELECT count() FROM hits WHERE AdvEngineID != 0" }, { - "query": "SELECT sum(AdvEngineID), count(), avg(ResolutionWidth) FROM hits", - "comment": "", + "query": "SELECT sum(AdvEngineID), count(), avg(ResolutionWidth) FROM hits" }, { - "query": "SELECT sum(UserID) FROM hits", - "comment": "", + "query": "SELECT sum(UserID) FROM hits" }, { - "query": "SELECT uniq(UserID) FROM hits", - "comment": "", + "query": "SELECT uniq(UserID) FROM hits" }, { - "query": "SELECT uniq(SearchPhrase) FROM hits", - "comment": "", + "query": "SELECT uniq(SearchPhrase) FROM hits" }, { - "query": "SELECT min(EventDate), max(EventDate) FROM hits", - "comment": "", + "query": "SELECT min(EventDate), max(EventDate) FROM hits" }, { - "query": "SELECT AdvEngineID, count() FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count() DESC", - "comment": "", + "query": "SELECT AdvEngineID, count() FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count() DESC" }, { - "query": "SELECT RegionID, uniq(UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10", - "comment": "", + "query": "SELECT RegionID, uniq(UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10" }, { - "query": "SELECT RegionID, sum(AdvEngineID), count() AS c, avg(ResolutionWidth), uniq(UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10", - "comment": "", + "query": "SELECT RegionID, sum(AdvEngineID), count() AS c, avg(ResolutionWidth), uniq(UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10" }, { - "query": "SELECT MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10", - "comment": "", + "query": "SELECT MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10" }, { - "query": "SELECT MobilePhone, MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10", - "comment": "", + "query": "SELECT MobilePhone, MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10" }, { - "query": "SELECT SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10", - "comment": "", + "query": "SELECT SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10" }, { - "query": "SELECT SearchPhrase, uniq(UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10", - "comment": "", + "query": "SELECT SearchPhrase, uniq(UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10" }, { - "query": "SELECT SearchEngineID, SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10", - "comment": "", + "query": "SELECT SearchEngineID, SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10" }, { - "query": "SELECT UserID, count() FROM hits GROUP BY UserID ORDER BY count() DESC LIMIT 10", - "comment": "", + "query": "SELECT UserID, count() FROM hits GROUP BY UserID ORDER BY count() DESC LIMIT 10" }, { - "query": "SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase ORDER BY count() DESC LIMIT 10", - "comment": "", + "query": "SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase ORDER BY count() DESC LIMIT 10" }, { - "query": "SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase LIMIT 10", - "comment": "", + "query": "SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase LIMIT 10" }, { - "query": "SELECT UserID, toMinute(EventTime) AS m, SearchPhrase, count() FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count() DESC LIMIT 10", - "comment": "", + "query": "SELECT UserID, toMinute(EventTime) AS m, SearchPhrase, count() FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count() DESC LIMIT 10" }, { - "query": "SELECT UserID FROM hits WHERE UserID = 12345678901234567890", - "comment": "", + "query": "SELECT UserID FROM hits WHERE UserID = 12345678901234567890" }, { - "query": "SELECT count() FROM hits WHERE URL LIKE '%metrika%'", - "comment": "", + "query": "SELECT count() FROM hits WHERE URL LIKE '%metrika%'" }, { - "query": "SELECT SearchPhrase, any(URL), count() AS c FROM hits WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10", - "comment": "", + "query": "SELECT SearchPhrase, any(URL), count() AS c FROM hits WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10" }, { - "query": "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", - "comment": "", + "query": "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" }, { - "query": "SELECT * FROM hits WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10", - "comment": "", + "query": "SELECT * FROM hits WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10" }, { - "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10", - "comment": "", + "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10" }, { - "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10", - "comment": "", + "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10" }, { - "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10", - "comment": "", + "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10" }, { - "query": "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", - "comment": "", + "query": "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" }, { - "query": "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", - "comment": "", + "query": "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" }, { - "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", - "comment": "", + "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" }, { - "query": "SELECT SearchEngineID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10", - "comment": "", + "query": "SELECT SearchEngineID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10" }, { - "query": "SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10", - "comment": "", + "query": "SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10" }, { - "query": "SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10", - "comment": "", + "query": "SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10" }, { - "query": "SELECT URL, count() AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10", - "comment": "", + "query": "SELECT URL, count() AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10" }, { - "query": "SELECT 1, URL, count() AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10", - "comment": "", + "query": "SELECT 1, URL, count() AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10" }, { - "query": "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", - "comment": "", + "query": "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" }, { - "query": "SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND notEmpty(URL) GROUP BY URL ORDER BY PageViews DESC LIMIT 10", - "comment": "", + "query": "SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND notEmpty(URL) GROUP BY URL ORDER BY PageViews DESC LIMIT 10" }, { - "query": "SELECT Title, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND notEmpty(Title) GROUP BY Title ORDER BY PageViews DESC LIMIT 10", - "comment": "", + "query": "SELECT Title, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND notEmpty(Title) GROUP BY Title ORDER BY PageViews DESC LIMIT 10" }, { - "query": "SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000", - "comment": "", + "query": "SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000" }, { - "query": "SELECT TraficSourceID, SearchEngineID, AdvEngineID, ((SearchEngineID = 0 AND AdvEngineID = 0) ? Referer : '') AS Src, URL AS Dst, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000", - "comment": "", + "query": "SELECT TraficSourceID, SearchEngineID, AdvEngineID, ((SearchEngineID = 0 AND AdvEngineID = 0) ? Referer : '') AS Src, URL AS Dst, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000" }, { - "query": "SELECT URLHash, EventDate, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = halfMD5('http://yandex.ru/') GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100", - "comment": "", + "query": "SELECT URLHash, EventDate, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = halfMD5('http://yandex.ru/') GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100" }, { - "query": "SELECT WindowClientWidth, WindowClientHeight, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = halfMD5('http://yandex.ru/') GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000;", - "comment": "", + "query": "SELECT WindowClientWidth, WindowClientHeight, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = halfMD5('http://yandex.ru/') GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000;" }, { - "query": "SELECT toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute;", - "comment": "", + "query": "SELECT toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute;" } ] diff --git a/website/benchmark/dbms/results/001_clickhouse_19_1_6.json b/website/benchmark/dbms/results/001_clickhouse_19_1_6.json deleted file mode 100644 index 7b3f68d0b16..00000000000 --- a/website/benchmark/dbms/results/001_clickhouse_19_1_6.json +++ /dev/null @@ -1,163 +0,0 @@ -[ - { - "system": "ClickHouse", - "version": "19.1.6", - "data_size": 10000000, - "time": "2016-06-01 03:00:00", - "comments": "FORMAT Null", - "result": - [ - [0.092, 0.012, 0.006], - [0.087, 0.006, 0.006], - [0.098, 0.010, 0.011], - [0.134, 0.016, 0.010], - [0.176, 0.037, 0.040], - [0.233, 0.057, 0.058], - [0.066, 0.006, 0.007], - [0.071, 0.006, 0.014], - [0.259, 0.131, 0.128], - [0.266, 0.094, 0.078], - [0.217, 0.033, 0.032], - [0.211, 0.040, 0.034], - [0.249, 0.102, 0.102], - [0.343, 0.122, 0.107], - [0.302, 0.116, 0.122], - [0.209, 0.090, 0.083], - [0.415, 0.222, 0.221], - [0.287, 0.112, 0.115], - [0.562, 0.323, 0.351], - [0.127, 0.009, 0.009], - [0.410, 0.068, 0.073], - [0.494, 0.108, 0.083], - [0.911, 0.197, 0.193], - [2.426, 0.303, 0.286], - [0.311, 0.041, 0.046], - [0.203, 0.037, 0.040], - [0.262, 0.042, 0.052], - [0.365, 0.067, 0.066], - [0.735, 0.158, 0.177], - [0.549, 0.344, 0.521], - [0.357, 0.098, 0.103], - [0.509, 0.136, 0.130], - [0.775, 0.531, 0.532], - [0.641, 0.342, 0.320], - [0.600, 0.351, 0.352], - [0.295, 0.122, 0.134], - [0.291, 0.118, 0.122], - [0.203, 0.053, 0.057], - [0.234, 0.041, 0.037], - [0.502, 0.223, 0.234], - [0.275, 0.016, 0.018], - [0.222, 0.014, 0.017], - [0.182, 0.016, 0.014] - ] - }, - - { - "system": "ClickHouse", - "version": "19.1.6", - "data_size": 100000000, - "time": "2016-06-01 03:00:00", - "comments": "FORMAT Null", - "result": - [ - [0.133, 0.016, 0.016], - [0.086, 0.012, 0.037], - [0.254, 0.041, 0.037], - [0.478, 0.047, 0.045], - [0.655, 0.104, 0.105], - [0.581, 0.228, 0.297], - [0.105, 0.048, 0.022], - [0.076, 0.012, 0.015], - [0.892, 0.509, 0.487], - [1.077, 0.542, 0.589], - [0.529, 0.171, 0.167], - [0.541, 0.188, 0.182], - [0.975, 0.659, 0.603], - [1.479, 0.812, 0.770], - [1.082, 0.734, 0.778], - [0.952, 0.728, 0.729], - [2.212, 1.756, 1.952], - [1.230, 1.033, 0.907], - [4.613, 4.030, 3.955], - [0.477, 0.074, 0.037], - [1.648, 0.532, 0.555], - [2.002, 0.645, 0.651], - [3.676, 1.479, 1.487], - [7.792, 0.759, 0.788], - [1.139, 0.239, 0.215], - [0.522, 0.220, 0.215], - [1.018, 0.230, 0.238], - [1.768, 0.602, 0.603], - [1.818, 0.899, 0.916], - [3.301, 3.174, 3.283], - [1.313, 0.639, 0.631], - [2.136, 0.938, 0.975], - [5.894, 5.412, 5.467], - [3.359, 3.053, 3.061], - [3.355, 2.999, 3.004], - [1.224, 1.103, 1.063], - [0.185, 0.086, 0.092], - [0.138, 0.038, 0.038], - [0.155, 0.031, 0.031], - [0.390, 0.183, 0.162], - [0.203, 0.013, 0.013], - [0.178, 0.011, 0.011], - [0.128, 0.007, 0.007] - ] - }, - - { - "system": "ClickHouse", - "version": "19.1.6", - "data_size": 1000000000, - "time": "2016-06-01 03:00:00", - "comments": "FORMAT Null", - "result": - [ - [0.747, 0.090, 0.075], - [0.197, 0.074, 0.061], - [1.370, 0.419, 0.334], - [3.911, 0.390, 0.365], - [4.096, 0.636, 0.638], - [4.739, 1.626, 1.542], - [0.270, 0.172, 0.188], - [0.184, 0.067, 0.061], - [6.182, 3.016, 2.913], - [7.068, 3.329, 3.409], - [4.332, 1.128, 1.296], - [4.698, 1.351, 1.328], - [7.721, 5.721, 5.802], - [11.683, 7.687, 7.629], - [9.144, 6.987, 6.974], - [8.069, 6.115, 6.386], - [22.657, 20.622, 20.842], - [11.727, 10.574, 10.708], - [52.193, 51.836, 53.738], - [4.179, 0.378, 0.279], - [24.400, 4.712, 4.591], - [29.045, 5.533, 5.502], - [56.733, 13.958, 13.791], - [86.314, 8.349, 7.448], - [11.360, 1.762, 1.781], - [4.890, 1.568, 1.508], - [11.667, 1.962, 1.889], - [24.491, 5.554, 5.556], - [28.096, 8.528, 8.481], - [24.084, 25.500, 26.527], - [10.820, 5.387, 5.406], - [25.187, 8.879, 8.852], - [70.218, 67.707, 68.059], - [42.018, 36.039, 36.391], - [43.128, 35.813, 36.154], - [9.646, 8.490, 8.418], - [0.639, 0.250, 0.289], - [0.350, 0.064, 0.066], - [0.443, 0.106, 0.100], - [0.923, 0.461, 0.460], - [0.479, 0.030, 0.029], - [0.372, 0.025, 0.023], - [0.224, 0.012, 0.013] - ] - } -] diff --git a/website/benchmark/dbms/results/002_vertica_7_1_1.json b/website/benchmark/dbms/results/002_vertica_7_1_1.json deleted file mode 100644 index 2b0c340747b..00000000000 --- a/website/benchmark/dbms/results/002_vertica_7_1_1.json +++ /dev/null @@ -1,163 +0,0 @@ -[ - { - "system": "Vertica", - "version": "7.1.1", - "data_size": 10000000, - "time": "", - "comments": "", - "result": - [ - [0.015899, 0.012807, 0.012184], - [0.035133, 0.021397, 0.018946], - [0.060938, 0.034013, 0.037085], - [0.051492, 0.021746, 0.022661], - [0.150695, 0.144041, 0.143313], - [0.412724, 0.276505, 0.27558], - [0.039736, 0.021215, 0.025029], - [0.049819, 0.045784, 0.049303], - [0.248834, 0.222963, 0.217386], - [0.43366, 0.437333, 0.4883], - [0.106483, 0.094236, 0.092362], - [0.122424, 0.10004, 0.100646], - [0.952346, 0.963712, 0.994094], - [0.64299, 0.647605, 0.644699], - [0.606994, 0.552117, 0.563657], - [0.221643, 0.205149, 0.216158], - [0.789877, 0.848421, 0.869198], - [0.439601, 0.438257, 0.424207], - [1.51968, 1.351311, 1.495538], - [0.038791, 0.03504, 0.050796], - [0.847444, 0.412624, 0.413898], - [0.491446, 0.474577, 0.484147], - [1.693912, 1.166251, 1.236441], - [1.905181, 1.257361, 1.437238], - [0.36171, 0.263338, 0.28958], - [0.46795, 0.515716, 0.443451], - [0.28467, 0.248823, 0.251787], - [2.326452, 2.152684, 2.073438], - [2.953462, 2.324174, 2.845123], - [0.631466, 0.822281, 0.873889], - [0.446697, 0.320824, 0.304922], - [0.489312, 0.417575, 0.440902], - [2.178985, 2.094694, 2.164159], - [1.375479, 1.331646, 1.360907], - [1.347677, 1.354772, 1.344533], - [0.429637, 0.43564, 0.436279], - [3.297413, 3.177341, 3.194704], - [0.835327, 0.743157, 0.746247], - [0.248143, 0.20795, 0.218004], - [1.495476, 1.322633, 1.374602], - [0.187092, 0.12099, 0.127517], - [0.148605, 0.109589, 0.107272], - [0.148173, 0.12809, 0.133435] - ] - }, - - { - "system": "Vertica", - "version": "7.1.1", - "data_size": 100000000, - "time": "", - "comments": "", - "result": - [ - [0.044914, 0.033471, 0.029564], - [0.124784, 0.044458, 0.04813], - [0.253575, 0.167392, 0.166981], - [0.267045, 0.060595, 0.059291], - [0.901021, 0.881088, 1.032132], - [1.284296, 0.991411, 1.011576], - [0.149862, 0.068061, 0.067217], - [0.148711, 0.083089, 0.078208], - [1.847624, 1.836724, 1.892968], - [4.278373, 4.313527, 4.564084], - [0.358281, 0.266623, 0.288294], - [0.42072, 0.38109, 0.371086], - [5.294134, 4.280055, 4.179055], - [8.849077, 8.744801, 8.750143], - [4.469753, 4.203493, 4.319043], - [1.542069, 1.506466, 1.605814], - [8.230461, 8.129543, 8.521089], - [5.063301, 5.177715, 4.989504], - [21.097494, 21.113905, 20.863796], - [0.238975, 0.163524, 0.162261], - [3.681673, 3.664944, 3.738555], - [3.996372, 3.875857, 3.897112], - [5.086255, 5.00034, 5.019747], - [5.105649, 4.51027, 4.780023], - [6.028593, 6.027804, 5.998026], - [4.324348, 4.357931, 4.451637], - [6.011405, 6.01204, 6.006612], - [13.744667, 14.174568, 14.053413], - [9.533647, 9.626582, 9.551671], - [2.652615, 2.689042, 2.617271], - [1.881931, 1.808578, 1.80198], - [3.926782, 3.813766, 3.806481], - [19.214651, 19.52602, 19.544008], - [27.55101, 27.641466, 28.128856], - [27.687275, 27.525594, 27.338331], - [4.989802, 5.243158, 4.861738], - [2.471475, 2.239634, 2.270449], - [0.814507, 0.733751, 0.760994], - [0.280513, 0.230994, 0.231817], - [1.479762, 1.293106, 1.277735], - [0.163974, 0.12314, 0.127756], - [0.148318, 0.112932, 0.107095], - [0.069868, 0.071664, 0.065618] - ] - }, - - { - "system": "Vertica", - "version": "7.1.1", - "data_size": 1000000000, - "time": "", - "comments": "", - "result": - [ - [0.09516, 0.094485, 0.092024], - [0.816135, 0.446083, 0.453634], - [2.11051, 1.773246, 1.791243], - [2.120462, 1.033154, 1.020443], - [8.149946, 7.740829, 7.659704], - [26.119083, 25.611287, 25.675649], - [1.035186, 0.818121, 0.822225], - [0.816869, 0.506568, 0.51574], - [13.108386, 13.20647, 13.556902], - [40.935852, 39.508237, 39.034314], - [3.183196, 2.862235, 2.86959], - [3.967054, 3.658079, 3.557328], - [30.73868, 30.722098, 31.301074], - [76.889072, 76.027064, 77.056729], - [32.033544, 31.866097, 32.772241], - [11.350141, 11.177469, 11.064054], - [74.059265, 73.061888, 73.041769], - [52.895306, 52.588376, 52.671085], - [186.891072, 184.998125, 185.913578], - [2.208113, 1.871682, 1.87381], - [21.705589, 21.595201, 21.415026], - [21.080841, 20.856408, 20.696303], - [45.916174, 45.351723, 45.433121], - [47.051243, 35.723046, 35.694351], - [53.209629, 53.406901, 53.069656], - [52.737858, 52.784361, 52.617806], - [53.430247, 53.206678, 53.309617], - [372.194119, 371.545597, 370.507236], - [76.594315, 76.464039, 76.319749], - [24.274602, 24.263616, 24.198579], - [14.400851, 13.927733, 13.747829], - [30.679117, 28.09498, 27.203538], - [210.606242, 214.108745, 214.521569], - [289.044749, 291.983512, 289.419234], - [289.46641, 290.132895, 289.4922], - [43.7288, 43.325352, 43.19419], - [5.028182, 4.798986, 4.728277], - [1.145844, 1.035948, 1.017571], - [0.398542, 0.3224, 0.324956], - [5.497337, 5.271507, 5.329618], - [0.210421, 0.162334, 0.161671], - [0.201661, 0.140586, 0.135319], - [0.178466, 0.162246, 0.159834] - ] - } -] diff --git a/website/benchmark/dbms/results/003_vertica_7_0_0_x3.json b/website/benchmark/dbms/results/003_vertica_7_0_0_x3.json deleted file mode 100644 index 8f5ddaf9e73..00000000000 --- a/website/benchmark/dbms/results/003_vertica_7_0_0_x3.json +++ /dev/null @@ -1,109 +0,0 @@ -[ - { - "system": "Vertica (x3)", - "version": "7.0.0-0", - "data_size": 1000000000, - "time": "", - "comments": "", - "result": - [ - [3.328, 0.397, 0.382], - [2.889, 0.24, 0.219], - [5.667, 0.781, 0.79], - [5.163, 0.58, 0.545], - [10.743, 7.414, 7.554], - [44.021, 43.629, 42.651], - [3.045, 0.416, 0.419], - [2.987, 0.366, 0.387], - [16.039, 13.626, 13.646], - [23.93, 21.818, 21.376], - [7.084, 2.683, 2.763], - [9.39, 3.176, 3.085], - [41.674, 42.039, 42.239], - [138.331, 136.452, 137.238], - [68.416, 67.551, 67.478], - [12.463, 10.125, 9.974], - [188.023, 186.817, 186.116], - [172.149, 170.75, 171.178], - [489.181, 488.154, 489.749], - [5.466, 0.916, 0.891], - [24.91, 18.009, 17.985], - [25.49, 18.525, 18.803], - [39.856, 17.993, 18.436], - [128.041, 9.876, 9.599], - [31.162, 30.831, 30.708], - [33.871, 32.901, 33.198], - [31.26, 30.795, 30.982], - [97.13, 93.233, 93.352], - [123.941, 123.625, 123.509], - [10.032, 9.418, 9.431], - [19.05, 9.184, 8.907], - [22.433, 19.726, 19.764], - [171.142, 162.149, 162.224], - [286.116, 283.672, 282.848], - [281.98, 278.234, 280.236], - [30.897, 30.486, 30.338], - [11.785, 11.42, 11.262], - [2.188, 1.739, 1.782], - [1.754, 1.582, 1.852], - [25.553, 24.89, 24.753], - [0.853, 0.324, 0.304], - [0.656, 0.368, 0.332], - [0.436, 0.356, 0.31] - ] - }, - - { - "system": "Vertica (x6)", - "version": "7.0.0-0", - "data_size": 1000000000, - "time": "", - "comments": "", - "result": - [ - [2.203, 0.392, 0.421], - [2.201, 0.336, 0.36], - [3.669, 0.704, 0.624], - [3.512, 0.516, 0.501], - [7.482, 5.696, 5.812], - [43.298, 48.75, 42.419], - [2.285, 0.411, 0.396], - [2.111, 0.454, 0.447], - [12.751, 10.454, 10.447], - [16.751, 15.247, 16.011], - [4.789, 2.06, 2.023], - [6.015, 2.207, 2.273], - [42.854, 41.299, 42.517], - [50.338, 48.5, 48.569], - [27.033, 25.38, 25.336], - [8.766, 6.73, 6.584], - [63.644, 64.514, 63.864], - [56.751, 56.018, 56.263], - [151.68, 149.595, 150.832], - [3.249, 0.701, 0.639], - [25.006, 18.019, 17.994], - [27.427, 18.74, 18.759], - [23.331, 9.915, 10.314], - [92.277, 6.708, 6.496], - [16.708, 15.827, 16.021], - [18.68, 18.99, 18.492], - [16.39, 16, 16.023], - [50.455, 48.204, 49.54], - [64.871, 64.565, 63.996], - [5.638, 5.103, 5.086], - [14.41, 6.785, 6.714], - [22.602, 14.089, 13.952], - [106.629, 105.692, 103.34], - [301.935, 293.388, 295.326], - [299.916, 290.717, 297.424], - [21.476, 21.447, 21.519], - [7.853, 7.158, 7.353], - [2.087, 1.573, 1.543], - [2.536, 1.788, 1.912], - [11.355, 10.46, 9.62], - [1.008, 0.401, 0.434], - [1.358, 0.373, 0.422], - [0.841, 0.438, 0.442] - ] - } -] diff --git a/website/benchmark/dbms/results/004_infinidb_3_6_23.json b/website/benchmark/dbms/results/004_infinidb_3_6_23.json deleted file mode 100644 index 0a446564af8..00000000000 --- a/website/benchmark/dbms/results/004_infinidb_3_6_23.json +++ /dev/null @@ -1,109 +0,0 @@ -[ - { - "system": "InfiniDB", - "version": "Enterprise 3.6.23", - "data_size": 10000000, - "time": "", - "comments": "", - "result": - [ - [1.15, 0.17, 0.16], - [0.31, 0.17, 0.16], - [0.47, 0.31, 0.30], - [null, null, null], - [0.97, 0.87, 0.92], - [6.14, 6.56, 5.43], - [0.35, 0.25, 0.26], - [0.22, 0.17, 0.16], - [1.19, 0.94, 1.02], - [1.34, 1.37, 1.33], - [0.51, 0.39, 0.39], - [0.45, 0.40, 0.38], - [8.06, 8.02, 8.02], - [8.43, 8.83, 8.89], - [8.21, 8.31, 8.42], - [1.73, 1.76, 1.78], - [18.95, 17.76, 19.98], - [12.59, 13.64, 12.24], - [37.04, 35.00, 36.76], - [0.25, 0.14, 0.13], - [2.28, 0.81, 0.86], - [0.61, 0.63, 0.57], - [3.02, 1.15, 1.17], - [12.44, 1.19, 1.19], - [8.92, 8.83, 9.07], - [8.15, 8.11, 8.11], - [10.39, 10.18, 10.33], - [5.70, 5.70, 5.82], - [13.77, 15.06, 13.88], - [8.57, 9.29, 8.58], - [2.03, 2.02, 2.00], - [3.18, 3.31, 3.26], - [20.23, 19.45, 20.16], - [183.55, 156.42, 124.94], - [160.14, 164.08, 162.15], - [3.49, 1.67, 1.71], - [23.03, 21.05, 21.21], - [3.14, 1.70, 1.65], - [1.64, 1.27, 1.23], - [82.86, 72.81, 77.55], - [0.32, 0.18, 0.18], - [0.28, 0.18, 0.19], - [3.43, 1.61, 1.53] - ] - }, - - { - "system": "InfiniDB", - "version": "Enterprise 3.6.23", - "data_size": 100000000, - "time": "", - "comments": "", - "result": - [ - [2.07, 0.34, 0.35], - [0.76, 0.3, 0.31], - [1.45, 1.23, 1.24], - [null, null, null], - [4.18, 3.89, 3.85], - [26.32, 28.07, 23.96], - [1.36, 1.04, 1.03], - [0.56, 0.32, 0.3], - [5.14, 4.54, 4.51], - [7.83, 8.18, 8.0], - [1.96, 1.4, 1.45], - [1.75, 1.52, 1.46], - [23.72, 23.01, 23.87], - [30.74, 30.86, 28.36], - [25.55, 24.76, 24.41], - [11.66, 11.59, 11.67], - [80.45, 85.49, 116.21], - [52.27, 50.76, 48.3], - [null, null, null], - [4.31, 0.24, 0.16], - [130.37, 7.24, 7.78], - [66.62, 10.19, 10.2], - [32.34, 19.66, 19.59], - [288.38, 58.86, 7.35], - [57.88, 57.95, 57.82], - [47.32, 52.59, 47.03], - [73.32, 65.1, 73.43], - [50.6, 51.5, 50.93], - [89.16, 85.75, 87.26], - [61.97, 60.49, 62.13], - [10.3, 10.4, 10.31], - [21.11, 20.86, 20.99], - [157.67, 151.81, 153.5], - [null, null, null], - [null, null, null], - [11.86, 11.08, 11.13], - [12.35, 12.49, 12.36], - [3.11, 3.12, 3.14], - [1.03, 0.89, 0.9], - [34.01, 45.75, 50.3], - [0.21, 0.23, 0.24], - [0.23, 0.21, 0.23], - [0.14, 0.15, 0.17] - ] - } -] diff --git a/website/benchmark/dbms/results/005_monetdb.json b/website/benchmark/dbms/results/005_monetdb.json deleted file mode 100644 index c378bb0a4fa..00000000000 --- a/website/benchmark/dbms/results/005_monetdb.json +++ /dev/null @@ -1,55 +0,0 @@ -[ - { - "system": "MonetDB", - "version": "11.37.11 (Jun2020-SP1)", - "data_size": 100000000, - "time": "2020-08-12", - "comments": "", - "result": - [ - [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] - ] - } -] diff --git a/website/benchmark/dbms/results/006_infobright_4_0_7.json b/website/benchmark/dbms/results/006_infobright_4_0_7.json deleted file mode 100644 index 25cf680f9d9..00000000000 --- a/website/benchmark/dbms/results/006_infobright_4_0_7.json +++ /dev/null @@ -1,55 +0,0 @@ -[ - { - "system": "Infobright", - "version": "CE 4.0.7", - "data_size": 10000000, - "time": "", - "comments": "", - "result": - [ - [0.00, 0.00, 0.00], - [0.40, 0.38, 0.39], - [0.10, 0.00, 0.00], - [null, null, null], - [2.83, 1.91, 1.95], - [9.16, 1.65, 1.70], - [0.03, 0.00, 0.00], - [0.46, 0.40, 0.41], - [4.13, 2.97, 3.43], - [5.12, 4.46, 4.15], - [1.98, 1.24, 1.36], - [1.58, 1.26, 1.27], - [13.37, 30.81, 29.76], - [32.59, 30.22, 13.00], - [12.93, 11.39, 30.46], - [2.98, 3.05, 2.96], - [9.90, 20.86, 25.17], - [10.38, 10.19, 10.28], - [162.43, 164.35, 169.28], - [1, 0, 22, 0, 24], - [47.80, 4.40, 4.47], - [4.83, 1.15, 1.14], - [43.82, 2.14, 2.16], - [6.14, 4.39, 4.35], - [0.41, 0.41, 0.41], - [0.80, 0.80, 0.80], - [0.41, 0.41, 0.40], - [10.39, 10.12, 9.88], - [139.25, 79.33, 78.92], - [145.8, 146.2, 144.97], - [3.34, 2.02, 2.06], - [3.68, 3.05, 3.02], - [14.76, 14.82, 14.76], - [77.49, 91.4, 90.9], - [75.37, 83.55, 63.55], - [10.80, 10.52, 10.67], - [23.77, 9.47, 9.30], - [50.21, 3.37, 3.36], - [1.26, 0.77, 0.76], - [98.25, 104.6, 94.29], - [2.04, 0.47, 0.54], - [1.36, 0.42, 0.41], - [4.58, 4.01, 3.98] - ] - } -] diff --git a/website/benchmark/dbms/results/007_hive_0_11.json b/website/benchmark/dbms/results/007_hive_0_11.json deleted file mode 100644 index a025fe9d884..00000000000 --- a/website/benchmark/dbms/results/007_hive_0_11.json +++ /dev/null @@ -1,109 +0,0 @@ -[ - { - "system": "Hive", - "version": "0.11, ORC File", - "data_size": 10000000, - "time": "", - "comments": "", - "result": - [ - [47.388, 44.55, 43.513], - [25.332, 22.592, 22.629], - [27.558, 23.861, 24.986], - [26.148, 23.564, 23.508], - [35.237, 31.445, 32.552], - [34.063, 29.607, 29.268], - [25.999, 22.443, 22.559], - [38.784, 37.082, 37.652], - [49.973, 47.282, 46.027], - [54.759, 50.301, 51.858], - [42.793, 39.001, 38.998], - [42.858, 38.928, 40.035], - [55.967, 53.253, 53.053], - [58.068, 54.393, 53.189], - [58.359, 53.181, 54.164], - [63.096, 58.614, 60.153], - [73.175, 70.386, 69.204], - [35.511, 31.512, 31.482], - [109.132, 107.333, 106.376], - [17.948, 14.47, 14.154], - [27.452, 24.527, 24.674], - [41.792, 40.17, 40.052], - [45.079, 42.12, 43.438], - [50.847, 46.004, 45.95], - [31.007, 26.473, 26.277], - [30.985, 27.724, 27.357], - [32.747, 28.329, 27.439], - [62.932, 57.159, 59.233], - [63.563, 63.375, 63.307], - [74.663, 67.206, 68.586], - [58.017, 52.364, 53.155], - [62.907, 60.202, 59.653], - [127.206, 124.701, 123.291], - [89.931, 87.6, 87.325], - [98.879, 89.299, 90.377], - [63.792, 61.127, 61.517], - [44.325, 39.995, 39.979], - [43.852, 40.178, 40.131], - [44.493, 40.17, 40.171], - [36.108, 36.293, 36.241], - [43.025, 39.168, 40.042], - [42.914, 40.129, 39.135], - [33.91, 34.161, 34.191] - ] - }, - - { - "system": "Hive", - "version": "0.11, ORC File", - "data_size": 100000000, - "time": "", - "comments": "", - "result": - [ - [110.676, 105.13, 107.358], - [55.195, 36.435, 32.201], - [39.991, 35.143, 35.085], - [44.465, 34.131, 34.032], - [110.69, 105.953, 107.343], - [68.119, 64.831, 64.269], - [37.809, 33.021, 33.13], - [53.788, 51.261, 48.653], - [87.479, 85.062, 85.039], - [106.577, 102.879, 101.705], - [60.4, 53.498, 53.516], - [61.275, 53.698, 53.577], - [87.924, 82.999, 82.867], - [94.281, 86.991, 87.084], - [91.05, 87.267, 87.731], - [132.697, 132.306, 130.91], - [141.357, 147.059, 140.75], - [60.884, 57.376, 57.367], - [237.554, 234.361, 234.271], - [34.019, 21.834, 21.08], - [41.195, 36.443, 35.979], - [60.385, 54.888, 56.541], - [67.257, 58.995, 59.828], - [87.697, 88.521, 89.324], - [53.796, 50.592, 50.118], - [68.786, 63.993, 62.886], - [60.715, 56.14, 55.303], - [112.58, 107.297, 106.493], - [115.068, 110.622, 109.541], - [136.36, 133.102, 135.896], - [113.348, 100.032, 99.905], - [124.002, 117.366, 109.524], - [301.77, 324.867, 294.034], - [233.937, 272.053, 238.167], - [241.283, 228.198, 246.999], - [120.684, 118.948, 118.18], - [70.292, 55.211, 55.076], - [63.4, 52.093, 52.895], - [67.483, 53.704, 54.814], - [60.588, 52.321, 53.356], - [62.644, 51.812, 53.23], - [69.068, 53.234, 52.853], - [46.67, 46.041, 45.95] - ] - } -] diff --git a/website/benchmark/dbms/results/008_mysql_5_5.json b/website/benchmark/dbms/results/008_mysql_5_5.json deleted file mode 100644 index c132ee3b2bb..00000000000 --- a/website/benchmark/dbms/results/008_mysql_5_5.json +++ /dev/null @@ -1,109 +0,0 @@ -[ - { - "system": "MySQL", - "version": "5.5.32, MyISAM", - "data_size": 10000000, - "time": "", - "comments": "", - "result": - [ - [0.01, 0.01, 0.01], - [21.55, 18.91, 18.28], - [22.71, 19.86, 20.53], - [21.3, 18.93, 19.25], - [26.77, 25.74, 25.65], - [29.14, 26.92, 26.53], - [7.47, 7.38, 7.23], - [20.56, 18.84, 18.41], - [27.53, 25.14, 24.45], - [30.08, 26.07, 26.75], - [22.93, 19.82, 20.23], - [21.79, 19.75, 19.64], - [51.3, 42.27, 46.45], - [43.75, 42.38, 42.36], - [43.76, 48.66, 46.8], - [106.76, 106.53, 105.32], - [172.51, 181.59, 177.7], - [189.92, 166.09, 172.03], - [185.61, 181.38, 206.92], - [20.3, 19.24, 18.49], - [21.43, 20.03, 19.99], - [21.88, 20.06, 20.3], - [25.51, 20.72, 20.58], - [24.14, 20.04, 19.29], - [21.65, 19.11, 19.0], - [22.44, 20.35, 20.02], - [21.41, 19.96, 19.91], - [27.15, 26.28, 25.32], - [135.45, 134.31, 133.12], - [107.44, 106.26, 106.16], - [40.47, 36.67, 37.07], - [39.2, 36.86, 37.22], - [134.5, 130.75, 133.55], - [1057.45, 1075.29, 928.38], - [867.64, 1023.33, 1063.4], - [111.01, 109.86, 109.34], - [1160.03, 23.32, 23.08], - [1109.2, 14.24, 14.38], - [1086.92, 7.29, 6.63], - [31.74, 31.79, null], - [1074.29, 5.51, 5.43], - [1069.16, 5.6, 5.3], - [652.84, 4.84, 4.57] - ] - }, - - { - "system": "MySQL", - "version": "5.5.32, MyISAM", - "data_size": 100000000, - "time": "", - "comments": "", - "result": - [ - [0.01, 0.01, 0.01], - [220.39, 234.32, 305.28], - [220.45, 198.31, 198.37], - [207.6, 190.59, 188.35], - [275.96, 250.84, 246.93], - [292.17, 254.14, 251.06], - [75.51, 76.11, 74.98], - [203.94, 184.14, 180.82], - [287.28, 252.52, 249.48], - [299.44, 282.02, 271.33], - [218.71, 197.51, 195.94], - [220.1, 197.17, 199.88], - [929.45, 869.74, 739.53], - [null, null, null], - [1196.42, null, null], - [null, null, null], - [null, null, null], - [null, null, null], - [null, null, null], - [447.72, 199.09, 185.82], - [582.73, 196.73, 195.8], - [582.54, 205.89, 199.15], - [568.75, 217.15, 209.65], - [562.72, 206.77, 203.19], - [602.47, 186.8, 186.62], - [565.26, 199.44, 199.24], - [657.78, 202.53, 196.95], - [675.84, 250.11, 248.9], - [null, null, null], - [1061.89, 1054.6, null], - [993.89, 918.67, null], - [604.48, 553.33, null], - [null, null, null], - [null, null, null], - [null, null, null], - [null, null, null], - [null, null, null], - [873.29, null, null], - [null, null, null], - [null, null, null], - [null, null, null], - [null, null, null], - [0.67, 0.0, 0.0] - ] - } -] diff --git a/website/benchmark/dbms/results/009_memsql_3_2.json b/website/benchmark/dbms/results/009_memsql_3_2.json deleted file mode 100644 index 7195f904778..00000000000 --- a/website/benchmark/dbms/results/009_memsql_3_2.json +++ /dev/null @@ -1,55 +0,0 @@ -[ - { - "system": "MemSQL", - "version": "3.2, column store", - "data_size": 10000000, - "time": "2015-04-05", - "comments": "", - "result": - [ - [0.01, 0.01, 0.01], - [0.29, 0.26, 0.25], - [0.48, 0.45, 0.39], - [0.75, 0.63, 0.62], - [1.03, 0.97, 0.89], - [2.76, 2.65, 2.57], - [0.36, 0.32, 0.32], - [0.29, 0.24, 0.24], - [1.71, 1.51, 1.47], - [1.91, 1.69, 1.67], - [0.83, 0.66, 0.65], - [0.88, 0.70, 0.70], - [2.55, 2.59, 2.32], - [null, null, null], - [4.36, 2.34, 2.39], - [1.23, 1.09, 1.09], - [3.26, 3.18, 2.81], - [2.76, 2.58, 2.58], - [5.23, 4.74, 4.45], - [null, null, null], - [5.12, 4.62, 4.81], - [5.43, 4.91, 4.90], - [7.32, 6.18, 6.14], - [22.61, 17.85, 17.89], - [1.04, 0.77, 0.75], - [0.93, 0.77, 0.90], - [1.04, 0.75, 0.76], - [2.84, 2.41, 2.31], - [18.64, 18.19, 18.38], - [5.78, 5.68, 5.67], - [2.24, 1.90, 1.85], - [2.65, 2.22, 2.22], - [8.82, 8.32, 8.01], - [11.30, 10.93, 11.21], - [11.22, 10.73, 10.72], - [1.60, 1.46, 1.45], - [1.86, 1.75, 1.83], - [1.16, 1.10, 1.11], - [0.54, 0.44, 0.44], - [3.79, 3.59, 3.58], - [null, null, null], - [null, null, null], - [0.37, 0.35, 0.35] - ] - } -] diff --git a/website/benchmark/dbms/results/010_greenplum_4_3_9.json b/website/benchmark/dbms/results/010_greenplum_4_3_9.json deleted file mode 100644 index c2c43b22de6..00000000000 --- a/website/benchmark/dbms/results/010_greenplum_4_3_9.json +++ /dev/null @@ -1,161 +0,0 @@ -[ - { - "system": "Greenplum", - "version": "4.3.9.1", - "data_size": 10000000, - "time": "", - "comments": "", - "result": - [ - [0.77, 0.77, 0.92], - [0.44, 0.27, 0.46], - [0.95, 0.90, 0.89], - [0.74, 0.91, 0.83], - [1.43, 1.47, 1.44], - [1.74, 1.63, 1.51], - [0.77, 0.83, 0.88], - [1.86, 1.95, 1.86], - [2.09, 1.92, 1.92], - [2.33, 2.41, 2.46], - [0.75, 0.78, 0.77], - [0.71, 0.70, 0.75], - [2.11, 2.22, 2.50], - [2.65, 2.47, 2.59], - [2.11, 2.40, 2.46], - [1.59, 1.66, 1.14], - [1.75, 1.96, 1.71], - [1.29, 1.30, 1.02], - [1.99, 2.25, 2.29], - [0.65, 0.53, 0.49], - [1.23, 0.98, 1.01], - [1.85, 1.81, 1.91], - [1.85, 1.24, 1.17], - [8.21, 3.48, 2.71], - [0.77, 0.49, 0.75], - [0.68, 0.68, 0.88], - [0.76, 0.46, 0.78], - [2.12, 2.07, 2.36], - [3.08, 2.86, 3.12], - [11.50, 11.90, 10.32], - [1.89, 1.85, 1.86], - [1.99, 1.96, 2.02], - [5.24, 4.93, 5.03], - [3.24, 3.40, 3.27], - [3.01, 2.64, 2.65], - [3.14, 2.80, 2.73], - [1.47, 1.14, 1.24], - [1.79, 1.05, 1.13], - [1.32, 1.11, 1.14], - [2.20, 1.51, 1.48], - [1.42, 1.01, 1.02], - [2.17, 1.74, 2.23], - [2.17, 1.70, 1.94] - ] - }, - { - "system": "Greenplum", - "version": "4.3.9.1", - "data_size": 100000000, - "time": "", - "comments": "", - "result": - [ - [4.11, 4.32, 4.36], - [2.56, 2.53, 2.80], - [4.98, 5.14, 5.45], - [6.64, 5.99, 5.69], - [5.86, 5.72, 5.98], - [6.71, 6.76, 7.36], - [4.25, 4.39, 4.62], - [4.53, 4.23, 4.14], - [9.12, 9.23, 9.18], - [14.77, 15.38, 14.37], - [3.22, 2.87, 3.89], - [4.41, 4.29, 4.89], - [6.78, 7.19, 6.36], - [15.04, 16.93, 16.14], - [7.08, 6.21, 6.96], - [6.83, 6.17, 5.02], - [8.90, 9.81, 9.29], - [6.84, 5.31, 7.67], - [15.33, 16.31, 15.42], - [3.18, 4.29, 3.59], - [8.47, 8.73, 9.91], - [10.40, 8.38, 9.67], - [10.68, 8.16, 7.90], - [50.37, 32.46, 30.45], - [5.21, 3.64, 4.13], - [5.47, 5.68, 5.91], - [4.33, 3.20, 3.83], - [10.76, 11.14, 11.22], - [18.45, 17.26, 16.28], - [96.58, 97.29, 92.61], - [7.23, 7.36, 7.57], - [10.35, 8.20, 9.87], - [38.32, 37.57, 40.99], - [16.27, 15.24, 16.74], - [15.90, 15.33, 16.27], - [13.44, 13.84, 13.24], - [1.92, 1.16, 1.10], - [1.88, 1.16, 1.10], - [1.87, 1.07, 0.98], - [2.71, 1.22, 1.22], - [1.71, 0.97, 1.07], - [2.44, 1.78, 1.68], - [2.19, 1.72, 2.24] - ] - }, - { - "system": "Greenplum", - "version": "4.3.9.1", - "data_size": 1000000000, - "time": "", - "comments": "", - "result": - [ - [29.01, 32.82, 30.09], - [20.93, 22.20, 20.63], - [34.41, 35.26, 36.59], - [44.01, 41.56, 41.36], - [36.87, 35.96, 39.32], - [46.44, 50.24, 45.50], - [29.22, 31.75, 30.19], - [20.58, 20.76, 24.18], - [56.29, 56.67, 57.36], - [79.13, 81.78, 78.60], - [32.11, 33.21, 29.69], - [33.69, 29.92, 30.92], - [44.85, 42.52, 40.64], - [63.16, 63.16, 64.79], - [47.16, 43.26, 42.45], - [41.04, 43.67, 41.76], - [63.45, 64.64, 60.74], - [48.56, 51.07, 48.81], - [79.89, 81.48, 81.27], - [25.73, 29.27, 31.48], - [72.20, 75.93, 71.44], - [74.50, 73.46, 74.82], - [81.19, 80.76, 78.67], - [339.14, 296.80, 296.95], - [28.51, 31.36, 28.67], - [36.49, 36.47, 38.96], - [31.26, 29.18, 31.65], - [80.03, 83.33, 80.84], - [73.36, 73.27, 73.57], - [961.62, 935.00, 944.02], - [51.55, 50.44, 45.14], - [73.03, 72.86, 70.11], - [29.24, 28.79, 29.78], - [37.51, 39.70, 39.66], - [53.86, 53.37, 53.77], - [84.54, 84.86, 85.62], - [7.01, 1.93, 2.16], - [5.39, 1.55, 1.69], - [6.85, 1.43, 1.57], - [13.18, 3.17, 3.08], - [5.97, 1.30, 1.47], - [5.69, 2.12, 1.96], - [4.11, 2.27, 2.43] - ] - } -] diff --git a/website/benchmark/dbms/results/011_greenplum_4_3_9_x2.json b/website/benchmark/dbms/results/011_greenplum_4_3_9_x2.json deleted file mode 100644 index 269f44d8dff..00000000000 --- a/website/benchmark/dbms/results/011_greenplum_4_3_9_x2.json +++ /dev/null @@ -1,161 +0,0 @@ -[ - { - "system": "Greenplum(x2)", - "version": "4.3.9.1", - "data_size": 100000000, - "time": "", - "comments": "", - "result": - [ - [2.11, 1.61, 1.60], - [1.30, 1.30, 1.25], - [2.19, 2.12, 2.10], - [2.48, 2.53, 2.50], - [2.96, 3.05, 3.12], - [4.02, 4.02, 3.98], - [1.99, 1.79, 1.81], - [4.26, 3.77, 3.80], - [7.83, 7.19, 6.67], - [11.48, 11.72, 11.62], - [2.12, 2.06, 2.02], - [2.18, 2.05, 2.03], - [5.23, 5.16, 5.29], - [7.01, 6.84, 6.91], - [5.11, 5.31, 5.25], - [3.47, 3.20, 3.24], - [4.19, 4.18, 4.19], - [3.25, 3.16, 3.25], - [6.72, 6.65, 6.84], - [1.70, 1.57, 1.55], - [3.84, 3.58, 3.46], - [6.80, 5.48, 5.31], - [10.50, 3.71, 3.86], - [31.39, 13.54, 14.30], - [3.00, 1.76, 1.70], - [2.13, 2.01, 2.11], - [2.69, 1.72, 1.67], - [6.26, 5.90, 5.58], - [16.77, 16.00, 15.89], - [45.96, 46.69, 47.78], - [4.61, 4.15, 4.22], - [7.08, 5.49, 5.64], - [28.43, 26.11, 28.32], - [19.05, 19.68, 19.23], - [19.23, 19.01, 20.48], - [7.71, 7.69, 7.61], - [2.50, 1.74, 1.74], - [2.15, 1.60, 1.77], - [2.21, 1.67, 1.97], - [2.88, 1.90, 1.82], - [2.25, 1.81, 1.84], - [3.36, 3.13, 3.18], - [3.16, 3.21, 2.90] - ] - }, - { - "system": "Greenplum(x2)", - "version": "4.3.9.1", - "data_size": 10000000, - "time": "", - "comments": "", - "result": - [ - [0.37, 0.17, 0.22], - [0.25, 0.13, 0.12], - [0.39, 0.22, 0.23], - [0.47, 0.26, 0.26], - [1.30, 1.49, 1.44], - [1.87, 1.85, 2.06], - [0.32, 0.19, 0.19], - [2.85, 3.32, 3.06], - [2.60, 2.44, 2.57], - [2.79, 2.46, 2.79], - [0.80, 0.57, 0.59], - [0.83, 0.60, 0.61], - [3.47, 3.21, 3.43], - [1.54, 1.33, 1.40], - [3.29, 3.11, 3.45], - [1.35, 1.45, 1.49], - [1.65, 1.54, 1.65], - [0.92, 0.81, 0.86], - [1.67, 1.25, 1.28], - [0.33, 0.19, 0.17], - [0.59, 0.40, 0.41], - [2.07, 2.22, 1.96], - [1.70, 0.95, 0.87], - [6.83, 1.74, 1.80], - [0.46, 0.20, 0.19], - [0.44, 0.28, 0.28], - [0.50, 0.19, 0.20], - [2.06, 2.07, 2.15], - [3.02, 2.94, 2.80], - [5.42, 5.62, 5.37], - [2.10, 1.95, 2.05], - [2.31, 2.16, 2.16], - [3.74, 3.59, 3.67], - [3.89, 3.99, 3.93], - [3.03, 2.89, 3.10], - [3.56, 3.41, 3.19], - [2.12, 1.96, 1.96], - [2.03, 1.87, 1.86], - [2.11, 1.83, 1.76], - [2.53, 2.10, 1.96], - [2.31, 1.68, 1.87], - [3.26, 3.22, 3.27], - [3.19, 3.23, 3.11] - ] - }, - { - "system": "Greenplum(x2)", - "version": "4.3.9.1", - "data_size": 1000000000, - "time": "", - "comments": "", - "result": - [ - [17.70, 12.63, 12.16], - [9.14, 9.22, 9.15], - [17.13, 17.56, 16.67], - [21.25, 20.67, 21.22], - [17.16, 16.49, 17.86], - [25.70, 25.60, 25.78], - [14.77, 15.00, 14.73], - [11.80, 12.09, 12.05], - [33.57, 33.21, 33.01], - [61.65, 59.21, 60.16], - [13.69, 13.74, 13.77], - [14.48, 14.19, 14.28], - [25.59, 26.22, 26.35], - [30.75, 31.32, 31.37], - [27.70, 28.49, 28.18], - [20.28, 20.50, 20.28], - [27.80, 27.13, 29.00], - [23.66, 14.42, 24.30], - [59.58, 58.06, 58.91], - [12.86, 13.18, 13.26], - [36.04, 32.46, 32.59], - [45.28, 34.80, 34.56], - [97.71, 34.57, 33.62], - [215.97, 121.61, 120.47], - [24.44, 13.65, 13.62], - [17.15, 17.01, 17.12], - [23.84, 13.32, 13.46], - [40.83, 39.39, 38.71], - [155.70, 155.18, 158.97], - [451.18, 448.88, 449.55], - [27.37, 25.11, 25.06], - [56.58, 42.46, 43.33], - [29.77, 29.24, 29.11], - [36.94, 31.05, 29.40], - [104.88, 102.26, 101.88], - [41.06, 41.52, 41.15], - [6.91, 2.29, 2.37], - [5.35, 2.00, 2.02], - [6.49, 1.98, 1.88], - [12.55, 3.30, 3.44], - [6.18, 1.95, 2.06], - [6.12, 3.02, 3.43], - [5.21, 3.66, 3.58] - ] - } -] diff --git a/website/benchmark/dbms/results/012_omnisci.json b/website/benchmark/dbms/results/012_omnisci.json deleted file mode 100644 index 3dad155f996..00000000000 --- a/website/benchmark/dbms/results/012_omnisci.json +++ /dev/null @@ -1,56 +0,0 @@ -[ - { - "system": "OmniSci", - "version": "2020-08-27", - "data_size": 100000000, - "time": "", - "comments": "", - "result": - [ - [23.471, 0.043, 0.035], - [17.329, 0.059, 0.059], - [17.31, 0.115, 0.13], - [26.091, 0.089, 0.072], - [21.72, 0.364, 0.345], - [19.315, 0.386, 0.382], - [19.432, 0.131, 0.148], - [20.661, 0.064, 0.089], - [21.472, 1.504, 1.505], - [22.285, 1.655, 1.658], - [22.344, 0.123, 0.118], - [21.695, 0.31, 0.292], - [23.36, 0.631, 0.624], - [68.618, 47.296, 46.866], - [25.984, 2.728, 2.789], - [26.953, 6.158, 6.286], - [33.581, 10.509, 10.047], - [39.575, 8.785, 8.665], - [57.764, 28.199, 27.336], - [18.976, 0.136, 0.136], - [32.445, 0.126, 0.136], - [null, null, null], - [null, null, null], - [96.166, 0.314, 0.305], - [27.494, 0.216, 0.222], - [38.308, 17.256, 17.31], - [115.714, 96.041, 94.383], - [31.779, 2.647, 2.937], - [null, null, null], - [28.854, 5.655, 5.581], - [31.925, 4.142, 4.162], - [44.297, 8.825, 8.711], - [null, null, null], - [29.715, 1.669, 1.586], - [35.201, 7.414, 7.567], - [26.724, 6.026, 5.92], - [31.908, 1.154, 1.168], - [27.997, 0.724, 0.744], - [34.661, 1.2, 1.159], - [30.136, 0.467, 0.451], - [19.991, 0.327, 0.326], - [18.66, 0.266, 0.255], - [25.227, 0.212, 0.2] - ] - } -] - diff --git a/website/benchmark/dbms/results/013_timescaledb.json b/website/benchmark/dbms/results/013_timescaledb.json deleted file mode 100644 index 836bf127910..00000000000 --- a/website/benchmark/dbms/results/013_timescaledb.json +++ /dev/null @@ -1,56 +0,0 @@ -[ - { - "system": "TimescaleDB", - "version": "2021-10-30", - "data_size": 100000000, - "time": "2021-10-30", - "comments": "", - "result": - [ -[3.259, 3.135, 3.135], -[146.854, 6.921, 6.619], -[146.568, 7.481, 7.258], -[146.864, 5.69, 5.381], -[227.507, 69.165, 72.216], -[323.644, 177.578, 175.055], -[146.147, 5.735, 5.428], -[148.658, 7.014, 6.599], -[202.423, 54.439, 54.8], -[201.152, 55.875, 55.2], -[146.042, 9.931, 10.037], -[150.811, 10.32, 9.993], -[173.071, 34.314, 34.42], -[172.874, 43.704, 43.918], -[178.484, 36.85, 35.789], -[169.72, 24.125, 23.782], -[182.335, 37.324, 37.124], -[163.799, 18.514, 18.968], -[294.799, 149.592, 149.466], -[140.797, 5.312, 5.02], -[143.092, 7.893, 7.661], -[143.682, 9.249, 9.073], -[150.965, 20.35, 20.074], -[4.674, 4.532, 4.555], -[0.005, 0.005, 0.004], -[141.152, 7.492, 7.3], -[0.03, 0.005, 0.005], -[144.034, 10.701, 10.348], -[191.575, 97.836, 97.673], -[143.652, 22.185, 21.887], -[153.481, 17.748, 17.551], -[167.448, 25.902, 25.592], -[299.183, 145.349, 143.214], -[389.851, 228.158, 231.811], -[407.458, 230.125, 230.764], -[174.098, 23.503, 24.322], -[145.906, 10.824, 10.484], -[144.063, 8.947, 8.608], -[141.883, 7.977, 7.673], -[147.1, 9.527, 9.457], -[144.585, 10.815, 10.594], -[145.738, 10.592, 10.181], -[145.023, 8.035, 7.865] - ] - } -] - diff --git a/website/benchmark/dbms/results/014_timescaledb.json b/website/benchmark/dbms/results/014_timescaledb.json deleted file mode 100644 index 85f125c05a6..00000000000 --- a/website/benchmark/dbms/results/014_timescaledb.json +++ /dev/null @@ -1,56 +0,0 @@ -[ - { - "system": "TimescaleDB (compressed)", - "version": "2021-10-31", - "data_size": 100000000, - "time": "2021-10-31", - "comments": "", - "result": - [ -[1.784, 1.223, 1.2], -[22.73, 1.379, 1.361], -[29.888, 3.16, 3.207], -[53.922, 2.301, 2.277], -[45.363, 43.765, 44.066], -[172.945, 136.944, 138.268], -[16.764, 2.579, 2.59], -[1.498, 1.434, 1.448], -[113.533, 78.465, 80.778], -[90.456, 87.05, 88.426], -[45.021, 12.486, 12.222], -[44.246, 15.606, 15.251], -[29.654, 29.441, 29.608], -[103.547, 104.733, 105.779], -[29.695, 15.395, 15.819], -[27.841, 29.521, 27.508], -[56.665, 56.459, 56.407], -[27.488, 25.557, 25.634], -[97.376, 96.047, 99.918], -[6.294, 6.407, 6.376], -[40.787, 11.206, 11.219], -[12.42, 12.548, 12.468], -[57.679, 35.466, 35.562], -[13.551, 13.417, 13.645], -[0.15, 0.055, 0.055], -[3.059, 3.038, 3.041], -[4.461, 4.446, 4.424], -[29.275, 17.558, 17.438], -[203.316, 190.037, 189.276], -[36.921, 36.963, 36.66], -[38.307, 17.597, 17.324], -[39.857, 26.776, 26.592], -[162.782, 160.722, 162.487], -[261.494, 263.594, 260.436], -[265.758, 270.087, 266.617], -[30.677, 28.933, 29.815], -[19.754, 16.851, 16.703], -[10.379, 10.267, 10.287], -[17.32, 9.786, 9.76], -[33.487, 26.056, 25.958], -[28.02, 5.609, 5.538], -[15.119, 5.057, 5.063], -[3.627, 3.645, 3.546] - ] - } -] - diff --git a/website/benchmark/dbms/results/015_postgresql.json b/website/benchmark/dbms/results/015_postgresql.json deleted file mode 100644 index 0298d9a54db..00000000000 --- a/website/benchmark/dbms/results/015_postgresql.json +++ /dev/null @@ -1,56 +0,0 @@ -[ - { - "system": "PostgreSQL", - "version": "13", - "data_size": 100000000, - "time": "2021-10-31", - "comments": "", - "result": - [ -[122.02, 5.06, 5.052], -[129.594, 8.079, 7.866], -[129.584, 8.276, 8.153], -[123.707, 6.835, 6.607], -[166.64, 75.401, 73.526], -[272.715, 182.721, 182.88], -[127.108, 6.542, 6.339], -[127.339, 8.376, 7.831], -[179.176, 58.559, 58.139], -[182.019, 58.435, 58.13], -[132.449, 11.203, 11.048], -[128.445, 11.602, 11.418], -[162.831, 41.51, 41.682], -[171.898, 47.379, 47.429], -[161.607, 41.674, 40.854], -[175.247, 46.721, 46.507], -[335.961, 248.535, 247.383], -[132.852, 14.939, 14.607], -[243.461, 157.307, 155.093], -[122.09, 6.411, 6.308], -[126.584, 8.836, 8.532], -[125.225, 10.236, 9.849], -[139.14, 21.797, 21.559], -[124.757, 8.728, 8.714], -[120.687, 8.366, 8.146], -[122.327, 8.698, 8.48], -[123.958, 8.595, 8.241], -[128.982, 11.252, 10.957], -[208.455, 102.53, 102.049], -[131.268, 21.094, 20.934], -[164.084, 77.418, 75.422], -[174.8, 87.859, 85.733], -[419.357, 339.047, 334.808], -[475.011, 344.406, 347.197], -[464.657, 332.084, 330.921], -[152.49, 30.954, 31.379], -[128.539, 12.802, 12.494], -[125.85, 10.318, 9.953], -[126.602, 8.935, 8.711], -[133.222, 11.848, 11.752], -[126.95, 11.26, 10.943], -[128.451, 10.984, 10.77], -[124.621, 8.885, 8.857] - ] - } -] - diff --git a/website/benchmark/dbms/results/016_duckdb.json b/website/benchmark/dbms/results/016_duckdb.json deleted file mode 100644 index 0b30e892c09..00000000000 --- a/website/benchmark/dbms/results/016_duckdb.json +++ /dev/null @@ -1,57 +0,0 @@ -[ - { - "system": "DuckDB", - "version": "v0.3.0", - "data_size": 100000000, - "time": "2021-10-31", - "comments": "", - "result": - [ -[0.189,0.01,0.012], -[0.063,0.639,0.154], -[0.097,0.053,0.049], -[0.095,0.039,0.471], -[10.279,8.105,8.319], -[12.691,12.975,12.888], -[0.119,0.338,0.183], -[0.05,0.03,0.048], -[10.522,9.257,10.04], -[12.087,10.892,10.689], -[2.262,2.245,2.253], -[2.487,2.461,2.528], -[1.772,1.045,1.127], -[20.063,14.072,13.811], -[1.371,1.297,1.17], -[1.016,1.002,0.961], -[3.555,2.324,2.309], -[2.36,1.908,2.013], -[8.635,4.255,4.94], -[0.257,0.258,0.258], -[7.228,2.665,2.626], -[0.829,0.721,0.716], -[7.2,8.537,3.669], -[36.001,72.104,38.169], -[0.255,0.232,0.227], -[0.251,0.248,0.271], -[0.232,0.231,0.231], -[0.443,0.439,0.426], -[121.613,119.865,123.649], -[6.264,6.176,6.374], -[0.996,0.991,0.999], -[2.661,1.552,1.531], -[20.238,55.584,63.046], -[12.508,14.208,7.564], -[4.31,4.586,4.014], -[2.423,4.737,2.404], -[0.065,0.064,0.077], -[0.046,0.045,0.051], -[0.04,0.04,0.045], -[0.126,0.132,0.137], -[0.014,0.014,0.016], -[0.019,0.019,0.022], -[0.012,0.012,0.015] - ] - } -] - - diff --git a/website/benchmark/dbms/results/athena.partitioned.json b/website/benchmark/dbms/results/athena.partitioned.json new file mode 100644 index 00000000000..e77974b51c4 --- /dev/null +++ b/website/benchmark/dbms/results/athena.partitioned.json @@ -0,0 +1,45 @@ +[{"system": "athena (partitioned)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[2.777,3.275,2.925], +[1.503,3.136,4.003], +[4.544,3.833,3.64], +[3.9,2.514,3.522], +[3.46,2.186,3.244], +[3.624,2.742,3.185], +[2.21,1.984,3.123], +[3.207,2.403,2.685], +[2.936,2.014,3.869], +[8.333,7.102,4.434], +[7.401,4.697,3.155], +[4.214,3.065,4.748], +[6.207,4.213,2.576], +[3.428,3.085,3.401], +[2.92,3.3,3.278], +[2.205,2.558,2.419], +[4.641,3.888,2.155], +[3.219,2.822,3.292], +[3.23,3.579,4.31], +[2.288,3.543,3.95], +[3.032,2.859,2.807], +[3.926,3.247,2.928], +[4.477,4.048,4.392], +[7.407,6.375,6.123], +[2.611,2.872,2.827], +[2.566,2.567,3.6], +[3.673,3.733,2.925], +[2.426,3.218,2.78], +[5.125,3.778,4.25], +[4.565,4.03,4.066], +[3.628,3.219,2.953], +[6.207,5.973,3.158], +[4.339,5.601,4.234], +[2.618,3.107,3.433], +[4.661,2.79,2.846], +[2.373,1.629,2.734], +[2.721,2.15,1.962], +[3.207,2.154,2.186], +[2.453,2.477,3.217], +[2.691,4.732,3.584], +[2.589,2.613,3.231], +[1.926,3.617,1.82], +[1.506,2.404,2.343] +]}] diff --git a/website/benchmark/dbms/results/athena.single.json b/website/benchmark/dbms/results/athena.single.json new file mode 100644 index 00000000000..7c004934c80 --- /dev/null +++ b/website/benchmark/dbms/results/athena.single.json @@ -0,0 +1,45 @@ +[{"system": "athena (single)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[2.268,1.327,2.137], +[3.427,2.248,3.605], +[3.254,2.548,2.316], +[3.025,2.314,3.003], +[2.264,2.876,4.213], +[3.044,2.745,2.698], +[2.732,2.199,2.659], +[2.022,3.692,3.072], +[2.746,2.477,2.785], +[3.53,2.782,4.031], +[2.709,2.047,2.853], +[2.318,1.969,3.4], +[2.635,1.935,2.707], +[3.049,3.38,3.071], +[3.661,2.387,2.476], +[2.479,2.591,2.21], +[3.093,3.698,4.351], +[3.479,3.236,2.274], +[4.36,2.97,3.457], +[2.525,2.384,3.328], +[3.34,3.174,3.409], +[3.163,2.971,3.034], +[2.999,3.539,2.906], +[6.454,7.597,7.858], +[2.754,1.951,2.645], +[2.852,3.018,2.718], +[2.513,2.678,2.417], +[3.293,2.521,2.771], +[4.392,3.863,3.981], +[3.658,4.246,4.027], +[3.028,3.87,2.337], +[2.923,3.635,3.591], +[3.142,4.105,3.15], +[3.66,3.187,4.745], +[2.652,2.695,2.742], +[2.262,2.776,1.815], +[1.881,2.212,2.053], +[1.934,2.551,1.524], +[2.069,2.26,1.805], +[2.626,2.902,2.793], +[1.791,2.082,2.481], +[3.757,2.6,1.946], +[2.608,1.994,3.967] +]}] diff --git a/website/benchmark/dbms/results/aurora-mysql.16acu.json b/website/benchmark/dbms/results/aurora-mysql.16acu.json new file mode 100644 index 00000000000..d0e35ffbc1a --- /dev/null +++ b/website/benchmark/dbms/results/aurora-mysql.16acu.json @@ -0,0 +1,45 @@ +[{"system": "aurora-mysql (16acu)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[740.42,739.91,746.65], +[828.2,835.67,832.87], +[830.08,830.98,832.38], +[829.88,832.83,830.87], +[845.99,842.4,843.21], +[869.51,870.69,869.75], +[823.77,829.08,825.54], +[827.74,832.87,829.25], +[916.26,909.46,929.17], +[946.49,939.27,932.32], +[852.37,857.69,854.74], +[857.99,864.05,825.14], +[null,null,null], +[863.37,860.2,865.62], +[null,null,null], +[891.84,895.28,893.68], +[null,null,null], +[null,null,null], +[1420.12,1419.34,1445.08], +[28.94,0.21,0.21], +[917.64,917.56,916.92], +[923.47,921.7,923.82], +[919.95,918.37,920.17], +[1002.19,1002.07,1001.2], +[902.23,902.65,901.8], +[901.17,900.02,898.3], +[900.04,898.89,903.35], +[901.78,902.71,901.28], +[null,null,null], +[1153.29,1154,1156.46], +[862.57,863.35,859.69], +[923.14,921.1,923.92], +[1370.78,1401.72,1401.44], +[1454.67,1455.55,1458.79], +[1463.31,1466.75,1461.83], +[941.03,944.07,937.23], +[7.42,2.80,2.77], +[2.57,2.52,2.59], +[1.50,1.52,1.59], +[3.62,3.57,3.61], +[0.95,0.94,0.94], +[0.90,0.92,0.91], +[1.69,1.72,1.69] +]}] diff --git a/website/benchmark/dbms/results/aurora-postgresql.16acu.json b/website/benchmark/dbms/results/aurora-postgresql.16acu.json new file mode 100644 index 00000000000..af9fca3d4f1 --- /dev/null +++ b/website/benchmark/dbms/results/aurora-postgresql.16acu.json @@ -0,0 +1,45 @@ +[{"system": "aurora-postgresql (16acu)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[12.8361,5.71812,5.8241], +[61.2565,62.1402,63.7173], +[68.0578,68.1218,67.609], +[7.83207,5.90193,6.0461], +[48.7194,48.0233,48.2198], +[289.492,304.639,282.436], +[6.30572,6.31857,6.21598], +[53.644,53.8931,53.5307], +[131.526,131.45,131.102], +[137.724,136.921,137.758], +[57.2079,56.2775,56.2152], +[56.5349,56.2048,55.9569], +[82.3897,82.8866,83.534], +[97.0569,97.1392,96.4731], +[85.6557,86.7783,86.2804], +[49.4325,42.4309,42.5743], +[111.537,114.59,111.807], +[88.4322,89.3756,87.7899], +[160.781,163.866,161.394], +[1025.04,2.10165,2.10065], +[106.741,56.2731,56.1535], +[59.2681,59.5272,59.536], +[58.6083,57.6054,57.3935], +[54.8271,55.1397,56.3487], +[54.718,52.469,53.271], +[53.5387,53.1926,52.4008], +[52.0042,51.9581,52.2453], +[60.1317,59.9695,59.2187], +[244.608,242.954,243.815], +[91.8674,92.4165,91.5884], +[63.7122,64.277,64.2783], +[69.2596,68.9535,69.4508], +[234.222,241.138,240.316], +[488.169,462.257,460.466], +[472.929,471.809,476.635], +[103.664,116.131,103.467], +[16.8124,3.34058,3.37782], +[0.852414,0.832073,0.859857], +[0.305464,0.31166,0.306694], +[4.55625,4.54098,4.58501], +[0.299746,0.297532,0.30334], +[0.275732,0.279817,0.27766], +[0.332107,0.324387,0.320099] +]}] diff --git a/website/benchmark/dbms/results/bigquery.serverless.json b/website/benchmark/dbms/results/bigquery.serverless.json new file mode 100644 index 00000000000..ef691fba636 --- /dev/null +++ b/website/benchmark/dbms/results/bigquery.serverless.json @@ -0,0 +1,45 @@ +[{"system": "bigquery (serverless)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[1.451,1.284,1.272], +[1.462,1.23,1.318], +[1.394,1.338,1.248], +[1.248,1.41,1.377], +[1.506,1.661,1.538], +[1.581,1.709,1.639], +[1.39,1.195,1.274], +[1.431,1.362,1.378], +[2.012,1.92,3.148], +[3.104,3.118,3.111], +[1.342,1.355,1.489], +[1.374,1.443,1.349], +[1.659,1.707,1.649], +[3.086,3.123,4.189], +[1.684,1.98,1.823], +[1.899,1.654,1.704], +[3.097,2.872,3.09], +[3.193,3.091,3.073], +[5.162,4.125,4.121], +[1.362,1.183,1.171], +[1.508,1.624,1.578], +[1.537,1.554,1.484], +[1.931,1.977,2.024], +[3.135,3.141,3.182], +[1.494,1.377,1.321], +[1.342,1.49,1.348], +[1.397,1.377,1.482], +[3.084,3.115,3.11], +[4.144,3.41,4.131], +[3.086,3.129,3.094], +[1.887,1.861,1.783], +[2.068,1.837,1.938], +[4.126,4.14,4.173], +[3.103,3.106,3.117], +[4.149,4.136,4.172], +[3.111,3.127,1.988], +[1.539,1.443,1.412], +[1.213,1.202,1.252], +[1.16,1.213,1.174], +[1.723,1.747,1.764], +[1.173,1.122,1.124], +[1.164,1.162,1.098], +[1.292,1.226,1.274] +]}] diff --git a/website/benchmark/dbms/results/citus.c6a.4xlarge.json b/website/benchmark/dbms/results/citus.c6a.4xlarge.json new file mode 100644 index 00000000000..0197ba4eb74 --- /dev/null +++ b/website/benchmark/dbms/results/citus.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "citus (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[7.58503,6.70447,6.52499], +[6.33941,5.06063,5.00238], +[11.7488,9.86417,9.93223], +[12.6306,9.36305,9.17061], +[40.6101,39.0803,38.1187], +[117.654,113.912,113.441], +[10.3404,8.08936,7.70732], +[6.31542,4.72821,4.72989], +[82.5425,77.2124,76.9219], +[91.1776,83.4492,82.4727], +[14.5474,10.0815,10.3873], +[15.4899,11.2922,11.1877], +[19.9794,15.5002,17.4492], +[76.9216,72.5172,72.7915], +[21.5446,17.5691,18.561], +[56.9438,54.6387,53.5745], +[75.0977,69.7842,70.0259], +[31.3299,27.0267,26.3216], +[129.417,122.956,121.182], +[3.73386,2.14148,2.12737], +[34.6021,27.9727,28.6878], +[37.152,29.6193,29.2966], +[52.2157,37.8589,37.6994], +[181.955,149.08,148.471], +[15.4687,11.3138,10.3856], +[10.2779,8.46868,8.8324], +[14.4687,10.4076,11.4263], +[47.009,40.2969,39.6888], +[749.946,742.979,744.461], +[69.4383,67.5636,67.2128], +[27.0317,21.4008,20.9524], +[36.6675,25.6347,26.4408], +[140.424,130.546,129.738], +[106.959,92.033,90.1609], +[110.98,94.4787,96.2656], +[64.4474,60.1853,60.6816], +[6.17549,6.25376,5.87004], +[1.99153,1.81776,1.80596], +[1.00141,0.800271,0.801975], +[7.91778,7.70928,8.33299], +[0.929845,0.642076,0.638478], +[0.866536,0.683567,0.680218], +[0.937823,0.784747,0.765929] +]}] diff --git a/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.partitioned.json b/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.partitioned.json new file mode 100644 index 00000000000..604192f26bf --- /dev/null +++ b/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.partitioned.json @@ -0,0 +1,45 @@ +[{"system": "clickhouse-local (c6a.4xlarge.partitioned)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.850, 0.102, 0.114], +[1.342, 0.090, 0.099], +[2.547, 0.179, 0.189], +[1.681, 0.245, 0.252], +[2.704, 1.680, 1.648], +[2.194, 1.460, 1.489], +[0.832, 0.113, 0.102], +[1.371, 0.106, 0.101], +[2.240, 0.790, 0.825], +[4.548, 1.021, 1.026], +[3.094, 0.552, 0.552], +[3.088, 0.623, 0.630], +[2.017, 1.170, 1.165], +[4.319, 1.677, 1.708], +[2.157, 1.496, 1.500], +[1.629, 1.138, 1.139], +[5.026, 3.267, 3.241], +[4.142, 2.303, 2.319], +[8.295, 5.569, 5.629], +[1.331, 0.255, 0.252], +[10.712, 3.668, 3.786], +[13.053, 4.185, 4.202], +[24.170, 7.935, 8.008], +[55.965, 23.933, 23.071], +[4.417, 0.947, 0.974], +[1.793, 0.698, 0.690], +[4.376, 0.955, 0.956], +[11.731, 4.385, 4.321], +[11.403, 8.549, 8.288], +[2.764, 2.754, 2.735], +[5.096, 1.262, 1.273], +[9.515, 1.682, 1.688], +[10.325, 6.745, 6.608], +[11.686, 6.261, 6.242], +[11.769, 6.301, 6.364], +[1.675, 1.490, 1.495], +[14.937, 3.631, 3.604], +[14.187, 3.609, 3.631], +[14.842, 3.769, 3.741], +[22.222, 6.355, 6.263], +[7.212, 0.836, 0.838], +[7.863, 0.716, 0.718], +[5.120, 0.587, 0.574] +]}] diff --git a/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.single.json b/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.single.json new file mode 100644 index 00000000000..e47a265585d --- /dev/null +++ b/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.single.json @@ -0,0 +1,45 @@ +[{"system": "clickhouse-local (c6a.4xlarge.single)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[1.176, 0.251, 0.249], +[1.037, 0.134, 0.111], +[1.609, 0.403, 0.369], +[1.616, 0.372, 0.370], +[3.008, 2.338, 2.266], +[9.061, 7.537, 7.535], +[1.206, 0.191, 0.187], +[0.882, 0.144, 0.135], +[4.610, 3.406, 3.256], +[6.712, 4.479, 4.469], +[4.081, 2.413, 2.394], +[3.894, 2.719, 2.691], +[7.651, 6.436, 6.243], +[10.765, 8.043, 7.894], +[9.860, 8.945, 8.235], +[7.159, 5.815, 5.814], +[20.916, 18.159, 18.013], +[20.952, 17.862, 17.850], +[37.585, 32.649, 32.487], +[1.767, 0.401, 0.393], +[23.713, 15.687, 15.755], +[28.700, 19.241, 19.198], +[50.740, 33.161, 33.011], +[152.485, 117.417, 118.178], +[7.606, 4.491, 5.326], +[4.331, 4.214, 3.587], +[6.743, 4.486, 5.357], +[22.910, 15.043, 15.183], +[43.342, 37.167, 36.842], +[11.807, 4.490, 4.546], +[9.557, 6.349, 6.263], +[13.964, 8.493, 8.464], +[38.110, 33.642, 33.996], +[41.266, 35.080, 27.073], +[34.056, 26.814, 26.902], +[8.855, 7.548, 7.475], +[22.596, 12.615, 12.669], +[23.217, 13.956, 13.831], +[22.528, 21.601, 13.207], +[37.890, 23.115, 22.955], +[6.490, 1.548, 1.522], +[6.413, 1.474, 1.416], +[3.796, 1.339, 1.316] +]}] diff --git a/website/benchmark/dbms/results/clickhouse.c6a.4xlarge.json b/website/benchmark/dbms/results/clickhouse.c6a.4xlarge.json new file mode 100644 index 00000000000..54f3eef41be --- /dev/null +++ b/website/benchmark/dbms/results/clickhouse.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "clickhouse (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.027, 0.001, 0.001], +[0.035, 0.015, 0.021], +[0.083, 0.034, 0.033], +[0.171, 0.044, 0.045], +[1.552, 1.495, 1.574], +[1.270, 1.075, 1.063], +[0.045, 0.026, 0.025], +[0.032, 0.016, 0.015], +[0.717, 0.615, 0.607], +[0.843, 0.821, 0.747], +[0.293, 0.219, 0.216], +[0.312, 0.226, 0.235], +[0.804, 0.694, 0.702], +[1.476, 1.047, 1.029], +[1.013, 0.898, 0.911], +[1.043, 0.964, 1.453], +[3.632, 2.715, 2.711], +[1.867, 1.750, 1.714], +[5.187, 4.797, 4.953], +[0.112, 0.068, 0.041], +[8.637, 1.761, 1.212], +[9.902, 0.902, 0.869], +[18.831, 2.067, 1.829], +[41.903, 4.476, 3.486], +[1.801, 0.254, 0.238], +[0.627, 0.214, 0.207], +[2.181, 0.241, 0.246], +[8.868, 0.748, 0.733], +[9.674, 6.891, 5.770], +[2.620, 2.355, 2.368], +[1.395, 0.533, 0.525], +[4.454, 0.730, 0.712], +[5.453, 4.990, 5.922], +[9.955, 3.968, 4.096], +[9.987, 4.035, 4.476], +[1.695, 1.236, 1.241], +[0.142, 0.079, 0.103], +[0.066, 0.033, 0.040], +[0.065, 0.030, 0.033], +[0.246, 0.207, 0.192], +[0.044, 0.019, 0.020], +[0.030, 0.023, 0.012], +[0.030, 0.018, 0.013] +]}] diff --git a/website/benchmark/dbms/results/clickhouse.c6a.metal.json b/website/benchmark/dbms/results/clickhouse.c6a.metal.json new file mode 100644 index 00000000000..e246b7df816 --- /dev/null +++ b/website/benchmark/dbms/results/clickhouse.c6a.metal.json @@ -0,0 +1,45 @@ +[{"system": "clickhouse (c6a.metal)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.011, 0.001, 0.001], +[0.040, 0.015, 0.013], +[0.045, 0.021, 0.023], +[0.090, 0.023, 0.023], +[1.922, 1.565, 1.576], +[0.961, 0.737, 0.739], +[0.040, 0.023, 0.018], +[0.032, 0.028, 0.028], +[0.321, 0.287, 0.275], +[0.632, 0.284, 0.287], +[0.166, 0.124, 0.118], +[0.235, 0.100, 0.102], +[1.006, 0.182, 0.159], +[1.637, 0.216, 0.213], +[0.871, 0.174, 0.177], +[0.258, 0.148, 0.148], +[1.804, 0.370, 0.358], +[1.235, 0.275, 0.278], +[3.143, 0.854, 0.815], +[0.071, 0.024, 0.016], +[8.816, 0.215, 0.155], +[10.239, 0.203, 0.173], +[19.179, 0.388, 0.357], +[43.152, 0.824, 0.823], +[1.821, 0.059, 0.052], +[0.992, 0.045, 0.051], +[2.539, 0.063, 0.058], +[9.258, 0.300, 0.278], +[7.923, 0.961, 0.936], +[0.445, 0.431, 0.428], +[1.367, 0.131, 0.113], +[4.819, 0.205, 0.175], +[3.808, 0.739, 0.726], +[8.935, 0.607, 0.600], +[8.988, 0.634, 0.615], +[0.242, 0.220, 0.226], +[0.075, 0.058, 0.056], +[0.038, 0.028, 0.026], +[0.043, 0.028, 0.021], +[0.172, 0.127, 0.119], +[0.028, 0.018, 0.017], +[0.027, 0.019, 0.014], +[0.018, 0.026, 0.015] +]}] diff --git a/website/benchmark/dbms/results/cratedb.c6a.4xlarge.json b/website/benchmark/dbms/results/cratedb.c6a.4xlarge.json new file mode 100644 index 00000000000..4a7bebcf51d --- /dev/null +++ b/website/benchmark/dbms/results/cratedb.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "cratedb (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.008162,0.005118,0.002553], +[0.350014,0.39977,0.133775], +[2.58426,2.47192,2.59779], +[2.12939,0.532981,0.507246], +[null,null,null], +[null,null,null], +[1.18488,1.06603,1.07219], +[0.209264,0.073284,0.067912], +[null,null,null], +[null,null,null], +[1.68892,1.2866,1.47428], +[1.62976,1.43073,1.26904], +[12.7517,13.0334,13.2685], +[18.8587,null,18.6951], +[11.2982,11.2108,11.577], +[20.2964,20.4035,19.1076], +[null,null,null], +[null,null,null], +[null,null,null], +[0.202044,0.010009,0.005566], +[9.22964,4.54606,0.774149], +[1.41673,1.09885,0.789775], +[12.3933,8.06911,1.69671], +[1.45018,0.969528,0.979718], +[0.357589,0.14887,0.153326], +[0.189282,0.133963,0.130279], +[0.153222,0.140756,0.139861], +[27.5195,19.6862,20.1825], +[72.7575,68.2,67.1238], +[144.533,146.579,152.144], +[8.76866,9.00563,8.46917], +[17.6652,16.6755,16.0558], +[null,null,null], +[null,null,null], +[null,null,null], +[42.2967,44.9621,44.4386], +[0.786911,0.4904,0.508416], +[0.602075,0.226261,0.182399], +[0.131407,0.058958,0.054518], +[0.954736,1.1361,1.14233], +[0.23764,0.139109,0.134472], +[0.110253,0.057695,0.056073], +[0.124285,0.150479,0.066226] +]}] diff --git a/website/benchmark/dbms/results/databend.c6a.4xlarge.json b/website/benchmark/dbms/results/databend.c6a.4xlarge.json new file mode 100644 index 00000000000..3904d77a03c --- /dev/null +++ b/website/benchmark/dbms/results/databend.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "databend (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.010087, 0.002961, 0.003271], +[0.127964, 0.080012, 0.075741], +[0.162388, 0.143967, 0.144762], +[0.252904, 0.217471, 0.217369], +[34.281026, 34.844158, 34.526942], +[25.290307, 25.793068, 25.620563], +[0.112484, 0.093867, 0.090891], +[0.086604, 0.07796, 0.076448], +[20.723203, 20.7483, 20.354869], +[20.81994, 20.72446, 20.696573], +[1.964378, 1.93559, 1.893824], +[1.846866, 1.789111, 1.763664], +[4.468158, 4.407959, 4.438036], +[19.947276, 19.8859, 19.853514], +[5.478573, 5.474461, 5.460604], +[5.509521, 5.513413, 5.363123], +[15.430359, 15.5406, 15.461211], +[14.905998, 15.029721, 15.019642], +[31.069663, 30.811763, 30.737336], +[0.281067, 0.220021, 0.217741], +[8.89374, 4.12692, 4.131689], +[10.38448, 4.603694, 4.571757], +[19.980572, 8.836322, 8.892694], +[59.786474, 52.452881, 39.941988], +[2.804019, 0.994794, 0.958224], +[0.765299, 0.730434, 0.723964], +[2.784648, 0.94665, 0.936684], +[8.905027, 5.418438, 5.386109], +[12.187652, 12.230066, 12.164123], +[3.35748, 3.395991, 3.319434], +[4.309389, 3.854977, 3.772506], +[9.958201, 7.027432, 6.888253], +[50.200569, 50.535126, 50.283066], +[24.469412, 21.222713, 21.010188], +[26.115852, 23.93507, 24.835342], +[7.511517, 7.296179, 7.324549], +[2.156784, 1.298258, 1.278441], +[2.155447, 1.314499, 1.331237], +[2.007053, 1.181676, 1.155612], +[null, null, null], +[0.485363, 0.420291, 0.416819], +[0.372131, 0.322068, 0.323578], +[null, null, null] +]}] diff --git a/website/benchmark/dbms/results/duckdb.c6a.4xlarge.json b/website/benchmark/dbms/results/duckdb.c6a.4xlarge.json new file mode 100644 index 00000000000..0b0d67edac7 --- /dev/null +++ b/website/benchmark/dbms/results/duckdb.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "duckdb (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.005694353996659629,0.003944558004150167,0.003837226002360694], +[0.16991353100456763,0.03919722700084094,0.03835860399703961], +[0.44898432699847035,0.04947217500011902,0.04852217998995911], +[0.07586832098604646,0.07051395199960098,0.07007493599667214], +[9.554053236002801,8.153356187991449,8.73448242500308], +[7.66042533799191,6.931124911992811,7.103380946995458], +[0.030703739990713075,0.027668555994750932,0.027583695002249442], +[0.1778664360026596,0.03942437999648973,0.03882004099432379], +[8.53439180701389,8.869582625004114,9.020313234999776], +[10.40215514000738,11.125320470004226,8.941559945000336], +[1.1747649609897053,1.04221136700653,1.004799570000614], +[1.2380354650085792,1.1211603130068397,2.4278587239969056], +[3.1751541379926493,0.9360461989999749,0.8868292279948946], +[6.855684430003748,7.300301584007684,5.712960822012974], +[3.70588762400439,1.0249276379909134,0.9473389159975341], +[2.1037107890006155,1.6215517020027619,1.5671920729946578], +[null,null,null], +[null,null,null], +[null,null,null], +[0.0002772739971987903,0.00016792300448287278,0.0001574420020915568], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[2.9310110910009826,0.19020285899750888,0.1736805049877148], +[2.939304119994631,0.18754731099761557,0.18073286200524308], +[2.8706370779982535,0.18822155400994234,0.17905898999015335], +[null,null,null], +[null,null,null], +[0.884408778991201,0.714329167996766,0.7135983259940986], +[5.3762675570033025,0.8803737630078103,0.8728962720051641], +[7.249190265996731,2.9648747390019707,2.866687831003219], +[null,null,null], +[null,null,null], +[null,null,null], +[4.515183198003797,4.030519469000865,4.014251719010645], +[0.11604027298744768,0.040539135996368714,0.04280066800129134], +[0.0457908230018802,0.021069509006338194,0.019683108999743126], +[0.0680370800109813,0.011889394998434,0.01056639499438461], +[0.22029169600864407,0.08547276000899728,0.09095505000732373], +[0.03759863799496088,0.008373684002435766,0.007633563989656977], +[0.025631797994719818,0.008081699008471332,0.007858585988287814], +[0.034359957004198804,0.025543516996549442,0.02533275399764534] +]}] diff --git a/website/benchmark/dbms/results/greenplum.c6a.4xlarge.json b/website/benchmark/dbms/results/greenplum.c6a.4xlarge.json new file mode 100644 index 00000000000..67db7aedd6c --- /dev/null +++ b/website/benchmark/dbms/results/greenplum.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "greenplum (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[2.20009,1.45883,1.13573], +[1.28661,1.07897,1.01856], +[2.2582,2.42193,2.59693], +[1.89371,1.9246,1.73866], +[9.54471,9.71956,10.0357], +[4.55957,4.10112,4.70458], +[1.65167,1.89767,1.97051], +[1.02537,0.999323,1.00524], +[7.89792,7.75718,8.01219], +[11.3779,11.6003,11.2552], +[2.11955,2.05003,1.9553], +[2.01997,1.97932,1.98284], +[2.88666,2.92828,2.96822], +[4.06272,4.01771,4.03011], +[3.46303,3.24779,3.06475], +[8.43717,8.18641,7.99202], +[9.02176,8.91572,8.92748], +[8.66982,8.39332,8.62497], +[15.1321,14.9502,15.2344], +[1.00877,0.934589,0.975342], +[11.0582,3.25807,3.17687], +[12.2634,3.5188,3.93705], +[24.1522,4.86328,4.88764], +[67.1517,20.4,20.261], +[2.5799,1.66114,1.90432], +[1.72831,1.89536,1.2515], +[2.44154,1.46595,1.88238], +[11.1175,6.09551,6.02292], +[83.041,82.9784,82.8678], +[82.5441,82.5427,83.9836], +[3.0962,3.22225,3.19431], +[5.77576,4.05973,4.00366], +[43.6826,40.4375,40.1076], +[18.0978,13.2741,12.8894], +[17.3187,13.5411,13.5927], +[9.72405,9.90029,10.0177], +[0.294344,0.169606,0.173804], +[0.162524,0.117489,0.115532], +[0.145205,0.098342,0.097275], +[0.334809,0.275365,0.265053], +[0.154522,0.107654,0.10529], +[0.158957,0.117284,0.119068], +[0.193756,0.144787,0.145485] +]}] diff --git a/website/benchmark/dbms/results/heavyai.c6a.4xlarge.json b/website/benchmark/dbms/results/heavyai.c6a.4xlarge.json new file mode 100644 index 00000000000..4f22107b3c9 --- /dev/null +++ b/website/benchmark/dbms/results/heavyai.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "heavyai (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[6.525,0.022,0.029], +[0.301,0.042,0.04], +[0.287,0.095,0.093], +[2.572,0.039,0.04], +[null,null,null], +[null,null,null], +[7.327,0.093,0.097], +[0.244,0.043,0.038], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[2.939,0.295,0.294], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[4.716,3.91,3.955], +[null,null,null], +[0.154,0.083,0.106], +[14.426,0.07,0.071], +[null,null,null], +[null,null,null], +[null,null,null], +[2.276,0.258,0.272], +[null,null,null], +[null,null,null], +[null,null,null], +[null,null,null], +[1.832,1.64,1.602], +[null,null,null], +[null,null,null], +[null,null,null], +[14.811,0.494,0.497], +[null,null,null], +[null,null,null], +[1.941,0.255,0.255], +[5.457,0.172,0.283], +[0.476,0.269,0.256], +[14.239,0.179,0.178], +[3.992,0.112,0.112], +[1.031,0.116,0.116], +[1.365,0.089,0.088] +]}] diff --git a/website/benchmark/dbms/results/mariadb-columnstore.c6a.4xlarge.json b/website/benchmark/dbms/results/mariadb-columnstore.c6a.4xlarge.json new file mode 100644 index 00000000000..0a2d10f98f2 --- /dev/null +++ b/website/benchmark/dbms/results/mariadb-columnstore.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "mariadb-columnstore (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.151,0.158,0.148], +[0.100,0.101,0.106], +[1.221,1.233,1.226], +[0.739,0.736,0.741], +[2.025,2.046,2.004], +[3.725,4.801,3.755], +[0.871,0.749,0.736], +[0.118,0.108,0.103], +[2.108,2.029,2.029], +[4.225,4.271,4.288], +[1.711,1.402,1.407], +[1.526,1.435,1.420], +[5.339,4.172,3.610], +[4.692,4.729,4.960], +[4.013,3.860,3.918], +[3.236,2.680,2.629], +[null,null,null], +[null,null,null], +[null,null,null], +[0.137,0.109,0.147], +[12.331,6.069,8.619], +[2.162,2.178,2.192], +[16.849,30.463,26.639], +[92,90.208,92.814], +[3.042,1.763,1.791], +[1.779,1.772,1.749], +[1.793,1.821,1.888], +[13.036,10.747,9.590], +[null,null,null], +[78.224,79.141,77.806], +[2.837,2.654,2.675], +[5.833,4.552,3.678], +[null,null,null], +[null,null,null], +[null,null,null], +[3.626,3.546,3.709], +[1.719,1.787,1.876], +[1.345,0.906,0.910], +[0.117,0.091,0.093], +[1.217,1.133,1.133], +[0.114,0.063,0.062], +[0.100,0.062,0.061], +[null,null,null] +]}] diff --git a/website/benchmark/dbms/results/monetdb.c6a.4xlarge.json b/website/benchmark/dbms/results/monetdb.c6a.4xlarge.json new file mode 100644 index 00000000000..73b1113bd4d --- /dev/null +++ b/website/benchmark/dbms/results/monetdb.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "monetdb (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.000218,0.000157,0.000155], +[0.101903,0.019908,0.018439], +[0.282431,0.035987,0.034938], +[2.868,0.029387,0.029207], +[4.675,4.515,4.511], +[6.584,4.269,4.650], +[0.528827,0.063135,0.065742], +[0.506863,0.020966,0.021687], +[8.343,4.457,4.408], +[7.224,6.548,7.576], +[0.267003,0.233353,0.230444], +[0.347206,0.28358,0.266085], +[5.389,3.099,3.074], +[7.653,7.759,8.596], +[3.276,3.326,3.292], +[5.310,3.465,3.578], +[9.341,9.143,9.536], +[9.584,9.604,9.419], +[19.539,19.783,19.611], +[0.004509,0.000702,0.000643], +[20.801,1.570,1.603], +[2.752,0.418221,0.395884], +[14.717,0.800894,0.395477], +[14.429,1.804,1.869], +[1.386,0.159602,0.156426], +[0.189736,0.167664,0.168781], +[0.164681,0.176666,0.17126], +[3.005,3.113,3.882], +[null,null,null], +[2.751,2.846,2.676], +[7.937,2.579,2.447], +[5.120,3.492,3.467], +[22.862,22.567,23.211], +[33.437,18.889,19.043], +[18.898,19.583,19.047], +[14.774,12.984,13.803], +[3.865,0.322143,0.323117], +[0.192149,0.177791,0.175984], +[0.194173,0.159398,0.165201], +[0.680778,0.592252,0.560738], +[0.106465,0.10638,0.102692], +[0.154871,0.153752,0.155782], +[0.11459,0.09639,0.095594] +]}] diff --git a/website/benchmark/dbms/results/mysql-myisam.c6a.4xlarge.json b/website/benchmark/dbms/results/mysql-myisam.c6a.4xlarge.json new file mode 100644 index 00000000000..88364415e90 --- /dev/null +++ b/website/benchmark/dbms/results/mysql-myisam.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "mysql-myisam (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.00,0.00,0.00], +[283.32,276.83,274.52], +[276.93,278.29,283.27], +[28.83,23.63,21.55], +[46.41,40.81,40.93], +[467.04,467.39,469.08], +[31.02,25.89,24.20], +[277.89,275.3,277.3], +[329.34,325.8,325.35], +[342.86,338.43,336.95], +[282.03,279.87,281.22], +[277.74,282.68,282], +[335.66,334.83,336.44], +[305.24,310.39,307.3], +[337.41,338.52,342.94], +[308.66,307.34,306.27], +[738.38,748.44,740.75], +[738.75,734.01,738.25], +[867.01,872.92,868.84], +[25.65,20.61,18.46], +[312.39,313.67,306.66], +[301.66,305.12,308.01], +[298.12,298.44,312.4], +[311.34,309.9,311.85], +[281.87,278.5,275], +[277.46,277.46,277.46], +[280.75,278.04,281.76], +[263.9,417.39,406.88], +[707.21,711.96,705], +[668.1,668.33,665.96], +[330.31,333.36,331.94], +[506.57,506.18,500.53], +[2604.49,2681.96,2703.12], +[830.65,832.88,831.14], +[831.98,830.46,833.41], +[608.49,608.51,613.68], +[4.56,4.13,4.16], +[3.80,3.80,3.70], +[1.65,1.45,1.46], +[6.33,5.14,6.15], +[1.60,1.41,1.41], +[1.56,1.42,1.39], +[7.04,1.17,1.13] +]}] diff --git a/website/benchmark/dbms/results/mysql.c6a.4xlarge.json b/website/benchmark/dbms/results/mysql.c6a.4xlarge.json new file mode 100644 index 00000000000..b8619c12848 --- /dev/null +++ b/website/benchmark/dbms/results/mysql.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "mysql (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[339.77,339.88,339.77], +[364.91,371.86,367.55], +[366.2,368.91,389.66], +[364.39,377.53,571.45], +[377.69,390.02,384.86], +[569.48,576.51,574.68], +[367.4,368.23,370.41], +[371.29,384.02,613.22], +[478.85,683.22,495.68], +[489.9,635.96,662.43], +[386.07,396.49,640.15], +[389.13,412.55,444.12], +[447.97,455.54,448.06], +[423.22,845.44,813.6], +[452.48,460.07,453.98], +[577.54,623.21,586.49], +[852.07,856.36,862.66], +[838.09,848.92,851.12], +[1006.37,1011.16,1023.17], +[369.76,375.61,415.28], +[412.45,419.9,456.62], +[411.65,432.88,482.2], +[412.73,420.73,429.5], +[551.16,577.62,545.45], +[382.89,394.76,386.37], +[380.9,391.4,385.05], +[385.3,394.67,460.32], +[388.95,394.7,387.21], +[800.33,807.90,807.11], +[706.03,745.27,718.9], +[450.9,489.59,530.97], +[625.5,651.93,647.32], +[2721.13,2792.12,2819.26], +[945.9,954.94,957.54], +[945.42,953.78,965.16], +[684.36,716.29,708.75], +[10.01,3.79,3.77], +[7.48,3.32,3.27], +[5.09,0.98,0.96], +[8.70,4.77,4.68], +[4.82,0.76,0.74], +[4.46,0.77,0.75], +[7.04,1.17,1.13] +]}] diff --git a/website/benchmark/dbms/results/postgresql.c6a.4xlarge.json b/website/benchmark/dbms/results/postgresql.c6a.4xlarge.json new file mode 100644 index 00000000000..a1d17e20065 --- /dev/null +++ b/website/benchmark/dbms/results/postgresql.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "postgresql (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[439.753,309.785,282.017], +[317.874,254.238,254.941], +[262.883,263.072,263.090], +[32.421,5.310,5.060], +[57.134,42.648,42.334], +[358.423,356.315,358.342], +[31.524,5.350,4.994], +[263.145,263.193,263.165], +[323.659,322.858,321.918], +[327.395,326.170,326.231], +[265.983,265.681,265.912], +[269.984,265.336,265.379], +[284.096,284.560,282.234], +[277.250,279.455,280.035], +[285.660,286.200,283.611], +[66.605,32.023,38.282], +[312.452,304.431,305.391], +[289.209,290.449,287.578], +[331.706,327.485,334.428], +[24.646,2.543,2.263], +[267.561,267.496,267.524], +[267.729,267.690,268.184], +[263.074,263.120,267.040], +[267.602,267.488,267.494], +[263.141,263.859,263.137], +[262.923,263.102,263.113], +[262.885,263.088,263.114], +[267.864,269.127,268.204], +[303.376,306.925,308.664], +[263.221,263.119,263.148], +[270.814,270.575,270.294], +[278.342,275.925,276.224], +[584.599,576.932,591.502], +[462.576,446.962,439.779], +[429.930,417.696,416.704], +[296.875,297.283,295.140], +[3.461,0.842,0.794], +[2.179,0.564,0.558], +[2.258,0.566,0.416], +[2.805,1.311,1.317], +[2.936,0.820,0.615], +[2.197,0.736,0.535], +[1.983,0.320,0.312] +]}] diff --git a/website/benchmark/dbms/results/questdb.c6a.4xlarge.json b/website/benchmark/dbms/results/questdb.c6a.4xlarge.json new file mode 100644 index 00000000000..4bbed6f1108 --- /dev/null +++ b/website/benchmark/dbms/results/questdb.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "questdb (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.0155878,0.000283657,0.000328587], +[0.586734,0.0486348,0.0485346], +[6.41881,5.57954,6.25638], +[6.8554,3.11997,3.11813], +[47.8723,54.5425,53.1198], +[null,null,null], +[5.63439,3.70619,3.92043], +[0.699764,0.130462,0.126272], +[55.7179,62.8433,62.0045], +[null,59.6416,64.7753], +[63.7134,null,39.8928], +[46.759,46.8544,null], +[16.7181,10.9064,10.0735], +[79.9885,null,52.5617], +[16.1015,12.1835,13.1469], +[17.0225,11.7342,10.6194], +[24.1167,15.6749,16.8193], +[23.0848,15.3149,17.375], +[42.0965,26.2421,25.7593], +[0.0356335,0.027459,0.0289404], +[70.7679,71.5825,71.4573], +[null,null,null], +[null,null,null], +[77.0625,13.2257,12.7578], +[11.0775,2.90421,2.33398], +[3.31611,3.38837,3.35419], +[3.13233,5.2785,3.07075], +[null,null,null], +[null,null,null], +[186.032,185.627,180.963], +[26.7279,16.4799,18.8758], +[63.8785,32.1097,32.1561], +[64.4635,31.4538,35.654], +[118.897,null,119.015], +[null,null,null], +[19.6853,17.4427,16.7998], +[1.50985,0.877967,0.885536], +[0.805639,0.577352,0.534731], +[0.509284,0.448942,0.467679], +[1.37609,0.966942,0.912858], +[0.567887,0.394619,0.438952], +[0.508977,0.441015,0.40528], +[null,null,null] +]}] diff --git a/website/benchmark/dbms/results/redshift-serverless.serverless.json b/website/benchmark/dbms/results/redshift-serverless.serverless.json new file mode 100644 index 00000000000..cd47d4a8c84 --- /dev/null +++ b/website/benchmark/dbms/results/redshift-serverless.serverless.json @@ -0,0 +1,45 @@ +[{"system": "redshift-serverless (serverless)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.436955,0.390225,0.387823], +[2.74529,0.280571,0.198074], +[7.24569,0.295612,0.386568], +[null,null,null], +[3.98825,0.488446,0.325645], +[4.24601,0.538891,0.690124], +[7.25492,0.288505,0.207515], +[5.99594,0.212732,0.281278], +[6.17534,0.681868,0.586073], +[6.37779,1.37865,1.31614], +[6.41405,0.549735,0.421345], +[6.64688,0.445678,0.477323], +[4.2294,0.361772,0.520471], +[4.73701,0.788258,0.743465], +[4.49038,0.670446,0.52727], +[6.19886,0.663381,0.583588], +[5.9867,0.679607,0.672772], +[2.76661,0.567555,0.51494], +[6.20219,0.816422,0.760568], +[2.51526,0.053,0.025066], +[2.96003,0.282737,0.226979], +[6.79648,0.57495,0.400798], +[6.37854,0.92746,1.05793], +[25.8462,0.902664,0.905365], +[5.00521,0.247895,0.308836], +[6.38373,0.308781,0.244082], +[4.25427,0.229966,0.247201], +[5.98382,0.398218,0.455249], +[null,null,null], +[6.52367,1.35877,1.30562], +[7.36935,0.536226,0.582304], +[7.05948,0.621982,0.639653], +[4.62901,0.954522,0.908651], +[4.95273,1.03062,1.10289], +[4.71404,1.06378,1.04157], +[4.8201,0.499996,0.575546], +[5.32757,0.566517,0.562058], +[5.37681,0.626458,0.658628], +[5.23137,0.470622,0.540079], +[6.09326,0.561312,0.574978], +[4.86561,0.595546,0.534209], +[4.34256,0.433804,0.414541], +[4.19814,0.288269,0.415328] +]}] diff --git a/website/benchmark/dbms/results/redshift.4x.ra3.xplus.json b/website/benchmark/dbms/results/redshift.4x.ra3.xplus.json new file mode 100644 index 00000000000..e902175ddf7 --- /dev/null +++ b/website/benchmark/dbms/results/redshift.4x.ra3.xplus.json @@ -0,0 +1,45 @@ +[{"system": "redshift (4x.ra3.xplus)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.081437,0.022376,0.022491], +[2.54868,0.024112,0.024072], +[2.62053,0.049948,0.049768], +[null,null,null], +[0.62714,0.447655,0.43262], +[1.14153,1.09479,1.09441], +[0.153399,0.053941,0.048224], +[3.76153,0.032235,0.030949], +[4.43439,0.746808,0.723719], +[6.13424,2.20458,2.20031], +[4.35338,0.220122,0.228684], +[5.15139,0.231856,0.230993], +[5.00006,1.10841,1.12871], +[5.80125,1.86531,1.90209], +[5.16246,1.21239,1.14848], +[0.579428,0.535859,0.546178], +[6.05764,2.17455,2.18286], +[5.86612,2.52634,2.48672], +[7.80075,3.35512,3.41153], +[2.54112,0.036378,0.035944], +[6.47189,2.26909,2.2673], +[6.95344,2.61929,2.65637], +[12.9508,6.85457,6.99], +[25.3022,11.8857,11.8493], +[4.37592,0.452737,0.452867], +[4.17199,0.469457,0.476302], +[6.24746,0.470935,0.482502], +[6.00065,2.08332,2.08059], +[null,null,null], +[22.2567,18.5376,18.3441], +[5.37492,0.868068,0.849486], +[7.29067,1.06155,1.11209], +[7.7832,4.07132,4.00384], +[8.95385,5.15488,5.21863], +[9.2232,5.32052,5.73207], +[4.99205,0.664347,0.618918], +[6.82279,1.89738,1.89398], +[7.09077,2.19008,2.26612], +[5.29731,0.19626,0.204603], +[6.07138,0.276315,0.267161], +[4.03108,0.054134,0.058568], +[4.35647,0.061157,0.053367], +[5.23605,0.037217,0.036335] +]}] diff --git a/website/benchmark/dbms/results/singlestore.c6a.4xlarge.json b/website/benchmark/dbms/results/singlestore.c6a.4xlarge.json new file mode 100644 index 00000000000..8b7b7ccd4e9 --- /dev/null +++ b/website/benchmark/dbms/results/singlestore.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "singlestore (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.09,0.00,0.00], +[0.23,0.04,0.01], +[0.47,0.15,0.15], +[0.63,0.09,0.08], +[1.65,1.23,1.20], +[7.96,2.79,2.63], +[0.10,0.00,0.00], +[0.17,0.02,0.02], +[1.90,1.40,1.39], +[4.79,3.52,3.48], +[0.94,0.22,0.23], +[0.89,0.24,0.23], +[5.82,2.26,2.25], +[6.97,4.62,4.66], +[3.05,2.28,2.31], +[3.92,2.70,2.28], +[5.83,4.48,4.42], +[4.76,4.13,4.12], +[14.03,null,null], +[0.57,0.05,0.04], +[18.18,1.74,1.94], +[20.85,2.17,0.98], +[31.98,3.12,1.22], +[78.96,3.35,108.85], +[2.82,0.39,0.32], +[1.83,0.44,0.35], +[2.81,0.33,0.32], +[18.33,2.57,1.15], +[null,null,null], +[3.56,2.40,2.40], +[3.83,1.11,1.11], +[7.35,1.73,1.70], +[null,null,null], +[null,null,null], +[null,null,null], +[2.53,1.92,1.84], +[0.92,0.23,0.19], +[0.84,0.15,0.08], +[0.70,0.05,0.05], +[3.12,0.38,0.36], +[0.29,0.03,0.03], +[0.22,0.06,0.02], +[0.27,0.11,0.12] +]}] diff --git a/website/benchmark/dbms/results/snowflake.2xl.json b/website/benchmark/dbms/results/snowflake.2xl.json new file mode 100644 index 00000000000..39054eb345f --- /dev/null +++ b/website/benchmark/dbms/results/snowflake.2xl.json @@ -0,0 +1,45 @@ +[{"system": "snowflake (2xl)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.177,0.052,0.090], +[0.903,0.324,0.782], +[0.458,2.909,0.275], +[0.881,0.316,0.166], +[0.404,0.257,0.256], +[0.481,0.325,0.339], +[0.056,0.062,0.060], +[0.183,0.324,0.280], +[0.444,0.314,0.320], +[0.408,0.426,0.417], +[0.345,0.241,0.253], +[0.406,0.245,0.235], +[0.521,1.259,0.326], +[0.466,0.493,0.526], +[0.447,0.349,0.362], +[0.327,0.322,0.302], +[0.462,0.508,0.461], +[0.489,0.481,0.455], +[0.731,0.659,0.674], +[0.151,0.156,0.160], +[0.832,0.339,0.312], +[0.289,0.292,0.340], +[0.591,0.484,0.419], +[2.661,0.716,0.696], +[0.190,0.199,0.377], +[0.181,0.182,0.194], +[0.220,0.209,0.195], +[0.368,0.330,0.347], +[0.677,0.645,0.646], +[0.877,0.886,0.871], +[0.415,0.282,0.301], +[1.265,0.404,0.550], +[0.786,0.750,0.757], +[0.905,0.835,0.841], +[0.864,0.865,0.823], +[0.352,0.360,0.364], +[0.201,0.186,0.200], +[0.143,0.137,0.276], +[0.290,0.132,0.146], +[0.310,0.301,0.304], +[0.212,0.197,0.170], +[0.223,0.174,0.177], +[0.172,0.172,0.172] +]}] diff --git a/website/benchmark/dbms/results/snowflake.3xl.json b/website/benchmark/dbms/results/snowflake.3xl.json new file mode 100644 index 00000000000..c4890b6b407 --- /dev/null +++ b/website/benchmark/dbms/results/snowflake.3xl.json @@ -0,0 +1,45 @@ +[{"system": "snowflake (3xl)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.165,0.061,0.049], +[1.356,1.252,1.502], +[1.287,0.470,0.325], +[0.627,0.331,0.181], +[0.265,0.265,0.483], +[0.887,0.450,0.298], +[0.054,0.066,0.062], +[0.182,0.222,0.194], +[0.408,0.438,0.319], +[0.434,0.462,0.411], +[1.357,0.247,0.412], +[0.343,0.250,0.517], +[0.273,0.289,0.600], +[0.404,0.405,0.392], +[0.379,0.304,0.455], +[0.275,0.271,0.264], +[0.418,0.386,0.388], +[0.417,0.434,0.567], +[0.753,0.759,0.506], +[0.291,0.307,0.400], +[0.954,0.423,0.280], +[0.568,0.562,0.248], +[0.568,0.477,0.496], +[1.458,0.492,0.514], +[0.179,0.300,0.184], +[0.165,0.169,0.176], +[0.197,0.186,0.190], +[0.289,0.547,0.397], +[0.513,0.544,0.632], +[0.766,0.754,0.775], +[0.389,0.374,0.383], +[0.484,0.297,0.286], +[0.505,0.505,0.734], +[0.656,0.598,0.621], +[0.634,0.646,0.609], +[0.309,0.298,0.370], +[0.192,0.219,0.212], +[0.840,0.174,0.139], +[0.172,0.163,0.151], +[0.323,0.296,0.347], +[0.200,0.154,0.144], +[0.191,0.121,0.125], +[0.137,0.233,0.148] +]}] diff --git a/website/benchmark/dbms/results/snowflake.4xl.json b/website/benchmark/dbms/results/snowflake.4xl.json new file mode 100644 index 00000000000..713ed9e445c --- /dev/null +++ b/website/benchmark/dbms/results/snowflake.4xl.json @@ -0,0 +1,45 @@ +[{"system": "snowflake (4xl)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.164,0.061,0.078], +[2.471,2.436,1.927], +[1.656,0.222,0.639], +[0.336,1.244,0.206], +[0.435,0.414,0.373], +[0.520,0.495,0.326], +[0.052,0.051,0.057], +[0.244,0.515,0.358], +[0.473,0.477,0.659], +[0.706,0.523,0.499], +[0.619,0.361,0.303], +[0.463,0.367,0.290], +[0.385,0.319,0.376], +[0.661,0.436,0.452], +[0.446,0.317,0.336], +[0.504,0.283,0.395], +[0.478,1.395,0.350], +[0.555,1.041,0.993], +[0.565,0.558,1.473], +[0.310,0.684,0.278], +[0.637,1.202,0.249], +[0.467,0.294,0.436], +[0.671,0.478,0.611], +[6.262,0.449,0.425], +[0.476,0.213,0.184], +[0.552,0.241,0.201], +[0.458,0.415,0.402], +[0.339,0.357,0.322], +[0.732,0.549,0.483], +[0.914,0.757,0.743], +[0.718,0.310,0.606], +[0.605,0.363,0.425], +[0.468,0.860,0.784], +[0.868,1.787,0.713], +[0.807,0.691,0.544], +[0.485,0.338,0.672], +[0.263,0.221,0.230], +[0.147,0.145,0.139], +[0.135,0.239,0.136], +[0.322,0.378,0.348], +[0.236,0.138,0.132], +[0.193,0.124,0.139], +[0.146,0.145,0.139] +]}] diff --git a/website/benchmark/dbms/results/snowflake.l.json b/website/benchmark/dbms/results/snowflake.l.json new file mode 100644 index 00000000000..a95126636ec --- /dev/null +++ b/website/benchmark/dbms/results/snowflake.l.json @@ -0,0 +1,45 @@ +[{"system": "snowflake (l)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.067,0.054,0.051], +[1.158,1.409,0.190], +[1.470,0.566,0.715], +[0.892,0.534,0.264], +[0.474,0.499,0.489], +[0.950,0.650,0.699], +[0.069,0.055,0.110], +[0.317,0.307,0.321], +[0.949,0.593,0.654], +[0.713,1.099,0.860], +[0.622,0.370,0.404], +[0.457,0.369,0.414], +[0.610,0.566,0.653], +[0.970,1.006,0.976], +[1.517,0.636,0.603], +[0.532,0.541,0.533], +[1.018,1.001,1.022], +[0.942,0.996,0.940], +[2.246,1.596,1.560], +[0.181,0.184,0.200], +[1.135,0.788,0.609], +[0.669,0.528,0.524], +[1.164,0.827,0.882], +[3.545,2.214,2.107], +[0.559,0.431,0.426], +[0.340,0.296,0.383], +[0.695,0.314,0.368], +[0.628,0.658,0.637], +[1.511,1.385,1,440], +[1.390,1.418,1.322], +[1.107,0.687,0.537], +[1.026,0.737,0.659], +[1.712,1.681,1.728], +[2.141,2.130,2.225], +[2.163,2.157,2.110], +[0.650,0.619,0.627], +[0.204,0.195,0.225], +[0.159,0.152,0.156], +[0.146,0.136,0.150], +[0.359,0.290,0.364], +[0.196,0.129,0.227], +[0.201,0.128,0.143], +[0.176,0.129,0.146] +]}] diff --git a/website/benchmark/dbms/results/snowflake.m.json b/website/benchmark/dbms/results/snowflake.m.json new file mode 100644 index 00000000000..3b0543d7b6b --- /dev/null +++ b/website/benchmark/dbms/results/snowflake.m.json @@ -0,0 +1,45 @@ +[{"system": "snowflake (m)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.054,0.062,0.064], +[0.698,0.563,0.407], +[0.586,0.412,0.266], +[1.386,0.363,0.301], +[0.814,0.825,0.984], +[1.303,1.024,1.048], +[0.069,0.066,0.062], +[0.334,0.254,0.270], +[1.043,0.952,0.993], +[1.210,1.209,1.171], +[0.667,0.483,0.456], +[0.543,0.495,0.500], +[1.005,0.889,0.888], +[1.644,1.646,1.652], +[1.054,1.044,0.966], +[0.893,0.874,0.907], +[1.737,1.779,1.837], +[1.518,1.539,1.526], +[3.082,2.818,2.842], +[0.309,0.286,0.256], +[1.594,1.017,0.993], +[0.781,0.853,0.735], +[1.461,1.226,1.080], +[5.308,2.974,2.642], +[0.511,0.625,0.467], +[0.405,0.382,0.439], +[0.601,0.535,0.471], +[0.947,1.624,1.192], +[2.631,2.486,2.490], +[1.938,1.960,1.954], +[1.930,0.830,0.835], +[1.359,1.140,1.062], +[3.599,3.623,3.621], +[3.619,3.741,3.663], +[3.725,3.614,3.786], +[1.149,1.126,1.055], +[0.202,0.207,0.196], +[0.152,0.139,0.145], +[0.149,0.144,0.148], +[0.383,0.287,0.294], +[0.203,0.137,0.119], +[0.200,0.312,0.137], +[0.149,0.130,0.214] +]}] diff --git a/website/benchmark/dbms/results/snowflake.s.json b/website/benchmark/dbms/results/snowflake.s.json new file mode 100644 index 00000000000..d6d096cb711 --- /dev/null +++ b/website/benchmark/dbms/results/snowflake.s.json @@ -0,0 +1,45 @@ +[{"system": "snowflake (s)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.186,0.060,0.062], +[0.980,0.574,0.311], +[0.977,0.554,0.426], +[0.686,0.573,0.404], +[1.386,1.384,1.349], +[1.871,1.697,1.704], +[0.052,0.059,0.227], +[0.309,0.536,0.508], +[1.768,1.631,1.635], +[2.039,2.219,1.908], +[0.807,0.647,0.587], +[0.763,0.690,0.631], +[1.403,1.586,1.404], +[2.593,2.584,2.554], +[1.670,1.538,1.653], +[1.659,1.509,1.514], +[2.875,2.990,2.998], +[2.605,2.549,2.598], +[6.120,5.894,5.766], +[0.320,0.431,0.416], +[2.406,1.703,1.609], +[1.189,1.186,1.163], +[2.104,1.441,1.370], +[7.144,5.174,4.139], +[0.839,0.659,0.641], +[0.527,0.518,0.509], +[0.633,0.621,0.695], +[1.491,1.509,1.514], +[4.848,4.485,4.571], +[3.067,3.106,3.098], +[1.521,1.224,1.236], +[1.839,1.690,1.497], +[5.692,5.751,6.087], +[6.733,6.755,6.712], +[6.722,6.709,6.676], +[1.704,1.686,1.676], +[0.203,0.231,0.218], +[0.151,0.134,0.214], +[0.140,0.156,0.163], +[0.317,0.328,0.319], +[0.166,0.133,0.141], +[0.166,0.120,0.140], +[0.120,0.119,0.126] +]}] diff --git a/website/benchmark/dbms/results/snowflake.xl.json b/website/benchmark/dbms/results/snowflake.xl.json new file mode 100644 index 00000000000..5b688792378 --- /dev/null +++ b/website/benchmark/dbms/results/snowflake.xl.json @@ -0,0 +1,45 @@ +[{"system": "snowflake (xl)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.071,0.053,0.057], +[0.998,0.610,0.240], +[0.420,1.138,1.051], +[0.653,0.264,0.178], +[0.352,0.312,0.349], +[1.126,0.431,0.420], +[0.067,0.057,0.054], +[0.225,0.217,0.200], +[0.617,0.366,0.371], +[1.006,0.541,0.498], +[0.463,0.425,0.293], +[0.431,0.360,0.339], +[0.392,0.371,0.386], +[0.588,0.581,0.590], +[0.634,0.414,0.400], +[0.368,0.410,0.388], +[0.594,0.639,0.663], +[0.616,0.581,0.569], +[1.092,0.933,0.901], +[0.493,0.213,0.160], +[0.886,0.480,0.442], +[0.448,0.337,0.399], +[0.840,0.572,0.505], +[2.251,1.230,0.959], +[0.295,0.253,0.241], +[0.214,0.239,0.278], +[0.261,0.232,0.314], +[0.422,0.429,0.403], +[0.892,0.934,0.883], +[1.041,1.017,1.009], +[0.715,0.442,0.363], +[0.845,0.413,0.461], +[1.101,1.085,1.102], +[1.294,1.272,1.339], +[1.839,1.327,1.241], +[0.439,0.399,0.393], +[0.199,0.211,0.190], +[0.157,0.143,0.140], +[0.145,0.157,0.141], +[0.331,0.291,0.333], +[0.173,0.214,0.138], +[0.189,0.150,0.159], +[0.135,0.149,0.138] +]}] diff --git a/website/benchmark/dbms/results/snowflake.xs.json b/website/benchmark/dbms/results/snowflake.xs.json new file mode 100644 index 00000000000..5cf9e5aac47 --- /dev/null +++ b/website/benchmark/dbms/results/snowflake.xs.json @@ -0,0 +1,45 @@ +[{"system": "snowflake (xs)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.169,0.055,0.056], +[1.184,0.582,0.386], +[1.350,0.560,0.568], +[1.270,0.554,0.538], +[2.516,2.564,2.506], +[2.935,2.649,2.670], +[0.052,0.050,0.064], +[0.383,0.387,0.397], +[3.249,2.993,3.014], +[3.589,3.627,3.887], +[1.243,0.986,0.966], +[1.325,1.080,1.073], +[2.038,2.046,2.035], +[3.738,3.626,3.718], +[2.318,2.159,2.176], +[2.733,2.637,2.668], +[5.607,5.683,5.667], +[3.978,3.923,3.879], +[10.085,9.871,9.844], +[0.450,0.375,0.469], +[5.474,3.103,3.060], +[2.012,1.982,1.971], +[3.365,2.471,2.501], +[11.960,10.619,9.518], +[1.074,1.059,1.026], +[0.856,0.846,0.879], +[1.100,1.085,1.083], +[3.057,3.228,3.117], +[9.406,9.019,9.158], +[6.196,6.243,6.911], +[2.906,2.343,2.017], +[2.954,2.666,2.565], +[9.459,9.565,9.557], +[9.555,9.529,9.368], +[9.409,9.185,9.294], +[2.796,2.880,2.685], +[0.299,0.249,0.262], +[0.156,0.145,0.180], +[0.147,0.146,0.160], +[0.371,0.357,0.356], +[0.166,0.133,0.155], +[0.218,0.140,0.135], +[0.140,0.152,0.158] +]}] diff --git a/website/benchmark/dbms/results/sqlite.c6a.4xlarge.json b/website/benchmark/dbms/results/sqlite.c6a.4xlarge.json new file mode 100644 index 00000000000..49cce70b560 --- /dev/null +++ b/website/benchmark/dbms/results/sqlite.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "sqlite (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[752.739,2.003,1.2], +[304.302,291.521,286.965], +[293.964,287.619,287.219], +[758.302,5.879,5.65], +[836.393,48.593,48.452], +[362.605,344.884,356.245], +[763.993,11.602,10.795], +[296.348,286.879,287.557], +[365.816,360.339,354.126], +[374.403,365.196,362.261], +[302.989,293.888,298.432], +[303.64,291.729,295.347], +[316.824,298.18,301.006], +[320.665,301.15,305.227], +[313.593,301.021,301.626], +[794.881,47,47.225], +[355.346,344.615,342.442], +[316.499,305.971,305.007], +[398.177,380.383,385.571], +[751.82,5.082,4.913], +[295.522,286.573,287.368], +[298.58,287.182,288.303], +[296.474,288.747,288.638], +[296.579,287.127,287.361], +[304.709,286.865,287.56], +[300.391,290.633,288.587], +[294.605,286.91,287.799], +[305.986,312.111,305.626], +[null,null,null], +[411.225,397.614,394.253], +[307.711,295.181,300.266], +[312.472,299.079,298.19], +[386.674,378.347,376.963], +[409.742,409.554,420.273], +[468.73,453.709,458.149], +[366.015,347.446,346.728], +[2.911,0.781,0.757], +[1.599,0.609,0.587], +[1.288,0.256,0.238], +[2.469,1.582,1.52], +[1.274,0.303,0.283], +[1.322,0.317,0.314], +[1.498,0.602,0.613] +]}] diff --git a/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.compression.json b/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.compression.json new file mode 100644 index 00000000000..148b06847a4 --- /dev/null +++ b/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.compression.json @@ -0,0 +1,45 @@ +[{"system": "timescaledb (c6a.4xlarge.compression)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[2.28686,1.63642,1.64263], +[32.6848,1.63476,1.40052], +[60.8633,3.70484,3.59342], +[36.4029,2.87091,2.80739], +[110.391,38.9688,38.0549], +[147.379,66.2513,65.6379], +[33.0294,2.92031,2.84375], +[33.0221,1.2984,1.19227], +[115.694,47.4651,47.0125], +[156.347,51.577,51.2694], +[68.3301,4.75521,4.68007], +[77.4356,5.55128,5.56577], +[49.7741,11.2911,11.3265], +[81.1014,14.9111,14.9541], +[82.9569,14.6156,14.6331], +[62.0338,26.399,26.3351], +[103.259,36.4122,36.6076], +[92.8828,26.2395,25.8991], +[144.281,63.5102,63.7661], +[7.00679,0.573073,0.536283], +[75.0203,7.86344,7.90495], +[81.2825,9.15868,9.01775], +[104.084,13.9528,13.8435], +[132.531,81.522,82.1561], +[80.6965,3.28231,3.16574], +[39.7693,2.51443,2.43849], +[80.4245,3.26941,3.13916], +[104.015,13.7044,13.5313], +[307.26,253.127,252.147], +[42.8549,22.4187,22.0325], +[137.601,14.9592,14.6804], +[136.767,22.8007,22.131], +[263.005,168.551,163.355], +[156.919,92.6308,91.702], +[160.842,96.0512,97.1773], +[62.8357,28.0336,28.7397], +[1.75869,0.561604,0.541215], +[0.46607,0.191863,0.19021], +[0.303671,0.137579,0.136615], +[2.32031,1.49223,1.52369], +[0.563764,0.14192,0.138234], +[0.372428,0.122989,0.123709], +[0.448574,0.159092,0.154687] +]}] diff --git a/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.json b/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.json new file mode 100644 index 00000000000..67bb1a09555 --- /dev/null +++ b/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "timescaledb (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[437.700,215.793,176.420], +[327.026,259.568,244.578], +[262.978,263.090,263.083], +[262.807,263.046,266.847], +[337.497,334.964,330.852], +[355.689,356.801,362.894], +[262.762,263.012,262.968], +[263.055,263.016,263.028], +[319.928,319.388,320.704], +[323.584,322.224,322.488], +[265.979,265.465,265.375], +[266.019,265.543,265.462], +[277.018,276.300,276.595], +[280.352,279.251,279.572], +[279.915,279.896,279.674], +[296.377,298.506,297.659], +[314.448,314.605,312.570], +[302.668,302.672,303.039], +[325.810,324.061,324.376], +[262.447,262.698,262.704], +[267.581,267.467,267.482], +[268.085,267.466,267.696], +[263.391,263.097,263.126], +[38.291,0.435,0.335], +[0.127,0.005,0.005], +[263.138,263.100,263.092], +[0.889,0.341,0.339], +[267.586,267.498,267.491], +[289.086,290.012,290.093], +[263.220,263.071,263.109], +[274.780,273.995,273.998], +[282.217,281.390,281.470], +[429.273,426.588,439.431], +[448.808,418.724,418.207], +[455.196,422.750,423.142], +[299.263,296.937,297.261], +[18.693,1.552,1.481], +[18.125,0.945,0.937], +[18.528,1.062,0.902], +[18.268,1.779,1.770], +[19.615,1.965,1.966], +[18.970,1.435,1.430], +[18.330,1.153,0.952] +]}] diff --git a/website/benchmark/dbms/results/vertica.c6a.4xlarge.json b/website/benchmark/dbms/results/vertica.c6a.4xlarge.json new file mode 100644 index 00000000000..a2064e0ea80 --- /dev/null +++ b/website/benchmark/dbms/results/vertica.c6a.4xlarge.json @@ -0,0 +1,45 @@ +[{"system": "vertica (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ +[0.031357,0.020759,0.019373], +[0.156612,0.033543,0.032149], +[0.404881,0.124383,0.124252], +[0.220711,0.074036,0.070379], +[2.19663,2.18441,2.1763], +[11.5901,11.4847,11.4573], +[0.160286,0.063093,0.064844], +[0.116135,0.047536,0.046322], +[2.36592,2.15174,2.15675], +[4.42049,4.37322,4.38596], +[1.24771,1.17476,1.21011], +[1.30116,1.24332,1.24521], +[14.0523,13.9236,13.9943], +[39.8961,39.7911,39.771], +[14.6776,14.5524,14.5899], +[2.82581,2.71968,2.72123], +[48.4932,48.4037,48.4298], +[36.8767,36.7432,36.9178], +[108.743,108.81,108.671], +[0.02734,0.018601,0.018892], +[8.98822,4.2958,4.38323], +[10.0097,4.31009,4.26843], +[14.2354,5.48257,5.47171], +[3.29109,0.405847,0.337797], +[0.088318,0.079666,0.091852], +[1.05864,1.26979,1.81786], +[0.087436,0.092269,0.087506], +[15.0424,15.0873,15.2561], +[22.1566,22.1208,21.9509], +[2.59797,2.46053,2.46406], +[3.63938,3.68086,3.68988], +[6.58816,4.84546,4.8853], +[20.7209,20.4806,20.5823], +[77.3779,78.1301,77.4366], +[77.2494,77.646,77.4976], +[6.79571,6.81133,6.67973], +[0.887148,0.782822,0.753053], +[0.369373,0.339676,0.338677], +[0.099038,0.089598,0.099019], +[2.15693,2.01152,2.00868], +[0.096018,0.052062,0.053813], +[0.058647,0.046352,0.045415], +[0.106915,0.077122,0.076134] +]}] From faa9580a58d01237197333add60a8bf01ee58847 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 15:12:44 +0200 Subject: [PATCH 157/627] Some results --- .../dbms/results/bigquery.serverless.json | 45 ------------------- .../dbms/results/vertica.c6a.4xlarge.json | 45 ------------------- 2 files changed, 90 deletions(-) delete mode 100644 website/benchmark/dbms/results/bigquery.serverless.json delete mode 100644 website/benchmark/dbms/results/vertica.c6a.4xlarge.json diff --git a/website/benchmark/dbms/results/bigquery.serverless.json b/website/benchmark/dbms/results/bigquery.serverless.json deleted file mode 100644 index ef691fba636..00000000000 --- a/website/benchmark/dbms/results/bigquery.serverless.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "bigquery (serverless)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[1.451,1.284,1.272], -[1.462,1.23,1.318], -[1.394,1.338,1.248], -[1.248,1.41,1.377], -[1.506,1.661,1.538], -[1.581,1.709,1.639], -[1.39,1.195,1.274], -[1.431,1.362,1.378], -[2.012,1.92,3.148], -[3.104,3.118,3.111], -[1.342,1.355,1.489], -[1.374,1.443,1.349], -[1.659,1.707,1.649], -[3.086,3.123,4.189], -[1.684,1.98,1.823], -[1.899,1.654,1.704], -[3.097,2.872,3.09], -[3.193,3.091,3.073], -[5.162,4.125,4.121], -[1.362,1.183,1.171], -[1.508,1.624,1.578], -[1.537,1.554,1.484], -[1.931,1.977,2.024], -[3.135,3.141,3.182], -[1.494,1.377,1.321], -[1.342,1.49,1.348], -[1.397,1.377,1.482], -[3.084,3.115,3.11], -[4.144,3.41,4.131], -[3.086,3.129,3.094], -[1.887,1.861,1.783], -[2.068,1.837,1.938], -[4.126,4.14,4.173], -[3.103,3.106,3.117], -[4.149,4.136,4.172], -[3.111,3.127,1.988], -[1.539,1.443,1.412], -[1.213,1.202,1.252], -[1.16,1.213,1.174], -[1.723,1.747,1.764], -[1.173,1.122,1.124], -[1.164,1.162,1.098], -[1.292,1.226,1.274] -]}] diff --git a/website/benchmark/dbms/results/vertica.c6a.4xlarge.json b/website/benchmark/dbms/results/vertica.c6a.4xlarge.json deleted file mode 100644 index a2064e0ea80..00000000000 --- a/website/benchmark/dbms/results/vertica.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "vertica (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.031357,0.020759,0.019373], -[0.156612,0.033543,0.032149], -[0.404881,0.124383,0.124252], -[0.220711,0.074036,0.070379], -[2.19663,2.18441,2.1763], -[11.5901,11.4847,11.4573], -[0.160286,0.063093,0.064844], -[0.116135,0.047536,0.046322], -[2.36592,2.15174,2.15675], -[4.42049,4.37322,4.38596], -[1.24771,1.17476,1.21011], -[1.30116,1.24332,1.24521], -[14.0523,13.9236,13.9943], -[39.8961,39.7911,39.771], -[14.6776,14.5524,14.5899], -[2.82581,2.71968,2.72123], -[48.4932,48.4037,48.4298], -[36.8767,36.7432,36.9178], -[108.743,108.81,108.671], -[0.02734,0.018601,0.018892], -[8.98822,4.2958,4.38323], -[10.0097,4.31009,4.26843], -[14.2354,5.48257,5.47171], -[3.29109,0.405847,0.337797], -[0.088318,0.079666,0.091852], -[1.05864,1.26979,1.81786], -[0.087436,0.092269,0.087506], -[15.0424,15.0873,15.2561], -[22.1566,22.1208,21.9509], -[2.59797,2.46053,2.46406], -[3.63938,3.68086,3.68988], -[6.58816,4.84546,4.8853], -[20.7209,20.4806,20.5823], -[77.3779,78.1301,77.4366], -[77.2494,77.646,77.4976], -[6.79571,6.81133,6.67973], -[0.887148,0.782822,0.753053], -[0.369373,0.339676,0.338677], -[0.099038,0.089598,0.099019], -[2.15693,2.01152,2.00868], -[0.096018,0.052062,0.053813], -[0.058647,0.046352,0.045415], -[0.106915,0.077122,0.076134] -]}] From cd97b279ac14b8b2a7cfc243239ebc93345f3ba1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jul 2022 15:13:39 +0200 Subject: [PATCH 158/627] Add .gitignore --- website/benchmark/dbms/results/.gitignore | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 website/benchmark/dbms/results/.gitignore diff --git a/website/benchmark/dbms/results/.gitignore b/website/benchmark/dbms/results/.gitignore new file mode 100644 index 00000000000..c128402e106 --- /dev/null +++ b/website/benchmark/dbms/results/.gitignore @@ -0,0 +1,4 @@ +# De-Witt Clause: + +vertica* +bigquery* From ac8a7947808fdcb2ffed53ab626050973984d9a6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 4 Jul 2022 15:40:23 +0200 Subject: [PATCH 159/627] Client -i file --- src/Client/ClientBase.cpp | 39 ++++++++++++------- src/Client/ClientBase.h | 1 + ...02352_interactive_queries_from_file.expect | 29 ++++++++++++++ ...52_interactive_queries_from_file.reference | 0 4 files changed, 56 insertions(+), 13 deletions(-) create mode 100755 tests/queries/0_stateless/02352_interactive_queries_from_file.expect create mode 100644 tests/queries/0_stateless/02352_interactive_queries_from_file.reference diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 396fd97368e..8230c97f49c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1834,9 +1834,21 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) bool ClientBase::processQueryText(const String & text) { - if (exit_strings.end() != exit_strings.find(trim(text, [](char c) { return isWhitespaceASCII(c) || c == ';'; }))) + auto trimmed_input = trim(text, [](char c) { return isWhitespaceASCII(c) || c == ';'; }); + + if (exit_strings.end() != exit_strings.find(trimmed_input)) return false; + if (trimmed_input.starts_with("\\i")) + { + size_t skip_prefix_size = std::strlen("\\i"); + auto file_name = trim( + trimmed_input.substr(skip_prefix_size, trimmed_input.size() - skip_prefix_size), + [](char c) { return isWhitespaceASCII(c); }); + + return processMultiQueryFromFile(file_name); + } + if (!is_multiquery) { assert(!query_fuzzer_runs); @@ -2019,6 +2031,17 @@ void ClientBase::runInteractive() } +bool ClientBase::processMultiQueryFromFile(const String & file_name) +{ + String queries_from_file; + + ReadBufferFromFile in(file_name); + readStringUntilEOF(queries_from_file, in); + + return executeMultiQuery(queries_from_file); +} + + void ClientBase::runNonInteractive() { if (delayed_interactive) @@ -2026,23 +2049,13 @@ void ClientBase::runNonInteractive() if (!queries_files.empty()) { - auto process_multi_query_from_file = [&](const String & file) - { - String queries_from_file; - - ReadBufferFromFile in(file); - readStringUntilEOF(queries_from_file, in); - - return executeMultiQuery(queries_from_file); - }; - for (const auto & queries_file : queries_files) { for (const auto & interleave_file : interleave_queries_files) - if (!process_multi_query_from_file(interleave_file)) + if (!processMultiQueryFromFile(interleave_file)) return; - if (!process_multi_query_from_file(queries_file)) + if (!processMultiQueryFromFile(queries_file)) return; } diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index d34fe282839..ec2267a3be6 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -154,6 +154,7 @@ private: protected: static bool isSyncInsertWithData(const ASTInsertQuery & insert_query, const ContextPtr & context); + bool processMultiQueryFromFile(const String & file_name); bool is_interactive = false; /// Use either interactive line editing interface or batch mode. bool is_multiquery = false; diff --git a/tests/queries/0_stateless/02352_interactive_queries_from_file.expect b/tests/queries/0_stateless/02352_interactive_queries_from_file.expect new file mode 100755 index 00000000000..ee310e88a46 --- /dev/null +++ b/tests/queries/0_stateless/02352_interactive_queries_from_file.expect @@ -0,0 +1,29 @@ +#!/usr/bin/expect -f +# tags: long, no-parallel + +set basedir [file dirname $argv0] +set basename [file tail $argv0] + +log_user 0 +set timeout 20 +match_max 100000 + +expect_after { + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } +} + +spawn bash -c "echo 'select 1;\nselect 2;\nselect 3' > queries_02352" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT --disable_suggestion" +expect ":) " + +send -- "\\i queries_02352\r" +expect "1" +expect "2" +expect "3" +expect ":) " + +send -- "exit\r" +expect eof diff --git a/tests/queries/0_stateless/02352_interactive_queries_from_file.reference b/tests/queries/0_stateless/02352_interactive_queries_from_file.reference new file mode 100644 index 00000000000..e69de29bb2d From 2746ede854f5778849d1dad320aeb5c3a358d6df Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 4 Jul 2022 15:58:06 +0200 Subject: [PATCH 160/627] Better test --- .../02352_interactive_queries_from_file.expect | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/tests/queries/0_stateless/02352_interactive_queries_from_file.expect b/tests/queries/0_stateless/02352_interactive_queries_from_file.expect index ee310e88a46..7823737d349 100755 --- a/tests/queries/0_stateless/02352_interactive_queries_from_file.expect +++ b/tests/queries/0_stateless/02352_interactive_queries_from_file.expect @@ -24,6 +24,21 @@ expect "1" expect "2" expect "3" expect ":) " +send -- "\\i queries_02352;\r" +expect "1" +expect "2" +expect "3" +expect ":) " +send -- " \\i queries_02352 ; \r" +expect "1" +expect "2" +expect "3" +expect ":) " +send -- " \\i queries_02352 ; \r" +expect "1" +expect "2" +expect "3" +expect ":) " send -- "exit\r" expect eof From 1eed72b525aab4bcf074f4b7b1ed8e7e4dc36937 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 29 Jun 2022 10:37:42 +0000 Subject: [PATCH 161/627] Make more multi-search methods work with non-const needles After making function multi[Fuzzy]Match(Any|AnyIndex|AllIndices)() work with non-const needles, 12 more functions started to fail in test "00233_position_function_family": multiSearchAny() multiSearchAnyCaseInsensitive() multiSearchAnyUTF8 multiSearchAnyCaseInsensitiveUTF8() multiSearchFirstPosition() multiSearchFirstPositionCaseInsensitive() multiSearchFirstPositionUTF8() multiSearchFirstPositionCaseInsensitiveUTF8() multiSearchFirstIndex() multiSearchFirstIndexCaseInsensitive() multiSearchFirstIndexUTF8() multiSearchFirstIndexCaseInsensitiveUTF8() Failing queries take the form select 0 = multiSearchAny('\0', CAST([], 'Array(String)')); --- src/Functions/MultiMatchAllIndicesImpl.h | 5 +- src/Functions/MultiMatchAnyImpl.h | 5 +- src/Functions/MultiSearchFirstIndexImpl.h | 57 +++++++++++++-- src/Functions/MultiSearchFirstPositionImpl.h | 70 +++++++++++++++++-- src/Functions/MultiSearchImpl.h | 55 +++++++++++++-- ...tringsearch_with_nonconst_needle.reference | 24 +++++++ ...2294_stringsearch_with_nonconst_needle.sql | 34 +++++++++ 7 files changed, 223 insertions(+), 27 deletions(-) diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index 7ff2376593c..adba8817b33 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -165,9 +165,8 @@ struct MultiMatchAllIndicesImpl size_t prev_haystack_offset = 0; for (size_t i = 0; i < haystack_offsets.size(); ++i) { - Field field; - needles_col.get(i, field); - Array & needles_arr = DB::get(field); + Field field = needles_col[i]; + const Array & needles_arr = DB::get(field); std::vector needles; needles.reserve(needles_arr.size()); diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 24cf6b53f30..fa56a5f0924 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -192,9 +192,8 @@ struct MultiMatchAnyImpl size_t prev_haystack_offset = 0; for (size_t i = 0; i < haystack_offsets.size(); ++i) { - Field field; - needles_col.get(i, field); - Array & needles_arr = DB::get(field); + Field field = needles_col[i]; + const Array & needles_arr = DB::get(field); std::vector needles; needles.reserve(needles_arr.size()); diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 9e717aaff23..56ac06551f6 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -30,7 +30,7 @@ struct MultiSearchFirstIndexImpl const ColumnString::Offsets & haystack_offsets, const Array & needles_arr, PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets, + PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/) @@ -47,13 +47,15 @@ struct MultiSearchFirstIndexImpl needles.emplace_back(needle.get()); auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); + + const size_t haystack_size = haystack_offsets.size(); + res.resize(haystack_size); + size_t iteration = 0; while (searcher.hasMoreToSearch()) { size_t prev_offset = 0; - for (size_t j = 0; j < haystack_string_size; ++j) + for (size_t j = 0; j < haystack_size; ++j) { const auto * haystack = &haystack_data[prev_offset]; const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; @@ -68,10 +70,51 @@ struct MultiSearchFirstIndexImpl std::fill(res.begin(), res.end(), 0); } - template - static void vectorVector(Args &&...) + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnArray & needles_col, + PaddedPODArray & res, + PaddedPODArray & /*offsets*/, + bool /*allow_hyperscan*/, + size_t /*max_hyperscan_regexp_length*/, + size_t /*max_hyperscan_regexp_total_length*/) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needles", name); + const size_t haystack_size = haystack_offsets.size(); + res.resize(haystack_size); + + size_t prev_offset = 0; + + for (size_t i = 0; i < haystack_size; ++i) + { + Field field = needles_col[i]; + const Array & needles_arr = DB::get(field); + + std::vector needles; + needles.reserve(needles_arr.size()); + for (const auto & needle : needles_arr) + needles.emplace_back(needle.get()); + + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal + + const auto * const haystack = &haystack_data[prev_offset]; + const auto * haystack_end = haystack + haystack_offsets[i] - prev_offset - 1; + + size_t iteration = 0; + while (searcher.hasMoreToSearch()) + { + if (iteration == 0 || res[i] == 0) + { + res[i] = searcher.searchOneFirstIndex(haystack, haystack_end); + } + ++iteration; + } + if (iteration == 0) + { + res[i] = 0; + } + prev_offset = haystack_offsets[i]; + } } }; diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index 1f0cf1e6463..f1d79b2685d 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -30,7 +30,7 @@ struct MultiSearchFirstPositionImpl const ColumnString::Offsets & haystack_offsets, const Array & needles_arr, PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets, + PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/) @@ -51,13 +51,15 @@ struct MultiSearchFirstPositionImpl return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); }; auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); + + const size_t haystack_size = haystack_offsets.size(); + res.resize(haystack_size); + size_t iteration = 0; while (searcher.hasMoreToSearch()) { size_t prev_offset = 0; - for (size_t j = 0; j < haystack_string_size; ++j) + for (size_t j = 0; j < haystack_size; ++j) { const auto * haystack = &haystack_data[prev_offset]; const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; @@ -77,10 +79,64 @@ struct MultiSearchFirstPositionImpl std::fill(res.begin(), res.end(), 0); } - template - static void vectorVector(Args &&...) + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnArray & needles_col, + PaddedPODArray & res, + PaddedPODArray & /*offsets*/, + bool /*allow_hyperscan*/, + size_t /*max_hyperscan_regexp_length*/, + size_t /*max_hyperscan_regexp_total_length*/) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needles", name); + const size_t haystack_size = haystack_offsets.size(); + res.resize(haystack_size); + + size_t prev_offset = 0; + + auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 + { + return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); + }; + + for (size_t i = 0; i < haystack_size; ++i) + { + Field field = needles_col[i]; + const Array & needles_arr = DB::get(field); + + std::vector needles; + needles.reserve(needles_arr.size()); + for (const auto & needle : needles_arr) + needles.emplace_back(needle.get()); + + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal + + const auto * const haystack = &haystack_data[prev_offset]; + const auto * haystack_end = haystack + haystack_offsets[i] - prev_offset - 1; + + size_t iteration = 0; + while (searcher.hasMoreToSearch()) + { + if (iteration == 0 || res[i] == 0) + { + res[i] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); + } + else + { + UInt64 result = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); + if (result != 0) + { + res[i] = std::min(result, res[i]); + } + } + ++iteration; + } + if (iteration == 0) + { + res[i] = 0; + } + prev_offset = haystack_offsets[i]; + } } }; diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 53406d9324d..86567625252 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -30,7 +30,7 @@ struct MultiSearchImpl const ColumnString::Offsets & haystack_offsets, const Array & needles_arr, PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets, + PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/) @@ -47,13 +47,15 @@ struct MultiSearchImpl needles.emplace_back(needle.get()); auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); + + const size_t haystack_size = haystack_offsets.size(); + res.resize(haystack_size); + size_t iteration = 0; while (searcher.hasMoreToSearch()) { size_t prev_offset = 0; - for (size_t j = 0; j < haystack_string_size; ++j) + for (size_t j = 0; j < haystack_size; ++j) { const auto * haystack = &haystack_data[prev_offset]; const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; @@ -67,10 +69,49 @@ struct MultiSearchImpl std::fill(res.begin(), res.end(), 0); } - template - static void vectorVector(Args &&...) + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnArray & needles_col, + PaddedPODArray & res, + PaddedPODArray & /*offsets*/, + bool /*allow_hyperscan*/, + size_t /*max_hyperscan_regexp_length*/, + size_t /*max_hyperscan_regexp_total_length*/) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needles", name); + const size_t haystack_size = haystack_offsets.size(); + res.resize(haystack_size); + + size_t prev_offset = 0; + + for (size_t i = 0; i < haystack_size; ++i) + { + const auto * const haystack = &haystack_data[prev_offset]; + const size_t haystack_length = haystack_offsets[i] - prev_offset - 1; + + Field field = needles_col[i]; + const Array & needles_arr = DB::get(field); + + std::vector needles; + needles.reserve(needles_arr.size()); + for (const auto & needle : needles_arr) + needles.emplace_back(needle.get()); + + size_t iteration = 0; + for (size_t j = 0; j < needles_arr.size(); ++j) + { + auto searcher = Impl::createSearcherInSmallHaystack(needles[j].data(), needles[j].size()); + if (iteration == 0 || !res[i]) + { + const auto * match = searcher.search(haystack, haystack_length); + res[i] = (match != haystack + haystack_length); + } + ++iteration; + } + if (iteration == 0) + res[i] = 0; + prev_offset = haystack_offsets[i]; + } } }; diff --git a/tests/queries/0_stateless/02294_stringsearch_with_nonconst_needle.reference b/tests/queries/0_stateless/02294_stringsearch_with_nonconst_needle.reference index 7471bcad00c..76ebc1fb99e 100644 --- a/tests/queries/0_stateless/02294_stringsearch_with_nonconst_needle.reference +++ b/tests/queries/0_stateless/02294_stringsearch_with_nonconst_needle.reference @@ -188,3 +188,27 @@ MATCH 35 Hello .*ell.* 1 36 Hello o$ 1 37 Hello hE.*lO 0 +MULTISEARCHANY +1 +1 +1 +1 +1 +1 +1 +MULTISEARCHFIRSTINDEX +1 +1 +1 +1 +1 +1 +1 +MULTISEARCHFIRSTPOSITION +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02294_stringsearch_with_nonconst_needle.sql b/tests/queries/0_stateless/02294_stringsearch_with_nonconst_needle.sql index 3057e342733..6dd4c4f396d 100644 --- a/tests/queries/0_stateless/02294_stringsearch_with_nonconst_needle.sql +++ b/tests/queries/0_stateless/02294_stringsearch_with_nonconst_needle.sql @@ -1,3 +1,5 @@ +-- tests of "(not) (i)like" functions + drop table if exists non_const_needle; create table non_const_needle @@ -34,3 +36,35 @@ select id, haystack, needle, match(haystack, needle) order by id; drop table if exists non_const_needle; + +-- rudimentary tests of "multiSearchFirstIndex()", "multiSearchAnyPosition()" and "multiSearchFirstIndex()" functions + +select 'MULTISEARCHANY'; +select multiSearchAny(materialize('Hello World'), materialize([])); -- { serverError 43 } +select 0 = multiSearchAny('Hello World', CAST([], 'Array(String)')); +select 1 = multiSearchAny(materialize('Hello World'), materialize(['orld'])); +select 0 = multiSearchAny(materialize('Hello World'), materialize(['Hallo', 'Welt'])); +select 1 = multiSearchAny(materialize('Hello World'), materialize(['Hallo', 'orld'])); +select 1 = multiSearchAnyCaseInsensitive(materialize('Hello World'), materialize(['WORLD'])); +select 1 = multiSearchAnyUTF8(materialize('Hello World £'), materialize(['WORLD', '£'])); +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('Hello World £'), materialize(['WORLD'])); + +select 'MULTISEARCHFIRSTINDEX'; +select multiSearchFirstIndex(materialize('Hello World'), materialize([])); -- { serverError 43 } +select 0 = multiSearchFirstIndex('Hello World', CAST([], 'Array(String)')); +select 1 = multiSearchFirstIndex(materialize('Hello World'), materialize(['orld'])); +select 0 = multiSearchFirstIndex(materialize('Hello World'), materialize(['Hallo', 'Welt'])); +select 2 = multiSearchFirstIndex(materialize('Hello World'), materialize(['Hallo', 'orld'])); +select 1 = multiSearchFirstIndexCaseInsensitive(materialize('Hello World'), materialize(['WORLD'])); +select 2 = multiSearchFirstIndexUTF8(materialize('Hello World £'), materialize(['WORLD', '£'])); +select 1 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('Hello World £'), materialize(['WORLD'])); + +select 'MULTISEARCHFIRSTPOSITION'; +select multiSearchFirstPosition(materialize('Hello World'), materialize([])); -- { serverError 43 } +select 0 = multiSearchFirstPosition('Hello World', CAST([], 'Array(String)')); +select 8 = multiSearchFirstPosition(materialize('Hello World'), materialize(['orld'])); +select 0 = multiSearchFirstPosition(materialize('Hello World'), materialize(['Hallo', 'Welt'])); +select 8 = multiSearchFirstPosition(materialize('Hello World'), materialize(['Hallo', 'orld'])); +select 7 = multiSearchFirstPositionCaseInsensitive(materialize('Hello World'), materialize(['WORLD'])); +select 13 = multiSearchFirstPositionUTF8(materialize('Hello World £'), materialize(['WORLD', '£'])); +select 7 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('Hello World £'), materialize(['WORLD'])); From f5bf7440284204834f7b088f468a3483153b53d3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 4 Jul 2022 13:52:53 +0200 Subject: [PATCH 162/627] Fix --- src/IO/ParallelReadBuffer.h | 1 + src/IO/WithFileSize.cpp | 8 ++++ tests/integration/test_storage_s3/test.py | 48 +++++++++++------------ 3 files changed, 33 insertions(+), 24 deletions(-) diff --git a/src/IO/ParallelReadBuffer.h b/src/IO/ParallelReadBuffer.h index 9881d463ed4..45b98f8c977 100644 --- a/src/IO/ParallelReadBuffer.h +++ b/src/IO/ParallelReadBuffer.h @@ -47,6 +47,7 @@ public: off_t getPosition() override; const ReadBufferFactory & getReadBufferFactory() const { return *reader_factory; } + ReadBufferFactory & getReadBufferFactory() { return *reader_factory; } private: /// Reader in progress with a list of read segments diff --git a/src/IO/WithFileSize.cpp b/src/IO/WithFileSize.cpp index 28542db7a73..f71690fcdee 100644 --- a/src/IO/WithFileSize.cpp +++ b/src/IO/WithFileSize.cpp @@ -33,6 +33,10 @@ size_t getFileSizeFromReadBuffer(ReadBuffer & in) { return getFileSize(compressed->getWrappedReadBuffer()); } + else if (auto * parallel = dynamic_cast(&in)) + { + return getFileSize(parallel->getReadBufferFactory()); + } return getFileSize(in); } @@ -47,6 +51,10 @@ bool isBufferWithFileSize(const ReadBuffer & in) { return isBufferWithFileSize(compressed->getWrappedReadBuffer()); } + else if (const auto * parallel = dynamic_cast(&in)) + { + return dynamic_cast(¶llel->getReadBufferFactory()) != nullptr; + } return dynamic_cast(&in) != nullptr; } diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index ec7c746c549..7f92325f44e 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1052,61 +1052,61 @@ def test_seekable_formats(started_cluster): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_function = f"s3(s3_parquet, structure='a Int32, b String', format='Parquet')" - instance.query( - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) settings s3_truncate_on_insert=1" + exec_query_with_retry( + instance, + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1000000) settings s3_truncate_on_insert=1", ) result = instance.query(f"SELECT count() FROM {table_function}") - assert int(result) == 5000000 + assert int(result) == 1000000 table_function = f"s3(s3_orc, structure='a Int32, b String', format='ORC')" exec_query_with_retry( instance, - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) settings s3_truncate_on_insert=1", + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1000000) settings s3_truncate_on_insert=1", ) - result = instance.query(f"SELECT count() FROM {table_function}") - assert int(result) == 5000000 + result = instance.query( + f"SELECT count() FROM {table_function} SETTINGS max_memory_usage='50M'" + ) + assert int(result) == 1000000 + + instance.query(f"SELECT * FROM {table_function} FORMAT Null") instance.query("SYSTEM FLUSH LOGS") result = instance.query( - f"SELECT formatReadableSize(memory_usage) FROM system.query_log WHERE startsWith(query, 'SELECT count() FROM s3') AND memory_usage > 0 ORDER BY event_time desc" + f"SELECT formatReadableSize(ProfileEvents['ReadBufferFromS3Bytes']) FROM system.query_log WHERE startsWith(query, 'SELECT * FROM s3') AND memory_usage > 0 AND type='QueryFinish' ORDER BY event_time desc" ) - + result = result.strip() + assert result.endswith("MiB") result = result[: result.index(".")] - assert int(result) < 200 + assert int(result) > 80 def test_seekable_formats_url(started_cluster): bucket = started_cluster.minio_bucket - instance = started_cluster.instances["dummy"] + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_function = f"s3(s3_parquet, structure='a Int32, b String', format='Parquet')" - instance.query( - f"insert into table function {table_function} select number, randomString(100) from numbers(5000000) settings s3_truncate_on_insert=1" + exec_query_with_retry( + instance, + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1000000) settings s3_truncate_on_insert=1", ) - table_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_parquet', 'Parquet', 'a Int32, b String')" result = instance.query(f"SELECT count() FROM {table_function}") - assert int(result) == 5000000 + assert int(result) == 1000000 table_function = f"s3(s3_orc, structure='a Int32, b String', format='ORC')" exec_query_with_retry( instance, - f"insert into table function {table_function} select number, randomString(100) from numbers(5000000) settings s3_truncate_on_insert=1", + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1000000) settings s3_truncate_on_insert=1", ) - table_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_orc', 'ORC', 'a Int32, b String')" - result = instance.query(f"SELECT count() FROM {table_function}") - assert int(result) == 5000000 - - instance.query("SYSTEM FLUSH LOGS") + table_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_parquet', 'Parquet', 'a Int32, b String')" result = instance.query( - f"SELECT formatReadableSize(memory_usage) FROM system.query_log WHERE startsWith(query, 'SELECT count() FROM url') AND memory_usage > 0 ORDER BY event_time desc" + f"SELECT count() FROM {table_function} SETTINGS max_memory_usage='50M'" ) - - result = result[: result.index(".")] - assert int(result) < 200 + assert int(result) == 1000000 def test_empty_file(started_cluster): From b4e62e06e7f2b493f800f9a022f9308214261c93 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 4 Jul 2022 14:07:29 +0000 Subject: [PATCH 163/627] Use native Map --- src/Interpreters/OpenTelemetrySpanLog.cpp | 25 +++---------- src/Interpreters/OpenTelemetrySpanLog.h | 3 +- src/Interpreters/ThreadStatusExt.cpp | 3 +- src/Interpreters/executeQuery.cpp | 45 +++++++---------------- 4 files changed, 21 insertions(+), 55 deletions(-) diff --git a/src/Interpreters/OpenTelemetrySpanLog.cpp b/src/Interpreters/OpenTelemetrySpanLog.cpp index e1199fa03b1..79000aab375 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.cpp +++ b/src/Interpreters/OpenTelemetrySpanLog.cpp @@ -12,6 +12,7 @@ #include #include +#include "Core/Field.h" namespace DB @@ -64,13 +65,7 @@ void OpenTelemetrySpanLogElement::appendToBlock(MutableColumns & columns) const // The user might add some ints values, and we will have Int Field, and the // insert will fail because the column requires Strings. Convert the fields // here, because it's hard to remember to convert them in all other places. - - Map map(attribute_names.size()); - for (size_t attr_idx = 0; attr_idx < map.size(); ++attr_idx) - { - map[attr_idx] = Tuple{attribute_names[attr_idx], toString(attribute_values[attr_idx])}; - } - columns[i++]->insert(map); + columns[i++]->insert(attributes); } @@ -158,9 +153,7 @@ void OpenTelemetrySpanHolder::addAttribute(const std::string& name, UInt64 value if (trace_id == UUID()) return; - this->attribute_names.push_back(name); - this->attribute_values.push_back(std::to_string(value)); - assert(this->attribute_names.size() == this->attribute_values.size()); + this->attributes.push_back(Tuple{name, toString(value)}); } void OpenTelemetrySpanHolder::addAttribute(const std::string& name, const std::string& value) @@ -168,9 +161,7 @@ void OpenTelemetrySpanHolder::addAttribute(const std::string& name, const std::s if (trace_id == UUID()) return; - this->attribute_names.push_back(name); - this->attribute_values.push_back(value); - assert(this->attribute_names.size() == this->attribute_values.size()); + this->attributes.push_back(Tuple{name, value}); } void OpenTelemetrySpanHolder::addAttribute(const Exception & e) @@ -178,9 +169,7 @@ void OpenTelemetrySpanHolder::addAttribute(const Exception & e) if (trace_id == UUID()) return; - this->attribute_names.push_back("clickhouse.exception"); - this->attribute_values.push_back(getExceptionMessage(e, false)); - assert(this->attribute_names.size() == this->attribute_values.size()); + this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); } void OpenTelemetrySpanHolder::addAttribute(std::exception_ptr e) @@ -188,9 +177,7 @@ void OpenTelemetrySpanHolder::addAttribute(std::exception_ptr e) if (trace_id == UUID() || e == nullptr) return; - this->attribute_names.push_back("clickhouse.exception"); - this->attribute_values.push_back(getExceptionMessage(e, false)); - assert(this->attribute_names.size() == this->attribute_values.size()); + this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); } bool OpenTelemetryTraceContext::parseTraceparentHeader(const std::string & traceparent, diff --git a/src/Interpreters/OpenTelemetrySpanLog.h b/src/Interpreters/OpenTelemetrySpanLog.h index 677a283bb56..34f4765c8c4 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.h +++ b/src/Interpreters/OpenTelemetrySpanLog.h @@ -15,8 +15,7 @@ struct OpenTelemetrySpan std::string operation_name; UInt64 start_time_us; UInt64 finish_time_us; - Array attribute_names; - Array attribute_values; + Map attributes; // I don't understand how Links work, namely, which direction should they // point to, and how they are related with parent_span_id, so no Links for now. }; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 42db91f47c0..53d7fd0457a 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -384,8 +384,7 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) span.finish_time_us = std::chrono::duration_cast( std::chrono::system_clock::now().time_since_epoch()).count(); - span.attribute_names.push_back("clickhouse.thread_id"); - span.attribute_values.push_back(thread_id); + span.attributes.push_back(Tuple{"clickhouse.thread_id", toString(thread_id)}); opentelemetry_span_log->add(span); } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 4b328f0466e..33ceef4ad7c 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -301,28 +301,16 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr span.operation_name = "query"; span.start_time_us = current_time_us; span.finish_time_us = time_in_microseconds(std::chrono::system_clock::now()); - - /// Keep values synchronized to type enum in QueryLogElement::createBlock. - span.attribute_names.push_back("clickhouse.query_status"); - span.attribute_values.push_back("ExceptionBeforeStart"); - - span.attribute_names.push_back("db.statement"); - span.attribute_values.push_back(elem.query); - - span.attribute_names.push_back("clickhouse.query_id"); - span.attribute_values.push_back(elem.client_info.current_query_id); - - span.attribute_names.push_back("clickhouse.exception"); - span.attribute_values.push_back(elem.exception); - - span.attribute_names.push_back("clickhouse.exception_code"); - span.attribute_values.push_back(elem.exception_code); - + span.attributes.reserve(7); + span.attributes.push_back(Tuple{"clickhouse.query_status", "ExceptionBeforeStart"}); + span.attributes.push_back(Tuple{"db.statement", "elem.query"}); + span.attributes.push_back(Tuple{"clickhouse.query_id", "elem.client_info.current_query_id"}); + span.attributes.push_back(Tuple{"clickhouse.exception", "elem.exception"}); + span.attributes.push_back(Tuple{"clickhouse.exception_code", "elem.exception_code"}); + span.attributes.push_back(Tuple{"clickhouse.query_status", "ExceptionBeforeStart"}); if (!context->query_trace_context.tracestate.empty()) { - span.attribute_names.push_back("clickhouse.tracestate"); - span.attribute_values.push_back( - context->query_trace_context.tracestate); + span.attributes.push_back(Tuple{"clickhouse.tracestate", context->query_trace_context.tracestate}); } opentelemetry_span_log->add(span); @@ -956,20 +944,13 @@ static std::tuple executeQueryImpl( span.start_time_us = elem.query_start_time_microseconds; span.finish_time_us = time_in_microseconds(finish_time); - /// Keep values synchronized to type enum in QueryLogElement::createBlock. - span.attribute_names.push_back("clickhouse.query_status"); - span.attribute_values.push_back("QueryFinish"); - - span.attribute_names.push_back("db.statement"); - span.attribute_values.push_back(elem.query); - - span.attribute_names.push_back("clickhouse.query_id"); - span.attribute_values.push_back(elem.client_info.current_query_id); + span.attributes.reserve(4); + span.attributes.push_back(Tuple{"clickhouse.query_status", "QueryFinish"}); + span.attributes.push_back(Tuple{"db.statement", elem.query}); + span.attributes.push_back(Tuple{"clickhouse.query_id", elem.client_info.current_query_id}); if (!context->query_trace_context.tracestate.empty()) { - span.attribute_names.push_back("clickhouse.tracestate"); - span.attribute_values.push_back( - context->query_trace_context.tracestate); + span.attributes.push_back(Tuple{"clickhouse.tracestate", context->query_trace_context.tracestate}); } opentelemetry_span_log->add(span); From 707142e9393d04bc2f7c55b5b1e2a90a3e434f36 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 4 Jul 2022 16:08:10 +0200 Subject: [PATCH 164/627] Update test.py --- tests/integration/test_storage_rabbitmq/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 2b6c2867b88..18b1e9d974b 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2829,5 +2829,5 @@ def test_rabbitmq_address(rabbitmq_cluster): time.sleep(1) assert result.strip() == "kek" - instance.query("drop table rabbit_in sync") - instance.query("drop table rabbit_out sync") + instance2.query("drop table rabbit_in sync") + instance2.query("drop table rabbit_out sync") From 35e616514c5c0bf7fa90c0f6f6927ee499133f8d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 4 Jul 2022 16:26:35 +0200 Subject: [PATCH 165/627] Update 02352_interactive_queries_from_file.expect --- .../0_stateless/02352_interactive_queries_from_file.expect | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02352_interactive_queries_from_file.expect b/tests/queries/0_stateless/02352_interactive_queries_from_file.expect index 7823737d349..d15b804b0b9 100755 --- a/tests/queries/0_stateless/02352_interactive_queries_from_file.expect +++ b/tests/queries/0_stateless/02352_interactive_queries_from_file.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] +exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 log_user 0 set timeout 20 From 3eb97313d8a051ed253bfc6ebc08577883f93377 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 4 Jul 2022 14:46:10 +0000 Subject: [PATCH 166/627] Allow null modifier in columns declaration for table functions --- src/Parsers/ParserCreateQuery.cpp | 2 +- src/Parsers/ParserCreateQuery.h | 12 ++++++++++++ .../parseColumnsListForTableFunction.cpp | 4 ++-- .../02323_null_modifier_in_table_function.reference | 8 ++++++++ .../02323_null_modifier_in_table_function.sql | 5 +++++ 5 files changed, 28 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02323_null_modifier_in_table_function.reference create mode 100644 tests/queries/0_stateless/02323_null_modifier_in_table_function.sql diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 4b6ab67e22f..ce79ccf708a 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -90,7 +90,7 @@ bool ParserNameTypePairList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe bool ParserColumnDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) + return ParserList(std::make_unique(require_type, allow_null_modifiers, check_keywords_after_name), std::make_unique(TokenType::Comma), false) .parse(pos, node, expected); } diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index daf27c0dc67..4ca238d7bc1 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -333,9 +333,21 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E class ParserColumnDeclarationList : public IParserBase { +public: + explicit ParserColumnDeclarationList(bool require_type_ = true, bool allow_null_modifiers_ = false, bool check_keywords_after_name_ = false) + : require_type(require_type_) + , allow_null_modifiers(allow_null_modifiers_) + , check_keywords_after_name(check_keywords_after_name_) + { + } + protected: const char * getName() const override { return "column declaration list"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + const bool require_type = true; + const bool allow_null_modifiers = false; + const bool check_keywords_after_name = false; }; diff --git a/src/TableFunctions/parseColumnsListForTableFunction.cpp b/src/TableFunctions/parseColumnsListForTableFunction.cpp index 911f2ae80f4..d9ae92b067a 100644 --- a/src/TableFunctions/parseColumnsListForTableFunction.cpp +++ b/src/TableFunctions/parseColumnsListForTableFunction.cpp @@ -16,7 +16,7 @@ namespace ErrorCodes ColumnsDescription parseColumnsListFromString(const std::string & structure, ContextPtr context) { - ParserColumnDeclarationList parser; + ParserColumnDeclarationList parser(true, true); const Settings & settings = context->getSettingsRef(); ASTPtr columns_list_raw = parseQuery(parser, structure, "columns declaration list", settings.max_query_size, settings.max_parser_depth); @@ -30,7 +30,7 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, Con bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, ContextPtr context) { - ParserColumnDeclarationList parser; + ParserColumnDeclarationList parser(true, true); const Settings & settings = context->getSettingsRef(); String error; diff --git a/tests/queries/0_stateless/02323_null_modifier_in_table_function.reference b/tests/queries/0_stateless/02323_null_modifier_in_table_function.reference new file mode 100644 index 00000000000..316141783da --- /dev/null +++ b/tests/queries/0_stateless/02323_null_modifier_in_table_function.reference @@ -0,0 +1,8 @@ +1 +\N +0 +1 +0 +\N +1 +\N diff --git a/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql b/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql new file mode 100644 index 00000000000..33fb4c43fa2 --- /dev/null +++ b/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql @@ -0,0 +1,5 @@ +select * from values('x UInt8 NOT NULL', 1); +select * from values('x UInt8 NULL', NULL); +insert into function file(data_02323.tsv) select number % 2 ? number : NULL from numbers(3) settings engine_file_truncate_on_insert=1; +select * from file(data_02323.tsv, auto, 'x UInt32 NOT NULL'); +select * from file(data_02323.tsv, auto, 'x UInt32 NULL'); From f7e0e4af7a0d7a2f95e30f29cd969b3fb7d4ee61 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 4 Jul 2022 17:03:09 +0200 Subject: [PATCH 167/627] Update src/Parsers/ParserCreateQuery.h Co-authored-by: Dmitry Novik --- src/Parsers/ParserCreateQuery.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 4ca238d7bc1..b8b9722df71 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -345,9 +345,9 @@ protected: const char * getName() const override { return "column declaration list"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; - const bool require_type = true; - const bool allow_null_modifiers = false; - const bool check_keywords_after_name = false; + const bool require_type; + const bool allow_null_modifiers; + const bool check_keywords_after_name; }; From 65dd81586bd5d1ff2430deda2eecfd3d722749f5 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 4 Jul 2022 17:30:22 +0200 Subject: [PATCH 168/627] Update run.sh --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 168851d5060..c16efdca876 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -273,7 +273,7 @@ previous_release_tag=$(clickhouse-client --query="SELECT version()" | get_previo echo $previous_release_tag echo "Clone previous release repository" -git clone git@github.com:ClickHouse/ClickHouse.git --no-tags --progress --branch=$previous_release_tag --no-recurse-submodules --depth=1 previous_release_repository +git clone https://github.com/ClickHouse/ClickHouse.git --no-tags --progress --branch=$previous_release_tag --no-recurse-submodules --depth=1 previous_release_repository echo "Download previous release server" mkdir previous_release_package_folder From f871cbc7d5c9dfd32ca3875f354ae0c887ea1e28 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 4 Jul 2022 17:52:06 +0200 Subject: [PATCH 169/627] Update src/Interpreters/OpenTelemetrySpanLog.cpp Co-authored-by: Dmitry Novik --- src/Interpreters/OpenTelemetrySpanLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/OpenTelemetrySpanLog.cpp b/src/Interpreters/OpenTelemetrySpanLog.cpp index 79000aab375..2683a5f7955 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.cpp +++ b/src/Interpreters/OpenTelemetrySpanLog.cpp @@ -12,7 +12,7 @@ #include #include -#include "Core/Field.h" +#include namespace DB From 867b3aa873185b81e7292248b35d0cc4a02dd9fe Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 4 Jul 2022 18:01:24 +0200 Subject: [PATCH 170/627] fix --- src/Interpreters/executeQuery.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 33ceef4ad7c..11bea5ad3bd 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -301,13 +301,12 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr span.operation_name = "query"; span.start_time_us = current_time_us; span.finish_time_us = time_in_microseconds(std::chrono::system_clock::now()); - span.attributes.reserve(7); - span.attributes.push_back(Tuple{"clickhouse.query_status", "ExceptionBeforeStart"}); - span.attributes.push_back(Tuple{"db.statement", "elem.query"}); - span.attributes.push_back(Tuple{"clickhouse.query_id", "elem.client_info.current_query_id"}); - span.attributes.push_back(Tuple{"clickhouse.exception", "elem.exception"}); - span.attributes.push_back(Tuple{"clickhouse.exception_code", "elem.exception_code"}); + span.attributes.reserve(6); span.attributes.push_back(Tuple{"clickhouse.query_status", "ExceptionBeforeStart"}); + span.attributes.push_back(Tuple{"db.statement", elem.query}); + span.attributes.push_back(Tuple{"clickhouse.query_id", elem.client_info.current_query_id}); + span.attributes.push_back(Tuple{"clickhouse.exception", elem.exception}); + span.attributes.push_back(Tuple{"clickhouse.exception_code", elem.exception_code}); if (!context->query_trace_context.tracestate.empty()) { span.attributes.push_back(Tuple{"clickhouse.tracestate", context->query_trace_context.tracestate}); From 8bce6451c63e735f0769586ae4d848999d195c4f Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 17 Jan 2022 14:06:40 +0300 Subject: [PATCH 171/627] add method StorageEmbeddedRocksDB::getByKeys --- .../RocksDB/StorageEmbeddedRocksDB.cpp | 115 ++++++++++-------- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 15 +++ 2 files changed, 79 insertions(+), 51 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 1bfaabe142b..74d847d3bc9 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -176,6 +176,18 @@ static std::pair getFilterKeys( return std::make_pair(res, !matched_keys); } +template +static void fillColumns(const K & key, const V & value, size_t key_pos, const Block & header, MutableColumns & columns) +{ + ReadBufferFromString key_buffer(key); + ReadBufferFromString value_buffer(value); + size_t idx = 0; + for (const auto & elem : header) + { + elem.type->getDefaultSerialization()->deserializeBinary(*columns[idx], idx == key_pos ? key_buffer : value_buffer); + ++idx; + } +} class EmbeddedRocksDBSource : public ISource { @@ -222,45 +234,10 @@ public: Chunk generateWithKeys() { - if (it >= end) - return {}; - - size_t num_keys = end - begin; - - std::vector serialized_keys(num_keys); - std::vector slices_keys(num_keys); - const auto & sample_block = getPort().getHeader(); - - const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey()).type; - - size_t rows_processed = 0; - while (it < end && rows_processed < max_block_size) - { - WriteBufferFromString wb(serialized_keys[rows_processed]); - key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); - wb.finalize(); - slices_keys[rows_processed] = serialized_keys[rows_processed]; - - ++it; - ++rows_processed; - } - - MutableColumns columns = sample_block.cloneEmptyColumns(); - std::vector values; - auto statuses = storage.multiGet(slices_keys, values); - for (size_t i = 0; i < statuses.size(); ++i) - { - if (statuses[i].ok()) - { - ReadBufferFromString key_buffer(slices_keys[i]); - ReadBufferFromString value_buffer(values[i]); - fillColumns(key_buffer, value_buffer, columns); - } - } - - UInt64 num_rows = columns.at(0)->size(); - return Chunk(std::move(columns), num_rows); + Chunk result; + it = storage.getByKeys(it, end, sample_block, result, max_block_size); + return result; } Chunk generateFullScan() @@ -273,9 +250,7 @@ public: for (size_t rows = 0; iterator->Valid() && rows < max_block_size; ++rows, iterator->Next()) { - ReadBufferFromString key_buffer(iterator->key()); - ReadBufferFromString value_buffer(iterator->value()); - fillColumns(key_buffer, value_buffer, columns); + fillColumns(iterator->key(), iterator->value(), primary_key_pos, getPort().getHeader(), columns); } if (!iterator->status().ok()) @@ -287,16 +262,6 @@ public: return Chunk(block.getColumns(), block.rows()); } - void fillColumns(ReadBufferFromString & key_buffer, ReadBufferFromString & value_buffer, MutableColumns & columns) - { - size_t idx = 0; - for (const auto & elem : getPort().getHeader()) - { - elem.type->getDefaultSerialization()->deserializeBinary(*columns[idx], idx == primary_key_pos ? key_buffer : value_buffer); - ++idx; - } - } - private: const StorageEmbeddedRocksDB & storage; @@ -527,6 +492,54 @@ std::vector StorageEmbeddedRocksDB::multiGet(const std::vector< return rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); } +FieldVector::const_iterator StorageEmbeddedRocksDB::getByKeys( + FieldVector::const_iterator begin, + FieldVector::const_iterator end, + const Block & sample_block, + Chunk & result, + size_t max_block_size) const +{ + if (begin >= end) + return {}; + + size_t num_keys = end - begin; + + std::vector serialized_keys(num_keys); + std::vector slices_keys(num_keys); + + const auto & key_column_type = sample_block.getByName(getPrimaryKey()).type; + + size_t rows_processed = 0; + auto it = begin; + while (it < end && (max_block_size == 0 || rows_processed < max_block_size)) + { + WriteBufferFromString wb(serialized_keys[rows_processed]); + key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); + wb.finalize(); + slices_keys[rows_processed] = serialized_keys[rows_processed]; + + ++it; + ++rows_processed; + } + + MutableColumns columns = sample_block.cloneEmptyColumns(); + std::vector values; + auto statuses = multiGet(slices_keys, values); + + size_t primary_key_pos = sample_block.getPositionByName(getPrimaryKey()); + + for (size_t i = 0; i < statuses.size(); ++i) + { + if (statuses[i].ok()) + { + fillColumns(slices_keys[i], values[i], primary_key_pos, sample_block, columns); + } + } + + result = Chunk(std::move(columns), columns.at(0)->size()); + return it; +} + void registerStorageEmbeddedRocksDB(StorageFactory & factory) { StorageFactory::StorageFeatures features{ diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index ca6b4436c72..6a7a6bb4a52 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -58,6 +58,21 @@ public: std::vector multiGet(const std::vector & slices_keys, std::vector & values) const; const String & getPrimaryKey() const { return primary_key; } + FieldVector::const_iterator getByKeys( + FieldVector::const_iterator begin, + FieldVector::const_iterator end, + const Block & sample_block, + Chunk & result, + size_t max_block_size) const override; + +protected: + StorageEmbeddedRocksDB(const StorageID & table_id_, + const String & relative_data_path_, + const StorageInMemoryMetadata & metadata, + bool attach, + ContextPtr context_, + const String & primary_key_); + private: const String primary_key; using RocksDBPtr = std::unique_ptr; From 744c692be30e7c6db256daf545017f39ea1eafc8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 21 Jan 2022 08:36:36 +0300 Subject: [PATCH 172/627] wip key value join --- src/Interpreters/ExpressionAnalyzer.cpp | 31 +++++--- src/Interpreters/HashJoin.cpp | 29 +++++++- src/Interpreters/HashJoin.h | 70 +++++++++++++++++++ src/Interpreters/JoinedTables.cpp | 2 + src/Interpreters/TableJoin.cpp | 5 ++ src/Interpreters/TableJoin.h | 9 ++- src/Storages/IKVStorage.h | 28 ++++++++ .../RocksDB/StorageEmbeddedRocksDB.cpp | 29 ++++++-- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 6 +- 9 files changed, 190 insertions(+), 19 deletions(-) create mode 100644 src/Storages/IKVStorage.h diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 8d0c4dee023..b085de83b75 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -43,9 +43,12 @@ #include #include + #include #include #include +#include + #include #include @@ -64,6 +67,7 @@ #include #include #include +#include namespace DB { @@ -1066,24 +1070,24 @@ static ActionsDAGPtr createJoinedBlockActions(ContextPtr context, const TableJoi return ExpressionAnalyzer(expression_list, syntax_result, context).getActionsDAG(true, false); } -static std::shared_ptr chooseJoinAlgorithm(std::shared_ptr analyzed_join, const Block & sample_block, ContextPtr context) +static std::shared_ptr chooseJoinAlgorithm(std::shared_ptr analyzed_join, const Block & right_sample_block, ContextPtr context) { /// HashJoin with Dictionary optimisation - if (analyzed_join->tryInitDictJoin(sample_block, context)) - return std::make_shared(analyzed_join, sample_block); + if (analyzed_join->tryInitDictJoin(right_sample_block, context)) + return std::make_shared(analyzed_join, right_sample_block); bool allow_merge_join = analyzed_join->allowMergeJoin(); if (analyzed_join->forceHashJoin() || (analyzed_join->preferMergeJoin() && !allow_merge_join)) { if (analyzed_join->allowParallelHashJoin()) { - return std::make_shared(context, analyzed_join, context->getSettings().max_threads, sample_block); + return std::make_shared(context, analyzed_join, context->getSettings().max_threads, right_sample_block); } - return std::make_shared(analyzed_join, sample_block); + return std::make_shared(analyzed_join, right_sample_block); } else if (analyzed_join->forceMergeJoin() || (analyzed_join->preferMergeJoin() && allow_merge_join)) - return std::make_shared(analyzed_join, sample_block); - return std::make_shared(analyzed_join, sample_block); + return std::make_shared(analyzed_join, right_sample_block); + return std::make_shared(analyzed_join, right_sample_block); } static std::unique_ptr buildJoinedPlan( @@ -1160,9 +1164,18 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( return storage->getJoinLocked(analyzed_join, getContext()); } + const Block & right_sample_block = joined_plan->getCurrentDataStream().header; + if (auto storage = analyzed_join->getStorageKeyValue()) + { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "DirectKeyValueJoin"); + std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, {}); + /// TODO: (vdimir@) check that we can perform this join (keys and so on) + return std::make_shared(analyzed_join, right_sample_block, storage); + } + joined_plan = buildJoinedPlan(getContext(), join_element, *analyzed_join, query_options); - const ColumnsWithTypeAndName & right_columns = joined_plan->getCurrentDataStream().header.getColumnsWithTypeAndName(); + const ColumnsWithTypeAndName & right_columns = right_sample_block.getColumnsWithTypeAndName(); std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns); if (right_convert_actions) { @@ -1171,7 +1184,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( joined_plan->addStep(std::move(converting_step)); } - JoinPtr join = chooseJoinAlgorithm(analyzed_join, joined_plan->getCurrentDataStream().header, getContext()); + JoinPtr join = chooseJoinAlgorithm(analyzed_join, right_sample_block, getContext()); /// Do not make subquery for join over dictionary. if (analyzed_join->getDictionaryReader()) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 0cec83e964b..00f2a75435a 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -22,9 +22,10 @@ #include #include - +#include #include +#include #include #include @@ -2138,4 +2139,30 @@ const ColumnWithTypeAndName & HashJoin::rightAsofKeyColumn() const return savedBlockSample().getByName(table_join->getOnlyClause().key_names_right.back()); } +void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) +{ + const String & key_name = storage->getPrimaryKey(); + const auto & key_col = block.getByName(key_name); + FieldVector keys; + keys.reserve(key_col.column->size()); + for (size_t i = 0; i < key_col.column->size(); ++i) + { + key_col.column->get(i, keys.emplace_back()); + } + + Chunk joined_chunk; + NullMap null_map(keys.size(), 0); + storage->getByKeys(keys.begin(), keys.end(), right_sample_block, joined_chunk, &null_map, 0); + + // const auto & key_names_right = table_join->getOnlyClause().key_names_right; + + sample_block_with_columns_to_add = materializeBlock(right_sample_block); + + JoinCommon::createMissedColumns(sample_block_with_columns_to_add); + if (table_join->forceNullableRight()) + JoinCommon::convertColumnsToNullable(sample_block_with_columns_to_add); + +} + + } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 17d09b25ea1..21fb6512910 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -25,6 +25,9 @@ #include +#include +#include + namespace DB { @@ -419,4 +422,71 @@ private: bool overDictionary() const; }; +class DirectKeyValueJoin : public IJoin +{ +public: + DirectKeyValueJoin(std::shared_ptr table_join_, const Block & right_sample_block_, std::shared_ptr storage_) + : table_join(table_join_) + , storage(storage_) + , right_sample_block(right_sample_block_) + , log(&Poco::Logger::get("DirectKeyValueJoin")) + { + LOG_TRACE(log, "Using direct join"); + } + + virtual const TableJoin & getTableJoin() const override { return *table_join; } + + virtual bool addJoinedBlock(const Block &, bool) override + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + } + + virtual void checkTypesOfKeys(const Block &) const override + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + } + + /// Join the block with data from left hand of JOIN to the right hand data (that was previously built by calls to addJoinedBlock). + /// Could be called from different threads in parallel. + virtual void joinBlock(Block & block, std::shared_ptr &) override; + + virtual void setTotals(const Block &) override + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + } + + virtual const Block & getTotals() const override + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + } + + virtual size_t getTotalRowCount() const override + { + return 0; + } + + virtual size_t getTotalByteCount() const override + { + return 0; + } + + virtual bool alwaysReturnsEmptySet() const override { return false; } + + virtual bool isFilled() const override { return true; } + + virtual std::shared_ptr + getNonJoinedBlocks(const Block &, const Block &, UInt64) const override + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + } + +private: + std::shared_ptr table_join; + std::shared_ptr storage; + Block right_sample_block; + Block sample_block_with_columns_to_add; + Poco::Logger * log; + +}; + } diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 482a813bfef..0a476b0ea30 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -308,6 +308,8 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se table_join->setStorageJoin(storage_join); else if (auto storage_dict = std::dynamic_pointer_cast(storage); storage_dict) table_join->setStorageJoin(storage_dict); + else if (auto storage_kv = std::dynamic_pointer_cast(storage); storage_kv) + table_join->setStorageJoin(storage_kv); } } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 87502e5965e..c5c7958c883 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -685,6 +685,11 @@ ActionsDAGPtr TableJoin::applyKeyConvertToTable( return dag_stage1; } +void TableJoin::setStorageJoin(std::shared_ptr storage) +{ + right_kv_storage = storage; +} + void TableJoin::setStorageJoin(std::shared_ptr storage) { if (right_storage_dictionary) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 37e9417bde7..1cf68fcad56 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -9,7 +9,8 @@ #include #include #include -#include +#include + #include #include @@ -31,6 +32,7 @@ class Block; class DictionaryReader; class StorageJoin; class StorageDictionary; +class IKeyValueStorage; struct ColumnWithTypeAndName; using ColumnsWithTypeAndName = std::vector; @@ -148,6 +150,8 @@ private: std::shared_ptr right_storage_dictionary; std::shared_ptr dictionary_reader; + std::shared_ptr right_kv_storage; + Names requiredJoinedNames() const; /// Create converting actions and change key column names if required @@ -294,6 +298,7 @@ public: std::unordered_map leftToRightKeyRemap() const; + void setStorageJoin(std::shared_ptr storage); void setStorageJoin(std::shared_ptr storage); void setStorageJoin(std::shared_ptr storage); @@ -303,6 +308,8 @@ public: bool isSpecialStorage() const { return right_storage_dictionary || right_storage_join; } const DictionaryReader * getDictionaryReader() const { return dictionary_reader.get(); } + + std::shared_ptr getStorageKeyValue() { return right_kv_storage; } }; } diff --git a/src/Storages/IKVStorage.h b/src/Storages/IKVStorage.h new file mode 100644 index 00000000000..e84cf3f6a00 --- /dev/null +++ b/src/Storages/IKVStorage.h @@ -0,0 +1,28 @@ +#pragma once + +#include + +namespace DB +{ + + +/// Storage that support key-value requests +class IKeyValueStorage : public IStorage +{ +public: + using IStorage::IStorage; + + /// Get key name that supports key-value requests + virtual const String & getPrimaryKey() const = 0; + + /// Get data directly by keys + virtual FieldVector::const_iterator getByKeys( + FieldVector::const_iterator /* begin */, + FieldVector::const_iterator /* end */, + const Block & /* sample_block */, + Chunk & /* result */, + PaddedPODArray * /* null_map */, + size_t /* max_block_size */) const = 0; +}; + +} diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 74d847d3bc9..ca8c8239a00 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -29,12 +29,14 @@ #include #include #include +#include #include #include #include #include +#include #include #include @@ -181,11 +183,10 @@ static void fillColumns(const K & key, const V & value, size_t key_pos, const Bl { ReadBufferFromString key_buffer(key); ReadBufferFromString value_buffer(value); - size_t idx = 0; - for (const auto & elem : header) + for (size_t i = 0; i < header.columns(); ++i) { - elem.type->getDefaultSerialization()->deserializeBinary(*columns[idx], idx == key_pos ? key_buffer : value_buffer); - ++idx; + const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); + serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer); } } @@ -236,7 +237,7 @@ public: { const auto & sample_block = getPort().getHeader(); Chunk result; - it = storage.getByKeys(it, end, sample_block, result, max_block_size); + it = storage.getByKeys(it, end, sample_block, result, nullptr, max_block_size); return result; } @@ -286,7 +287,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, bool attach, ContextPtr context_, const String & primary_key_) - : IStorage(table_id_) + : IKeyValueStorage(table_id_) , WithContext(context_->getGlobalContext()) , primary_key{primary_key_} { @@ -497,6 +498,7 @@ FieldVector::const_iterator StorageEmbeddedRocksDB::getByKeys( FieldVector::const_iterator end, const Block & sample_block, Chunk & result, + PaddedPODArray * null_map, size_t max_block_size) const { if (begin >= end) @@ -534,6 +536,21 @@ FieldVector::const_iterator StorageEmbeddedRocksDB::getByKeys( { fillColumns(slices_keys[i], values[i], primary_key_pos, sample_block, columns); } + else if (statuses[i].IsNotFound()) + { + if (null_map) + { + (*null_map)[i] = 1; + for (size_t col_idx = 0; col_idx < sample_block.columns(); ++col_idx) + { + columns[col_idx]->insert(sample_block.getByPosition(col_idx).type->getDefault()); + } + } + } + else + { + throw DB::Exception(ErrorCodes::ROCKSDB_ERROR, "rocksdb error {}", statuses[i].ToString()); + } } result = Chunk(std::move(columns), columns.at(0)->size()); diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 6a7a6bb4a52..9947b752e50 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -18,7 +19,7 @@ namespace DB class Context; -class StorageEmbeddedRocksDB final : public IStorage, WithContext +class StorageEmbeddedRocksDB final : public IKeyValueStorage, WithContext { friend class EmbeddedRocksDBSink; public: @@ -56,13 +57,14 @@ public: std::shared_ptr getRocksDBStatistics() const; std::vector multiGet(const std::vector & slices_keys, std::vector & values) const; - const String & getPrimaryKey() const { return primary_key; } + const String & getPrimaryKey() const override { return primary_key; } FieldVector::const_iterator getByKeys( FieldVector::const_iterator begin, FieldVector::const_iterator end, const Block & sample_block, Chunk & result, + PaddedPODArray * null_map, size_t max_block_size) const override; protected: From 980000d04ced1100bd627d7c311e1a9b589e9936 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Mar 2022 16:46:32 +0000 Subject: [PATCH 173/627] Change getByKeys, add StorageEmbeddedRocksDB::getByKeysImpl --- src/Interpreters/ExpressionAnalyzer.cpp | 18 +-- src/Interpreters/HashJoin.cpp | 33 +++--- src/Interpreters/HashJoin.h | 7 +- src/Interpreters/TableJoin.cpp | 5 +- src/Interpreters/TableJoin.h | 2 +- src/Storages/IKVStorage.h | 11 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 108 ++++++++++++------ src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 14 ++- 8 files changed, 121 insertions(+), 77 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b085de83b75..dfa22de10e8 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1164,16 +1164,8 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( return storage->getJoinLocked(analyzed_join, getContext()); } - const Block & right_sample_block = joined_plan->getCurrentDataStream().header; - if (auto storage = analyzed_join->getStorageKeyValue()) - { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "DirectKeyValueJoin"); - std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, {}); - /// TODO: (vdimir@) check that we can perform this join (keys and so on) - return std::make_shared(analyzed_join, right_sample_block, storage); - } - joined_plan = buildJoinedPlan(getContext(), join_element, *analyzed_join, query_options); + const Block & right_sample_block = joined_plan->getCurrentDataStream().header; const ColumnsWithTypeAndName & right_columns = right_sample_block.getColumnsWithTypeAndName(); std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns); @@ -1184,6 +1176,14 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( joined_plan->addStep(std::move(converting_step)); } + if (auto storage = analyzed_join->getStorageKeyValue()) + { + Block rblock = right_sample_block; + joined_plan.reset(); + /// TODO: (vdimir@) check that we can perform this join (keys and so on) + return std::make_shared(analyzed_join, rblock, storage); + } + JoinPtr join = chooseJoinAlgorithm(analyzed_join, right_sample_block, getContext()); /// Do not make subquery for join over dictionary. diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 00f2a75435a..0ac26a5522c 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -2141,27 +2141,30 @@ const ColumnWithTypeAndName & HashJoin::rightAsofKeyColumn() const void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) { - const String & key_name = storage->getPrimaryKey(); - const auto & key_col = block.getByName(key_name); - FieldVector keys; - keys.reserve(key_col.column->size()); - for (size_t i = 0; i < key_col.column->size(); ++i) + if (!table_join->oneDisjunct() + || table_join->getOnlyClause().key_names_left.size() != 1 + || table_join->getOnlyClause().key_names_right.size() != 1) { - key_col.column->get(i, keys.emplace_back()); + throw DB::Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Not suppoted by direct JOIN"); } - Chunk joined_chunk; - NullMap null_map(keys.size(), 0); - storage->getByKeys(keys.begin(), keys.end(), right_sample_block, joined_chunk, &null_map, 0); + const auto & key_names_left = table_join->getOnlyClause().key_names_left; - // const auto & key_names_right = table_join->getOnlyClause().key_names_right; + const String & key_name = key_names_left[0]; + const auto & key_col = block.getByName(key_name); + if (!key_col.column) + return; - sample_block_with_columns_to_add = materializeBlock(right_sample_block); - - JoinCommon::createMissedColumns(sample_block_with_columns_to_add); - if (table_join->forceNullableRight()) - JoinCommon::convertColumnsToNullable(sample_block_with_columns_to_add); + NullMap null_map(key_col.column->size(), 0); + Chunk joined_chunk = storage->getByKeys(key_col, right_sample_block, &null_map); + Columns cols = joined_chunk.detachColumns(); + for (size_t i = 0; i < cols.size(); ++i) + { + ColumnWithTypeAndName col = right_sample_block.getByPosition(i); + col.column = std::move(cols[i]); + block.insert(std::move(col)); + } } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 21fb6512910..db42c6e099e 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -443,7 +443,7 @@ public: virtual void checkTypesOfKeys(const Block &) const override { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + // throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); } /// Join the block with data from left hand of JOIN to the right hand data (that was previously built by calls to addJoinedBlock). @@ -457,7 +457,8 @@ public: virtual const Block & getTotals() const override { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + static const Block totals; + return totals; } virtual size_t getTotalRowCount() const override @@ -477,7 +478,7 @@ public: virtual std::shared_ptr getNonJoinedBlocks(const Block &, const Block &, UInt64) const override { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + return nullptr; } private: diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index c5c7958c883..902965e1629 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -513,6 +513,7 @@ TableJoin::createConvertingActions(const ColumnsWithTypeAndName & left_sample_co { if (dag) { + /// Just debug message std::vector input_cols; for (const auto & col : dag->getRequiredColumns()) input_cols.push_back(col.name + ": " + col.type->getName()); @@ -591,7 +592,7 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig catch (DB::Exception & ex) { throw DB::Exception(ErrorCodes::TYPE_MISMATCH, - "Can't infer common type for joined columns: {}: {} at left, {}: {} at right. {}", + "Can't infer common type for joined columns: {}: {} at left, {}: {} at right ({})", left_key_name, ltype->second->getName(), right_key_name, rtype->second->getName(), ex.message()); @@ -599,7 +600,7 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig if (!allow_right && !common_type->equals(*rtype->second)) { throw DB::Exception(ErrorCodes::TYPE_MISMATCH, - "Can't change type for right table: {}: {} -> {}.", + "Can't change type for right table: {}: {} -> {}", right_key_name, rtype->second->getName(), common_type->getName()); } left_type_map[left_key_name] = right_type_map[right_key_name] = common_type; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 1cf68fcad56..020b3c51198 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -306,7 +306,7 @@ public: bool tryInitDictJoin(const Block & sample_block, ContextPtr context); - bool isSpecialStorage() const { return right_storage_dictionary || right_storage_join; } + bool isSpecialStorage() const { return right_storage_dictionary || right_storage_join || right_kv_storage; } const DictionaryReader * getDictionaryReader() const { return dictionary_reader.get(); } std::shared_ptr getStorageKeyValue() { return right_kv_storage; } diff --git a/src/Storages/IKVStorage.h b/src/Storages/IKVStorage.h index e84cf3f6a00..994a228405b 100644 --- a/src/Storages/IKVStorage.h +++ b/src/Storages/IKVStorage.h @@ -16,13 +16,10 @@ public: virtual const String & getPrimaryKey() const = 0; /// Get data directly by keys - virtual FieldVector::const_iterator getByKeys( - FieldVector::const_iterator /* begin */, - FieldVector::const_iterator /* end */, - const Block & /* sample_block */, - Chunk & /* result */, - PaddedPODArray * /* null_map */, - size_t /* max_block_size */) const = 0; + virtual Chunk getByKeys( + const ColumnWithTypeAndName & col, + const Block & sample_block, + PaddedPODArray * null_map) const = 0; }; } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index ca8c8239a00..f4f4b694874 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -190,7 +190,55 @@ static void fillColumns(const K & key, const V & value, size_t key_pos, const Bl } } -class EmbeddedRocksDBSource : public ISource + +static std::vector getSlicedKeys( + FieldVector::const_iterator & it, + FieldVector::const_iterator end, + DataTypePtr key_column_type, + size_t max_block_size) +{ + size_t num_keys = end - it; + + std::vector slices_keys(num_keys); + + size_t rows_processed = 0; + while (it < end && (max_block_size == 0 || rows_processed < max_block_size)) + { + std::string serialized_key; + WriteBufferFromString wb(serialized_key); + key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); + wb.finalize(); + slices_keys[rows_processed] = serialized_key; + + ++it; + ++rows_processed; + } + return slices_keys; +} + +static std::vector getSlicedKeys(const ColumnWithTypeAndName & col) +{ + if (!col.column) + return {}; + + size_t num_keys = col.column->size(); + + std::vector slices_keys(num_keys); + for (size_t i = 0; i < num_keys; ++i) + { + std::string serialized_key; + WriteBufferFromString wb(serialized_key); + Field field; + col.column->get(i, field); + /// TODO(@vdimir): use serializeBinaryBulk + col.type->getDefaultSerialization()->serializeBinary(field, wb); + wb.finalize(); + slices_keys[i] = serialized_key; + } + return slices_keys; +} + +class EmbeddedRocksDBSource : public SourceWithProgress { public: EmbeddedRocksDBSource( @@ -236,9 +284,15 @@ public: Chunk generateWithKeys() { const auto & sample_block = getPort().getHeader(); - Chunk result; - it = storage.getByKeys(it, end, sample_block, result, nullptr, max_block_size); - return result; + if (it >= end) + { + it = {}; + return {}; + } + + const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey()).type; + auto slices_keys = getSlicedKeys(it, end, key_column_type, max_block_size); + return storage.getByKeysImpl(slices_keys, sample_block, nullptr); } Chunk generateFullScan() @@ -493,43 +547,29 @@ std::vector StorageEmbeddedRocksDB::multiGet(const std::vector< return rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); } -FieldVector::const_iterator StorageEmbeddedRocksDB::getByKeys( - FieldVector::const_iterator begin, - FieldVector::const_iterator end, +Chunk StorageEmbeddedRocksDB::getByKeys( + const ColumnWithTypeAndName & col, const Block & sample_block, - Chunk & result, - PaddedPODArray * null_map, - size_t max_block_size) const + PaddedPODArray * null_map) const { - if (begin >= end) + auto sliced_keys = getSlicedKeys(col); + return getByKeysImpl(sliced_keys, sample_block, null_map); +} + +Chunk StorageEmbeddedRocksDB::getByKeysImpl( + const std::vector & slices_keys, + const Block & sample_block, + PaddedPODArray * null_map) const +{ + if (!sample_block) return {}; - size_t num_keys = end - begin; - - std::vector serialized_keys(num_keys); - std::vector slices_keys(num_keys); - - const auto & key_column_type = sample_block.getByName(getPrimaryKey()).type; - - size_t rows_processed = 0; - auto it = begin; - while (it < end && (max_block_size == 0 || rows_processed < max_block_size)) - { - WriteBufferFromString wb(serialized_keys[rows_processed]); - key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); - wb.finalize(); - slices_keys[rows_processed] = serialized_keys[rows_processed]; - - ++it; - ++rows_processed; - } - - MutableColumns columns = sample_block.cloneEmptyColumns(); std::vector values; auto statuses = multiGet(slices_keys, values); size_t primary_key_pos = sample_block.getPositionByName(getPrimaryKey()); + MutableColumns columns = sample_block.cloneEmptyColumns(); for (size_t i = 0; i < statuses.size(); ++i) { if (statuses[i].ok()) @@ -553,8 +593,8 @@ FieldVector::const_iterator StorageEmbeddedRocksDB::getByKeys( } } - result = Chunk(std::move(columns), columns.at(0)->size()); - return it; + size_t num_rows = columns.at(0)->size(); + return Chunk(std::move(columns), num_rows); } void registerStorageEmbeddedRocksDB(StorageFactory & factory) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 9947b752e50..7829eb696b1 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -59,13 +59,15 @@ public: std::vector multiGet(const std::vector & slices_keys, std::vector & values) const; const String & getPrimaryKey() const override { return primary_key; } - FieldVector::const_iterator getByKeys( - FieldVector::const_iterator begin, - FieldVector::const_iterator end, + Chunk getByKeys( + const ColumnWithTypeAndName & col, const Block & sample_block, - Chunk & result, - PaddedPODArray * null_map, - size_t max_block_size) const override; + PaddedPODArray * null_map) const override; + + Chunk getByKeysImpl( + const std::vector & slices_keys, + const Block & sample_block, + PaddedPODArray * null_map) const; protected: StorageEmbeddedRocksDB(const StorageID & table_id_, From 085e70c7a376087c2f75144b587b95bcf2d0a5bb Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 7 Mar 2022 11:06:53 +0000 Subject: [PATCH 174/627] wip left kvjoin --- src/Interpreters/HashJoin.cpp | 16 ++++++++++++++-- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 0ac26a5522c..2ce4381ceb4 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -2151,11 +2151,11 @@ void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) const auto & key_names_left = table_join->getOnlyClause().key_names_left; const String & key_name = key_names_left[0]; - const auto & key_col = block.getByName(key_name); + const ColumnWithTypeAndName & key_col = block.getByName(key_name); if (!key_col.column) return; - NullMap null_map(key_col.column->size(), 0); + NullMap null_map(key_col.column->size(), 1); Chunk joined_chunk = storage->getByKeys(key_col, right_sample_block, &null_map); Columns cols = joined_chunk.detachColumns(); @@ -2165,6 +2165,18 @@ void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) col.column = std::move(cols[i]); block.insert(std::move(col)); } + + if (!isLeftOrFull(table_join->kind())) + { + MutableColumns dst_columns = block.mutateColumns(); + + for (auto & col : dst_columns) + { + col = IColumn::mutate(col->filter(null_map, -1)); + } + block.setColumns(std::move(dst_columns)); + } + } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index f4f4b694874..e9bffe25cf3 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -580,7 +580,7 @@ Chunk StorageEmbeddedRocksDB::getByKeysImpl( { if (null_map) { - (*null_map)[i] = 1; + (*null_map)[i] = 0; for (size_t col_idx = 0; col_idx < sample_block.columns(); ++col_idx) { columns[col_idx]->insert(sample_block.getByPosition(col_idx).type->getDefault()); From 5ed0911ff47ef59a264089b8eaf4872a5425e670 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 16 Mar 2022 17:53:36 +0000 Subject: [PATCH 175/627] Fix rocksdb::getSlicedKeys --- .../RocksDB/StorageEmbeddedRocksDB.cpp | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index e9bffe25cf3..3c9946a3ce1 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -195,16 +195,17 @@ static std::vector getSlicedKeys( FieldVector::const_iterator & it, FieldVector::const_iterator end, DataTypePtr key_column_type, - size_t max_block_size) + size_t max_block_size, + std::vector & holder) { size_t num_keys = end - it; - + holder.reserve(num_keys); std::vector slices_keys(num_keys); size_t rows_processed = 0; while (it < end && (max_block_size == 0 || rows_processed < max_block_size)) { - std::string serialized_key; + std::string & serialized_key = holder.emplace_back(); WriteBufferFromString wb(serialized_key); key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); wb.finalize(); @@ -216,17 +217,18 @@ static std::vector getSlicedKeys( return slices_keys; } -static std::vector getSlicedKeys(const ColumnWithTypeAndName & col) +static std::vector getSlicedKeys(const ColumnWithTypeAndName & col, std::vector & holder) { if (!col.column) return {}; size_t num_keys = col.column->size(); + holder.reserve(num_keys); std::vector slices_keys(num_keys); for (size_t i = 0; i < num_keys; ++i) { - std::string serialized_key; + std::string & serialized_key = holder.emplace_back(); WriteBufferFromString wb(serialized_key); Field field; col.column->get(i, field); @@ -291,7 +293,8 @@ public: } const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey()).type; - auto slices_keys = getSlicedKeys(it, end, key_column_type, max_block_size); + std::vector holder; + auto slices_keys = getSlicedKeys(it, end, key_column_type, max_block_size, holder); return storage.getByKeysImpl(slices_keys, sample_block, nullptr); } @@ -552,7 +555,12 @@ Chunk StorageEmbeddedRocksDB::getByKeys( const Block & sample_block, PaddedPODArray * null_map) const { - auto sliced_keys = getSlicedKeys(col); + std::vector holder; + auto sliced_keys = getSlicedKeys(col, holder); + + if (sliced_keys.size() != col.column->size()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", sliced_keys.size(), col.column->size()); + return getByKeysImpl(sliced_keys, sample_block, null_map); } From 0dfd400f518c73702bce34545e8c03cbba7e6096 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 17 Mar 2022 11:38:49 +0000 Subject: [PATCH 176/627] check conditions before using direct join --- src/Interpreters/ExpressionAnalyzer.cpp | 34 ++++++++++++++++++++++--- src/Parsers/ASTTablesInSelectQuery.h | 1 + 2 files changed, 31 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index dfa22de10e8..fd2eff177e8 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1143,6 +1143,34 @@ static std::unique_ptr buildJoinedPlan( return joined_plan; } +std::shared_ptr tryKeyValueJoin(std::shared_ptr analyzed_join, const Block & right_sample_block) +{ + if (!isInnerOrLeft(analyzed_join->kind())) + return nullptr; + + if (analyzed_join->strictness() != ASTTableJoin::Strictness::All) + return nullptr; + + auto storage = analyzed_join->getStorageKeyValue(); + if (!storage) + return nullptr; + + const auto & clauses = analyzed_join->getClauses(); + bool only_one_key = clauses.size() == 1 && + clauses[0].key_names_left.size() == 1 && + clauses[0].key_names_right.size() == 1 && + !clauses[0].on_filter_condition_left && + !clauses[0].on_filter_condition_right; + + if (!only_one_key) + return nullptr; + + if (storage->getPrimaryKey() != clauses[0].key_names_right[0]) + return nullptr; + + return std::make_shared(analyzed_join, right_sample_block, storage); +} + JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( const ASTTablesInSelectQueryElement & join_element, const ColumnsWithTypeAndName & left_columns, @@ -1176,12 +1204,10 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( joined_plan->addStep(std::move(converting_step)); } - if (auto storage = analyzed_join->getStorageKeyValue()) + if (JoinPtr kvjoin = tryKeyValueJoin(analyzed_join, right_sample_block)) { - Block rblock = right_sample_block; joined_plan.reset(); - /// TODO: (vdimir@) check that we can perform this join (keys and so on) - return std::make_shared(analyzed_join, rblock, storage); + return kvjoin; } JoinPtr join = chooseJoinAlgorithm(analyzed_join, right_sample_block, getContext()); diff --git a/src/Parsers/ASTTablesInSelectQuery.h b/src/Parsers/ASTTablesInSelectQuery.h index 500337936d1..2b07d31fb7d 100644 --- a/src/Parsers/ASTTablesInSelectQuery.h +++ b/src/Parsers/ASTTablesInSelectQuery.h @@ -121,6 +121,7 @@ inline bool isCrossOrComma(ASTTableJoin::Kind kind) { return kind == ASTTableJoi inline bool isRightOrFull(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Right || kind == ASTTableJoin::Kind::Full; } inline bool isLeftOrFull(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Left || kind == ASTTableJoin::Kind::Full; } inline bool isInnerOrRight(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Right; } +inline bool isInnerOrLeft(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Left; } /// Specification of ARRAY JOIN. From 51a6c347ade7d31661f9a10bc860665fa640a97c Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 17 Mar 2022 12:51:44 +0000 Subject: [PATCH 177/627] make join_alorithm settings with multiple choices --- src/Core/SettingsEnums.cpp | 5 +++-- src/Core/SettingsEnums.h | 3 ++- src/Interpreters/ExpressionAnalyzer.cpp | 23 +++++++++++++++++------ src/Interpreters/TableJoin.h | 15 ++++++++++----- 4 files changed, 32 insertions(+), 14 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index bff1971bad9..66d38337243 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -30,12 +30,13 @@ IMPLEMENT_SETTING_ENUM(JoinStrictness, ErrorCodes::UNKNOWN_JOIN, {"ANY", JoinStrictness::ANY}}) -IMPLEMENT_SETTING_ENUM(JoinAlgorithm, ErrorCodes::UNKNOWN_JOIN, +IMPLEMENT_SETTING_MULTI_ENUM(JoinAlgorithm, ErrorCodes::UNKNOWN_JOIN, {{"auto", JoinAlgorithm::AUTO}, {"hash", JoinAlgorithm::HASH}, {"partial_merge", JoinAlgorithm::PARTIAL_MERGE}, {"prefer_partial_merge", JoinAlgorithm::PREFER_PARTIAL_MERGE}, - {"parallel_hash", JoinAlgorithm::PARALLEL_HASH}}) + {"parallel_hash", JoinAlgorithm::PARALLEL_HASH}, + {"direct", JoinAlgorithm::DIRECT}}) IMPLEMENT_SETTING_ENUM(TotalsMode, ErrorCodes::UNKNOWN_TOTALS_MODE, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 83a65f2a320..bb9311a808b 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -43,9 +43,10 @@ enum class JoinAlgorithm PARTIAL_MERGE, PREFER_PARTIAL_MERGE, PARALLEL_HASH, + DIRECT, }; -DECLARE_SETTING_ENUM(JoinAlgorithm) +DECLARE_SETTING_MULTI_ENUM(JoinAlgorithm) /// Which rows should be included in TOTALS. diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index fd2eff177e8..abd3c910aea 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -43,7 +43,7 @@ #include #include - +#include #include #include #include @@ -85,6 +85,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int UNKNOWN_IDENTIFIER; extern const int UNKNOWN_TYPE_OF_AST_NODE; + extern const int UNSUPPORTED_METHOD; } namespace @@ -1145,15 +1146,25 @@ static std::unique_ptr buildJoinedPlan( std::shared_ptr tryKeyValueJoin(std::shared_ptr analyzed_join, const Block & right_sample_block) { - if (!isInnerOrLeft(analyzed_join->kind())) + auto error_or_null = [&](const String & msg) + { + if (analyzed_join->isForcedAlgorithm(JoinAlgorithm::DIRECT)) + throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, "Can't use '{}' join algorithm: {}", JoinAlgorithm::DIRECT, msg); return nullptr; + }; - if (analyzed_join->strictness() != ASTTableJoin::Strictness::All) + if (!analyzed_join->isAllowedAlgorithm(JoinAlgorithm::DIRECT)) return nullptr; auto storage = analyzed_join->getStorageKeyValue(); if (!storage) - return nullptr; + return error_or_null("unsupported storage"); + + if (!isInnerOrLeft(analyzed_join->kind())) + return error_or_null("illegal kind"); + + if (analyzed_join->strictness() != ASTTableJoin::Strictness::All) + return error_or_null("illegal strictness"); const auto & clauses = analyzed_join->getClauses(); bool only_one_key = clauses.size() == 1 && @@ -1163,10 +1174,10 @@ std::shared_ptr tryKeyValueJoin(std::shared_ptr a !clauses[0].on_filter_condition_right; if (!only_one_key) - return nullptr; + return error_or_null("multiple keys is not allowed"); if (storage->getPrimaryKey() != clauses[0].key_names_right[0]) - return nullptr; + return error_or_null("key doesn't match storage"); return std::make_shared(analyzed_join, right_sample_block, storage); } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 020b3c51198..f2ff6076f9c 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -109,7 +109,7 @@ private: const size_t default_max_bytes = 0; const bool join_use_nulls = false; const size_t max_joined_block_rows = 0; - JoinAlgorithm join_algorithm = JoinAlgorithm::AUTO; + MultiEnum join_algorithm = MultiEnum(JoinAlgorithm::AUTO); const size_t partial_merge_join_rows_in_right_blocks = 0; const size_t partial_merge_join_left_table_buffer_bytes = 0; const size_t max_files_to_merge = 0; @@ -193,13 +193,18 @@ public: const SizeLimits & sizeLimits() const { return size_limits; } VolumePtr getTemporaryVolume() { return tmp_volume; } bool allowMergeJoin() const; - bool preferMergeJoin() const { return join_algorithm == JoinAlgorithm::PREFER_PARTIAL_MERGE; } - bool forceMergeJoin() const { return join_algorithm == JoinAlgorithm::PARTIAL_MERGE; } - bool allowParallelHashJoin() const; + + bool isAllowedAlgorithm(JoinAlgorithm val) const { return join_algorithm.isSet(val) || join_algorithm.isSet(JoinAlgorithm::AUTO); } + bool isForcedAlgorithm(JoinAlgorithm val) const { return join_algorithm == MultiEnum(val); } + + bool preferMergeJoin() const { return join_algorithm == MultiEnum(JoinAlgorithm::PREFER_PARTIAL_MERGE); } + bool forceMergeJoin() const { return join_algorithm == MultiEnum(JoinAlgorithm::PARTIAL_MERGE); } bool forceHashJoin() const { /// HashJoin always used for DictJoin - return dictionary_reader || join_algorithm == JoinAlgorithm::HASH || join_algorithm == JoinAlgorithm::PARALLEL_HASH; + return dictionary_reader + || join_algorithm == MultiEnum(JoinAlgorithm::HASH) + || join_algorithm == MultiEnum(JoinAlgorithm::PARALLEL_HASH); } bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(table_join.kind); } From ad06dd3fe9acb373a468d25952a91c89850adb03 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 17 Mar 2022 12:52:55 +0000 Subject: [PATCH 178/627] add 02241_join_rocksdb --- .../0_stateless/02241_join_rocksdb.reference | 34 +++++++++++++++ .../0_stateless/02241_join_rocksdb.sql.j2 | 41 +++++++++++++++++++ 2 files changed, 75 insertions(+) create mode 100644 tests/queries/0_stateless/02241_join_rocksdb.reference create mode 100644 tests/queries/0_stateless/02241_join_rocksdb.sql.j2 diff --git a/tests/queries/0_stateless/02241_join_rocksdb.reference b/tests/queries/0_stateless/02241_join_rocksdb.reference new file mode 100644 index 00000000000..4600557506b --- /dev/null +++ b/tests/queries/0_stateless/02241_join_rocksdb.reference @@ -0,0 +1,34 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02241_join_rocksdb.sql.j2 b/tests/queries/0_stateless/02241_join_rocksdb.sql.j2 new file mode 100644 index 00000000000..69cff73598a --- /dev/null +++ b/tests/queries/0_stateless/02241_join_rocksdb.sql.j2 @@ -0,0 +1,41 @@ +SET join_algorithm = 'direct'; + +{% for table_size in [10, 65555, 100000] -%} + +CREATE OR REPLACE TABLE rdb (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY (key); +INSERT INTO rdb SELECT (sipHash64(number) % {{ table_size }}) as key, ('val' || toString(key)) as value FROM numbers_mt({{ table_size }}); + +{% for block_size in [10, 11, 128, 129, 65505, 65506, 70000] -%} + +{% if block_size * 5000 > table_size -%} + +SET max_block_size = {{ block_size }}; + +SELECT count() == (SELECT count() FROM rdb WHERE key < {{ table_size // 2 }}) +FROM (SELECT number as k FROM numbers_mt({{ table_size // 2 }})) as t1 +INNER JOIN rdb +ON rdb.key == t1.k; + +SELECT count() == (SELECT count() FROM rdb WHERE key < {{ table_size + table_size // 4 + 1 }}) +FROM (SELECT number as k FROM numbers_mt({{ table_size + table_size // 4 + 1 }})) as t1 +INNER JOIN rdb +ON rdb.key == t1.k; + +{% endif -%} + +{% endfor -%} +{% endfor -%} + +SELECT count() +FROM (SELECT number as k FROM numbers_mt(10)) as t1 +INNER JOIN rdb +ON rdb.key + 1 == t1.k; -- { serverError UNSUPPORTED_METHOD } + +SELECT count() +FROM (SELECT number as k FROM numbers_mt(10)) as t1 +INNER JOIN (SELECT * FROM rdb) AS rdb +ON rdb.key == t1.k; -- { serverError UNSUPPORTED_METHOD } + +DROP TABLE rdb; + + From 8359753a8569b9da30e86b83661b34d10c81033f Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 21 Mar 2022 11:54:32 +0000 Subject: [PATCH 179/627] Move DirectJoin to separate file --- src/Interpreters/DirectJoin.cpp | 53 +++++++++++ src/Interpreters/DirectJoin.h | 88 +++++++++++++++++++ src/Interpreters/ExpressionAnalyzer.cpp | 1 + src/Interpreters/HashJoin.cpp | 41 --------- src/Interpreters/HashJoin.h | 68 -------------- .../RocksDB/StorageEmbeddedRocksDB.cpp | 3 +- 6 files changed, 144 insertions(+), 110 deletions(-) create mode 100644 src/Interpreters/DirectJoin.cpp create mode 100644 src/Interpreters/DirectJoin.h diff --git a/src/Interpreters/DirectJoin.cpp b/src/Interpreters/DirectJoin.cpp new file mode 100644 index 00000000000..361dd6c4959 --- /dev/null +++ b/src/Interpreters/DirectJoin.cpp @@ -0,0 +1,53 @@ +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int UNSUPPORTED_JOIN_KEYS; +} + +void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) +{ + if (!table_join->oneDisjunct() + || table_join->getOnlyClause().key_names_left.size() != 1 + || table_join->getOnlyClause().key_names_right.size() != 1) + { + throw DB::Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Not supported by direct JOIN"); + } + + const auto & key_names_left = table_join->getOnlyClause().key_names_left; + + const String & key_name = key_names_left[0]; + const ColumnWithTypeAndName & key_col = block.getByName(key_name); + if (!key_col.column) + return; + + NullMap null_map(key_col.column->size(), 1); + Chunk joined_chunk = storage->getByKeys(key_col, right_sample_block, &null_map); + + Columns cols = joined_chunk.detachColumns(); + for (size_t i = 0; i < cols.size(); ++i) + { + ColumnWithTypeAndName col = right_sample_block.getByPosition(i); + col.column = std::move(cols[i]); + block.insert(std::move(col)); + } + + if (!isLeftOrFull(table_join->kind())) + { + MutableColumns dst_columns = block.mutateColumns(); + + for (auto & col : dst_columns) + { + col = IColumn::mutate(col->filter(null_map, -1)); + } + block.setColumns(std::move(dst_columns)); + } +} + +} diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h new file mode 100644 index 00000000000..6b0c14f84d6 --- /dev/null +++ b/src/Interpreters/DirectJoin.h @@ -0,0 +1,88 @@ +#pragma once + +#include + +#include + +#include +#include + +#include + +#include +#include + +namespace DB +{ + +class NotJoinedBlocks; + +class DirectKeyValueJoin : public IJoin +{ +public: + DirectKeyValueJoin(std::shared_ptr table_join_, const Block & right_sample_block_, std::shared_ptr storage_) + : table_join(table_join_) + , storage(storage_) + , right_sample_block(right_sample_block_) + , log(&Poco::Logger::get("DirectKeyValueJoin")) + { + LOG_TRACE(log, "Using direct join"); + } + + virtual const TableJoin & getTableJoin() const override { return *table_join; } + + virtual bool addJoinedBlock(const Block &, bool) override + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + } + + virtual void checkTypesOfKeys(const Block &) const override + { + // throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + } + + /// Join the block with data from left hand of JOIN to the right hand data (that was previously built by calls to addJoinedBlock). + /// Could be called from different threads in parallel. + virtual void joinBlock(Block & block, std::shared_ptr &) override; + + virtual void setTotals(const Block &) override + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + } + + virtual const Block & getTotals() const override + { + static const Block totals; + return totals; + } + + virtual size_t getTotalRowCount() const override + { + return 0; + } + + virtual size_t getTotalByteCount() const override + { + return 0; + } + + virtual bool alwaysReturnsEmptySet() const override { return false; } + + virtual bool isFilled() const override { return true; } + + virtual std::shared_ptr + getNonJoinedBlocks(const Block &, const Block &, UInt64) const override + { + return nullptr; + } + +private: + std::shared_ptr table_join; + std::shared_ptr storage; + Block right_sample_block; + Block sample_block_with_columns_to_add; + Poco::Logger * log; + +}; + +} diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index abd3c910aea..f436eb24dc4 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 2ce4381ceb4..a5bbcf9a373 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -2139,45 +2139,4 @@ const ColumnWithTypeAndName & HashJoin::rightAsofKeyColumn() const return savedBlockSample().getByName(table_join->getOnlyClause().key_names_right.back()); } -void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) -{ - if (!table_join->oneDisjunct() - || table_join->getOnlyClause().key_names_left.size() != 1 - || table_join->getOnlyClause().key_names_right.size() != 1) - { - throw DB::Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Not suppoted by direct JOIN"); - } - - const auto & key_names_left = table_join->getOnlyClause().key_names_left; - - const String & key_name = key_names_left[0]; - const ColumnWithTypeAndName & key_col = block.getByName(key_name); - if (!key_col.column) - return; - - NullMap null_map(key_col.column->size(), 1); - Chunk joined_chunk = storage->getByKeys(key_col, right_sample_block, &null_map); - - Columns cols = joined_chunk.detachColumns(); - for (size_t i = 0; i < cols.size(); ++i) - { - ColumnWithTypeAndName col = right_sample_block.getByPosition(i); - col.column = std::move(cols[i]); - block.insert(std::move(col)); - } - - if (!isLeftOrFull(table_join->kind())) - { - MutableColumns dst_columns = block.mutateColumns(); - - for (auto & col : dst_columns) - { - col = IColumn::mutate(col->filter(null_map, -1)); - } - block.setColumns(std::move(dst_columns)); - } - -} - - } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index db42c6e099e..c94b580c6b2 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -422,72 +422,4 @@ private: bool overDictionary() const; }; -class DirectKeyValueJoin : public IJoin -{ -public: - DirectKeyValueJoin(std::shared_ptr table_join_, const Block & right_sample_block_, std::shared_ptr storage_) - : table_join(table_join_) - , storage(storage_) - , right_sample_block(right_sample_block_) - , log(&Poco::Logger::get("DirectKeyValueJoin")) - { - LOG_TRACE(log, "Using direct join"); - } - - virtual const TableJoin & getTableJoin() const override { return *table_join; } - - virtual bool addJoinedBlock(const Block &, bool) override - { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); - } - - virtual void checkTypesOfKeys(const Block &) const override - { - // throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); - } - - /// Join the block with data from left hand of JOIN to the right hand data (that was previously built by calls to addJoinedBlock). - /// Could be called from different threads in parallel. - virtual void joinBlock(Block & block, std::shared_ptr &) override; - - virtual void setTotals(const Block &) override - { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); - } - - virtual const Block & getTotals() const override - { - static const Block totals; - return totals; - } - - virtual size_t getTotalRowCount() const override - { - return 0; - } - - virtual size_t getTotalByteCount() const override - { - return 0; - } - - virtual bool alwaysReturnsEmptySet() const override { return false; } - - virtual bool isFilled() const override { return true; } - - virtual std::shared_ptr - getNonJoinedBlocks(const Block &, const Block &, UInt64) const override - { - return nullptr; - } - -private: - std::shared_ptr table_join; - std::shared_ptr storage; - Block right_sample_block; - Block sample_block_with_columns_to_add; - Poco::Logger * log; - -}; - } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 3c9946a3ce1..f7298a51226 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -49,8 +49,9 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int ROCKSDB_ERROR; + extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ROCKSDB_ERROR; } using FieldVectorPtr = std::shared_ptr; From a1388813e47874ef85d9b1a0ae0d4293b15e481a Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 21 Mar 2022 12:14:53 +0000 Subject: [PATCH 180/627] fix src/Interpreters/ExpressionAnalyzer.cpp --- src/Interpreters/ExpressionAnalyzer.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f436eb24dc4..88f4496e40e 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -68,7 +68,6 @@ #include #include #include -#include namespace DB { @@ -1205,9 +1204,8 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( } joined_plan = buildJoinedPlan(getContext(), join_element, *analyzed_join, query_options); - const Block & right_sample_block = joined_plan->getCurrentDataStream().header; - const ColumnsWithTypeAndName & right_columns = right_sample_block.getColumnsWithTypeAndName(); + const ColumnsWithTypeAndName & right_columns = joined_plan->getCurrentDataStream().header.getColumnsWithTypeAndName(); std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns); if (right_convert_actions) { @@ -1216,6 +1214,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( joined_plan->addStep(std::move(converting_step)); } + const Block & right_sample_block = joined_plan->getCurrentDataStream().header; if (JoinPtr kvjoin = tryKeyValueJoin(analyzed_join, right_sample_block)) { joined_plan.reset(); From b3c581a1f98a4ec6917f5d80ed8633d08dba2a61 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 21 Mar 2022 12:24:20 +0000 Subject: [PATCH 181/627] drop table in 02241_join_rocksdb --- tests/queries/0_stateless/02241_join_rocksdb.sql.j2 | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02241_join_rocksdb.sql.j2 b/tests/queries/0_stateless/02241_join_rocksdb.sql.j2 index 69cff73598a..0627101b51f 100644 --- a/tests/queries/0_stateless/02241_join_rocksdb.sql.j2 +++ b/tests/queries/0_stateless/02241_join_rocksdb.sql.j2 @@ -2,7 +2,9 @@ SET join_algorithm = 'direct'; {% for table_size in [10, 65555, 100000] -%} -CREATE OR REPLACE TABLE rdb (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY (key); +DROP TABLE IF EXISTS rdb; + +CREATE TABLE rdb (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY (key); INSERT INTO rdb SELECT (sipHash64(number) % {{ table_size }}) as key, ('val' || toString(key)) as value FROM numbers_mt({{ table_size }}); {% for block_size in [10, 11, 128, 129, 65505, 65506, 70000] -%} From 87794f9b5b5f8dfd8be304da19b646cd368a6f01 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 21 Mar 2022 13:12:17 +0000 Subject: [PATCH 182/627] get/setTotas to IJoin --- src/Interpreters/DirectJoin.h | 11 ----------- src/Interpreters/HashJoin.h | 7 ------- src/Interpreters/IJoin.h | 13 +++++++++---- src/Interpreters/MergeJoin.cpp | 2 +- src/Interpreters/MergeJoin.h | 2 -- 5 files changed, 10 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index 6b0c14f84d6..d93d99d2018 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -45,17 +45,6 @@ public: /// Could be called from different threads in parallel. virtual void joinBlock(Block & block, std::shared_ptr &) override; - virtual void setTotals(const Block &) override - { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); - } - - virtual const Block & getTotals() const override - { - static const Block totals; - return totals; - } - virtual size_t getTotalRowCount() const override { return 0; diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index c94b580c6b2..e8f8afcc125 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -170,11 +170,6 @@ public: /// Used by joinGet function that turns StorageJoin into a dictionary. ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; - /** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. - */ - void setTotals(const Block & block) override { totals = block; } - const Block & getTotals() const override { return totals; } - bool isFilled() const override { return from_storage_join || data->type == Type::DICT; } /** For RIGHT and FULL JOINs. @@ -393,8 +388,6 @@ private: Poco::Logger * log; - Block totals; - /// Should be set via setLock to protect hash table from modification from StorageJoin /// If set HashJoin instance is not available for modification (addJoinedBlock) TableLockHolder storage_join_lock = nullptr; diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 64b576d3b96..a9022b3d151 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -4,12 +4,12 @@ #include #include +#include #include namespace DB { -class Block; struct ExtraBlock; using ExtraBlockPtr = std::shared_ptr; @@ -33,9 +33,11 @@ public: /// Could be called from different threads in parallel. virtual void joinBlock(Block & block, std::shared_ptr & not_processed) = 0; - /// Set/Get totals for right table - virtual void setTotals(const Block & block) = 0; - virtual const Block & getTotals() const = 0; + /** Set/Get totals for right table + * Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. + */ + virtual void setTotals(const Block & block) { totals = block; } + virtual const Block & getTotals() const { return totals; } virtual size_t getTotalRowCount() const = 0; virtual size_t getTotalByteCount() const = 0; @@ -50,6 +52,9 @@ public: virtual std::shared_ptr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0; + +private: + Block totals; }; using JoinPtr = std::shared_ptr; diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 9ddd4ac0be0..1dea769f724 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -563,7 +563,7 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right /// Has to be called even if totals are empty void MergeJoin::setTotals(const Block & totals_block) { - totals = totals_block; + IJoin::setTotals(totals_block); mergeRightBlocks(); if (is_right || is_full) diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 89570015d0f..ab36599e6f4 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -29,7 +29,6 @@ public: void joinBlock(Block &, ExtraBlockPtr & not_processed) override; void setTotals(const Block &) override; - const Block & getTotals() const override { return totals; } size_t getTotalRowCount() const override { return right_blocks.row_count; } size_t getTotalByteCount() const override { return right_blocks.bytes; } @@ -100,7 +99,6 @@ private: std::unique_ptr disk_writer; /// Set of files with sorted blocks SortedBlocksWriter::SortedFiles flushed_right_blocks; - Block totals; std::atomic is_in_memory{true}; const bool is_any_join; const bool is_all_join; From e40f799f969442d75bb21f9ff8b8be12aa4299f0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 21 Mar 2022 15:01:34 +0000 Subject: [PATCH 183/627] handle right rey rename in direct join --- src/Interpreters/DirectJoin.cpp | 18 ++++++++++++++- src/Interpreters/ExpressionAnalyzer.cpp | 29 +++++++++++++++---------- src/Interpreters/TableJoin.cpp | 16 ++++++++++---- src/Interpreters/TableJoin.h | 3 ++- 4 files changed, 49 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/DirectJoin.cpp b/src/Interpreters/DirectJoin.cpp index 361dd6c4959..6a01d4db0f4 100644 --- a/src/Interpreters/DirectJoin.cpp +++ b/src/Interpreters/DirectJoin.cpp @@ -11,6 +11,14 @@ namespace ErrorCodes extern const int UNSUPPORTED_JOIN_KEYS; } +static Block originalRightBlock(const Block & block, const TableJoin & table_join) +{ + Block original_right_block; + for (const auto & col : block) + original_right_block.insert({col.column, col.type, table_join.getOriginalName(col.name)}); + return original_right_block; +} + void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) { if (!table_join->oneDisjunct() @@ -20,6 +28,13 @@ void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) throw DB::Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Not supported by direct JOIN"); } + if (table_join->strictness() != ASTTableJoin::Strictness::All && + table_join->strictness() != ASTTableJoin::Strictness::Any && + table_join->strictness() != ASTTableJoin::Strictness::RightAny) + { + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Not supported by direct JOIN"); + } + const auto & key_names_left = table_join->getOnlyClause().key_names_left; const String & key_name = key_names_left[0]; @@ -28,7 +43,8 @@ void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) return; NullMap null_map(key_col.column->size(), 1); - Chunk joined_chunk = storage->getByKeys(key_col, right_sample_block, &null_map); + Block original_right_block = originalRightBlock(right_sample_block, *table_join); + Chunk joined_chunk = storage->getByKeys(key_col, original_right_block, &null_map); Columns cols = joined_chunk.detachColumns(); for (size_t i = 0; i < cols.size(); ++i) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 88f4496e40e..9d02569a6da 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1099,12 +1099,12 @@ static std::unique_ptr buildJoinedPlan( { /// Actions which need to be calculated on joined block. auto joined_block_actions = createJoinedBlockActions(context, analyzed_join); - Names original_right_columns; - NamesWithAliases required_columns_with_aliases = analyzed_join.getRequiredColumns( Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); + + Names original_right_column_names; for (auto & pr : required_columns_with_aliases) - original_right_columns.push_back(pr.first); + original_right_column_names.push_back(pr.first); /** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs * - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1, @@ -1115,18 +1115,18 @@ static std::unique_ptr buildJoinedPlan( auto interpreter = interpretSubquery( join_element.table_expression, context, - original_right_columns, + original_right_column_names, query_options.copy().setWithAllColumns().ignoreProjections(false).ignoreAlias(false)); auto joined_plan = std::make_unique(); interpreter->buildQueryPlan(*joined_plan); { - auto sample_block = interpreter->getSampleBlock(); - auto rename_dag = std::make_unique(sample_block.getColumnsWithTypeAndName()); + Block original_right_columns = interpreter->getSampleBlock(); + auto rename_dag = std::make_unique(original_right_columns.getColumnsWithTypeAndName()); for (const auto & name_with_alias : required_columns_with_aliases) { - if (sample_block.has(name_with_alias.first)) + if (name_with_alias.first != name_with_alias.second && original_right_columns.has(name_with_alias.first)) { - auto pos = sample_block.getPositionByName(name_with_alias.first); + auto pos = original_right_columns.getPositionByName(name_with_alias.first); const auto & alias = rename_dag->addAlias(*rename_dag->getInputs()[pos], name_with_alias.second); rename_dag->getIndex()[pos] = &alias; } @@ -1163,7 +1163,9 @@ std::shared_ptr tryKeyValueJoin(std::shared_ptr a if (!isInnerOrLeft(analyzed_join->kind())) return error_or_null("illegal kind"); - if (analyzed_join->strictness() != ASTTableJoin::Strictness::All) + if (analyzed_join->strictness() != ASTTableJoin::Strictness::All && + analyzed_join->strictness() != ASTTableJoin::Strictness::Any && + analyzed_join->strictness() != ASTTableJoin::Strictness::RightAny) return error_or_null("illegal strictness"); const auto & clauses = analyzed_join->getClauses(); @@ -1176,8 +1178,13 @@ std::shared_ptr tryKeyValueJoin(std::shared_ptr a if (!only_one_key) return error_or_null("multiple keys is not allowed"); - if (storage->getPrimaryKey() != clauses[0].key_names_right[0]) - return error_or_null("key doesn't match storage"); + String key_name = clauses[0].key_names_right[0]; + String original_key_name = analyzed_join->getOriginalName(key_name); + if (storage->getPrimaryKey() != original_key_name) + { + return error_or_null(fmt::format("key '{}'{} doesn't match storage '{}'", + key_name, (key_name != original_key_name ? " (aka '" + original_key_name + "')" : ""), storage->getPrimaryKey())); + } return std::make_shared(analyzed_join, right_sample_block, storage); } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 902965e1629..d3a291ed895 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -190,14 +190,22 @@ void TableJoin::deduplicateAndQualifyColumnNames(const NameSet & left_table_colu columns_from_joined_table.swap(dedup_columns); } +String TableJoin::getOriginalName(const String & column_name) const +{ + auto it = original_names.find(column_name); + if (it != original_names.end()) + return it->second; + return column_name; +} + NamesWithAliases TableJoin::getNamesWithAliases(const NameSet & required_columns) const { NamesWithAliases out; - for (const auto & column : required_columns) + out.reserve(required_columns.size()); + for (const auto & name : required_columns) { - auto it = original_names.find(column); - if (it != original_names.end()) - out.emplace_back(it->second, it->first); /// {original_name, name} + auto original_name = getOriginalName(name); + out.emplace_back(original_name, name); } return out; } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index f2ff6076f9c..39cd3bf004c 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -57,7 +57,7 @@ public: struct JoinOnClause { Names key_names_left; - Names key_names_right; /// Duplicating right key names are qualified. + Names key_names_right; /// Duplicating right key names are qualified ASTPtr on_filter_condition_left; ASTPtr on_filter_condition_right; @@ -252,6 +252,7 @@ public: bool hasUsing() const { return table_join.using_expression_list != nullptr; } bool hasOn() const { return table_join.on_expression != nullptr; } + String getOriginalName(const String & column_name) const; NamesWithAliases getNamesWithAliases(const NameSet & required_columns) const; NamesWithAliases getRequiredColumns(const Block & sample, const Names & action_required_columns) const; From a1ef707a8d6e02360ec8d50cea67fc7d7d9496eb Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 21 Mar 2022 15:02:52 +0000 Subject: [PATCH 184/627] update 02241_join_rocksdb --- .../0_stateless/02241_join_rocksdb.sql.j2 | 43 ------------------- ...erence => 02241_join_rocksdb_bs.reference} | 34 +++++++++++++++ .../0_stateless/02241_join_rocksdb_bs.sql.j2 | 40 +++++++++++++++++ 3 files changed, 74 insertions(+), 43 deletions(-) delete mode 100644 tests/queries/0_stateless/02241_join_rocksdb.sql.j2 rename tests/queries/0_stateless/{02241_join_rocksdb.reference => 02241_join_rocksdb_bs.reference} (50%) create mode 100644 tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 diff --git a/tests/queries/0_stateless/02241_join_rocksdb.sql.j2 b/tests/queries/0_stateless/02241_join_rocksdb.sql.j2 deleted file mode 100644 index 0627101b51f..00000000000 --- a/tests/queries/0_stateless/02241_join_rocksdb.sql.j2 +++ /dev/null @@ -1,43 +0,0 @@ -SET join_algorithm = 'direct'; - -{% for table_size in [10, 65555, 100000] -%} - -DROP TABLE IF EXISTS rdb; - -CREATE TABLE rdb (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY (key); -INSERT INTO rdb SELECT (sipHash64(number) % {{ table_size }}) as key, ('val' || toString(key)) as value FROM numbers_mt({{ table_size }}); - -{% for block_size in [10, 11, 128, 129, 65505, 65506, 70000] -%} - -{% if block_size * 5000 > table_size -%} - -SET max_block_size = {{ block_size }}; - -SELECT count() == (SELECT count() FROM rdb WHERE key < {{ table_size // 2 }}) -FROM (SELECT number as k FROM numbers_mt({{ table_size // 2 }})) as t1 -INNER JOIN rdb -ON rdb.key == t1.k; - -SELECT count() == (SELECT count() FROM rdb WHERE key < {{ table_size + table_size // 4 + 1 }}) -FROM (SELECT number as k FROM numbers_mt({{ table_size + table_size // 4 + 1 }})) as t1 -INNER JOIN rdb -ON rdb.key == t1.k; - -{% endif -%} - -{% endfor -%} -{% endfor -%} - -SELECT count() -FROM (SELECT number as k FROM numbers_mt(10)) as t1 -INNER JOIN rdb -ON rdb.key + 1 == t1.k; -- { serverError UNSUPPORTED_METHOD } - -SELECT count() -FROM (SELECT number as k FROM numbers_mt(10)) as t1 -INNER JOIN (SELECT * FROM rdb) AS rdb -ON rdb.key == t1.k; -- { serverError UNSUPPORTED_METHOD } - -DROP TABLE rdb; - - diff --git a/tests/queries/0_stateless/02241_join_rocksdb.reference b/tests/queries/0_stateless/02241_join_rocksdb_bs.reference similarity index 50% rename from tests/queries/0_stateless/02241_join_rocksdb.reference rename to tests/queries/0_stateless/02241_join_rocksdb_bs.reference index 4600557506b..8416a2991c1 100644 --- a/tests/queries/0_stateless/02241_join_rocksdb.reference +++ b/tests/queries/0_stateless/02241_join_rocksdb_bs.reference @@ -32,3 +32,37 @@ 1 1 1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 b/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 new file mode 100644 index 00000000000..773c909b3c1 --- /dev/null +++ b/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 @@ -0,0 +1,40 @@ +-- Tags: use-rocksdb + +SET join_algorithm = 'direct'; + +{% for table_size in [10, 65555, 100000] -%} + +DROP TABLE IF EXISTS rdb; + +CREATE TABLE rdb (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY (key); +INSERT INTO rdb SELECT (sipHash64(number) % {{ table_size }}) as key, ('val' || toString(key)) as value FROM numbers_mt({{ table_size }}); + +{% for block_size in [10, 11, 128, 129, 65505, 65506, 70000] -%} + +{% if block_size * 5000 > table_size -%} + +SET max_block_size = {{ block_size }}; + +{% for right_size in [table_size // 2, table_size + table_size // 4 + 1] -%} + +SELECT count() == (SELECT count() FROM rdb WHERE key < {{ right_size }}) +FROM (SELECT number as k FROM numbers_mt({{ right_size }})) as t1 +INNER JOIN rdb +ON rdb.key == t1.k; + +SELECT count() == {{ right_size }} and countIf(value != '') == (SELECT count() FROM rdb WHERE key < {{ right_size }}) +FROM (SELECT number as k FROM numbers_mt({{ right_size }})) as t1 +LEFT JOIN rdb +ON rdb.key == t1.k; + +{% endfor -%} + +{% endif -%} + +{% endfor -%} +{% endfor -%} + + +DROP TABLE rdb; + + From 0c12adc93453e6f709c3ee3d046e7615142fecbb Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 21 Mar 2022 15:35:47 +0000 Subject: [PATCH 185/627] fix style --- src/Interpreters/DirectJoin.cpp | 12 ++++++++++++ src/Interpreters/DirectJoin.h | 21 ++++----------------- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/DirectJoin.cpp b/src/Interpreters/DirectJoin.cpp index 6a01d4db0f4..2722a072eb1 100644 --- a/src/Interpreters/DirectJoin.cpp +++ b/src/Interpreters/DirectJoin.cpp @@ -19,6 +19,18 @@ static Block originalRightBlock(const Block & block, const TableJoin & table_joi return original_right_block; } + +bool DirectKeyValueJoin::addJoinedBlock(const Block &, bool) +{ + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); +} + + +void DirectKeyValueJoin::checkTypesOfKeys(const Block &) const +{ + // throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); +} + void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) { if (!table_join->oneDisjunct() diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index d93d99d2018..95ab8f1c20a 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -31,29 +31,16 @@ public: virtual const TableJoin & getTableJoin() const override { return *table_join; } - virtual bool addJoinedBlock(const Block &, bool) override - { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); - } - - virtual void checkTypesOfKeys(const Block &) const override - { - // throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); - } + virtual bool addJoinedBlock(const Block &, bool) override; + virtual void checkTypesOfKeys(const Block &) const override; /// Join the block with data from left hand of JOIN to the right hand data (that was previously built by calls to addJoinedBlock). /// Could be called from different threads in parallel. virtual void joinBlock(Block & block, std::shared_ptr &) override; - virtual size_t getTotalRowCount() const override - { - return 0; - } + virtual size_t getTotalRowCount() const override { return 0; } - virtual size_t getTotalByteCount() const override - { - return 0; - } + virtual size_t getTotalByteCount() const override { return 0; } virtual bool alwaysReturnsEmptySet() const override { return false; } From 65ac97ce8945d7736ec6d5fedecfa92ca0d964f2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 21 Mar 2022 18:08:35 +0000 Subject: [PATCH 186/627] direct join: handle nullable, add tests --- src/Columns/ColumnNullable.cpp | 26 +++++--- src/Columns/ColumnNullable.h | 4 +- src/Interpreters/DirectJoin.cpp | 13 ++-- src/Interpreters/TableJoin.cpp | 6 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 46 +++++++++++-- .../0_stateless/02242_join_rocksdb.reference | 60 +++++++++++++++++ .../0_stateless/02242_join_rocksdb.sql | 64 +++++++++++++++++++ 7 files changed, 197 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02242_join_rocksdb.reference create mode 100644 tests/queries/0_stateless/02242_join_rocksdb.sql diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 435e6bf1fbc..d8e98ec9406 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -715,29 +715,37 @@ ColumnPtr ColumnNullable::replicate(const Offsets & offsets) const template -void ColumnNullable::applyNullMapImpl(const ColumnUInt8 & map) +void ColumnNullable::applyNullMapImpl(const NullMap & map) { - NullMap & arr1 = getNullMapData(); - const NullMap & arr2 = map.getData(); + NullMap & arr = getNullMapData(); - if (arr1.size() != arr2.size()) + if (arr.size() != map.size()) throw Exception{"Inconsistent sizes of ColumnNullable objects", ErrorCodes::LOGICAL_ERROR}; - for (size_t i = 0, size = arr1.size(); i < size; ++i) - arr1[i] |= negative ^ arr2[i]; + for (size_t i = 0, size = arr.size(); i < size; ++i) + arr[i] |= negative ^ map[i]; } - -void ColumnNullable::applyNullMap(const ColumnUInt8 & map) +void ColumnNullable::applyNullMap(const NullMap & map) { applyNullMapImpl(map); } -void ColumnNullable::applyNegatedNullMap(const ColumnUInt8 & map) +void ColumnNullable::applyNullMap(const ColumnUInt8 & map) +{ + applyNullMapImpl(map.getData()); +} + +void ColumnNullable::applyNegatedNullMap(const NullMap & map) { applyNullMapImpl(map); } +void ColumnNullable::applyNegatedNullMap(const ColumnUInt8 & map) +{ + applyNullMapImpl(map.getData()); +} + void ColumnNullable::applyNullMap(const ColumnNullable & other) { diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 60951dfcc2e..52e57f7f0d0 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -199,7 +199,9 @@ public: /// columns. void applyNullMap(const ColumnNullable & other); void applyNullMap(const ColumnUInt8 & map); + void applyNullMap(const NullMap & map); void applyNegatedNullMap(const ColumnUInt8 & map); + void applyNegatedNullMap(const NullMap & map); /// Check that size of null map equals to size of nested column. void checkConsistency() const; @@ -209,7 +211,7 @@ private: WrappedPtr null_map; template - void applyNullMapImpl(const ColumnUInt8 & map); + void applyNullMapImpl(const NullMap & map); int compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint, const Collator * collator=nullptr) const; diff --git a/src/Interpreters/DirectJoin.cpp b/src/Interpreters/DirectJoin.cpp index 2722a072eb1..12aca6f715f 100644 --- a/src/Interpreters/DirectJoin.cpp +++ b/src/Interpreters/DirectJoin.cpp @@ -8,6 +8,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; extern const int UNSUPPORTED_JOIN_KEYS; } @@ -19,16 +20,17 @@ static Block originalRightBlock(const Block & block, const TableJoin & table_joi return original_right_block; } - bool DirectKeyValueJoin::addJoinedBlock(const Block &, bool) { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable code reached"); } - -void DirectKeyValueJoin::checkTypesOfKeys(const Block &) const +void DirectKeyValueJoin::checkTypesOfKeys(const Block & block) const { - // throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented"); + for (const auto & onexpr : table_join->getClauses()) + { + JoinCommon::checkTypesOfKeys(block, onexpr.key_names_left, right_sample_block, onexpr.key_names_right); + } } void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) @@ -69,7 +71,6 @@ void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) if (!isLeftOrFull(table_join->kind())) { MutableColumns dst_columns = block.mutateColumns(); - for (auto & col : dst_columns) { col = IColumn::mutate(col->filter(null_map, -1)); diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index d3a291ed895..b8257a36e29 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -605,7 +605,8 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig right_key_name, rtype->second->getName(), ex.message()); } - if (!allow_right && !common_type->equals(*rtype->second)) + bool right_side_changed = !common_type->equals(*rtype->second); + if (right_side_changed && !allow_right) { throw DB::Exception(ErrorCodes::TYPE_MISMATCH, "Can't change type for right table: {}: {} -> {}", @@ -635,7 +636,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src, bool has_some_to_do = false; for (auto & col : cols_dst) { - if (auto it = type_mapping.find(col.name); it != type_mapping.end()) + if (auto it = type_mapping.find(col.name); it != type_mapping.end() && col.type != it->second) { col.type = it->second; col.column = nullptr; @@ -644,6 +645,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src, } if (!has_some_to_do) return nullptr; + return ActionsDAG::makeConvertingActions(cols_src, cols_dst, ActionsDAG::MatchColumnsMode::Name, true, add_new_cols, &key_column_rename); } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index f7298a51226..df84893ae7d 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -20,6 +20,7 @@ #include #include +#include #include #include #include @@ -50,6 +51,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; + extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ROCKSDB_ERROR; } @@ -565,20 +567,53 @@ Chunk StorageEmbeddedRocksDB::getByKeys( return getByKeysImpl(sliced_keys, sample_block, null_map); } +static MutableColumns getColumnsFromBlock( + const Block & sample_block, MutableColumns && columns, const Block & output_sample_block, PaddedPODArray * null_map) +{ + MutableColumns result_columns; + for (const auto & out_sample_col : output_sample_block) + { + auto i = sample_block.getPositionByName(out_sample_col.name); + if (columns[i] == nullptr) + { + throw DB::Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Can't find column '{}'", out_sample_col.name); + } + + ColumnWithTypeAndName col = sample_block.getByPosition(i); + if (!col.type->equals(*out_sample_col.type)) + { + col.column = std::move(columns[i]); + result_columns.push_back(IColumn::mutate(castColumnAccurate(col, out_sample_col.type))); + } + else + { + result_columns.push_back(std::move(columns[i])); + } + columns[i] = nullptr; + + if (null_map && result_columns.back()->isNullable()) + { + assert_cast(result_columns.back().get())->applyNegatedNullMap(*null_map); + } + } + return result_columns; +} + Chunk StorageEmbeddedRocksDB::getByKeysImpl( const std::vector & slices_keys, - const Block & sample_block, + const Block & output_sample_block, PaddedPODArray * null_map) const { - if (!sample_block) + if (!output_sample_block) return {}; std::vector values; - auto statuses = multiGet(slices_keys, values); + Block sample_block = getInMemoryMetadataPtr()->getSampleBlock(); size_t primary_key_pos = sample_block.getPositionByName(getPrimaryKey()); MutableColumns columns = sample_block.cloneEmptyColumns(); + auto statuses = multiGet(slices_keys, values); for (size_t i = 0; i < statuses.size(); ++i) { if (statuses[i].ok()) @@ -603,7 +638,10 @@ Chunk StorageEmbeddedRocksDB::getByKeysImpl( } size_t num_rows = columns.at(0)->size(); - return Chunk(std::move(columns), num_rows); + /// The `output_sample_block` may be different from `sample_block`. + /// It may constains subset of columns and types can be different (but should be convertable) + auto result_columns = getColumnsFromBlock(sample_block, std::move(columns), output_sample_block, null_map); + return Chunk(std::move(result_columns), num_rows); } void registerStorageEmbeddedRocksDB(StorageFactory & factory) diff --git a/tests/queries/0_stateless/02242_join_rocksdb.reference b/tests/queries/0_stateless/02242_join_rocksdb.reference new file mode 100644 index 00000000000..b1f7307ff4f --- /dev/null +++ b/tests/queries/0_stateless/02242_join_rocksdb.reference @@ -0,0 +1,60 @@ +-- key rename +0 0 [0,1] val20 +2 2 [2,3] val22 +3 3 [3,4] val23 +6 6 [6,7] val26 +7 7 [7,8] val27 +8 8 [8,9] val28 +9 9 [9,10] val29 +-- using +0 [0,1] val20 +2 [2,3] val22 +3 [3,4] val23 +6 [6,7] val26 +7 [7,8] val27 +8 [8,9] val28 +9 [9,10] val29 +-- join_use_nulls left +0 0 Nullable(String) val20 +1 \N Nullable(String) \N +2 2 Nullable(String) val22 +3 3 Nullable(String) val23 +4 \N Nullable(String) \N +5 \N Nullable(String) \N +6 6 Nullable(String) val26 +7 7 Nullable(String) val27 +8 8 Nullable(String) val28 +9 9 Nullable(String) val29 +-- join_use_nulls inner +0 0 String val20 +2 2 String val22 +3 3 String val23 +6 6 String val26 +7 7 String val27 +8 8 String val28 +9 9 String val29 +-- columns subset +val20 + +val22 +val23 + + +val26 +val27 +val28 +val29 +--- key types +0 0 [0,1] val20 +2 2 [2,3] val22 +3 3 [3,4] val23 +6 6 [6,7] val26 +7 7 [7,8] val27 +8 8 [8,9] val28 +9 9 [9,10] val29 +--- totals +0 16 val28 +1 19 val29 + +0 35 val29 +--- diff --git a/tests/queries/0_stateless/02242_join_rocksdb.sql b/tests/queries/0_stateless/02242_join_rocksdb.sql new file mode 100644 index 00000000000..1759311163b --- /dev/null +++ b/tests/queries/0_stateless/02242_join_rocksdb.sql @@ -0,0 +1,64 @@ +-- Tags: use-rocksdb + +DROP TABLE IF EXISTS rdb; +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE rdb (key UInt32, value Array(UInt32), value2 String) ENGINE = EmbeddedRocksDB PRIMARY KEY (key); +INSERT INTO rdb + SELECT + toUInt32(sipHash64(number) % 10) as key, + [key, key+1] as value, + ('val2' || toString(key)) as value2 + FROM numbers_mt(10); + +CREATE TABLE t1 (k UInt32) ENGINE = TinyLog; +INSERT INTO t1 SELECT number as k from numbers_mt(10); + +CREATE TABLE t2 (k UInt16) ENGINE = TinyLog; +INSERT INTO t2 SELECT number as k from numbers_mt(10); + +SET join_algorithm = 'direct'; + +SELECT '-- key rename'; +SELECT * FROM (SELECT k as key FROM t2) as t2 INNER JOIN rdb ON rdb.key == t2.key ORDER BY key; + +SELECT '-- using'; +SELECT * FROM (SELECT k as key FROM t2) as t2 INNER JOIN rdb USING key ORDER BY key; + +SELECT '-- join_use_nulls left'; +SELECT k, key, toTypeName(value2), value2 FROM t2 LEFT JOIN rdb ON rdb.key == t2.k ORDER BY k SETTINGS join_use_nulls = 1; + +SELECT '-- join_use_nulls inner'; +SELECT k, key, toTypeName(value2), value2 FROM t2 INNER JOIN rdb ON rdb.key == t2.k ORDER BY k SETTINGS join_use_nulls = 1; + +SELECT '-- columns subset'; +SELECT value2 FROM t2 LEFT JOIN rdb ON rdb.key == t2.k ORDER BY k; + +SELECT '--- key types'; +SELECT * FROM t2 INNER JOIN rdb ON rdb.key == t2.k ORDER BY rdb.key; + +-- can't promote right table type +SELECT * FROM (SELECT toUInt64(k) as k FROM t2) as t2 INNER JOIN rdb ON rdb.key == t2.k; -- { serverError TYPE_MISMATCH } +-- TODO: support fallcack when right table key type can't be changed +-- SELECT * FROM (SELECT toUInt64(k) as k FROM t2) as t2 INNER JOIN rdb ON rdb.key == t2.k FORMAT Null SETTINGS join_algorithm = 'direct,hash'; + +SELECT '--- totals'; +SELECT rdb.key % 2, sum(k), max(value2) FROM t2 INNER JOIN rdb ON rdb.key == t2.k GROUP BY (rdb.key % 2) WITH TOTALS; + +SELECT '---'; +SELECT * FROM t1 RIGHT JOIN rdb ON rdb.key == t1.k; -- { serverError UNSUPPORTED_METHOD } +SELECT * FROM t1 RIGHT JOIN rdb ON rdb.key == t1.k FORMAT Null SETTINGS join_algorithm = 'direct,hash'; + +SELECT * FROM t1 FULL JOIN rdb ON rdb.key == t1.k; -- { serverError UNSUPPORTED_METHOD } +SELECT * FROM t1 FULL JOIN rdb ON rdb.key == t1.k FORMAT Null SETTINGS join_algorithm = 'direct,hash'; + +SELECT * FROM t1 INNER JOIN rdb ON rdb.key + 1 == t1.k; -- { serverError UNSUPPORTED_METHOD } +SELECT * FROM t1 INNER JOIN rdb ON rdb.key + 1 == t1.k FORMAT Null SETTINGS join_algorithm = 'direct,hash'; + +SELECT * FROM t1 INNER JOIN (SELECT * FROM rdb) AS rdb ON rdb.key == t1.k; -- { serverError UNSUPPORTED_METHOD } +SELECT * FROM t1 INNER JOIN (SELECT * FROM rdb) AS rdb ON rdb.key == t1.k FORMAT Null SETTINGS join_algorithm = 'direct,hash'; + +DROP TABLE IF EXISTS rdb; +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; From 43afda408bfa102ad61ac9edcfe97b7ba44e6243 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 22 Mar 2022 10:52:55 +0000 Subject: [PATCH 187/627] do not call setStorageJoin when setting isn't set --- src/Interpreters/JoinedTables.cpp | 12 +++++++++++- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 0a476b0ea30..df47e8acdca 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -1,5 +1,7 @@ #include +#include + #include #include #include @@ -292,6 +294,7 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se return {}; auto settings = context->getSettingsRef(); + MultiEnum join_algorithm = settings.join_algorithm; auto table_join = std::make_shared(settings, context->getTemporaryVolume()); const ASTTablesInSelectQueryElement * ast_join = select_query.join(); @@ -305,11 +308,18 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se if (storage) { if (auto storage_join = std::dynamic_pointer_cast(storage); storage_join) + { table_join->setStorageJoin(storage_join); + } else if (auto storage_dict = std::dynamic_pointer_cast(storage); storage_dict) + { table_join->setStorageJoin(storage_dict); - else if (auto storage_kv = std::dynamic_pointer_cast(storage); storage_kv) + } + else if (auto storage_kv = std::dynamic_pointer_cast(storage); + storage_kv && join_algorithm.isSet(JoinAlgorithm::DIRECT)) + { table_join->setStorageJoin(storage_kv); + } } } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index df84893ae7d..79d0a9b1a64 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -639,7 +639,7 @@ Chunk StorageEmbeddedRocksDB::getByKeysImpl( size_t num_rows = columns.at(0)->size(); /// The `output_sample_block` may be different from `sample_block`. - /// It may constains subset of columns and types can be different (but should be convertable) + /// It may contains subset of columns and types can be different (but should be convertible) auto result_columns = getColumnsFromBlock(sample_block, std::move(columns), output_sample_block, null_map); return Chunk(std::move(result_columns), num_rows); } From 54cf5c51de1f3fd91816d3f3860b2a0089bbb4c3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 22 Mar 2022 16:33:03 +0000 Subject: [PATCH 188/627] upd 02241_join_rocksdb_bs.sql --- .../0_stateless/02241_join_rocksdb_bs.sql.j2 | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 b/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 index 773c909b3c1..e08df1688fa 100644 --- a/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 +++ b/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 @@ -3,11 +3,15 @@ SET join_algorithm = 'direct'; {% for table_size in [10, 65555, 100000] -%} +DROP TABLE IF EXISTS rdb_{{ table_size }}; +{% endfor -%} -DROP TABLE IF EXISTS rdb; +{% for table_size in [10, 65555, 100000] -%} -CREATE TABLE rdb (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY (key); -INSERT INTO rdb SELECT (sipHash64(number) % {{ table_size }}) as key, ('val' || toString(key)) as value FROM numbers_mt({{ table_size }}); +CREATE TABLE rdb_{{ table_size }} (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY (key); +INSERT INTO rdb_{{ table_size }} + SELECT (sipHash64(number) % {{ table_size }}) as key, ('val' || toString(key)) AS value + FROM numbers_mt({{ table_size }}); {% for block_size in [10, 11, 128, 129, 65505, 65506, 70000] -%} @@ -17,14 +21,14 @@ SET max_block_size = {{ block_size }}; {% for right_size in [table_size // 2, table_size + table_size // 4 + 1] -%} -SELECT count() == (SELECT count() FROM rdb WHERE key < {{ right_size }}) +SELECT count() == (SELECT count() FROM rdb_{{ table_size }} WHERE key < {{ right_size }}) FROM (SELECT number as k FROM numbers_mt({{ right_size }})) as t1 -INNER JOIN rdb +INNER JOIN rdb_{{ table_size }} as rdb ON rdb.key == t1.k; -SELECT count() == {{ right_size }} and countIf(value != '') == (SELECT count() FROM rdb WHERE key < {{ right_size }}) +SELECT count() == {{ right_size }} and countIf(value != '') == (SELECT count() FROM rdb_{{ table_size }} WHERE key < {{ right_size }}) FROM (SELECT number as k FROM numbers_mt({{ right_size }})) as t1 -LEFT JOIN rdb +LEFT JOIN rdb_{{ table_size }} as rdb ON rdb.key == t1.k; {% endfor -%} @@ -34,7 +38,7 @@ ON rdb.key == t1.k; {% endfor -%} {% endfor -%} - -DROP TABLE rdb; - +{% for table_size in [10, 65555, 100000] -%} +DROP TABLE IF EXISTS rdb_{{ table_size }}; +{% endfor -%} From 1cad4a7e9b18194b2e539181712f2dc067f02fbf Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 23 Mar 2022 16:10:50 +0000 Subject: [PATCH 189/627] Mark 02241_join_rocksdb_bs long --- tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 b/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 index e08df1688fa..6121db6d6a2 100644 --- a/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 +++ b/tests/queries/0_stateless/02241_join_rocksdb_bs.sql.j2 @@ -1,4 +1,4 @@ --- Tags: use-rocksdb +-- Tags: use-rocksdb, long SET join_algorithm = 'direct'; From c0cb588e454b2901d4e28ba686bf8038ea984abb Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 4 Jul 2022 13:19:46 +0000 Subject: [PATCH 190/627] Fix build --- src/Interpreters/DirectJoin.h | 2 +- src/Interpreters/HashJoin.h | 1 + src/Interpreters/TableJoin.cpp | 2 +- src/Interpreters/TableJoin.h | 1 + src/Storages/IKVStorage.h | 1 + src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 7 ------- 7 files changed, 6 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index 95ab8f1c20a..435b33f6dab 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index e8f8afcc125..ae6b059696c 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index b8257a36e29..20842023bab 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -800,7 +800,7 @@ void TableJoin::resetToCross() bool TableJoin::allowParallelHashJoin() const { - if (dictionary_reader || join_algorithm != JoinAlgorithm::PARALLEL_HASH) + if (dictionary_reader || !join_algorithm.isSet(JoinAlgorithm::PARALLEL_HASH)) return false; if (table_join.kind != ASTTableJoin::Kind::Left && table_join.kind != ASTTableJoin::Kind::Inner) return false; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 39cd3bf004c..ff8c62100b6 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -206,6 +206,7 @@ public: || join_algorithm == MultiEnum(JoinAlgorithm::HASH) || join_algorithm == MultiEnum(JoinAlgorithm::PARALLEL_HASH); } + bool allowParallelHashJoin() const; bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(table_join.kind); } bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(table_join.kind); } diff --git a/src/Storages/IKVStorage.h b/src/Storages/IKVStorage.h index 994a228405b..06ac42f2d80 100644 --- a/src/Storages/IKVStorage.h +++ b/src/Storages/IKVStorage.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 79d0a9b1a64..064c1d850ff 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -243,7 +243,7 @@ static std::vector getSlicedKeys(const ColumnWithTypeAndName & c return slices_keys; } -class EmbeddedRocksDBSource : public SourceWithProgress +class EmbeddedRocksDBSource : public ISource { public: EmbeddedRocksDBSource( diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 7829eb696b1..6894c2e684e 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -69,13 +69,6 @@ public: const Block & sample_block, PaddedPODArray * null_map) const; -protected: - StorageEmbeddedRocksDB(const StorageID & table_id_, - const String & relative_data_path_, - const StorageInMemoryMetadata & metadata, - bool attach, - ContextPtr context_, - const String & primary_key_); private: const String primary_key; From 8a270c01e9aa6872911da6ce305eea4ec2205f30 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 4 Jul 2022 20:45:05 +0300 Subject: [PATCH 191/627] fix floating point in intervals --- src/Common/IntervalKind.cpp | 35 ++++++++++++++++--- src/Common/IntervalKind.h | 6 +++- src/Processors/Transforms/WindowTransform.cpp | 2 +- 3 files changed, 37 insertions(+), 6 deletions(-) diff --git a/src/Common/IntervalKind.cpp b/src/Common/IntervalKind.cpp index 75c2a83e9fb..4e923fdba55 100644 --- a/src/Common/IntervalKind.cpp +++ b/src/Common/IntervalKind.cpp @@ -7,15 +7,16 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; + extern const int BAD_ARGUMENTS; } -Float64 IntervalKind::toAvgSeconds() const +Int32 IntervalKind::toAvgSeconds() const { switch (kind) { - case IntervalKind::Nanosecond: return 0.000000001; - case IntervalKind::Microsecond: return 0.000001; - case IntervalKind::Millisecond: return 0.001; + case IntervalKind::Nanosecond: + case IntervalKind::Microsecond: + case IntervalKind::Millisecond: return 0; case IntervalKind::Second: return 1; case IntervalKind::Minute: return 60; case IntervalKind::Hour: return 3600; @@ -28,6 +29,32 @@ Float64 IntervalKind::toAvgSeconds() const __builtin_unreachable(); } +Float64 IntervalKind::toSeconds() const +{ + switch (kind) + { + case IntervalKind::Nanosecond: + return 0.000000001; + case IntervalKind::Microsecond: + return 0.000001; + case IntervalKind::Millisecond: + return 0.001; + case IntervalKind::Second: + return 1; + case IntervalKind::Minute: + return 60; + case IntervalKind::Hour: + return 3600; + case IntervalKind::Day: + return 86400; + case IntervalKind::Week: + return 604800; + default: + throw Exception("Not possible to get precise number of seconds in non-precise interval", ErrorCodes::BAD_ARGUMENTS); + } + __builtin_unreachable(); +} + bool IntervalKind::isFixedLength() const { switch (kind) diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index 65c14515e34..59e8d32f3e7 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -31,7 +31,11 @@ struct IntervalKind /// Returns number of seconds in one interval. /// For `Month`, `Quarter` and `Year` the function returns an average number of seconds. - Float64 toAvgSeconds() const; + Int32 toAvgSeconds() const; + + /// Returns exact number of seconds in one interval. + /// For `Month`, `Quarter` and `Year` the function raises an error. + Float64 toSeconds() const; /// Chooses an interval kind based on number of seconds. /// For example, `IntervalKind::fromAvgSeconds(3600)` returns `IntervalKind::Hour`. diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index d05c703a7cd..5c833cf8f69 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2272,7 +2272,7 @@ struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction "The INTERVAL must be a week or shorter, '{}' given", argument_types[ARGUMENT_INTERVAL]->getName()); } - interval_length = interval_datatype->getKind().toAvgSeconds(); + interval_length = interval_datatype->getKind().toSeconds(); interval_specified = true; } } From 45ac74617b43b4fa6b83b600f24217b430945d0e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 4 Jul 2022 20:38:21 +0200 Subject: [PATCH 192/627] Fix --- docker/test/stress/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index c16efdca876..be781309277 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -285,10 +285,10 @@ stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.clean.log # Check if we cloned previous release repository successfully -if ! [ -z "$(ls -A previous_release_repository/tests/queries)" ] +if ! [ "$(ls -A previous_release_repository/tests/queries)" ] then echo -e "Backward compatibility check: Failed to clone previous release tests\tFAIL" >> /test_output/test_results.tsv -elif ! [ -z "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ] +elif ! [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ] then echo -e "Backward compatibility check: Failed to download previous release packets\tFAIL" >> /test_output/test_results.tsv else From 74bd625cd5f39493ba831f5030661c8eecbd4b43 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 4 Jul 2022 20:48:35 +0200 Subject: [PATCH 193/627] Fix test --- .../0_stateless/02323_null_modifier_in_table_function.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql b/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql index 33fb4c43fa2..810480ed868 100644 --- a/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql +++ b/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql @@ -1,3 +1,4 @@ +-- Tags: no-parallel select * from values('x UInt8 NOT NULL', 1); select * from values('x UInt8 NULL', NULL); insert into function file(data_02323.tsv) select number % 2 ? number : NULL from numbers(3) settings engine_file_truncate_on_insert=1; From b8ab20bcb3825c9a85ef7dda75c17f591c96d9ec Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 4 Jul 2022 20:56:34 +0200 Subject: [PATCH 194/627] Update test.py --- tests/integration/test_storage_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 7f92325f44e..5dd09ddd362 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1075,7 +1075,7 @@ def test_seekable_formats(started_cluster): instance.query("SYSTEM FLUSH LOGS") result = instance.query( - f"SELECT formatReadableSize(ProfileEvents['ReadBufferFromS3Bytes']) FROM system.query_log WHERE startsWith(query, 'SELECT * FROM s3') AND memory_usage > 0 AND type='QueryFinish' ORDER BY event_time desc" + f"SELECT formatReadableSize(ProfileEvents['ReadBufferFromS3Bytes']) FROM system.query_log WHERE startsWith(query, 'SELECT * FROM s3') AND memory_usage > 0 AND type='QueryFinish' ORDER BY event_time_microseconds DESC LIMIT 1" ) result = result.strip() assert result.endswith("MiB") From 03f82f73fb534b84302cff8529faa348126c9b96 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 4 Jul 2022 19:56:52 +0000 Subject: [PATCH 195/627] Add test for segfault in Map combinator --- .../02324_map_combinator_bug.reference | 1 + .../0_stateless/02324_map_combinator_bug.sql | 26 +++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/02324_map_combinator_bug.reference create mode 100644 tests/queries/0_stateless/02324_map_combinator_bug.sql diff --git a/tests/queries/0_stateless/02324_map_combinator_bug.reference b/tests/queries/0_stateless/02324_map_combinator_bug.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02324_map_combinator_bug.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02324_map_combinator_bug.sql b/tests/queries/0_stateless/02324_map_combinator_bug.sql new file mode 100644 index 00000000000..aa9eefaa9ba --- /dev/null +++ b/tests/queries/0_stateless/02324_map_combinator_bug.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS segfault; +DROP TABLE IF EXISTS segfault_mv; + +CREATE TABLE segfault +( + id UInt32, + uuid UUID, + tags_ids Array(UInt32) +) ENGINE = MergeTree() +ORDER BY (id); + +CREATE MATERIALIZED VIEW segfault_mv + ENGINE = AggregatingMergeTree() + ORDER BY (id) +AS SELECT + id, + uniqState(uuid) as uniq_uuids, + uniqMapState(CAST((tags_ids, arrayMap(_ -> toString(uuid), tags_ids)), 'Map(UInt32, String)')) as uniq_tags_ids +FROM segfault +GROUP BY id; + +INSERT INTO segfault SELECT * FROM generateRandom('id UInt32, uuid UUID, c Array(UInt32)', 10, 5, 5) LIMIT 100; +INSERT INTO segfault SELECT * FROM generateRandom('id UInt32, uuid UUID, c Array(UInt32)', 10, 5, 5) LIMIT 100; +INSERT INTO segfault SELECT * FROM generateRandom('id UInt32, uuid UUID, c Array(UInt32)', 10, 5, 5) LIMIT 100; + +SELECT ignore(CAST((arrayMap(k -> toString(k), mapKeys(uniqMapMerge(uniq_tags_ids) AS m)), mapValues(m)), 'Map(String, UInt32)')) FROM segfault_mv; From 4a557201c85dcb8f0531a01fdec2dddd3d98840c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 4 Jul 2022 21:59:46 +0200 Subject: [PATCH 196/627] Fix style --- tests/ci/download_release_packets.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/tests/ci/download_release_packets.py b/tests/ci/download_release_packets.py index 919b54e7a6b..0e0f1884fbc 100755 --- a/tests/ci/download_release_packets.py +++ b/tests/ci/download_release_packets.py @@ -1,6 +1,5 @@ #!/usr/bin/env python3 -import re import os import logging @@ -54,16 +53,16 @@ def download_packets(release, dest_path=PACKETS_DIR): return os.path.join(dest_path, pkg_name) for pkg in ( - CLICKHOUSE_COMMON_STATIC_PACKET_NAME, - CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME, + CLICKHOUSE_COMMON_STATIC_PACKET_NAME, + CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME, ): url = (DOWNLOAD_PREFIX + pkg).format(version=release.version, type=release.type) pkg_name = get_dest_path(pkg.format(version=release.version)) download_packet(url, pkg_name) for pkg, fallback in ( - (CLICKHOUSE_SERVER_PACKET_NAME, CLICKHOUSE_SERVER_PACKET_FALLBACK), - (CLICKHOUSE_CLIENT_PACKET_NAME, CLICKHOUSE_CLIENT_PACKET_FALLBACK), + (CLICKHOUSE_SERVER_PACKET_NAME, CLICKHOUSE_SERVER_PACKET_FALLBACK), + (CLICKHOUSE_CLIENT_PACKET_NAME, CLICKHOUSE_CLIENT_PACKET_FALLBACK), ): url = (DOWNLOAD_PREFIX + pkg).format(version=release.version, type=release.type) pkg_name = get_dest_path(pkg.format(version=release.version)) @@ -77,7 +76,6 @@ def download_packets(release, dest_path=PACKETS_DIR): download_packet(url, pkg_name) - def download_last_release(dest_path): current_release = get_previous_release(None) download_packets(current_release, dest_path=dest_path) From 315e6dcdb9ade99a07d51bedb375ba6d152fef5d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 4 Jul 2022 22:06:04 +0200 Subject: [PATCH 197/627] Fix style --- tests/ci/get_previous_release_tag.py | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index 26e4bee8875..a6b24810d0f 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -65,12 +65,7 @@ def get_previous_release(server_version): + " server version" ) - releases = list( - map( - lambda x: ReleaseInfo(x), - releases_str, - ) - ) + releases = [ReleaseInfo(release) for release releases_str] found, previous_release = find_previous_release(server_version, releases) page += 1 From 04948e81cc7e0ae9f5fee2476b22d7d4dfd4b9e6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 5 Jul 2022 00:27:48 +0200 Subject: [PATCH 198/627] Get files changed in master since release is branched --- tests/ci/pr_info.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index a06c5a75b0f..99a06de5f28 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -152,6 +152,13 @@ class PRInfo: self.user_orgs = set(org["id"] for org in response_json) self.diff_urls.append(github_event["pull_request"]["diff_url"]) + if "release" in self.labels: + # For release PRs we must get not only files changed in the PR + # itself, but as well files changed since we branched out + self.diff_urls.append( + f"https://github.com/{GITHUB_REPOSITORY}/" + f"compare/{self.head_ref}...master.diff" + ) elif "commits" in github_event: self.sha = github_event["after"] pull_request = get_pr_for_commit(self.sha, github_event["ref"]) From bb281481ee740e7b480191114509ca544b752905 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jul 2022 05:03:23 +0200 Subject: [PATCH 199/627] Add README, 50% --- benchmark/README.md | 283 +++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 282 insertions(+), 1 deletion(-) diff --git a/benchmark/README.md b/benchmark/README.md index c7fa9cf5adf..2022c9f7f8d 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -1,4 +1,142 @@ -Run all tests on c6a.4xlarge, 500 GB gp2. +# ClickBench: a Benchmark For Analytical Databases + +## Overview + +The benchmark represents typical workload in the following areas: click stream and traffic analysis, web analytics, machine-generated data, structured logs and events data. It covers the typical queries in ad-hoc analytics and realtime dashboards. + +The dataset from this benchmark is obtained from the real traffic recording of one of the world largest web analysis platform. It has been anonymized while keeping all the important distributions of the data. The set of queries was improvised to reflect the realistic workloads, while the queries are not directly from production. + +The main goals of this benchmark are: + +### Reproducibility + +Every test can be easily reproduced in near 20 minutes in semi-automated way. The test setup is documented and using inexpensive cloud VMs. The test process is documented in a form of a shell script, covering installation of every system, loading of the data, running the workload and collecting the result numbers. The dataset is published in multiple formats. + +### Compatibility + +The tables and queries are using mostly standard SQL and require minimum or no adaptation for most of SQL DBMS. The dataset has been filtered to avoid caveats with parsing and loading. + +### Diversity + +The benchmark process is easy enough to cover a wide range of systems. It includes: modern and historical self-managed OLAP DBMS; traditional OLTP DBMS are included for comparison baseline; managed databases as a service offerings are included, as well as serverless cloud-native databases; some NoSQL, document and specialized time-series databases are included as well for a reference, even if they should not be comparable on the same workload. + +### Realism + +The dataset is derived from real production data. The realistic data distributions allow to correctly account for compression, indices, codecs, custom data structures, etc. which is not possible with most of the random dataset generators. The workload consists of 43 queries and can test the effiency of full scan and filtered scan, as well as index lookups, and main relational operations. It can test various aspects of hardware as well: some queries require high storage throughput; some queries benefit from large number of CPU cores and some benefit from single-core speed; some queries benefit from main memory bandwidth. + +## Limitations + +The limitations of this benchmark allow to keep it easy to reproduce and to include more systems in the comparison. The benchmark represents only a subset of all possible workloads and scenarios. While it aims to be as fair as possible, the focus on a specific subset of workloads may give advantage to the systems specialized on that workloads. + +The following limitations should be acknowledged: + +1. The dataset is represented by one flat table. This is not representative for classical data warehouses which are using a normalized star or snowflake data model. The systems for classical data warehouses may get unfair disadvantage on this benchmark. + +2. The table consists of exactly 99 997 497 records. This is rather small by modern standard, but allows to perform tests in reasonable time. + +3. While this benchmark allows testing distributed systems, and it includes multi-node and serverless cloud-native setups, most of the results so far have been obtained on a single node setup. + +4. The benchmark runs queries one after another and does not test a workload with concurrent requests; neither does it test for system capacity. Every query is run only a few times and this allows some variability in the results. + +6. Many setups and systems are different enough to make direct comparison tricky. It is not possible to test the efficiency of storage use for in-memory databases, or the time of data loading for stateless query engines. The goal of the benchmark is to give the numbers for comparison and let you derive the conclusions on your own. + +TLDR: *All Benchmarks Are Bastards*. + +## Rules and Contribution + +### How to add a new result + +To introduce a new system, simply copy-paste one of the directories and edit the files accordingly: +- `benchmark.sh`: this is the main script to run the benchmark on a fresh VM; Ubuntu 22.04 or newer should be used by default, or any other system if this is specified in the comments. The script is not necessarily can be run in a fully automated manner - it is recommended to always copy-paste the commands one by one and observe the results. For managed databases, if the setup requires clicking on the UI, write a `README.md` instead. +- `README.md`: contains comments and observations if needed. For managed databases, it can describe the setup procedure instead of a shell script. +- `create.sql`: a CREATE TABLE statement. If it's a NoSQL system, another file like `wtf.json` can be presented. +- `queries.sql`: contains 43 queries to run; +- `run.sh`: a loop of running the queries; every query is run three times; if it's a database with local on-disk storage, the first query should be run after dropping the page cache; +- `results`: put the .txt files with the results for every hardware configuration there. + +To introduce a new result for existing system on different hardware configuration, add a new file to `results`. + +To introduce a new result for existing system with different usage scenario, either copy-paste the whole directory and name it differently (e.g. `timescaledb`, `timescaledb-compression`) or add a new file to the `results` directory. + +### Installation and fine-tuning + +The systems can be installed or used in any reasonable way: from binary distribution, from Docker container, from package manager, or compiled - whatever is more natural and simple or gives better results. + +It's better to use the default settings and avoid fine-tuning. Configuration changes can be applied if it is considered strictly necessary and documented. + +Fine-tuning and optimization for the benchmark is not recommended but allowed. In this case, add the results on vanilla configuration and fine-tuned configuration separately. + +### Indexing + +The benchmark table has one index - the primary key. The primary key not necessary to be unique. The index of the primary key can be made clustered / ordered / partitioned / sharded. + +Manual creation of other indices is not recommended, although if the system create indexes automatically, it is considered ok. + +### Preaggregation + +Creation of preaggregated tables or indices, projections or materialized views is not recommended for the purpose of this benchmark. Although you can add results on fine-tuned setup for the reference, but they will be out of competition. + +If a system is of "multidimensional OLAP" kind, so always or implicitly doing preaggregations, it can be added for comparison. + +### Caching + +If the system contains a cache for query results, it should be disabled. + +If the system performs caching for source data, it is ok. If the cache can be flushed, it should be flushed before the first run of every query. + +If the system contains a cache for intermediate data, it should be disabled if this cache is located near the end of the query execution pipeline, thus similar to a query result cache. + +### Incomplete Results + +Many systems cannot run full benchmark suite successfully due to OOMs, crashes or unsupported queries. The partial results should be included nevertheless. Put `null` for the missing numbers. + +### If The Results Cannot Be Published + +Some vendors don't allow publishing the benchmark results due to the infamous [DeWitt Clause](https://cube.dev/blog/dewitt-clause-or-can-you-benchmark-a-database). Most of them still allow to use the system for benchmarks. In this case, please submit the full information about installation and reproduction, but without the `results` directory. A `.gitignore` file can be added to prevent accidental publishing. + +We allow both open-source and proprietary systems in our benchmark, as well as managed services, even if registration, credit card or salesperson call is required - you still can submit the testing description if you don't violate the ToS. + +Please let us know if some results were published by mistake by opening an issue on GitHub. + +### If a Mistake Or Misrepresentation Is Found + +It is easy to accidentally misrepresent some systems. While acting in a good faith, the authors admit their lack of deep knowledge of most systems. Please send a pull request to correct the mistakes. + +### Results Usage And Scoreboards + +The results can be used for comparison of various systems, but always take them with a grain of salt due to vast amount of caveats and hidden details. Always reference the original benchmark and this text. + +We allow but not recommend to create scoreboards from this benchmark or tell that one system is better (faster, cheaper, etc) than another. + +## History and Motivation + +The benchmark has been created in October 2013 for evaluating various DBMS to use for a web analytics system. It has been made by taking a 1/50th of one week of production pageviews (a.k.a. "hits") data and taking the first one billion, one hundred million and ten million records from it. It has been run on a 3-node cluster of Xeon E2650v2 with 128 GiB RAM, 8x6TB HDD in md-RAID-6 and 10 Gbit network in a private datacenter in Finland. + +The following systems were tested in 2013: ClickHouse, MonetDB, InfiniDB, Infobright, LucidDB, Vertica, Hive and MySQL. To ensure fairness, the benchmark has been conducted by a person without ClickHouse experience. ClickHouse has been selected for production usage by the results of this benchmark. + +The benchmark continued to be occasionally used privately until 2016, when the results has been published with the ClickHouse release in open-source. While the results were made public, the datasets were not, as they contain customer data. + +We needed to publish the dataset to facilitate open-source development and testing, but it was not possible to do as is. In 2019, `clickhouse-obfuscator` tool has been introduced to anonymize the data, and the dataset has been published. Read more about the challenge of data obfuscation [here](https://habr.com/en/company/yandex/blog/485096/). + +More systems were included in the benchmark over time: Greenplum, MemSQL (now SingleStore), OmniSci (now HeavyAI), DuckDB, PostgreSQL and TimescaleDB. + +In [2021](https://clickhouse.com/blog/introducing-click-house-inc/) the original cluster for benchmark stopped to be used, and we were unable to add new results without rerunning the old results on a different hardware. Rerunning the old results appeared to be difficult: due to natural churn of the software the old step-by-step instructions become stale. + +The original benchmark dataset included many details that were natural for ClickHouse and web analytics data but hard for other systems: unsigned integers (not supported by standard SQL), strings with zero bytes, fixed length string data types, etc. Only ClickHouse is being able to load the dataset as is, while most other databases require non-trivial adjustments to the data and queries. + +The idea of the new benchmark is: +- normalize the dataset to a "common denominator", so it can be loaded to most of the systems without a hassle. +- normalize the queries to use only standard SQL - they will not use any advantages of ClickHouse, but will be runnable on every system. +- ideally make it automated. At least make it simple - runnable by a short shell script that can be run by copy-pasting a few commands in the terminal, in worst case. +- run everything on widely available cloud VMs and allow to record the results from various type of instances. + +The benchmark is created and used by ClickHouse team. It can be surprising, but we [did not perform](https://clickhouse.com/blog/clickhouse-over-the-years-with-benchmarks/) any specific optimizations in ClickHouse for the queries in the benchmark, which allowed to keep some reasonable sense of fairness with respect to other systems. + +Now the new benchmark is easy to use and the results for any system can be reproduced in around 20 minutes. + +We also introduced the [Hardware Benchmark](https://clickhouse.com/benchmark/hardware/) for testing servers and VMs. + +## Systems Included - [x] ClickHouse - [ ] ClickHouse operating like "Athena" @@ -63,3 +201,146 @@ Run all tests on c6a.4xlarge, 500 GB gp2. - [ ] LocustDB - [ ] EventQL - [ ] Apache Drill + +By default, all tests are run on c6a.4xlarge VM in AWS with 500 GB gp2. + +Please help us add more systems and running the benchmarks on more types of VMs. + +## Similar Projects + +There are many alternative benchmarks applicable to OLAP DBMS with their own advantages and disadvantages. + +### Brown University Mgbench + +https://github.com/crottyan/mgbench + +A new analytical benchmark for machine-generated log data. By Andrew Crottyan from Brown University. + +Advantages: +- somewhat realistic dataset; +- diverse set of queries; +- good coverage of systems; +- easy to reproduce; + +Disadvantages: +- very small dataset size; +- favors in-memory databases; +- mostly abandoned. + +### UC Berkeley AMPLab Big Data Benchmark + +https://amplab.cs.berkeley.edu/benchmark/ + +Poor coverage of queries that are too simple. The benchmark is abandoned. + +### Mark Litwinschik's NYC Taxi + +https://tech.marksblogg.com/benchmarks.html + +Advantages: +- real-world dataset; +- good coverage of systems; many unusual entries; +- contains a story for every benchmark entry; + +Disadvantages: +- unreasonably small set of queries: 4 mostly trivial queries don't represent any realistic workload and are subjects for overoptimizations; +- compares different systems on different hardware; +- many results are outdated; +- no automated or easy way to reproduce the results; +- while many results are performed independently of corporations or academia, some benchmark entries may have been sponsored; +- the dataset is not readily available for downloads: originally 1.1 billion records are used, while it's more 4 billion records in 2022. + +### Database-like ops Benchmark from h2o.ai + +https://h2oai.github.io/db-benchmark/ + +A benchmark for dataframe libraries and embedded databases. Good coverage of dataframe libraries and a few full featured DBMS as well. + +### A benchmark for querying large JSON datasets + +https://colab.research.google.com/github/dcmoura/spyql/blob/master/notebooks/json_benchmark.ipynb + +A good benchmark for command line tools for processing semistructured data. Can be used to test DBMS as well. + +### Star Schema Benchmark + +Pat O'Neil, Betty O'Neil, Xuedong Chen +https://www.cs.umb.edu/~poneil/StarSchemaB.PDF + +It is a simplified version of TPC-H. + +Advantages: +- well specified; +- popular in academia; + +Disadvantages: +- represents a classic data warehouse schema; +- database generator produces random distributions that are not realistic and the benchmark does not allow to capture the difference in various optimizations that matter on real world data; +- many research systems in academia targeting for this benchmark which makes many aspects of it exhausted; + +### TPC-H + +A benchmark suite from Transaction Processing Council - one of the oldest organization specialized in DBMS benchmarks. + +Advantages: +- well specified; + +Disadvantages: +- requires official certification; +- represents a classic data warehouse schema; +- database generator produces random distributions that are not realistic and the benchmark does not allow to capture the difference in various optimizations that matter on real world data; +- many systems are targeting for this benchmark which makes many aspects of it exhausted; + +### TPC-DS + +More advanced than TPC-H, focused on complex ad-hoc queries. Requires official certification as well. + +Advantages: +- extensive collection of complex queries. + +Disadvantages: +- requires official certification; +- official results have only sparse coverage of systems; +- biased towards complex queries over many tables. + +### A benchmark on Ontime Flight Data + +Introduced by Vadim Tkachenko from Percona [in 2009](https://www.percona.com/blog/2009/10/02/analyzing-air-traffic-performance-with-infobright-and-monetdb/). + +Based on the US Bureau of Transportation Statistics open data. + +Advantages: +- real-world dataset; + +Disadvantages: +- not widely used; +- the set of queries is not standardized; +- the table contains too much redundancy; + +### TSBS + +Time Series Benchmark Suite. https://github.com/timescale/tsbs +Originally from InfluxDB, and supported by TimescaleDB. + +Advantages: +- a benchmark for time-series scenarios + +Disadvantages: +- not applicable for scenarios with data analytics. + +### STAC + +https://www.stacresearch.com/ + +Disadvantages: +- requires paid membership. + +### More + +If you know more well-defined, realistic and reproducible benchmarks for analytical workloads, please let me know. + +I collect every benchmark that includes ClickHouse [here](https://github.com/ClickHouse/ClickHouse/issues/22398). + +### References and Citation. + +Alexey Milovidov, 2022. From 431d0c801e10aa8bf1ad5e684c5ee94692f50559 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jul 2022 05:35:00 +0200 Subject: [PATCH 200/627] Add README, 50% --- benchmark/README.md | 104 ++++++++++++++++++++++---------------------- 1 file changed, 53 insertions(+), 51 deletions(-) diff --git a/benchmark/README.md b/benchmark/README.md index 2022c9f7f8d..2212b87c6bd 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -2,43 +2,45 @@ ## Overview -The benchmark represents typical workload in the following areas: click stream and traffic analysis, web analytics, machine-generated data, structured logs and events data. It covers the typical queries in ad-hoc analytics and realtime dashboards. +The benchmark represents typical workload in the following areas: click stream and traffic analysis, web analytics, machine-generated data, structured logs, and events data. It covers the typical queries in ad-hoc analytics and real-time dashboards. -The dataset from this benchmark is obtained from the real traffic recording of one of the world largest web analysis platform. It has been anonymized while keeping all the important distributions of the data. The set of queries was improvised to reflect the realistic workloads, while the queries are not directly from production. +The dataset from this benchmark is obtained from the real traffic recording of one of the world's largest web analytics platforms. It has been anonymized while keeping all the important distributions of the data. The set of queries was improvised to reflect the realistic workloads, while the queries are not directly from production. + +## Goals The main goals of this benchmark are: ### Reproducibility -Every test can be easily reproduced in near 20 minutes in semi-automated way. The test setup is documented and using inexpensive cloud VMs. The test process is documented in a form of a shell script, covering installation of every system, loading of the data, running the workload and collecting the result numbers. The dataset is published in multiple formats. +Every test can be easily reproduced in nearly 20 minutes in a semi-automated way. The test setup is documented and uses inexpensive cloud VMs. The test process is documented in a form of a shell script, covering the installation of every system, loading of the data, running the workload, and collecting the result numbers. The dataset is published and made available for download in multiple formats. ### Compatibility -The tables and queries are using mostly standard SQL and require minimum or no adaptation for most of SQL DBMS. The dataset has been filtered to avoid caveats with parsing and loading. +The tables and queries are using mostly standard SQL and require minimum or no adaptation for most SQL DBMS. The dataset has been filtered to avoid difficulties with parsing and loading. ### Diversity -The benchmark process is easy enough to cover a wide range of systems. It includes: modern and historical self-managed OLAP DBMS; traditional OLTP DBMS are included for comparison baseline; managed databases as a service offerings are included, as well as serverless cloud-native databases; some NoSQL, document and specialized time-series databases are included as well for a reference, even if they should not be comparable on the same workload. +The benchmark process is easy enough to cover a wide range of systems. It includes: modern and historical self-managed OLAP DBMS; traditional OLTP DBMS are included for comparison baseline; managed database-as-a-service offerings are included, as well as serverless cloud-native databases; some NoSQL, document, and specialized time-series databases are included as well for a reference, even if they should not be comparable on the same workload. ### Realism -The dataset is derived from real production data. The realistic data distributions allow to correctly account for compression, indices, codecs, custom data structures, etc. which is not possible with most of the random dataset generators. The workload consists of 43 queries and can test the effiency of full scan and filtered scan, as well as index lookups, and main relational operations. It can test various aspects of hardware as well: some queries require high storage throughput; some queries benefit from large number of CPU cores and some benefit from single-core speed; some queries benefit from main memory bandwidth. +The dataset is derived from real production data. The realistic data distributions allow to correctly account for compression, indices, codecs, custom data structures, etc. which is not possible with most of the random dataset generators. The workload consists of 43 queries and can test the efficiency of full scan and filtered scan, as well as index lookups, and main relational operations. It can test various aspects of hardware as well: some queries require high storage throughput; some queries benefit from a large number of CPU cores and some benefit from single-core speed; some queries benefit from high main memory bandwidth. ## Limitations -The limitations of this benchmark allow to keep it easy to reproduce and to include more systems in the comparison. The benchmark represents only a subset of all possible workloads and scenarios. While it aims to be as fair as possible, the focus on a specific subset of workloads may give advantage to the systems specialized on that workloads. +The limitations of this benchmark allow keeping it easy to reproduce and to include more systems in the comparison. The benchmark represents only a subset of all possible workloads and scenarios. While it aims to be as fair as possible, the focus on a specific subset of workloads may give an advantage to the systems specialized in that workloads. The following limitations should be acknowledged: -1. The dataset is represented by one flat table. This is not representative for classical data warehouses which are using a normalized star or snowflake data model. The systems for classical data warehouses may get unfair disadvantage on this benchmark. +1. The dataset is represented by one flat table. This is not representative of classical data warehouses which are using a normalized star or snowflake data model. The systems for classical data warehouses may get an unfair disadvantage on this benchmark. -2. The table consists of exactly 99 997 497 records. This is rather small by modern standard, but allows to perform tests in reasonable time. +2. The table consists of exactly 99 997 497 records. This is rather small by modern standards but allows to perform tests in a reasonable time. 3. While this benchmark allows testing distributed systems, and it includes multi-node and serverless cloud-native setups, most of the results so far have been obtained on a single node setup. 4. The benchmark runs queries one after another and does not test a workload with concurrent requests; neither does it test for system capacity. Every query is run only a few times and this allows some variability in the results. -6. Many setups and systems are different enough to make direct comparison tricky. It is not possible to test the efficiency of storage use for in-memory databases, or the time of data loading for stateless query engines. The goal of the benchmark is to give the numbers for comparison and let you derive the conclusions on your own. +6. Many setups and systems are different enough to make direct comparison tricky. It is not possible to test the efficiency of storage used for in-memory databases, or the time of data loading for stateless query engines. The goal of the benchmark is to give the numbers for comparison and let you derive the conclusions on your own. TLDR: *All Benchmarks Are Bastards*. @@ -47,36 +49,36 @@ TLDR: *All Benchmarks Are Bastards*. ### How to add a new result To introduce a new system, simply copy-paste one of the directories and edit the files accordingly: -- `benchmark.sh`: this is the main script to run the benchmark on a fresh VM; Ubuntu 22.04 or newer should be used by default, or any other system if this is specified in the comments. The script is not necessarily can be run in a fully automated manner - it is recommended to always copy-paste the commands one by one and observe the results. For managed databases, if the setup requires clicking on the UI, write a `README.md` instead. +- `benchmark.sh`: this is the main script to run the benchmark on a fresh VM; Ubuntu 22.04 or newer should be used by default, or any other system if this is specified in the comments. The script is not necessarily can be run in a fully automated manner - it is recommended to always copy-paste the commands one by one and observe the results. For managed databases, if the setup requires clicking in the UI, write a `README.md` instead. - `README.md`: contains comments and observations if needed. For managed databases, it can describe the setup procedure instead of a shell script. - `create.sql`: a CREATE TABLE statement. If it's a NoSQL system, another file like `wtf.json` can be presented. - `queries.sql`: contains 43 queries to run; - `run.sh`: a loop of running the queries; every query is run three times; if it's a database with local on-disk storage, the first query should be run after dropping the page cache; - `results`: put the .txt files with the results for every hardware configuration there. -To introduce a new result for existing system on different hardware configuration, add a new file to `results`. +To introduce a new result for the existing system on different hardware configurations, add a new file to `results`. -To introduce a new result for existing system with different usage scenario, either copy-paste the whole directory and name it differently (e.g. `timescaledb`, `timescaledb-compression`) or add a new file to the `results` directory. +To introduce a new result for an existing system with a different usage scenario, either copy the whole directory and name it differently (e.g. `timescaledb`, `timescaledb-compression`) or add a new file to the `results` directory. ### Installation and fine-tuning -The systems can be installed or used in any reasonable way: from binary distribution, from Docker container, from package manager, or compiled - whatever is more natural and simple or gives better results. +The systems can be installed or used in any reasonable way: from a binary distribution, from a Docker container, from the package manager, or compiled - whatever is more natural and simple or gives better results. It's better to use the default settings and avoid fine-tuning. Configuration changes can be applied if it is considered strictly necessary and documented. -Fine-tuning and optimization for the benchmark is not recommended but allowed. In this case, add the results on vanilla configuration and fine-tuned configuration separately. +Fine-tuning and optimization for the benchmark are not recommended but allowed. In this case, add the results on vanilla configuration and fine-tuned configuration separately. ### Indexing -The benchmark table has one index - the primary key. The primary key not necessary to be unique. The index of the primary key can be made clustered / ordered / partitioned / sharded. +The benchmark table has one index - the primary key. The primary key is not necessary to be unique. The index of the primary key can be made clustered (ordered, partitioned, sharded). -Manual creation of other indices is not recommended, although if the system create indexes automatically, it is considered ok. +Manual creation of other indices is not recommended, although if the system creates indexes automatically, it is considered ok. ### Preaggregation -Creation of preaggregated tables or indices, projections or materialized views is not recommended for the purpose of this benchmark. Although you can add results on fine-tuned setup for the reference, but they will be out of competition. +The creation of pre-aggregated tables or indices, projections, or materialized views is not recommended for the purpose of this benchmark. Although you can add results on fine-tuned setup for reference, they will be out of competition. -If a system is of "multidimensional OLAP" kind, so always or implicitly doing preaggregations, it can be added for comparison. +If a system is of a "multidimensional OLAP" kind, so always or implicitly doing aggregations, it can be added for comparison. ### Caching @@ -88,13 +90,13 @@ If the system contains a cache for intermediate data, it should be disabled if t ### Incomplete Results -Many systems cannot run full benchmark suite successfully due to OOMs, crashes or unsupported queries. The partial results should be included nevertheless. Put `null` for the missing numbers. +Many systems cannot run the full benchmark suite successfully due to OOMs, crashes, or unsupported queries. The partial results should be included nevertheless. Put `null` for the missing numbers. ### If The Results Cannot Be Published -Some vendors don't allow publishing the benchmark results due to the infamous [DeWitt Clause](https://cube.dev/blog/dewitt-clause-or-can-you-benchmark-a-database). Most of them still allow to use the system for benchmarks. In this case, please submit the full information about installation and reproduction, but without the `results` directory. A `.gitignore` file can be added to prevent accidental publishing. +Some vendors don't allow publishing the benchmark results due to the infamous [DeWitt Clause](https://cube.dev/blog/dewitt-clause-or-can-you-benchmark-a-database). Most of them are still allowed to use the system for benchmarks. In this case, please submit the full information about installation and reproduction, but without the `results` directory. A `.gitignore` file can be added to prevent accidental publishing. -We allow both open-source and proprietary systems in our benchmark, as well as managed services, even if registration, credit card or salesperson call is required - you still can submit the testing description if you don't violate the ToS. +We allow both open-source and proprietary systems in our benchmark, as well as managed services, even if registration, credit card, or salesperson call is required - you still can submit the testing description if you don't violate the TOS. Please let us know if some results were published by mistake by opening an issue on GitHub. @@ -104,33 +106,33 @@ It is easy to accidentally misrepresent some systems. While acting in a good fai ### Results Usage And Scoreboards -The results can be used for comparison of various systems, but always take them with a grain of salt due to vast amount of caveats and hidden details. Always reference the original benchmark and this text. +The results can be used for comparison of various systems, but always take them with a grain of salt due to the vast amount of caveats and hidden details. Always reference the original benchmark and this text. -We allow but not recommend to create scoreboards from this benchmark or tell that one system is better (faster, cheaper, etc) than another. +We allow but do not recommend creating scoreboards from this benchmark or tell that one system is better (faster, cheaper, etc) than another. ## History and Motivation -The benchmark has been created in October 2013 for evaluating various DBMS to use for a web analytics system. It has been made by taking a 1/50th of one week of production pageviews (a.k.a. "hits") data and taking the first one billion, one hundred million and ten million records from it. It has been run on a 3-node cluster of Xeon E2650v2 with 128 GiB RAM, 8x6TB HDD in md-RAID-6 and 10 Gbit network in a private datacenter in Finland. +The benchmark has been created in October 2013 for evaluating various DBMS to use for a web analytics system. It has been made by taking a 1/50th of one week of production pageviews (a.k.a. "hits") data and taking the first one billion, one hundred million, and ten million records from it. It has been run on a 3-node cluster of Xeon E2650v2 with 128 GiB RAM, 8x6TB HDD in md-RAID-6, and 10 Gbit network in a private datacenter in Finland. The following systems were tested in 2013: ClickHouse, MonetDB, InfiniDB, Infobright, LucidDB, Vertica, Hive and MySQL. To ensure fairness, the benchmark has been conducted by a person without ClickHouse experience. ClickHouse has been selected for production usage by the results of this benchmark. -The benchmark continued to be occasionally used privately until 2016, when the results has been published with the ClickHouse release in open-source. While the results were made public, the datasets were not, as they contain customer data. +The benchmark continued to be occasionally used privately until 2016 when the results has been published with the ClickHouse release in open-source. While the results were made public, the datasets were not, as they contain customer data. -We needed to publish the dataset to facilitate open-source development and testing, but it was not possible to do as is. In 2019, `clickhouse-obfuscator` tool has been introduced to anonymize the data, and the dataset has been published. Read more about the challenge of data obfuscation [here](https://habr.com/en/company/yandex/blog/485096/). +We needed to publish the dataset to facilitate open-source development and testing, but it was not possible to do it as is. In 2019, the `clickhouse-obfuscator` tool has been introduced to anonymize the data, and the dataset has been published. Read more about the challenge of data obfuscation [here](https://habr.com/en/company/yandex/blog/485096/). -More systems were included in the benchmark over time: Greenplum, MemSQL (now SingleStore), OmniSci (now HeavyAI), DuckDB, PostgreSQL and TimescaleDB. +More systems were included in the benchmark over time: Greenplum, MemSQL (now SingleStore), OmniSci (now HeavyAI), DuckDB, PostgreSQL, and TimescaleDB. -In [2021](https://clickhouse.com/blog/introducing-click-house-inc/) the original cluster for benchmark stopped to be used, and we were unable to add new results without rerunning the old results on a different hardware. Rerunning the old results appeared to be difficult: due to natural churn of the software the old step-by-step instructions become stale. +In [2021](https://clickhouse.com/blog/introducing-click-house-inc/) the original cluster for benchmark stopped being used, and we were unable to add new results without rerunning the old results on different hardware. Rerunning the old results appeared to be difficult: due to the natural churn of the software, the old step-by-step instructions become stale. -The original benchmark dataset included many details that were natural for ClickHouse and web analytics data but hard for other systems: unsigned integers (not supported by standard SQL), strings with zero bytes, fixed length string data types, etc. Only ClickHouse is being able to load the dataset as is, while most other databases require non-trivial adjustments to the data and queries. +The original benchmark dataset included many details that were natural for ClickHouse and web analytics data but hard for other systems: unsigned integers (not supported by standard SQL), strings with zero bytes, fixed-length string data types, etc. Only ClickHouse is being able to load the dataset as is, while most other databases require non-trivial adjustments to the data and queries. The idea of the new benchmark is: - normalize the dataset to a "common denominator", so it can be loaded to most of the systems without a hassle. -- normalize the queries to use only standard SQL - they will not use any advantages of ClickHouse, but will be runnable on every system. -- ideally make it automated. At least make it simple - runnable by a short shell script that can be run by copy-pasting a few commands in the terminal, in worst case. -- run everything on widely available cloud VMs and allow to record the results from various type of instances. +- normalize the queries to use only standard SQL - they will not use any advantages of ClickHouse but will be runnable on every system. +- ideally make it automated. At least make it simple - runnable by a short shell script that can be run by copy-pasting a few commands in the terminal, in the worst case. +- run everything on widely available cloud VMs and allow to record the results from various types of instances. -The benchmark is created and used by ClickHouse team. It can be surprising, but we [did not perform](https://clickhouse.com/blog/clickhouse-over-the-years-with-benchmarks/) any specific optimizations in ClickHouse for the queries in the benchmark, which allowed to keep some reasonable sense of fairness with respect to other systems. +The benchmark is created and used by the ClickHouse team. It can be surprising, but we [did not perform](https://clickhouse.com/blog/clickhouse-over-the-years-with-benchmarks/) any specific optimizations in ClickHouse for the queries in the benchmark, which allowed us to keep some reasonable sense of fairness with respect to other systems. Now the new benchmark is easy to use and the results for any system can be reproduced in around 20 minutes. @@ -204,7 +206,7 @@ We also introduced the [Hardware Benchmark](https://clickhouse.com/benchmark/har By default, all tests are run on c6a.4xlarge VM in AWS with 500 GB gp2. -Please help us add more systems and running the benchmarks on more types of VMs. +Please help us add more systems and run the benchmarks on more types of VMs. ## Similar Projects @@ -218,7 +220,7 @@ A new analytical benchmark for machine-generated log data. By Andrew Crottyan fr Advantages: - somewhat realistic dataset; -- diverse set of queries; +- a diverse set of queries; - good coverage of systems; - easy to reproduce; @@ -243,24 +245,24 @@ Advantages: - contains a story for every benchmark entry; Disadvantages: -- unreasonably small set of queries: 4 mostly trivial queries don't represent any realistic workload and are subjects for overoptimizations; +- unreasonably small set of queries: 4 mostly trivial queries don't represent any realistic workload and are subjects for over-optimization; - compares different systems on different hardware; - many results are outdated; - no automated or easy way to reproduce the results; - while many results are performed independently of corporations or academia, some benchmark entries may have been sponsored; -- the dataset is not readily available for downloads: originally 1.1 billion records are used, while it's more 4 billion records in 2022. +- the dataset is not readily available for downloads: originally 1.1 billion records are used, while it's more than 4 billion records in 2022. ### Database-like ops Benchmark from h2o.ai https://h2oai.github.io/db-benchmark/ -A benchmark for dataframe libraries and embedded databases. Good coverage of dataframe libraries and a few full featured DBMS as well. +A benchmark for data-frame libraries and embedded databases. Good coverage of data-frame libraries and a few full-featured DBMS as well. ### A benchmark for querying large JSON datasets https://colab.research.google.com/github/dcmoura/spyql/blob/master/notebooks/json_benchmark.ipynb -A good benchmark for command line tools for processing semistructured data. Can be used to test DBMS as well. +A good benchmark for command-line tools for processing semistructured data. Can be used to test DBMS as well. ### Star Schema Benchmark @@ -270,40 +272,40 @@ https://www.cs.umb.edu/~poneil/StarSchemaB.PDF It is a simplified version of TPC-H. Advantages: -- well specified; +- well-specified; - popular in academia; Disadvantages: - represents a classic data warehouse schema; -- database generator produces random distributions that are not realistic and the benchmark does not allow to capture the difference in various optimizations that matter on real world data; +- database generator produces random distributions that are not realistic and the benchmark does not allow to capture the difference in various optimizations that matter on real-world data; - many research systems in academia targeting for this benchmark which makes many aspects of it exhausted; ### TPC-H -A benchmark suite from Transaction Processing Council - one of the oldest organization specialized in DBMS benchmarks. +A benchmark suite from Transaction Processing Council - one of the oldest organizations specializing in DBMS benchmarks. Advantages: -- well specified; +- well-specified; Disadvantages: - requires official certification; - represents a classic data warehouse schema; -- database generator produces random distributions that are not realistic and the benchmark does not allow to capture the difference in various optimizations that matter on real world data; -- many systems are targeting for this benchmark which makes many aspects of it exhausted; +- database generator produces random distributions that are not realistic and the benchmark does not allow to capture the difference in various optimizations that matter on real-world data; +- many systems are targeting this benchmark which makes many aspects of it exhausted; ### TPC-DS More advanced than TPC-H, focused on complex ad-hoc queries. Requires official certification as well. Advantages: -- extensive collection of complex queries. +- an extensive collection of complex queries. Disadvantages: - requires official certification; - official results have only sparse coverage of systems; - biased towards complex queries over many tables. -### A benchmark on Ontime Flight Data +### Ontime Introduced by Vadim Tkachenko from Percona [in 2009](https://www.percona.com/blog/2009/10/02/analyzing-air-traffic-performance-with-infobright-and-monetdb/). @@ -323,7 +325,7 @@ Time Series Benchmark Suite. https://github.com/timescale/tsbs Originally from InfluxDB, and supported by TimescaleDB. Advantages: -- a benchmark for time-series scenarios +- a benchmark for time-series scenarios; Disadvantages: - not applicable for scenarios with data analytics. @@ -333,11 +335,11 @@ Disadvantages: https://www.stacresearch.com/ Disadvantages: -- requires paid membership. +- requires a paid membership. ### More -If you know more well-defined, realistic and reproducible benchmarks for analytical workloads, please let me know. +If you know more well-defined, realistic, and reproducible benchmarks for analytical workloads, please let me know. I collect every benchmark that includes ClickHouse [here](https://github.com/ClickHouse/ClickHouse/issues/22398). From 4a3e6870e88fef7f983734cbeb78b0aeac08cd9e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jul 2022 05:40:14 +0200 Subject: [PATCH 201/627] Add acknowledge from @BohuTANG --- benchmark/databend/README.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/benchmark/databend/README.md b/benchmark/databend/README.md index 4e77d54094f..fd7877efb27 100644 --- a/benchmark/databend/README.md +++ b/benchmark/databend/README.md @@ -1 +1,7 @@ -It is written in Rust and is blazing slow. +It is written in Rust and is blazing. + +Update from @BohuTANG: + +> Thanks for the benchmark! +> Databend is a cloud warehouse designed for object storage(like Amazon S3), not the local file system. The FS model is only used for testing for some cases, we didn't do any optimization, and we know it has some performance issues. +> I believe that ClickHouse is also being designed for the cloud, and looking forward to the S3 benchmark results :) From 9e1b3be0c948f206d4f5f303bce0a227428cb67f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jul 2022 05:47:55 +0200 Subject: [PATCH 202/627] Add README, 51% --- benchmark/README.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/benchmark/README.md b/benchmark/README.md index 2212b87c6bd..7c4c6f69211 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -328,7 +328,14 @@ Advantages: - a benchmark for time-series scenarios; Disadvantages: -- not applicable for scenarios with data analytics. +- not applicable for scenarios with data analytics. + +### Fair Database Benchmarks + +https://github.com/db-benchmarks/db-benchmarks + +A benchmark suite inspired by ClickHouse benchmarks. +Used mostly to compare search engines: Elasticsearch and Manticore. ### STAC From 76813ee3c05f5d3bbfc7361b53c1ba9008a02e50 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 5 Jul 2022 11:49:55 +0800 Subject: [PATCH 203/627] update codes --- src/Storages/Hive/StorageHive.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 040a59f8cb4..6e52b002bf0 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -925,7 +925,6 @@ void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPt void StorageHive::alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder & /*alter_lock_holder*/) { auto table_id = getStorageID(); - checkAlterIsPossible(params, local_context); auto metadata_snapshot = getInMemoryMetadataPtr(); StorageInMemoryMetadata new_metadata = *metadata_snapshot; params.apply(new_metadata, local_context); From 51923e43995cf3ad63bd67feba2c5536cd114a1d Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Tue, 5 Jul 2022 09:25:32 +0530 Subject: [PATCH 204/627] update integration tests doc --- tests/integration/README.md | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index ef0b5a4b334..2d44ff70861 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -77,25 +77,25 @@ Notes: You can run tests via `./runner` script and pass pytest arguments as last arg: ``` -$ ./runner --binary $HOME/ClickHouse/programs/clickhouse --bridge-binary $HOME/ClickHouse/programs/clickhouse-odbc-bridge --base-configs-dir $HOME/ClickHouse/programs/server/ 'test_odbc_interaction -ss' +$ ./runner --binary $HOME/ClickHouse/programs/clickhouse --odbc-bridge-binary $HOME/ClickHouse/programs/clickhouse-odbc-bridge --base-configs-dir $HOME/ClickHouse/programs/server/ 'test_ssl_cert_authentication -ss' Start tests -============================= test session starts ============================== -platform linux2 -- Python 2.7.15rc1, pytest-4.0.0, py-1.7.0, pluggy-0.8.0 -rootdir: /ClickHouse/tests/integration, inifile: pytest.ini -collected 6 items +====================================================================================================== test session starts ====================================================================================================== +platform linux -- Python 3.8.10, pytest-7.1.2, pluggy-1.0.0 -- /usr/bin/python3 +cachedir: .pytest_cache +rootdir: /ClickHouse/tests/integration, configfile: pytest.ini +plugins: repeat-0.9.1, xdist-2.5.0, forked-1.4.0, order-1.0.0, timeout-2.1.0 +timeout: 900.0s +timeout method: signal +timeout func_only: False +collected 4 items -test_odbc_interaction/test.py Removing network clickhouse_default -... +test_ssl_cert_authentication/test.py::test_https Copy common default production configuration from /clickhouse-config. Files: config.xml, users.xml +PASSED +test_ssl_cert_authentication/test.py::test_https_wrong_cert PASSED +test_ssl_cert_authentication/test.py::test_https_non_ssl_auth PASSED +test_ssl_cert_authentication/test.py::test_create_user PASSED -Killing roottestodbcinteraction_node1_1 ... done -Killing roottestodbcinteraction_mysql1_1 ... done -Killing roottestodbcinteraction_postgres1_1 ... done -Removing roottestodbcinteraction_node1_1 ... done -Removing roottestodbcinteraction_mysql1_1 ... done -Removing roottestodbcinteraction_postgres1_1 ... done -Removing network roottestodbcinteraction_default - -==================== 6 passed, 1 warnings in 95.21 seconds ===================== +================================================================================================= 4 passed in 118.58s (0:01:58) ================================================================================================= ``` From 0def9a89f75d6bdca9d60b5c9e32cfc04c8924a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jul 2022 06:08:44 +0200 Subject: [PATCH 205/627] Update simdjson --- contrib/simdjson | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/simdjson b/contrib/simdjson index de196dd7a3a..1075e8609c4 160000 --- a/contrib/simdjson +++ b/contrib/simdjson @@ -1 +1 @@ -Subproject commit de196dd7a3a16e4056b0551ffa3b85c2f52581e1 +Subproject commit 1075e8609c4afa253162d441437af929c29e31bb From edad44ba03ee0a6211c603111d1d3d67415502f8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jul 2022 08:08:09 +0200 Subject: [PATCH 206/627] Add README, 52% --- benchmark/README.md | 25 +++++++++++++++++++++---- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/benchmark/README.md b/benchmark/README.md index 7c4c6f69211..bfbac6f36d2 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -46,7 +46,7 @@ TLDR: *All Benchmarks Are Bastards*. ## Rules and Contribution -### How to add a new result +### How To Add a New Result To introduce a new system, simply copy-paste one of the directories and edit the files accordingly: - `benchmark.sh`: this is the main script to run the benchmark on a fresh VM; Ubuntu 22.04 or newer should be used by default, or any other system if this is specified in the comments. The script is not necessarily can be run in a fully automated manner - it is recommended to always copy-paste the commands one by one and observe the results. For managed databases, if the setup requires clicking in the UI, write a `README.md` instead. @@ -60,7 +60,7 @@ To introduce a new result for the existing system on different hardware configur To introduce a new result for an existing system with a different usage scenario, either copy the whole directory and name it differently (e.g. `timescaledb`, `timescaledb-compression`) or add a new file to the `results` directory. -### Installation and fine-tuning +### Installation And Fine-Tuning The systems can be installed or used in any reasonable way: from a binary distribution, from a Docker container, from the package manager, or compiled - whatever is more natural and simple or gives better results. @@ -68,6 +68,19 @@ It's better to use the default settings and avoid fine-tuning. Configuration cha Fine-tuning and optimization for the benchmark are not recommended but allowed. In this case, add the results on vanilla configuration and fine-tuned configuration separately. +### Data Loading + +The dataset is available in `CSV`, `TSV`, `JSONlines` and `Parquet` formats by the following links: + +https://datasets.clickhouse.com/hits_compatible/hits.csv.gz +https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz +https://datasets.clickhouse.com/hits_compatible/hits.json.gz +https://datasets.clickhouse.com/hits_compatible/hits.parquet +https://datasets.clickhouse.com/hits_compatible/athena/hits.parquet +https://datasets.clickhouse.com/hits_compatible/athena_partitioned/hits_{0..99}.parquet + +To correctly compare insertion time, the dataset should be downloaded and decompressed before loading. The dataset should be loaded as a single file in the most straightforward way. Splitting the dataset for parallel loading is not recommended, as it will make comparisons more difficult. Splitting the dataset is possible if the system cannot eat it as a whole due to its limitations. + ### Indexing The benchmark table has one index - the primary key. The primary key is not necessary to be unique. The index of the primary key can be made clustered (ordered, partitioned, sharded). @@ -344,12 +357,16 @@ https://www.stacresearch.com/ Disadvantages: - requires a paid membership. -### More +### More... If you know more well-defined, realistic, and reproducible benchmarks for analytical workloads, please let me know. I collect every benchmark that includes ClickHouse [here](https://github.com/ClickHouse/ClickHouse/issues/22398). -### References and Citation. +## Additional Outcomes + +This benchmark can be used to collect the snippets for installation and data loading across a wide variety of DBMS. The usability and quality of the documentation can be compared. It has been used to improve the quality of the participants as demonstrated in [duckdb#3969](https://github.com/duckdb/duckdb/issues/3969), [timescaledb#4473](https://github.com/timescale/timescaledb/issues/4473), [mariadb-corporation#16](https://github.com/mariadb-corporation/mariadb-community-columnstore-docker/issues/16), [MonetDB#7309](https://github.com/duckdb/duckdb/issues/3969), [questdb#2272](https://github.com/questdb/questdb/issues/2272), [crate#12654](https://github.com/crate/crate/issues/12654), [LocustDB#152](https://github.com/cswinter/LocustDB/issues/152), etc; + +### References and Citation Alexey Milovidov, 2022. From 849c46e6fa906f4ee00255bf31d3df82d8dba08c Mon Sep 17 00:00:00 2001 From: lokax Date: Tue, 5 Jul 2022 15:23:07 +0800 Subject: [PATCH 207/627] feat(Function): isNullable --- src/Functions/isNullable.cpp | 60 +++++++++++++++++++++++++ src/Functions/registerFunctionsNull.cpp | 2 + 2 files changed, 62 insertions(+) create mode 100644 src/Functions/isNullable.cpp diff --git a/src/Functions/isNullable.cpp b/src/Functions/isNullable.cpp new file mode 100644 index 00000000000..27a559c3dc4 --- /dev/null +++ b/src/Functions/isNullable.cpp @@ -0,0 +1,60 @@ +#include +#include +#include +#include + +namespace DB +{ +namespace +{ + +class FunctionIsNullable : public IFunction +{ +public: + static constexpr auto name = "isNullable"; + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + bool useDefaultImplementationForNothing() const override { return false; } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & elem = arguments[0]; + return ColumnUInt8::create(input_rows_count, isColumnNullable(*elem.column)); + } +}; + +} + +void registerFunctionIsNullable(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + diff --git a/src/Functions/registerFunctionsNull.cpp b/src/Functions/registerFunctionsNull.cpp index 238133fbb67..d94ee95d28f 100644 --- a/src/Functions/registerFunctionsNull.cpp +++ b/src/Functions/registerFunctionsNull.cpp @@ -11,6 +11,7 @@ void registerFunctionNullIf(FunctionFactory & factory); void registerFunctionAssumeNotNull(FunctionFactory & factory); void registerFunctionToNullable(FunctionFactory & factory); void registerFunctionIsZeroOrNull(FunctionFactory & factory); +void registerFunctionIsNullable(FunctionFactory & factory); void registerFunctionsNull(FunctionFactory & factory) @@ -23,6 +24,7 @@ void registerFunctionsNull(FunctionFactory & factory) registerFunctionAssumeNotNull(factory); registerFunctionToNullable(factory); registerFunctionIsZeroOrNull(factory); + registerFunctionIsNullable(factory); } } From 83613753e283e989148ca44c222a26894c5d6ff2 Mon Sep 17 00:00:00 2001 From: lokax Date: Tue, 5 Jul 2022 15:24:24 +0800 Subject: [PATCH 208/627] test(Function): isNullable --- tests/queries/0_stateless/02353_isnullable.reference | 4 ++++ tests/queries/0_stateless/02353_isnullable.sql | 5 +++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02353_isnullable.reference create mode 100644 tests/queries/0_stateless/02353_isnullable.sql diff --git a/tests/queries/0_stateless/02353_isnullable.reference b/tests/queries/0_stateless/02353_isnullable.reference new file mode 100644 index 00000000000..74240c4b196 --- /dev/null +++ b/tests/queries/0_stateless/02353_isnullable.reference @@ -0,0 +1,4 @@ +0 +1 +1 +1 diff --git a/tests/queries/0_stateless/02353_isnullable.sql b/tests/queries/0_stateless/02353_isnullable.sql new file mode 100644 index 00000000000..567e294d498 --- /dev/null +++ b/tests/queries/0_stateless/02353_isnullable.sql @@ -0,0 +1,5 @@ +SELECT isNullable(3); +SELECT isNullable(toNullable(3)); + +SELECT isNullable(NULL); +SELECT isNullable(materialize(NULL)); From e6bd0105b14fa2ef68f2fccf8574bd7f7541e4ed Mon Sep 17 00:00:00 2001 From: lokax Date: Tue, 5 Jul 2022 15:51:53 +0800 Subject: [PATCH 209/627] feat(Function): isNullable Signed-off-by: lokax --- src/Functions/isNullable.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/isNullable.cpp b/src/Functions/isNullable.cpp index 27a559c3dc4..35cefdbfe63 100644 --- a/src/Functions/isNullable.cpp +++ b/src/Functions/isNullable.cpp @@ -8,6 +8,7 @@ namespace DB namespace { +/// Return true if the column is nullable. class FunctionIsNullable : public IFunction { public: From c9d1cbbc49d272fbfb769175585b2683e90e745e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 5 Jul 2022 11:17:41 +0200 Subject: [PATCH 210/627] Update 02324_map_combinator_bug.sql --- tests/queries/0_stateless/02324_map_combinator_bug.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02324_map_combinator_bug.sql b/tests/queries/0_stateless/02324_map_combinator_bug.sql index aa9eefaa9ba..b4e039b95ab 100644 --- a/tests/queries/0_stateless/02324_map_combinator_bug.sql +++ b/tests/queries/0_stateless/02324_map_combinator_bug.sql @@ -1,3 +1,5 @@ +-- Tags: no-backward-compatibility-check:22.6 + DROP TABLE IF EXISTS segfault; DROP TABLE IF EXISTS segfault_mv; From 46cfe419d229a235c0d3abd983d7f4394b526bce Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 5 Jul 2022 11:21:25 +0200 Subject: [PATCH 211/627] Make allow_null_modifiers true by default for columns list parser --- src/Parsers/ParserCreateQuery.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index b8b9722df71..79da3defdac 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -334,7 +334,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E class ParserColumnDeclarationList : public IParserBase { public: - explicit ParserColumnDeclarationList(bool require_type_ = true, bool allow_null_modifiers_ = false, bool check_keywords_after_name_ = false) + explicit ParserColumnDeclarationList(bool require_type_ = true, bool allow_null_modifiers_ = true, bool check_keywords_after_name_ = false) : require_type(require_type_) , allow_null_modifiers(allow_null_modifiers_) , check_keywords_after_name(check_keywords_after_name_) From 6e0f27be2329a0ae4074dae9a02ad869c19f5578 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 5 Jul 2022 11:23:35 +0200 Subject: [PATCH 212/627] Fix --- tests/ci/get_previous_release_tag.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index a6b24810d0f..bfce69a17d9 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -65,7 +65,7 @@ def get_previous_release(server_version): + " server version" ) - releases = [ReleaseInfo(release) for release releases_str] + releases = [ReleaseInfo(release) for release in releases_str] found, previous_release = find_previous_release(server_version, releases) page += 1 From 5c54a259df474cda033bee165d74dc7b1d349187 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Tue, 5 Jul 2022 09:39:05 +0000 Subject: [PATCH 213/627] Disable query caching in Druid --- benchmark/druid/benchmark.sh | 6 ++-- benchmark/druid/ingest.json | 2 +- benchmark/druid/results/c6a.4xlarge.txt | 43 +++++++++++++++++++++++++ 3 files changed, 48 insertions(+), 3 deletions(-) create mode 100644 benchmark/druid/results/c6a.4xlarge.txt diff --git a/benchmark/druid/benchmark.sh b/benchmark/druid/benchmark.sh index 54ac6d76137..ad5634f6be9 100755 --- a/benchmark/druid/benchmark.sh +++ b/benchmark/druid/benchmark.sh @@ -15,6 +15,10 @@ tar xf apache-druid-${VERSION}-bin.tar.gz # Have to increase indexer memory limit sed -i 's MaxDirectMemorySize=1g MaxDirectMemorySize=5g g' apache-druid-$VERSION/conf/druid/single-server/medium/middleManager/runtime.properties +# Disable cache to test query performance +sed -i 's druid.historical.cache.useCache=true druid.historical.cache.useCache=false g' apache-druid-$VERSION/conf/druid/single-server/medium/historical/runtime.properties +sed -i 's druid.historical.cache.populateCache=true druid.historical.cache.populateCache=false g' apache-druid-$VERSION/conf/druid/single-server/medium/historical/runtime.properties + # Druid launcher does not start Druid as a daemon. Run it in background ./apache-druid-${VERSION}/bin/start-single-server-medium & @@ -22,9 +26,7 @@ sed -i 's MaxDirectMemorySize=1g MaxDirectMemorySize=5g g' apache-druid-$VERSION wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz -split --additional-suffix .tsv --verbose -n l/10 hits.tsv input -# Running 10 tasks one by one to make it work in parallel ./apache-druid-${VERSION}/bin/post-index-task --file ingest.json --url http://localhost:8081 # Run the queries diff --git a/benchmark/druid/ingest.json b/benchmark/druid/ingest.json index 45bb40482af..8d4c741934c 100644 --- a/benchmark/druid/ingest.json +++ b/benchmark/druid/ingest.json @@ -6,7 +6,7 @@ "inputSource": { "type": "local", "baseDir": "../", - "filter": "inputa?.tsv" + "filter": "hits.tsv" }, "inputFormat": { "type": "tsv", diff --git a/benchmark/druid/results/c6a.4xlarge.txt b/benchmark/druid/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..676f2058f68 --- /dev/null +++ b/benchmark/druid/results/c6a.4xlarge.txt @@ -0,0 +1,43 @@ +[0.179351, 0.040782, 0.026180], +[0.305672, 0.123548, 0.112512], +[0.187141, 0.072973, 0.071330], +[1.451711, 0.060012, 0.056498], +[1.819509, 1.405169, 1.377259], +[2.993129, 2.383569, 2.725656], +[0.061959, 0.016261, 0.010215], +[0.285995, 0.143307, 0.131404], +[3.561017, 2.008080, 2.019541], +[2.549107, 2.432374, 2.419320], +[0.302380, 0.235096, 0.178124], +[0.404714, 0.227526, 0.209203], +[0.866275, 0.745312, 0.715667], +[2.929871, 2.943361, 2.793336], +[2.031640, 1.839312, 1.598095], +[5.304266, 5.124386, 5.266312], +[4.370606, 3.720471, 3.679889], +[3.603636, 3.407057, 3.503119], +[0.030800, 0.004050, 0.003638], +[0.963703, 0.898970, 0.902731], +[18.642888, 18.434240, 18.451905], +[0.053461, 0.014235, 0.015694], +[0.040045, 0.020073, 0.021030], +[7.561157, 6.471910, 6.453816], +[0.111474, 0.019374, 0.027832], +[0.016728, 0.015407, 0.013660], +[0.013881, 0.012076, 0.012084], +[300.155122, 300.081168, 300.067467], +[0.006903, 0.004498, 0.003752], +[1.758529, 1.328889, 1.393226], +[2.106015, 1.906621, 1.653751], +[2.708141, 2.064777, 1.868830], +[7.690441, 7.650803, 7.350766], +[17.321933, 18.088249, 18.109673], +[300.088064, 300.153875, 300.069589], +[4.378734, 3.495874, 3.754953], +[0.623633, 0.603495, 0.580749], +[0.551690, 0.483025, 0.494859], +[0.516076, 0.189721, 0.184861], +[0.005613, 0.005072, 0.004628], +[0.375339, 0.276121, 0.264940], +[0.207965, 0.187909, 0.183420], +[0.176383, 0.107790, 0.106120] From e56ffd665fb54302c2cae0a5504d3848f66b192d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 4 Jul 2022 16:13:58 +0200 Subject: [PATCH 214/627] Fix test --- tests/integration/test_storage_postgresql/test.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index df262a69843..a3ebbe97451 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -676,14 +676,21 @@ def test_auto_close_connection(started_cluster): result = node2.query("SELECT * FROM test.test_table LIMIT 100", user="default") + node2.query( + f""" + CREATE TABLE test.stat (numbackends UInt32, datname String) + ENGINE = PostgreSQL(postgres1, database='{database_name}', table='pg_stat_database') + """ + ) + count = int( node2.query( - f"SELECT numbackends FROM postgresql(postgres1, database='{database_name}', table='pg_stat_database') WHERE datname = '{database_name}'" + f"SELECT numbackends FROM test.stat WHERE datname = '{database_name}'" ) ) - # Connection from python - assert count == 1 + # Connection from python + pg_stat table also has a connection at the moment of current query + assert count == 2 if __name__ == "__main__": From f048db23e2b1c743cf737eabe058cd2bd6797fea Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 5 Jul 2022 12:10:33 +0200 Subject: [PATCH 215/627] Release branches run CI on pushes, fix PRInfo --- tests/ci/pr_info.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 99a06de5f28..4b7c100c300 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -152,13 +152,6 @@ class PRInfo: self.user_orgs = set(org["id"] for org in response_json) self.diff_urls.append(github_event["pull_request"]["diff_url"]) - if "release" in self.labels: - # For release PRs we must get not only files changed in the PR - # itself, but as well files changed since we branched out - self.diff_urls.append( - f"https://github.com/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master.diff" - ) elif "commits" in github_event: self.sha = github_event["after"] pull_request = get_pr_for_commit(self.sha, github_event["ref"]) @@ -207,6 +200,13 @@ class PRInfo: ] else: self.diff_urls.append(pull_request["diff_url"]) + if "release" in self.labels: + # For release PRs we must get not only files changed in the PR + # itself, but as well files changed since we branched out + self.diff_urls.append( + f"https://github.com/{GITHUB_REPOSITORY}/" + f"compare/{self.head_ref}...master.diff" + ) else: print("event.json does not match pull_request or push:") print(json.dumps(github_event, sort_keys=True, indent=4)) From 7a4be3b748c7e9d0ca3ad06c1c172e977b6537af Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 5 Jul 2022 12:10:49 +0200 Subject: [PATCH 216/627] Fix build, disable for freebsd --- contrib/nats-io-cmake/CMakeLists.txt | 11 ++++-- src/Storages/NATS/NATSSource.cpp | 15 ++++++- src/Storages/NATS/NATSSource.h | 11 +++++- src/Storages/NATS/StorageNATS.cpp | 48 ++++++++++++++--------- src/Storages/NATS/StorageNATS.h | 11 +++--- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 3 -- 6 files changed, 66 insertions(+), 33 deletions(-) diff --git a/contrib/nats-io-cmake/CMakeLists.txt b/contrib/nats-io-cmake/CMakeLists.txt index 9cd5d0daa51..5588d5750c4 100644 --- a/contrib/nats-io-cmake/CMakeLists.txt +++ b/contrib/nats-io-cmake/CMakeLists.txt @@ -1,7 +1,12 @@ -option (ENABLE_KAFKA "Enable kafka" ${ENABLE_LIBRARIES}) +option (ENABLE_NATS "Enable NATS" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_KAFKA) - message(STATUS "Not using librdkafka") +if (OS_FREEBSD) + set(ENABLE_NATS OFF) + message (STATUS "Using internal nats-io library on FreeBSD is not supported") +endif() + +if (NOT ENABLE_NATS) + message(STATUS "Not using nats-io") return() endif() diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index 5479856411e..f5e5e4f8b91 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -42,7 +42,7 @@ NATSSource::NATSSource( ContextPtr context_, const Names & columns, size_t max_block_size_) - : SourceWithProgress(getSampleBlock(headers.first, headers.second)) + : ISource(getSampleBlock(headers.first, headers.second)) , storage(storage_) , storage_snapshot(storage_snapshot_) , context(context_) @@ -66,6 +66,19 @@ NATSSource::~NATSSource() storage.pushReadBuffer(buffer); } +bool NATSSource::checkTimeLimit() const +{ + if (max_execution_time != 0) + { + auto elapsed_ns = total_stopwatch.elapsed(); + + if (elapsed_ns > static_cast(max_execution_time.totalMicroseconds()) * 1000) + return false; + } + + return true; +} + Chunk NATSSource::generate() { if (!buffer) diff --git a/src/Storages/NATS/NATSSource.h b/src/Storages/NATS/NATSSource.h index 6c1ea5bed0d..e4e94d2347a 100644 --- a/src/Storages/NATS/NATSSource.h +++ b/src/Storages/NATS/NATSSource.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -8,7 +8,7 @@ namespace DB { -class NATSSource : public SourceWithProgress +class NATSSource : public ISource { public: NATSSource( @@ -27,7 +27,11 @@ public: bool queueEmpty() const { return !buffer || buffer->queueEmpty(); } + void setTimeLimit(Poco::Timespan max_execution_time_) { max_execution_time = max_execution_time_; } + private: + bool checkTimeLimit() const; + StorageNATS & storage; StorageSnapshotPtr storage_snapshot; ContextPtr context; @@ -40,6 +44,9 @@ private: ConsumerBufferPtr buffer; + Poco::Timespan max_execution_time = 0; + Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; + NATSSource( StorageNATS & storage_, const StorageSnapshotPtr & storage_snapshot_, diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 308ded9a6c2..3c1a04c7824 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -1,12 +1,15 @@ #include #include #include +#include #include #include #include #include #include #include +#include +#include #include #include #include @@ -14,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -261,20 +265,21 @@ size_t StorageNATS::getMaxBlockSize() const } -Pipe StorageNATS::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & /* query_info */, - ContextPtr local_context, - QueryProcessingStage::Enum /* processed_stage */, - size_t /* max_block_size */, - unsigned /* num_streams */) +void StorageNATS::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + unsigned /* num_streams */) { if (!consumers_ready) throw Exception("NATS consumers setup not finished. Connection might be lost", ErrorCodes::CANNOT_CONNECT_NATS); if (num_created_consumers == 0) - return {}; + return; if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) throw Exception( @@ -317,9 +322,19 @@ Pipe StorageNATS::read( startLoop(); LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); - auto united_pipe = Pipe::unitePipes(std::move(pipes)); - united_pipe.addInterpreterContext(modified_context); - return united_pipe; + auto pipe = Pipe::unitePipes(std::move(pipes)); + + if (pipe.empty()) + { + auto header = storage_snapshot->getSampleBlockForColumns(column_names); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, local_context); + } + else + { + auto read_step = std::make_unique(std::move(pipe), getName(), query_info.storage_limits); + query_plan.addStep(std::move(read_step)); + query_plan.addInterpreterContext(modified_context); + } } @@ -617,16 +632,11 @@ bool StorageNATS::streamToViews() sources.emplace_back(source); pipes.emplace_back(source); - // Limit read batch to maximum block size to allow DDL - StreamLocalLimits limits; - - limits.speed_limits.max_execution_time = nats_settings->nats_flush_interval_ms.changed + Poco::Timespan max_execution_time = nats_settings->nats_flush_interval_ms.changed ? nats_settings->nats_flush_interval_ms : getContext()->getSettingsRef().stream_flush_interval_ms; - limits.timeout_overflow_mode = OverflowMode::BREAK; - - source->setLimits(limits); + source->setTimeLimit(max_execution_time); } block_io.pipeline.complete(Pipe::unitePipes(std::move(pipes))); diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index b1030cbb788..185b39250c8 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -39,14 +39,15 @@ public: void checkTableCanBeDropped() const override { drop_table = true; } /// Always return virtual columns in addition to required columns - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) override; + ContextPtr local_context, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + unsigned /* num_streams */) override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 6a94b1a28dc..73f0c8bd44e 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1086,9 +1086,6 @@ bool StorageRabbitMQ::streamToViews() sources.emplace_back(source); pipes.emplace_back(source); - // Limit read batch to maximum block size to allow DDL - StreamLocalLimits limits; - Poco::Timespan max_execution_time = rabbitmq_settings->rabbitmq_flush_interval_ms.changed ? rabbitmq_settings->rabbitmq_flush_interval_ms : getContext()->getSettingsRef().stream_flush_interval_ms; From 736fa4bb7217cff4a7dffdbcd67a877e33fba52a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 5 Jul 2022 12:31:50 +0200 Subject: [PATCH 217/627] Update libprotobuf-mutator + fix build (#38834) --- contrib/libprotobuf-mutator | 2 +- contrib/libprotobuf-mutator-cmake/CMakeLists.txt | 7 +++++-- src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt | 3 ++- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index ffd86a32874..a304ec48dcf 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit ffd86a32874e5c08a143019aad1aaf0907294c9f +Subproject commit a304ec48dcf15d942607032151f7e9ee504b5dcf diff --git a/contrib/libprotobuf-mutator-cmake/CMakeLists.txt b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt index a623f95c418..9bbd6c17caa 100644 --- a/contrib/libprotobuf-mutator-cmake/CMakeLists.txt +++ b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt @@ -14,8 +14,11 @@ add_library(_protobuf-mutator ${LIBRARY_DIR}/src/text_format.cc ${LIBRARY_DIR}/src/utf8_fix.cc) -target_include_directories(_protobuf-mutator BEFORE INTERFACE "${LIBRARY_DIR}") -target_include_directories(_protobuf-mutator BEFORE INTERFACE "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") +# codegen_select_fuzzer includes ... +target_include_directories(_protobuf-mutator BEFORE PUBLIC "${LIBRARY_DIR}/src") +# ... which includes +target_include_directories(_protobuf-mutator BEFORE PUBLIC "${LIBRARY_DIR}") +target_include_directories(_protobuf-mutator BEFORE PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") target_link_libraries(_protobuf-mutator ch_contrib::protobuf) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 3d416544419..86eb8bf36a5 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -31,7 +31,8 @@ add_custom_command( DEPENDS "${CURRENT_DIR_IN_BINARY}/clickhouse.g" ) -PROTOBUF_GENERATE_CPP(PROTO_SRCS PROTO_HDRS "${CURRENT_DIR_IN_BINARY}/out.proto") +set(PROTOBUF_GENERATE_CPP_APPEND_PATH TRUE) +protobuf_generate_cpp(PROTO_SRCS PROTO_HDRS "${CURRENT_DIR_IN_BINARY}/out.proto") set(FUZZER_SRCS codegen_select_fuzzer.cpp "${CURRENT_DIR_IN_BINARY}/out.cpp" ${PROTO_SRCS} ${PROTO_HDRS}) set(CMAKE_INCLUDE_CURRENT_DIR TRUE) From d1d717d4e841ec44c159724f5198d1a2c3e04efc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 5 Jul 2022 13:41:22 +0300 Subject: [PATCH 218/627] tests: make aggregate_state_exception_memory_leak tests more deterministic Sometimes if the 30 seconds is not enough for the query to fail (i.e. under TSan [1]), the error will be ignored and the test will fail. [1]: https://s3.amazonaws.com/clickhouse-test-reports/38748/d863f6ce1ece810c66ddf8bd89575825d3f2595f/stateless_tests__thread__actions__%5B2/3%5D.html Follow-up for: #11496 (cc @alexey-milovidov) Signed-off-by: Azat Khuzhin --- ...1_aggregate_state_exception_memory_leak.sh | 21 ++++++++++--------- ...2_aggregate_state_exception_memory_leak.sh | 20 +++++++++--------- 2 files changed, 21 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh index 010c66b9bb1..2d18c45406c 100755 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh @@ -5,15 +5,16 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function test() -{ - for _ in {1..1000}; do - $CLICKHOUSE_CLIENT --max_memory_usage 1G <<< "SELECT uniqExactState(number) FROM system.numbers_mt GROUP BY number % 10"; - done -} - -export -f test; - +start=$SECONDS # If the memory leak exists, it will lead to OOM fairly quickly. -timeout 30 bash -c test 2>&1 | grep -o -F 'Memory limit (for query) exceeded' | uniq +for _ in {1..1000}; do + $CLICKHOUSE_CLIENT --max_memory_usage 1G <<< "SELECT uniqExactState(number) FROM system.numbers_mt GROUP BY number % 10"; + + # NOTE: we cannot use timeout here since this will not guarantee that the query will be executed at least once. + # (since graceful wait of clickhouse-client had been reverted) + elapsed=$((SECONDS - start)) + if [[ $elapsed -gt 30 ]]; then + break + fi +done 2>&1 | grep -o -F 'Memory limit (for query) exceeded' | uniq echo 'Ok' diff --git a/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh index 7e10ab475d4..b9f1f81da1a 100755 --- a/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh @@ -5,14 +5,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function test() -{ - for _ in {1..250}; do - $CLICKHOUSE_CLIENT --query "SELECT groupArrayIfState(('Hello, world' AS s) || s || s || s || s || s || s || s || s || s, NOT throwIf(number > 10000000, 'Ok')) FROM system.numbers_mt GROUP BY number % 10"; - done -} +start=$SECONDS +for _ in {1..250}; do + $CLICKHOUSE_CLIENT --query "SELECT groupArrayIfState(('Hello, world' AS s) || s || s || s || s || s || s || s || s || s, NOT throwIf(number > 10000000, 'Ok')) FROM system.numbers_mt GROUP BY number % 10"; -export -f test; - -# If the memory leak exists, it will lead to OOM fairly quickly. -timeout 30 bash -c test 2>&1 | grep -o -F 'Ok' | uniq + # NOTE: we cannot use timeout here since this will not guarantee that the query will be executed at least once. + # (since graceful wait of clickhouse-client had been reverted) + elapsed=$((SECONDS - start)) + if [[ $elapsed -gt 30 ]]; then + break + fi +done 2>&1 | grep -o -F 'Ok' | uniq From 17903117d2c5c76b66dbac4ca89dbec6d427b5f3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 4 Jul 2022 17:10:34 +0000 Subject: [PATCH 219/627] better --- src/Interpreters/DirectJoin.cpp | 94 +++++++++--- src/Interpreters/DirectJoin.h | 12 +- src/Interpreters/ExpressionAnalyzer.cpp | 5 +- src/Interpreters/IJoin.h | 3 + src/Storages/IKVStorage.h | 23 ++- .../RocksDB/StorageEmbeddedRocksDB.cpp | 137 ++++++++---------- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 22 +-- 7 files changed, 165 insertions(+), 131 deletions(-) diff --git a/src/Interpreters/DirectJoin.cpp b/src/Interpreters/DirectJoin.cpp index 12aca6f715f..21baeb31945 100644 --- a/src/Interpreters/DirectJoin.cpp +++ b/src/Interpreters/DirectJoin.cpp @@ -1,6 +1,7 @@ #include #include +#include namespace DB { @@ -10,6 +11,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int UNSUPPORTED_JOIN_KEYS; + extern const int NOT_FOUND_COLUMN_IN_BLOCK; } static Block originalRightBlock(const Block & block, const TableJoin & table_join) @@ -20,6 +22,65 @@ static Block originalRightBlock(const Block & block, const TableJoin & table_joi return original_right_block; } +/// Converts `columns` from `source_sample_block` structure to `result_sample_block`. +/// Can select subset of columns and change types. +static MutableColumns convertBlockStructure( + const Block & source_sample_block, const Block & result_sample_block, MutableColumns && columns, const PaddedPODArray & null_map) +{ + MutableColumns result_columns; + for (const auto & out_sample_col : result_sample_block) + { + auto i = source_sample_block.getPositionByName(out_sample_col.name); + if (columns[i] == nullptr) + { + throw DB::Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Can't find column '{}'", out_sample_col.name); + } + + ColumnWithTypeAndName col = source_sample_block.getByPosition(i); + if (!col.type->equals(*out_sample_col.type)) + { + col.column = std::move(columns[i]); + result_columns.push_back(IColumn::mutate(castColumnAccurate(col, out_sample_col.type))); + } + else + { + result_columns.push_back(std::move(columns[i])); + } + columns[i] = nullptr; + + if (result_columns.back()->isNullable()) + { + assert_cast(result_columns.back().get())->applyNegatedNullMap(null_map); + } + } + return result_columns; +} + +DirectKeyValueJoin::DirectKeyValueJoin(std::shared_ptr table_join_, + const Block & right_sample_block_, + std::shared_ptr storage_) + : table_join(table_join_) + , storage(storage_) + , right_sample_block(right_sample_block_) + , log(&Poco::Logger::get("DirectKeyValueJoin")) +{ + if (!table_join->oneDisjunct() + || table_join->getOnlyClause().key_names_left.size() != 1 + || table_join->getOnlyClause().key_names_right.size() != 1) + { + throw DB::Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Not supported by direct JOIN"); + } + + if (table_join->strictness() != ASTTableJoin::Strictness::All && + table_join->strictness() != ASTTableJoin::Strictness::Any && + table_join->strictness() != ASTTableJoin::Strictness::RightAny) + { + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Not supported by direct JOIN"); + } + + LOG_TRACE(log, "Using direct join"); +} + bool DirectKeyValueJoin::addJoinedBlock(const Block &, bool) { throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable code reached"); @@ -35,36 +96,23 @@ void DirectKeyValueJoin::checkTypesOfKeys(const Block & block) const void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) { - if (!table_join->oneDisjunct() - || table_join->getOnlyClause().key_names_left.size() != 1 - || table_join->getOnlyClause().key_names_right.size() != 1) - { - throw DB::Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Not supported by direct JOIN"); - } - - if (table_join->strictness() != ASTTableJoin::Strictness::All && - table_join->strictness() != ASTTableJoin::Strictness::Any && - table_join->strictness() != ASTTableJoin::Strictness::RightAny) - { - throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Not supported by direct JOIN"); - } - - const auto & key_names_left = table_join->getOnlyClause().key_names_left; - - const String & key_name = key_names_left[0]; + const String & key_name = table_join->getOnlyClause().key_names_left[0]; const ColumnWithTypeAndName & key_col = block.getByName(key_name); if (!key_col.column) return; - NullMap null_map(key_col.column->size(), 1); - Block original_right_block = originalRightBlock(right_sample_block, *table_join); - Chunk joined_chunk = storage->getByKeys(key_col, original_right_block, &null_map); + NullMap null_map; + Chunk joined_chunk = storage->getByKeys({key_col}, null_map); - Columns cols = joined_chunk.detachColumns(); - for (size_t i = 0; i < cols.size(); ++i) + /// Expected right block may differ from structure in storage, because of `join_use_nulls` or we just select not all columns. + Block original_right_block = originalRightBlock(right_sample_block, *table_join); + Block sample_storage_block = storage->getInMemoryMetadataPtr()->getSampleBlock(); + MutableColumns result_columns = convertBlockStructure(sample_storage_block, original_right_block, joined_chunk.mutateColumns(), null_map); + + for (size_t i = 0; i < result_columns.size(); ++i) { ColumnWithTypeAndName col = right_sample_block.getByPosition(i); - col.column = std::move(cols[i]); + col.column = std::move(result_columns[i]); block.insert(std::move(col)); } diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index 435b33f6dab..f7da06ef826 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -20,14 +20,10 @@ class NotJoinedBlocks; class DirectKeyValueJoin : public IJoin { public: - DirectKeyValueJoin(std::shared_ptr table_join_, const Block & right_sample_block_, std::shared_ptr storage_) - : table_join(table_join_) - , storage(storage_) - , right_sample_block(right_sample_block_) - , log(&Poco::Logger::get("DirectKeyValueJoin")) - { - LOG_TRACE(log, "Using direct join"); - } + DirectKeyValueJoin( + std::shared_ptr table_join_, + const Block & right_sample_block_, + std::shared_ptr storage_); virtual const TableJoin & getTableJoin() const override { return *table_join; } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 9d02569a6da..7db76fc420d 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1180,10 +1180,11 @@ std::shared_ptr tryKeyValueJoin(std::shared_ptr a String key_name = clauses[0].key_names_right[0]; String original_key_name = analyzed_join->getOriginalName(key_name); - if (storage->getPrimaryKey() != original_key_name) + const auto & storage_primary_key = storage->getPrimaryKey(); + if (storage_primary_key.size() != 1 || storage_primary_key[0] != original_key_name) { return error_or_null(fmt::format("key '{}'{} doesn't match storage '{}'", - key_name, (key_name != original_key_name ? " (aka '" + original_key_name + "')" : ""), storage->getPrimaryKey())); + key_name, (key_name != original_key_name ? " (aka '" + original_key_name + "')" : ""), fmt::join(storage_primary_key, ","))); } return std::make_shared(analyzed_join, right_sample_block, storage); diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index a9022b3d151..2cc4ce175ba 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -39,8 +39,11 @@ public: virtual void setTotals(const Block & block) { totals = block; } virtual const Block & getTotals() const { return totals; } + /// Number of rows/bytes stored in memory virtual size_t getTotalRowCount() const = 0; virtual size_t getTotalByteCount() const = 0; + + /// Returns true if no data to join with. virtual bool alwaysReturnsEmptySet() const = 0; /// StorageJoin/Dictionary is already filled. No need to call addJoinedBlock. diff --git a/src/Storages/IKVStorage.h b/src/Storages/IKVStorage.h index 06ac42f2d80..667ccda0c41 100644 --- a/src/Storages/IKVStorage.h +++ b/src/Storages/IKVStorage.h @@ -1,26 +1,33 @@ #pragma once +#include #include #include namespace DB { - /// Storage that support key-value requests class IKeyValueStorage : public IStorage { public: using IStorage::IStorage; - /// Get key name that supports key-value requests - virtual const String & getPrimaryKey() const = 0; + /// Get primary key name that supports key-value requests. + /// Primary key can constist of multiple columns. + virtual Names getPrimaryKey() const = 0; - /// Get data directly by keys - virtual Chunk getByKeys( - const ColumnWithTypeAndName & col, - const Block & sample_block, - PaddedPODArray * null_map) const = 0; + /* + * Get data from storage directly by keys. + * + * @param keys - keys to get data for. Key can be compound and represented by several columns. + * @param out_null_map - output parameter indicating which keys were not found. + * + * @return - chunk of data corresponding for keys. + * Number of rows in chunk is equal to size of columns in keys. + * If the key was not found row would have a default value. + */ + virtual Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & out_null_map) const = 0; }; } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 064c1d850ff..fd943fbe1c5 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -51,7 +51,6 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; - extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ROCKSDB_ERROR; } @@ -59,7 +58,6 @@ namespace ErrorCodes using FieldVectorPtr = std::shared_ptr; using RocksDBOptions = std::unordered_map; - static RocksDBOptions getOptionsFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & path) { RocksDBOptions options; @@ -76,7 +74,6 @@ static RocksDBOptions getOptionsFromConfig(const Poco::Util::AbstractConfigurati return options; } - // returns keys may be filter by condition static bool traverseASTFilter( const String & primary_key, const DataTypePtr & primary_key_type, const ASTPtr & elem, const PreparedSets & sets, FieldVectorPtr & res) @@ -165,7 +162,6 @@ static bool traverseASTFilter( return false; } - /** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. * TODO support key like search */ @@ -193,54 +189,59 @@ static void fillColumns(const K & key, const V & value, size_t key_pos, const Bl } } - -static std::vector getSlicedKeys( +static std::vector serializeKeysToRawString( FieldVector::const_iterator & it, FieldVector::const_iterator end, DataTypePtr key_column_type, - size_t max_block_size, - std::vector & holder) + size_t max_block_size) { size_t num_keys = end - it; - holder.reserve(num_keys); - std::vector slices_keys(num_keys); + + std::vector result; + result.reserve(num_keys); size_t rows_processed = 0; while (it < end && (max_block_size == 0 || rows_processed < max_block_size)) { - std::string & serialized_key = holder.emplace_back(); + std::string & serialized_key = result.emplace_back(); WriteBufferFromString wb(serialized_key); key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); wb.finalize(); - slices_keys[rows_processed] = serialized_key; ++it; ++rows_processed; } - return slices_keys; + return result; } -static std::vector getSlicedKeys(const ColumnWithTypeAndName & col, std::vector & holder) +static std::vector serializeKeysToRawString(const ColumnWithTypeAndName & keys) { - if (!col.column) + if (!keys.column) return {}; - size_t num_keys = col.column->size(); - holder.reserve(num_keys); + size_t num_keys = keys.column->size(); + std::vector result; + result.reserve(num_keys); - std::vector slices_keys(num_keys); for (size_t i = 0; i < num_keys; ++i) { - std::string & serialized_key = holder.emplace_back(); + std::string & serialized_key = result.emplace_back(); WriteBufferFromString wb(serialized_key); Field field; - col.column->get(i, field); + keys.column->get(i, field); /// TODO(@vdimir): use serializeBinaryBulk - col.type->getDefaultSerialization()->serializeBinary(field, wb); + keys.type->getDefaultSerialization()->serializeBinary(field, wb); wb.finalize(); - slices_keys[i] = serialized_key; } - return slices_keys; + return result; +} + +/// In current implementation rocks db can have key with only one column. +static size_t getPrimaryKeyPos(const Block & header, const Names & primary_key) +{ + if (primary_key.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB: only one primary key is supported"); + return header.getPositionByName(primary_key[0]); } class EmbeddedRocksDBSource : public ISource @@ -255,7 +256,7 @@ public: const size_t max_block_size_) : ISource(header) , storage(storage_) - , primary_key_pos(header.getPositionByName(storage.getPrimaryKey())) + , primary_key_pos(getPrimaryKeyPos(header, storage.getPrimaryKey())) , keys(keys_) , begin(begin_) , end(end_) @@ -271,7 +272,7 @@ public: const size_t max_block_size_) : ISource(header) , storage(storage_) - , primary_key_pos(header.getPositionByName(storage.getPrimaryKey())) + , primary_key_pos(getPrimaryKeyPos(header, storage.getPrimaryKey())) , iterator(std::move(iterator_)) , max_block_size(max_block_size_) { @@ -295,10 +296,9 @@ public: return {}; } - const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey()).type; - std::vector holder; - auto slices_keys = getSlicedKeys(it, end, key_column_type, max_block_size, holder); - return storage.getByKeysImpl(slices_keys, sample_block, nullptr); + const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey().at(0)).type; + auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size); + return storage.getBySerializedKeys(raw_keys, nullptr); } Chunk generateFullScan() @@ -453,6 +453,7 @@ void StorageEmbeddedRocksDB::initDB() throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to open rocksdb path at: {}: {}", rocksdb_dir, status.ToString()); } + /// It's ok just to wrap db with unique_ptr, from rdb documentation: "when you are done with a database, just delete the database object" rocksdb_ptr = std::unique_ptr(db); } @@ -554,66 +555,45 @@ std::vector StorageEmbeddedRocksDB::multiGet(const std::vector< } Chunk StorageEmbeddedRocksDB::getByKeys( - const ColumnWithTypeAndName & col, - const Block & sample_block, - PaddedPODArray * null_map) const + const ColumnsWithTypeAndName & keys, + PaddedPODArray & null_map) const { - std::vector holder; - auto sliced_keys = getSlicedKeys(col, holder); + if (keys.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "StorageEmbeddedRocksDB supports only one key, got: {}", keys.size()); - if (sliced_keys.size() != col.column->size()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", sliced_keys.size(), col.column->size()); + auto raw_keys = serializeKeysToRawString(keys[0]); - return getByKeysImpl(sliced_keys, sample_block, null_map); + if (raw_keys.size() != keys[0].column->size()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); + + return getBySerializedKeys(raw_keys, &null_map); } -static MutableColumns getColumnsFromBlock( - const Block & sample_block, MutableColumns && columns, const Block & output_sample_block, PaddedPODArray * null_map) -{ - MutableColumns result_columns; - for (const auto & out_sample_col : output_sample_block) - { - auto i = sample_block.getPositionByName(out_sample_col.name); - if (columns[i] == nullptr) - { - throw DB::Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Can't find column '{}'", out_sample_col.name); - } - - ColumnWithTypeAndName col = sample_block.getByPosition(i); - if (!col.type->equals(*out_sample_col.type)) - { - col.column = std::move(columns[i]); - result_columns.push_back(IColumn::mutate(castColumnAccurate(col, out_sample_col.type))); - } - else - { - result_columns.push_back(std::move(columns[i])); - } - columns[i] = nullptr; - - if (null_map && result_columns.back()->isNullable()) - { - assert_cast(result_columns.back().get())->applyNegatedNullMap(*null_map); - } - } - return result_columns; -} - -Chunk StorageEmbeddedRocksDB::getByKeysImpl( - const std::vector & slices_keys, - const Block & output_sample_block, +Chunk StorageEmbeddedRocksDB::getBySerializedKeys( + const std::vector & keys, PaddedPODArray * null_map) const { - if (!output_sample_block) - return {}; - std::vector values; Block sample_block = getInMemoryMetadataPtr()->getSampleBlock(); - size_t primary_key_pos = sample_block.getPositionByName(getPrimaryKey()); + size_t primary_key_pos = getPrimaryKeyPos(sample_block, getPrimaryKey()); MutableColumns columns = sample_block.cloneEmptyColumns(); + + /// Convert from vector of string to vector of string refs (rocksdb::Slice), because multiGet api expects them. + std::vector slices_keys; + slices_keys.reserve(keys.size()); + for (const auto & key : keys) + slices_keys.emplace_back(key); + + auto statuses = multiGet(slices_keys, values); + if (null_map) + { + null_map->clear(); + null_map->resize_fill(statuses.size(), 1); + } + for (size_t i = 0; i < statuses.size(); ++i) { if (statuses[i].ok()) @@ -638,10 +618,7 @@ Chunk StorageEmbeddedRocksDB::getByKeysImpl( } size_t num_rows = columns.at(0)->size(); - /// The `output_sample_block` may be different from `sample_block`. - /// It may contains subset of columns and types can be different (but should be convertible) - auto result_columns = getColumnsFromBlock(sample_block, std::move(columns), output_sample_block, null_map); - return Chunk(std::move(result_columns), num_rows); + return Chunk(std::move(columns), num_rows); } void registerStorageEmbeddedRocksDB(StorageFactory & factory) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 6894c2e684e..62c9a0eeae7 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -19,6 +19,10 @@ namespace DB class Context; +/// Wrapper for rocksdb storage. +/// Operates with rocksdb data structures via rocksdb API (holds pointer to rocksdb::DB inside for that). +/// Storage have one primary key. +/// Values are serialized into raw strings to store in rocksdb. class StorageEmbeddedRocksDB final : public IKeyValueStorage, WithContext { friend class EmbeddedRocksDBSink; @@ -57,18 +61,16 @@ public: std::shared_ptr getRocksDBStatistics() const; std::vector multiGet(const std::vector & slices_keys, std::vector & values) const; - const String & getPrimaryKey() const override { return primary_key; } + Names getPrimaryKey() const override { return {primary_key}; } - Chunk getByKeys( - const ColumnWithTypeAndName & col, - const Block & sample_block, - PaddedPODArray * null_map) const override; - - Chunk getByKeysImpl( - const std::vector & slices_keys, - const Block & sample_block, - PaddedPODArray * null_map) const; + Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map) const override; + /// Return chunk with data for given serialized keys. + /// If out_null_map is passed, fill it with 1/0 depending on key was/wasn't found. Result chunk may contain default values. + /// If out_null_map is not passed. Not found rows excluded from result chunk. + Chunk getBySerializedKeys( + const std::vector & keys, + PaddedPODArray * out_null_map) const; private: const String primary_key; From d0b2f13f9d267e7794070c4439eed5c154ce5b3d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Jul 2022 13:41:52 +0200 Subject: [PATCH 220/627] Fix style check --- src/Functions/MultiSearchFirstIndexImpl.h | 1 - src/Functions/MultiSearchFirstPositionImpl.h | 1 - src/Functions/MultiSearchImpl.h | 1 - 3 files changed, 3 deletions(-) diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 56ac06551f6..2f781de3214 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -10,7 +10,6 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index f1d79b2685d..8029fc703c4 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -10,7 +10,6 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 86567625252..9a7801a586d 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -10,7 +10,6 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } From 2b31d68cefa0017eb97244a96f3cd76d194194bb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Jul 2022 13:47:48 +0200 Subject: [PATCH 221/627] Fix 00928_multi_match_constant_constant.sql Difficult to test because x86 and ARM behave differently due to vectorscan not available everyehwere ... The same stuff (multiFuzzyMatch) is tested elsewhere. --- .../00928_multi_match_constant_constant.reference | 4 ---- .../0_stateless/00928_multi_match_constant_constant.sql | 9 --------- 2 files changed, 13 deletions(-) diff --git a/tests/queries/0_stateless/00928_multi_match_constant_constant.reference b/tests/queries/0_stateless/00928_multi_match_constant_constant.reference index 653ddcf0d58..cc2b1466fcb 100644 --- a/tests/queries/0_stateless/00928_multi_match_constant_constant.reference +++ b/tests/queries/0_stateless/00928_multi_match_constant_constant.reference @@ -1,7 +1,3 @@ 1 -1 2 -2 -[2] -[2] [1,8] diff --git a/tests/queries/0_stateless/00928_multi_match_constant_constant.sql b/tests/queries/0_stateless/00928_multi_match_constant_constant.sql index 6b4beea2b79..fc3e8ca6b2c 100644 --- a/tests/queries/0_stateless/00928_multi_match_constant_constant.sql +++ b/tests/queries/0_stateless/00928_multi_match_constant_constant.sql @@ -1,12 +1,3 @@ --- Tags: no-fasttest - SELECT multiMatchAny('goodbye', ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); -SELECT multiFuzzyMatchAny('goodbye', 1, ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); - SELECT multiMatchAnyIndex('goodbye', ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); -SELECT multiFuzzyMatchAnyIndex('goodbye', 1, ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); - -SELECT multiMatchAllIndices('goodbye', ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); -SELECT multiFuzzyMatchAllIndices('goodbye', 1, ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']); - SELECT multiSearchAllPositions('hello, world', ['hello', 'world']); From 0232c9267de355b660667eb11f59742590ac535b Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 3 Jun 2022 15:26:06 +0000 Subject: [PATCH 222/627] Pass CHPC_DATABASE_URL to perf tests --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- tests/ci/performance_comparison_check.py | 19 +++++++++++++++++-- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 28438a86e47..33b6a2531a9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1269,13 +1269,13 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

Date: Tue, 7 Jun 2022 09:54:55 +0000 Subject: [PATCH 223/627] fix performance_comparison_check.py --- tests/ci/performance_comparison_check.py | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index f97e9879d4a..6bcf947552a 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -148,14 +148,12 @@ if __name__ == "__main__": logging.info("Going to run command %s", run_command) popen_env = os.environ.copy() - popen_env["CHPC_DATABASE_URL"] = ( - get_parameter_from_ssm("clickhouse-test-stat-url"), + popen_env["CHPC_DATABASE_URL"] = get_parameter_from_ssm("clickhouse-test-stat-url") + popen_env["CHPC_DATABASE_USER"] = get_parameter_from_ssm( + "clickhouse-test-stat-login" ) - popen_env["CHPC_DATABASE_USER"] = ( - get_parameter_from_ssm("clickhouse-test-stat-login"), - ) - popen_env["CHPC_DATABASE_PASSWORD"] = ( - get_parameter_from_ssm("clickhouse-test-stat-password"), + popen_env["CHPC_DATABASE_PASSWORD"] = get_parameter_from_ssm( + "clickhouse-test-stat-password" ) run_log_path = os.path.join(temp_path, "runlog.log") From aac2f8aefc29acc88dc1c30a146eab0333a07c08 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Jun 2022 10:00:31 +0000 Subject: [PATCH 224/627] Rename envs for performance comparsion --- docker/test/performance-comparison/compare.sh | 16 +++++++------- tests/ci/performance_comparison_check.py | 21 ++++++++++++------- 2 files changed, 21 insertions(+), 16 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 046a394a686..fa03ecde25a 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -561,7 +561,7 @@ create table query_metric_stats_denorm engine File(TSVWithNamesAndTypes, " 2> >(tee -a analyze/errors.log 1>&2) # Fetch historical query variability thresholds from the CI database -if [ -v CHPC_DATABASE_URL ] +if [ -v CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL ] then set +x # Don't show password in the log client=(clickhouse-client @@ -569,10 +569,10 @@ then # so I have to extract host and port with clickhouse-local. I tried to use # Poco URI parser to support this in the client, but it's broken and can't # parse host:port. - $(clickhouse-local --query "with '${CHPC_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV") + $(clickhouse-local --query "with '${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV") --secure - --user "${CHPC_DATABASE_USER}" - --password "${CHPC_DATABASE_PASSWORD}" + --user "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER}" + --password "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD}" --config "right/config/client_config.xml" --database perftest --date_time_input_format=best_effort) @@ -1244,7 +1244,7 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') ; " - if ! [ -v CHPC_DATABASE_URL ] + if ! [ -v CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL ] then echo Database for test results is not specified, will not upload them. return 0 @@ -1256,10 +1256,10 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') # so I have to extract host and port with clickhouse-local. I tried to use # Poco URI parser to support this in the client, but it's broken and can't # parse host:port. - $(clickhouse-local --query "with '${CHPC_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV") + $(clickhouse-local --query "with '${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV") --secure - --user "${CHPC_DATABASE_USER}" - --password "${CHPC_DATABASE_PASSWORD}" + --user "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER}" + --password "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD}" --config "right/config/client_config.xml" --database perftest --date_time_input_format=best_effort) diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index 6bcf947552a..e3f39c47fea 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -134,8 +134,11 @@ if __name__ == "__main__": os.makedirs(result_path) docker_env += ( - f" -e CHPC_DATABASE_URL -e CHPC_DATABASE_USER -e CHPC_DATABASE_PASSWORD" + " -e CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL" + " -e CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER" + " -e CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD" ) + run_command = get_run_command( result_path, result_path, @@ -148,13 +151,15 @@ if __name__ == "__main__": logging.info("Going to run command %s", run_command) popen_env = os.environ.copy() - popen_env["CHPC_DATABASE_URL"] = get_parameter_from_ssm("clickhouse-test-stat-url") - popen_env["CHPC_DATABASE_USER"] = get_parameter_from_ssm( - "clickhouse-test-stat-login" - ) - popen_env["CHPC_DATABASE_PASSWORD"] = get_parameter_from_ssm( - "clickhouse-test-stat-password" - ) + popen_env[ + "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL" + ] = get_parameter_from_ssm("clickhouse-test-stat-url") + popen_env[ + "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER" + ] = get_parameter_from_ssm("clickhouse-test-stat-login") + popen_env[ + "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD" + ] = get_parameter_from_ssm("clickhouse-test-stat-password") run_log_path = os.path.join(temp_path, "runlog.log") with TeePopen(run_command, run_log_path, env=popen_env) as process: From c2d774d598f186e28d7003f382a44761dbd21478 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 9 Jun 2022 14:49:47 +0200 Subject: [PATCH 225/627] [testing] run only one perftest --- docker/test/performance-comparison/compare.sh | 2 +- tests/ci/performance_comparison_check.py | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index fa03ecde25a..d73aed7f2b7 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -218,7 +218,7 @@ function run_tests if [ -v CHPC_TEST_RUN_BY_HASH_TOTAL ]; then # filter tests array in bash https://stackoverflow.com/a/40375567 for index in "${!test_files[@]}"; do - [ $(( index % CHPC_TEST_RUN_BY_HASH_TOTAL )) != "$CHPC_TEST_RUN_BY_HASH_NUM" ] && \ + [ $(( index )) != "$CHPC_TEST_RUN_BY_HASH_NUM" ] && \ unset -v 'test_files[$index]' done # to have sequential indexes... diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index e3f39c47fea..b1e617662f5 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -102,6 +102,10 @@ if __name__ == "__main__": if "RUN_BY_HASH_TOTAL" in os.environ: run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "1")) run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "1")) + if run_by_hash_num != 1: + print("Skipped ", run_by_hash_num) + sys.exit(0) + docker_env += ( f" -e CHPC_TEST_RUN_BY_HASH_TOTAL={run_by_hash_total}" f" -e CHPC_TEST_RUN_BY_HASH_NUM={run_by_hash_num}" From 45b2b1ca8a26d7e1befa0381b710e1c9c75d3d89 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 10 Jun 2022 12:53:09 +0200 Subject: [PATCH 226/627] [wip] persist pertest results --- docker/test/performance-comparison/compare.sh | 13 ++++++------ tests/ci/performance_comparison_check.py | 21 +++++++++++-------- 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index d73aed7f2b7..95475f27ab7 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -574,7 +574,6 @@ then --user "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER}" --password "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD}" --config "right/config/client_config.xml" - --database perftest --date_time_input_format=best_effort) @@ -1304,13 +1303,15 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') /^old-sha/ { old_sha=$2 } /^new-sha/ { new_sha=$2 } /^metric/ { print old_sha, new_sha, $2, $3 }' \ - | "${client[@]}" --query "INSERT INTO run_attributes_v1 FORMAT TSV" + | cat + + # | "${client[@]}" --query "INSERT INTO run_attributes_v1 FORMAT TSV" # Grepping numactl results from log is too crazy, I'll just call it again. - "${client[@]}" --query "INSERT INTO run_attributes_v1 FORMAT TSV" < Date: Fri, 10 Jun 2022 16:04:31 +0200 Subject: [PATCH 227/627] create tables for perf tests --- docker/test/performance-comparison/compare.sh | 41 +++++++++++++++---- 1 file changed, 34 insertions(+), 7 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 95475f27ab7..7948adee42d 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1263,6 +1263,35 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') --database perftest --date_time_input_format=best_effort) + "${client[@]}" --query ' + CREATE TABLE IF NOT EXISTS query_metrics_v2 ( + `event_date` Date, + `event_time` DateTime, + `pr_number` UInt32, + `old_sha` String, + `new_sha` String, + `test` LowCardinality(String), + `query_index` UInt32, + `query_display_name` String, + `metric_name` LowCardinality(String), + `old_value` Float64, + `new_value` Float64, + `diff` Float64, + `stat_threshold` Float64 + ) ENGINE = ReplicatedMergeTree + ORDER BY event_date + ' + + "${client[@]}" --query ' + CREATE TABLE IF NOT EXISTS run_attributes_v1 ( + `old_sha` String, + `new_sha` String, + `metric_name` LowCardinality(String), + `metric_value` String + ) ENGINE = ReplicatedMergeTree + ORDER BY old_sha, new_sha + ' + "${client[@]}" --query " insert into query_metrics_v2 select @@ -1303,15 +1332,13 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') /^old-sha/ { old_sha=$2 } /^new-sha/ { new_sha=$2 } /^metric/ { print old_sha, new_sha, $2, $3 }' \ - | cat - - # | "${client[@]}" --query "INSERT INTO run_attributes_v1 FORMAT TSV" + | "${client[@]}" --query "INSERT INTO run_attributes_v1 FORMAT TSV" # Grepping numactl results from log is too crazy, I'll just call it again. -# "${client[@]}" --query "INSERT INTO run_attributes_v1 FORMAT TSV" < Date: Fri, 10 Jun 2022 17:35:04 +0200 Subject: [PATCH 228/627] Use default database for perf tests --- docker/test/performance-comparison/compare.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 7948adee42d..48b8e2d7bc3 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1260,7 +1260,6 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') --user "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER}" --password "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD}" --config "right/config/client_config.xml" - --database perftest --date_time_input_format=best_effort) "${client[@]}" --query ' From 18dc718e4e1bf6e0ec1c00f417f68eba2403349a Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 14 Jun 2022 19:39:16 +0200 Subject: [PATCH 229/627] do not create tables for perf tests --- docker/test/performance-comparison/compare.sh | 50 +++++++++---------- 1 file changed, 23 insertions(+), 27 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 48b8e2d7bc3..1cedbc8a7be 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1262,34 +1262,30 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') --config "right/config/client_config.xml" --date_time_input_format=best_effort) - "${client[@]}" --query ' - CREATE TABLE IF NOT EXISTS query_metrics_v2 ( - `event_date` Date, - `event_time` DateTime, - `pr_number` UInt32, - `old_sha` String, - `new_sha` String, - `test` LowCardinality(String), - `query_index` UInt32, - `query_display_name` String, - `metric_name` LowCardinality(String), - `old_value` Float64, - `new_value` Float64, - `diff` Float64, - `stat_threshold` Float64 - ) ENGINE = ReplicatedMergeTree - ORDER BY event_date - ' + # CREATE TABLE IF NOT EXISTS query_metrics_v2 ( + # `event_date` Date, + # `event_time` DateTime, + # `pr_number` UInt32, + # `old_sha` String, + # `new_sha` String, + # `test` LowCardinality(String), + # `query_index` UInt32, + # `query_display_name` String, + # `metric_name` LowCardinality(String), + # `old_value` Float64, + # `new_value` Float64, + # `diff` Float64, + # `stat_threshold` Float64 + # ) ENGINE = ReplicatedMergeTree + # ORDER BY event_date - "${client[@]}" --query ' - CREATE TABLE IF NOT EXISTS run_attributes_v1 ( - `old_sha` String, - `new_sha` String, - `metric_name` LowCardinality(String), - `metric_value` String - ) ENGINE = ReplicatedMergeTree - ORDER BY old_sha, new_sha - ' + # CREATE TABLE IF NOT EXISTS run_attributes_v1 ( + # `old_sha` String, + # `new_sha` String, + # `metric_name` LowCardinality(String), + # `metric_value` String + # ) ENGINE = ReplicatedMergeTree + # ORDER BY (old_sha, new_sha) "${client[@]}" --query " insert into query_metrics_v2 From b1ee16ef77a0ddd513548c2c4d19d3f8549d2e27 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 15 Jun 2022 13:40:52 +0200 Subject: [PATCH 230/627] show grants in compare.sh --- docker/test/performance-comparison/compare.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 1cedbc8a7be..2660a25b198 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1287,6 +1287,8 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') # ) ENGINE = ReplicatedMergeTree # ORDER BY (old_sha, new_sha) + "${client[@]}" --query "show grants for ci" + "${client[@]}" --query " insert into query_metrics_v2 select From c1d44fee9fba61c8ff878d38eac04a05ac3b99fd Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 15 Jun 2022 22:25:36 +0200 Subject: [PATCH 231/627] upd compare.sh --- docker/test/performance-comparison/compare.sh | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 2660a25b198..d180b48d168 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1195,16 +1195,14 @@ unset IFS function upload_results { # Prepare info for the CI checks table. - rm ci-checks.tsv + rm -f ci-checks.tsv + clickhouse-local --query " -create view queries as select * from file('report/queries.tsv', TSVWithNamesAndTypes, - 'changed_fail int, changed_show int, unstable_fail int, unstable_show int, - left float, right float, diff float, stat_threshold float, - test text, query_index int, query_display_name text'); +create view queries as select * from file('report/queries.tsv', TSVWithNamesAndTypes); create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') as select - $PR_TO_TEST pull_request_number, + $PR_TO_TEST :: UInt32 AS pull_request_number, '$SHA_TO_TEST' commit_sha, 'Performance' check_name, '$(sed -n 's/.*/\1/p' report.html)' check_status, @@ -1271,7 +1269,7 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') # `test` LowCardinality(String), # `query_index` UInt32, # `query_display_name` String, - # `metric_name` LowCardinality(String), + # `metric` LowCardinality(String), # `old_value` Float64, # `new_value` Float64, # `diff` Float64, @@ -1282,7 +1280,7 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') # CREATE TABLE IF NOT EXISTS run_attributes_v1 ( # `old_sha` String, # `new_sha` String, - # `metric_name` LowCardinality(String), + # `metric` LowCardinality(String), # `metric_value` String # ) ENGINE = ReplicatedMergeTree # ORDER BY (old_sha, new_sha) @@ -1300,7 +1298,7 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') test, query_index, query_display_name, - metric_name, + metric_name as metric, old_value, new_value, diff, @@ -1310,7 +1308,6 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') test text, query_index int, query_display_name text') settings date_time_input_format='best_effort' format TSV - settings date_time_input_format='best_effort' " < report/all-query-metrics.tsv # Don't leave whitespace after INSERT: https://github.com/ClickHouse/ClickHouse/issues/16652 # Upload some run attributes. I use this weird form because it is the same @@ -1337,6 +1334,8 @@ $REF_SHA $SHA_TO_TEST $(numactl --show | sed -n 's/^cpubind:[[:space:]]\+/numact $REF_SHA $SHA_TO_TEST $(numactl --hardware | sed -n 's/^available:[[:space:]]\+/numactl-available /p') EOF + head ci-checks.tsv + # Also insert some data about the check into the CI checks table. "${client[@]}" --query "INSERT INTO "'"'"default"'"'".checks FORMAT TSVWithNamesAndTypes" \ < ci-checks.tsv From 051dd63a620ba07d58c9ff6c81e2fc45aa435e00 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 17 Jun 2022 16:14:18 +0200 Subject: [PATCH 232/627] Fix types in ci-checks.tsv in compare.sh --- docker/test/performance-comparison/compare.sh | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index d180b48d168..d0e7c587211 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1203,15 +1203,15 @@ create view queries as select * from file('report/queries.tsv', TSVWithNamesAndT create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') as select $PR_TO_TEST :: UInt32 AS pull_request_number, - '$SHA_TO_TEST' commit_sha, - 'Performance' check_name, - '$(sed -n 's/.*/\1/p' report.html)' check_status, + '$SHA_TO_TEST' :: LowCardinality(String) AS commit_sha, + 'Performance' :: LowCardinality(String) AS check_name, + '$(sed -n 's/.*/\1/p' report.html)' :: LowCardinality(String) AS check_status, -- TODO toDateTime() can't parse output of 'date', so no time for now. - ($(date +%s) - $CHPC_CHECK_START_TIMESTAMP) * 1000 check_duration_ms, + (($(date +%s) - $CHPC_CHECK_START_TIMESTAMP) * 1000) :: UInt64 AS check_duration_ms, fromUnixTimestamp($CHPC_CHECK_START_TIMESTAMP) check_start_time, - test_name, - test_status, - test_duration_ms, + test_name :: LowCardinality(String) AS test_name , + test_status :: LowCardinality(String) AS test_status, + test_duration_ms :: UInt64 AS test_duration_ms, report_url, $PR_TO_TEST = 0 ? 'https://github.com/ClickHouse/ClickHouse/commit/$SHA_TO_TEST' From 3731bd44c129f120270329d25e51b56e099bbe8a Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 27 Jun 2022 15:19:13 +0200 Subject: [PATCH 233/627] Revert "[testing] run only one perftest" This reverts commit 3b079827b78e04ac20043346998ce9d546969746. --- docker/test/performance-comparison/compare.sh | 2 +- tests/ci/performance_comparison_check.py | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index d0e7c587211..e90903b29cc 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -218,7 +218,7 @@ function run_tests if [ -v CHPC_TEST_RUN_BY_HASH_TOTAL ]; then # filter tests array in bash https://stackoverflow.com/a/40375567 for index in "${!test_files[@]}"; do - [ $(( index )) != "$CHPC_TEST_RUN_BY_HASH_NUM" ] && \ + [ $(( index % CHPC_TEST_RUN_BY_HASH_TOTAL )) != "$CHPC_TEST_RUN_BY_HASH_NUM" ] && \ unset -v 'test_files[$index]' done # to have sequential indexes... diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index f6cd9eb18b8..c2d3597b6c8 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -102,10 +102,6 @@ if __name__ == "__main__": if "RUN_BY_HASH_TOTAL" in os.environ: run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "1")) run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "1")) - if run_by_hash_num != 1: - print("Skipped ", run_by_hash_num) - sys.exit(0) - docker_env += ( f" -e CHPC_TEST_RUN_BY_HASH_TOTAL={run_by_hash_total}" f" -e CHPC_TEST_RUN_BY_HASH_NUM={run_by_hash_num}" From e4917914aeda94f3aaaa4b1cf95f339eeeaaa856 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 27 Jun 2022 15:21:23 +0200 Subject: [PATCH 234/627] Revert "show grants in compare.sh" This reverts commit 4d3a64b318aa665bf1494cca6ae94e58dc8d361e. --- docker/test/performance-comparison/compare.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index e90903b29cc..f4b4ca31437 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1285,8 +1285,6 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') # ) ENGINE = ReplicatedMergeTree # ORDER BY (old_sha, new_sha) - "${client[@]}" --query "show grants for ci" - "${client[@]}" --query " insert into query_metrics_v2 select From 45b1c3fe204ec55024b3df39c439ee9d312e1904 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 5 Jul 2022 15:15:11 +0300 Subject: [PATCH 235/627] Revert "Smallish updates of dev guide" --- docs/en/development/build.md | 15 ++++ docs/en/development/contrib.md | 85 +++++++++++++++++++- docs/en/development/developer-instruction.md | 20 ----- 3 files changed, 97 insertions(+), 23 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index dbb90f8e537..4f06c52a1b5 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -164,3 +164,18 @@ ClickHouse is available in pre-built binaries and packages. Binaries are portabl They are built for stable, prestable and testing releases as long as for every commit to master and for every pull request. To find the freshest build from `master`, go to [commits page](https://github.com/ClickHouse/ClickHouse/commits/master), click on the first green check mark or red cross near commit, and click to the “Details” link right after “ClickHouse Build Check”. + +## Faster builds for development: Split build configuration {#split-build} + +Normally, ClickHouse is statically linked into a single static `clickhouse` binary with minimal dependencies. This is convenient for distribution, but it means that on every change the entire binary needs to be linked, which is slow and may be inconvenient for development. There is an alternative configuration which instead creates dynamically loaded shared libraries and separate binaries `clickhouse-server`, `clickhouse-client` etc., allowing for faster incremental builds. To use it, add the following flags to your `cmake` invocation: +``` +-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 +``` + +Note that the split build has several drawbacks: +* There is no single `clickhouse` binary, and you have to run `clickhouse-server`, `clickhouse-client`, etc. +* Risk of segfault if you run any of the programs while rebuilding the project. +* You cannot run the integration tests since they only work a single complete binary. +* You can't easily copy the binaries elsewhere. Instead of moving a single binary you'll need to copy all binaries and libraries. + +[Original article](https://clickhouse.com/docs/en/development/build/) diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 13af1be5097..8c1f6b5fc9e 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -6,14 +6,93 @@ description: A list of third-party libraries used # Third-Party Libraries Used -ClickHouse utilizes third-party libraries for different purposes, e.g., to connect to other databases, to decode (encode) data during load (save) from (to) disk or to implement certain specialized SQL functions. To be independent of the available libraries in the target system, each third-party library is imported as a Git submodule into ClickHouse's source tree and compiled and linked with ClickHouse. A list of third-party libraries and their licenses can be obtained by the following query: +The list of third-party libraries: + +| Library name | License type | +|:-|:-| +| abseil-cpp | [Apache](https://github.com/ClickHouse-Extras/abseil-cpp/blob/4f3b686f86c3ebaba7e4e926e62a79cb1c659a54/LICENSE) | +| AMQP-CPP | [Apache](https://github.com/ClickHouse-Extras/AMQP-CPP/blob/1a6c51f4ac51ac56610fa95081bd2f349911375a/LICENSE) | +| arrow | [Apache](https://github.com/ClickHouse-Extras/arrow/blob/078e21bad344747b7656ef2d7a4f7410a0a303eb/LICENSE.txt) | +| avro | [Apache](https://github.com/ClickHouse-Extras/avro/blob/e43c46e87fd32eafdc09471e95344555454c5ef8/LICENSE.txt) | +| aws | [Apache](https://github.com/ClickHouse-Extras/aws-sdk-cpp/blob/7d48b2c8193679cc4516e5bd68ae4a64b94dae7d/LICENSE.txt) | +| aws-c-common | [Apache](https://github.com/ClickHouse-Extras/aws-c-common/blob/736a82d1697c108b04a277e66438a7f4e19b6857/LICENSE) | +| aws-c-event-stream | [Apache](https://github.com/ClickHouse-Extras/aws-c-event-stream/blob/3bc33662f9ccff4f4cbcf9509cc78c26e022fde0/LICENSE) | +| aws-checksums | [Apache](https://github.com/ClickHouse-Extras/aws-checksums/blob/519d6d9093819b6cf89ffff589a27ef8f83d0f65/LICENSE) | +| base58 | [MIT](https://github.com/ClickHouse/base-x/blob/3e58874643c087f57e82b0ff03825c933fab945a/LICENSE) | +| base64 | [BSD 2-clause](https://github.com/ClickHouse-Extras/Turbo-Base64/blob/af9b331f2b4f30b41c70f3a571ff904a8251c1d3/LICENSE) | +| boost | [Boost](https://github.com/ClickHouse-Extras/boost/blob/9cf09dbfd55a5c6202dedbdf40781a51b02c2675/LICENSE_1_0.txt) | +| boringssl | [BSD](https://github.com/ClickHouse-Extras/boringssl/blob/a6a2e2ab3e44d97ce98e51c558e989f211de7eb3/LICENSE) | +| brotli | [MIT](https://github.com/google/brotli/blob/63be8a99401992075c23e99f7c84de1c653e39e2/LICENSE) | +| capnproto | [MIT](https://github.com/capnproto/capnproto/blob/a00ccd91b3746ef2ab51d40fe3265829949d1ace/LICENSE) | +| cassandra | [Apache](https://github.com/ClickHouse-Extras/cpp-driver/blob/eb9b68dadbb4417a2c132ad4a1c2fa76e65e6fc1/LICENSE.txt) | +| cctz | [Apache](https://github.com/ClickHouse-Extras/cctz/blob/c0f1bcb97fd2782f7c3f972fadd5aad5affac4b8/LICENSE.txt) | +| cityhash102 | [MIT](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/cityhash102/COPYING) | +| cppkafka | [BSD 2-clause](https://github.com/mfontanini/cppkafka/blob/5a119f689f8a4d90d10a9635e7ee2bee5c127de1/LICENSE) | +| croaring | [Apache](https://github.com/RoaringBitmap/CRoaring/blob/2c867e9f9c9e2a3a7032791f94c4c7ae3013f6e0/LICENSE) | +| curl | [Apache](https://github.com/curl/curl/blob/3b8bbbbd1609c638a3d3d0acb148a33dedb67be3/docs/LICENSE-MIXING.md) | +| cyrus-sasl | [BSD 2-clause](https://github.com/ClickHouse-Extras/cyrus-sasl/blob/e6466edfd638cc5073debe941c53345b18a09512/COPYING) | +| double-conversion | [BSD 3-clause](https://github.com/google/double-conversion/blob/cf2f0f3d547dc73b4612028a155b80536902ba02/LICENSE) | +| dragonbox | [Apache](https://github.com/ClickHouse-Extras/dragonbox/blob/923705af6fd953aa948fc175f6020b15f7359838/LICENSE-Apache2-LLVM) | +| fast_float | [Apache](https://github.com/fastfloat/fast_float/blob/7eae925b51fd0f570ccd5c880c12e3e27a23b86f/LICENSE) | +| fastops | [MIT](https://github.com/ClickHouse-Extras/fastops/blob/88752a5e03cf34639a4a37a4b41d8b463fffd2b5/LICENSE) | +| flatbuffers | [Apache](https://github.com/ClickHouse-Extras/flatbuffers/blob/eb3f827948241ce0e701516f16cd67324802bce9/LICENSE.txt) | +| fmtlib | [Unknown](https://github.com/fmtlib/fmt/blob/c108ee1d590089ccf642fc85652b845924067af2/LICENSE.rst) | +| gcem | [Apache](https://github.com/kthohr/gcem/blob/8d4f1b5d76ea8f6ff12f3f4f34cda45424556b00/LICENSE) | +| googletest | [BSD 3-clause](https://github.com/google/googletest/blob/e7e591764baba0a0c3c9ad0014430e7a27331d16/LICENSE) | +| grpc | [Apache](https://github.com/ClickHouse-Extras/grpc/blob/60c986e15cae70aade721d26badabab1f822fdd6/LICENSE) | +| h3 | [Apache](https://github.com/ClickHouse-Extras/h3/blob/c7f46cfd71fb60e2fefc90e28abe81657deff735/LICENSE) | +| vectorscan | [Boost](https://github.com/ClickHouse-Extras/hyperscan/blob/73695e419c27af7fe2a099c7aa57931cc02aea5d/LICENSE) | +| icu | [Public Domain](https://github.com/unicode-org/icu/blob/a56dde820dc35665a66f2e9ee8ba58e75049b668/icu4c/LICENSE) | +| icudata | [Public Domain](https://github.com/ClickHouse-Extras/icudata/blob/72d9a4a7febc904e2b0a534ccb25ae40fac5f1e5/LICENSE) | +| jemalloc | [BSD 2-clause](https://github.com/ClickHouse-Extras/jemalloc/blob/e6891d9746143bf2cf617493d880ba5a0b9a3efd/COPYING) | +| krb5 | [MIT](https://github.com/ClickHouse-Extras/krb5/blob/5149dea4e2be0f67707383d2682b897c14631374/src/lib/gssapi/LICENSE) | +| libc-headers | [LGPL](https://github.com/ClickHouse-Extras/libc-headers/blob/a720b7105a610acbd7427eea475a5b6810c151eb/LICENSE) | +| libcpuid | [BSD 2-clause](https://github.com/ClickHouse-Extras/libcpuid/blob/8db3b8d2d32d22437f063ce692a1b9bb15e42d18/COPYING) | +| libcxx | [Apache](https://github.com/ClickHouse-Extras/libcxx/blob/2fa892f69acbaa40f8a18c6484854a6183a34482/LICENSE.TXT) | +| libcxxabi | [Apache](https://github.com/ClickHouse-Extras/libcxxabi/blob/df8f1e727dbc9e2bedf2282096fa189dc3fe0076/LICENSE.TXT) | +| libdivide | [zLib](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libdivide/LICENSE.txt) | +| libfarmhash | [MIT](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libfarmhash/COPYING) | +| libgsasl | [LGPL](https://github.com/ClickHouse-Extras/libgsasl/blob/383ee28e82f69fa16ed43b48bd9c8ee5b313ab84/LICENSE) | +| libhdfs3 | [Apache](https://github.com/ClickHouse-Extras/libhdfs3/blob/095b9d48b400abb72d967cb0539af13b1e3d90cf/LICENSE.txt) | +| libmetrohash | [Apache](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libmetrohash/LICENSE) | +| libpq | [Unknown](https://github.com/ClickHouse-Extras/libpq/blob/e071ea570f8985aa00e34f5b9d50a3cfe666327e/COPYRIGHT) | +| libpqxx | [BSD 3-clause](https://github.com/ClickHouse-Extras/libpqxx/blob/357608d11b7a1961c3fb7db2ef9a5dbb2e87da77/COPYING) | +| librdkafka | [MIT](https://github.com/ClickHouse-Extras/librdkafka/blob/b8554f1682062c85ba519eb54ef2f90e02b812cb/LICENSE.murmur2) | +| libunwind | [Apache](https://github.com/ClickHouse-Extras/libunwind/blob/6b816d2fba3991f8fd6aaec17d92f68947eab667/LICENSE.TXT) | +| libuv | [BSD](https://github.com/ClickHouse-Extras/libuv/blob/e2e9b7e9f978ce8a1367b5fe781d97d1ce9f94ab/LICENSE) | +| llvm | [Apache](https://github.com/ClickHouse-Extras/llvm/blob/e5751459412bce1391fb7a2e9bbc01e131bf72f1/llvm/LICENSE.TXT) | +| lz4 | [BSD](https://github.com/lz4/lz4/blob/f39b79fb02962a1cd880bbdecb6dffba4f754a11/LICENSE) | +| mariadb-connector-c | [LGPL](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/5f4034a3a6376416504f17186c55fe401c6d8e5e/COPYING.LIB) | +| miniselect | [Boost](https://github.com/danlark1/miniselect/blob/be0af6bd0b6eb044d1acc4f754b229972d99903a/LICENSE_1_0.txt) | +| msgpack-c | [Boost](https://github.com/msgpack/msgpack-c/blob/46684265d50b5d1b062d4c5c428ba08462844b1d/LICENSE_1_0.txt) | +| murmurhash | [Public Domain](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/murmurhash/LICENSE) | +| NuRaft | [Apache](https://github.com/ClickHouse-Extras/NuRaft/blob/7ecb16844af6a9c283ad432d85ecc2e7d1544676/LICENSE) | +| openldap | [Unknown](https://github.com/ClickHouse-Extras/openldap/blob/0208811b6043ca06fda8631a5e473df1ec515ccb/LICENSE) | +| orc | [Apache](https://github.com/ClickHouse-Extras/orc/blob/0a936f6bbdb9303308973073f8623b5a8d82eae1/LICENSE) | +| poco | [Boost](https://github.com/ClickHouse-Extras/poco/blob/7351c4691b5d401f59e3959adfc5b4fa263b32da/LICENSE) | +| protobuf | [BSD 3-clause](https://github.com/ClickHouse-Extras/protobuf/blob/75601841d172c73ae6bf4ce8121f42b875cdbabd/LICENSE) | +| rapidjson | [MIT](https://github.com/ClickHouse-Extras/rapidjson/blob/c4ef90ccdbc21d5d5a628d08316bfd301e32d6fa/bin/jsonschema/LICENSE) | +| re2 | [BSD 3-clause](https://github.com/google/re2/blob/13ebb377c6ad763ca61d12dd6f88b1126bd0b911/LICENSE) | +| replxx | [BSD 3-clause](https://github.com/ClickHouse-Extras/replxx/blob/c81be6c68b146f15f2096b7ef80e3f21fe27004c/LICENSE.md) | +| rocksdb | [BSD 3-clause](https://github.com/ClickHouse-Extras/rocksdb/blob/b6480c69bf3ab6e298e0d019a07fd4f69029b26a/LICENSE.leveldb) | +| s2geometry | [Apache](https://github.com/ClickHouse-Extras/s2geometry/blob/20ea540d81f4575a3fc0aea585aac611bcd03ede/LICENSE) | +| sentry-native | [MIT](https://github.com/ClickHouse-Extras/sentry-native/blob/94644e92f0a3ff14bd35ed902a8622a2d15f7be4/LICENSE) | +| simdjson | [Apache](https://github.com/simdjson/simdjson/blob/8df32cea3359cb30120795da6020b3b73da01d38/LICENSE) | +| snappy | [Public Domain](https://github.com/google/snappy/blob/3f194acb57e0487531c96b97af61dcbd025a78a3/COPYING) | +| sparsehash-c11 | [BSD 3-clause](https://github.com/sparsehash/sparsehash-c11/blob/cf0bffaa456f23bc4174462a789b90f8b6f5f42f/LICENSE) | +| stats | [Apache](https://github.com/kthohr/stats/blob/b6dd459c10a88c7ea04693c007e9e35820c5d9ad/LICENSE) | +| thrift | [Apache](https://github.com/apache/thrift/blob/010ccf0a0c7023fea0f6bf4e4078ebdff7e61982/LICENSE) | +| unixodbc | [LGPL](https://github.com/ClickHouse-Extras/UnixODBC/blob/b0ad30f7f6289c12b76f04bfb9d466374bb32168/COPYING) | +| xz | [Public Domain](https://github.com/xz-mirror/xz/blob/869b9d1b4edd6df07f819d360d306251f8147353/COPYING) | +| zlib-ng | [zLib](https://github.com/ClickHouse-Extras/zlib-ng/blob/6a5e93b9007782115f7f7e5235dedc81c4f1facb/LICENSE.md) | +| zstd | [BSD](https://github.com/facebook/zstd/blob/a488ba114ec17ea1054b9057c26a046fc122b3b6/LICENSE) | + +The list of third-party libraries can be obtained by the following query: ``` sql SELECT library_name, license_type, license_path FROM system.licenses ORDER BY library_name COLLATE 'en'; ``` -(Note that the listed libraries are the ones located in the `contrib/` directory of the ClickHouse repository. Depending on the build options, some of of the libraries may have not been compiled, and as a result, their functionality may not be available at runtime. - [Example](https://play.clickhouse.com/play?user=play#U0VMRUNUIGxpYnJhcnlfbmFtZSwgbGljZW5zZV90eXBlLCBsaWNlbnNlX3BhdGggRlJPTSBzeXN0ZW0ubGljZW5zZXMgT1JERVIgQlkgbGlicmFyeV9uYW1lIENPTExBVEUgJ2VuJw==) ## Adding new third-party libraries and maintaining patches in third-party libraries {#adding-third-party-libraries} diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 77ddae6a756..38d99430193 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -276,23 +276,3 @@ Testing will commence as soon as ClickHouse employees label your PR with a tag The system will prepare ClickHouse binary builds for your pull request individually. To retrieve these builds click the “Details” link next to “ClickHouse build check” entry in the list of checks. There you will find direct links to the built .deb packages of ClickHouse which you can deploy even on your production servers (if you have no fear). Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page, you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways. - -## Faster builds for development: Split build configuration {#split-build} - -ClickHouse is normally statically linked into a single static `clickhouse` binary with minimal dependencies. This is convenient for distribution, but it means that for every change the entire binary needs to be re-linked, which is slow and inconvenient for development. As an alternative, you can instead build dynamically linked shared libraries and separate binaries `clickhouse-server`, `clickhouse-client` etc., allowing for faster incremental builds. To use it, add the following flags to your `cmake` invocation: -``` --DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 -``` - -Note that the split build has several drawbacks: -* There is no single `clickhouse` binary, and you have to run `clickhouse-server`, `clickhouse-client`, etc. -* Risk of segfault if you run any of the programs while rebuilding the project. -* You cannot run the integration tests since they only work a single complete binary. -* You can't easily copy the binaries elsewhere. Instead of moving a single binary you'll need to copy all binaries and libraries. - -If you are not interested in functionality provided by third-party libraries, you can further speed up the build using `cmake` options -``` --DENABLE_LIBRARIES=0 -DENABLE_EMBEDDED_COMPILER=0 -``` - -In case of problems with any of the development options, you are on your own! From e2d3bb80f4e11abfc318bd8c9e72f8ee2d664a67 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Jul 2022 14:24:00 +0200 Subject: [PATCH 236/627] Revert "Revert "Smallish updates of dev guide"" This reverts commit 45b1c3fe204ec55024b3df39c439ee9d312e1904. --- docs/en/development/build.md | 15 ---- docs/en/development/contrib.md | 85 +------------------- docs/en/development/developer-instruction.md | 20 +++++ 3 files changed, 23 insertions(+), 97 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 4f06c52a1b5..dbb90f8e537 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -164,18 +164,3 @@ ClickHouse is available in pre-built binaries and packages. Binaries are portabl They are built for stable, prestable and testing releases as long as for every commit to master and for every pull request. To find the freshest build from `master`, go to [commits page](https://github.com/ClickHouse/ClickHouse/commits/master), click on the first green check mark or red cross near commit, and click to the “Details” link right after “ClickHouse Build Check”. - -## Faster builds for development: Split build configuration {#split-build} - -Normally, ClickHouse is statically linked into a single static `clickhouse` binary with minimal dependencies. This is convenient for distribution, but it means that on every change the entire binary needs to be linked, which is slow and may be inconvenient for development. There is an alternative configuration which instead creates dynamically loaded shared libraries and separate binaries `clickhouse-server`, `clickhouse-client` etc., allowing for faster incremental builds. To use it, add the following flags to your `cmake` invocation: -``` --DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 -``` - -Note that the split build has several drawbacks: -* There is no single `clickhouse` binary, and you have to run `clickhouse-server`, `clickhouse-client`, etc. -* Risk of segfault if you run any of the programs while rebuilding the project. -* You cannot run the integration tests since they only work a single complete binary. -* You can't easily copy the binaries elsewhere. Instead of moving a single binary you'll need to copy all binaries and libraries. - -[Original article](https://clickhouse.com/docs/en/development/build/) diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 8c1f6b5fc9e..13af1be5097 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -6,93 +6,14 @@ description: A list of third-party libraries used # Third-Party Libraries Used -The list of third-party libraries: - -| Library name | License type | -|:-|:-| -| abseil-cpp | [Apache](https://github.com/ClickHouse-Extras/abseil-cpp/blob/4f3b686f86c3ebaba7e4e926e62a79cb1c659a54/LICENSE) | -| AMQP-CPP | [Apache](https://github.com/ClickHouse-Extras/AMQP-CPP/blob/1a6c51f4ac51ac56610fa95081bd2f349911375a/LICENSE) | -| arrow | [Apache](https://github.com/ClickHouse-Extras/arrow/blob/078e21bad344747b7656ef2d7a4f7410a0a303eb/LICENSE.txt) | -| avro | [Apache](https://github.com/ClickHouse-Extras/avro/blob/e43c46e87fd32eafdc09471e95344555454c5ef8/LICENSE.txt) | -| aws | [Apache](https://github.com/ClickHouse-Extras/aws-sdk-cpp/blob/7d48b2c8193679cc4516e5bd68ae4a64b94dae7d/LICENSE.txt) | -| aws-c-common | [Apache](https://github.com/ClickHouse-Extras/aws-c-common/blob/736a82d1697c108b04a277e66438a7f4e19b6857/LICENSE) | -| aws-c-event-stream | [Apache](https://github.com/ClickHouse-Extras/aws-c-event-stream/blob/3bc33662f9ccff4f4cbcf9509cc78c26e022fde0/LICENSE) | -| aws-checksums | [Apache](https://github.com/ClickHouse-Extras/aws-checksums/blob/519d6d9093819b6cf89ffff589a27ef8f83d0f65/LICENSE) | -| base58 | [MIT](https://github.com/ClickHouse/base-x/blob/3e58874643c087f57e82b0ff03825c933fab945a/LICENSE) | -| base64 | [BSD 2-clause](https://github.com/ClickHouse-Extras/Turbo-Base64/blob/af9b331f2b4f30b41c70f3a571ff904a8251c1d3/LICENSE) | -| boost | [Boost](https://github.com/ClickHouse-Extras/boost/blob/9cf09dbfd55a5c6202dedbdf40781a51b02c2675/LICENSE_1_0.txt) | -| boringssl | [BSD](https://github.com/ClickHouse-Extras/boringssl/blob/a6a2e2ab3e44d97ce98e51c558e989f211de7eb3/LICENSE) | -| brotli | [MIT](https://github.com/google/brotli/blob/63be8a99401992075c23e99f7c84de1c653e39e2/LICENSE) | -| capnproto | [MIT](https://github.com/capnproto/capnproto/blob/a00ccd91b3746ef2ab51d40fe3265829949d1ace/LICENSE) | -| cassandra | [Apache](https://github.com/ClickHouse-Extras/cpp-driver/blob/eb9b68dadbb4417a2c132ad4a1c2fa76e65e6fc1/LICENSE.txt) | -| cctz | [Apache](https://github.com/ClickHouse-Extras/cctz/blob/c0f1bcb97fd2782f7c3f972fadd5aad5affac4b8/LICENSE.txt) | -| cityhash102 | [MIT](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/cityhash102/COPYING) | -| cppkafka | [BSD 2-clause](https://github.com/mfontanini/cppkafka/blob/5a119f689f8a4d90d10a9635e7ee2bee5c127de1/LICENSE) | -| croaring | [Apache](https://github.com/RoaringBitmap/CRoaring/blob/2c867e9f9c9e2a3a7032791f94c4c7ae3013f6e0/LICENSE) | -| curl | [Apache](https://github.com/curl/curl/blob/3b8bbbbd1609c638a3d3d0acb148a33dedb67be3/docs/LICENSE-MIXING.md) | -| cyrus-sasl | [BSD 2-clause](https://github.com/ClickHouse-Extras/cyrus-sasl/blob/e6466edfd638cc5073debe941c53345b18a09512/COPYING) | -| double-conversion | [BSD 3-clause](https://github.com/google/double-conversion/blob/cf2f0f3d547dc73b4612028a155b80536902ba02/LICENSE) | -| dragonbox | [Apache](https://github.com/ClickHouse-Extras/dragonbox/blob/923705af6fd953aa948fc175f6020b15f7359838/LICENSE-Apache2-LLVM) | -| fast_float | [Apache](https://github.com/fastfloat/fast_float/blob/7eae925b51fd0f570ccd5c880c12e3e27a23b86f/LICENSE) | -| fastops | [MIT](https://github.com/ClickHouse-Extras/fastops/blob/88752a5e03cf34639a4a37a4b41d8b463fffd2b5/LICENSE) | -| flatbuffers | [Apache](https://github.com/ClickHouse-Extras/flatbuffers/blob/eb3f827948241ce0e701516f16cd67324802bce9/LICENSE.txt) | -| fmtlib | [Unknown](https://github.com/fmtlib/fmt/blob/c108ee1d590089ccf642fc85652b845924067af2/LICENSE.rst) | -| gcem | [Apache](https://github.com/kthohr/gcem/blob/8d4f1b5d76ea8f6ff12f3f4f34cda45424556b00/LICENSE) | -| googletest | [BSD 3-clause](https://github.com/google/googletest/blob/e7e591764baba0a0c3c9ad0014430e7a27331d16/LICENSE) | -| grpc | [Apache](https://github.com/ClickHouse-Extras/grpc/blob/60c986e15cae70aade721d26badabab1f822fdd6/LICENSE) | -| h3 | [Apache](https://github.com/ClickHouse-Extras/h3/blob/c7f46cfd71fb60e2fefc90e28abe81657deff735/LICENSE) | -| vectorscan | [Boost](https://github.com/ClickHouse-Extras/hyperscan/blob/73695e419c27af7fe2a099c7aa57931cc02aea5d/LICENSE) | -| icu | [Public Domain](https://github.com/unicode-org/icu/blob/a56dde820dc35665a66f2e9ee8ba58e75049b668/icu4c/LICENSE) | -| icudata | [Public Domain](https://github.com/ClickHouse-Extras/icudata/blob/72d9a4a7febc904e2b0a534ccb25ae40fac5f1e5/LICENSE) | -| jemalloc | [BSD 2-clause](https://github.com/ClickHouse-Extras/jemalloc/blob/e6891d9746143bf2cf617493d880ba5a0b9a3efd/COPYING) | -| krb5 | [MIT](https://github.com/ClickHouse-Extras/krb5/blob/5149dea4e2be0f67707383d2682b897c14631374/src/lib/gssapi/LICENSE) | -| libc-headers | [LGPL](https://github.com/ClickHouse-Extras/libc-headers/blob/a720b7105a610acbd7427eea475a5b6810c151eb/LICENSE) | -| libcpuid | [BSD 2-clause](https://github.com/ClickHouse-Extras/libcpuid/blob/8db3b8d2d32d22437f063ce692a1b9bb15e42d18/COPYING) | -| libcxx | [Apache](https://github.com/ClickHouse-Extras/libcxx/blob/2fa892f69acbaa40f8a18c6484854a6183a34482/LICENSE.TXT) | -| libcxxabi | [Apache](https://github.com/ClickHouse-Extras/libcxxabi/blob/df8f1e727dbc9e2bedf2282096fa189dc3fe0076/LICENSE.TXT) | -| libdivide | [zLib](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libdivide/LICENSE.txt) | -| libfarmhash | [MIT](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libfarmhash/COPYING) | -| libgsasl | [LGPL](https://github.com/ClickHouse-Extras/libgsasl/blob/383ee28e82f69fa16ed43b48bd9c8ee5b313ab84/LICENSE) | -| libhdfs3 | [Apache](https://github.com/ClickHouse-Extras/libhdfs3/blob/095b9d48b400abb72d967cb0539af13b1e3d90cf/LICENSE.txt) | -| libmetrohash | [Apache](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libmetrohash/LICENSE) | -| libpq | [Unknown](https://github.com/ClickHouse-Extras/libpq/blob/e071ea570f8985aa00e34f5b9d50a3cfe666327e/COPYRIGHT) | -| libpqxx | [BSD 3-clause](https://github.com/ClickHouse-Extras/libpqxx/blob/357608d11b7a1961c3fb7db2ef9a5dbb2e87da77/COPYING) | -| librdkafka | [MIT](https://github.com/ClickHouse-Extras/librdkafka/blob/b8554f1682062c85ba519eb54ef2f90e02b812cb/LICENSE.murmur2) | -| libunwind | [Apache](https://github.com/ClickHouse-Extras/libunwind/blob/6b816d2fba3991f8fd6aaec17d92f68947eab667/LICENSE.TXT) | -| libuv | [BSD](https://github.com/ClickHouse-Extras/libuv/blob/e2e9b7e9f978ce8a1367b5fe781d97d1ce9f94ab/LICENSE) | -| llvm | [Apache](https://github.com/ClickHouse-Extras/llvm/blob/e5751459412bce1391fb7a2e9bbc01e131bf72f1/llvm/LICENSE.TXT) | -| lz4 | [BSD](https://github.com/lz4/lz4/blob/f39b79fb02962a1cd880bbdecb6dffba4f754a11/LICENSE) | -| mariadb-connector-c | [LGPL](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/5f4034a3a6376416504f17186c55fe401c6d8e5e/COPYING.LIB) | -| miniselect | [Boost](https://github.com/danlark1/miniselect/blob/be0af6bd0b6eb044d1acc4f754b229972d99903a/LICENSE_1_0.txt) | -| msgpack-c | [Boost](https://github.com/msgpack/msgpack-c/blob/46684265d50b5d1b062d4c5c428ba08462844b1d/LICENSE_1_0.txt) | -| murmurhash | [Public Domain](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/murmurhash/LICENSE) | -| NuRaft | [Apache](https://github.com/ClickHouse-Extras/NuRaft/blob/7ecb16844af6a9c283ad432d85ecc2e7d1544676/LICENSE) | -| openldap | [Unknown](https://github.com/ClickHouse-Extras/openldap/blob/0208811b6043ca06fda8631a5e473df1ec515ccb/LICENSE) | -| orc | [Apache](https://github.com/ClickHouse-Extras/orc/blob/0a936f6bbdb9303308973073f8623b5a8d82eae1/LICENSE) | -| poco | [Boost](https://github.com/ClickHouse-Extras/poco/blob/7351c4691b5d401f59e3959adfc5b4fa263b32da/LICENSE) | -| protobuf | [BSD 3-clause](https://github.com/ClickHouse-Extras/protobuf/blob/75601841d172c73ae6bf4ce8121f42b875cdbabd/LICENSE) | -| rapidjson | [MIT](https://github.com/ClickHouse-Extras/rapidjson/blob/c4ef90ccdbc21d5d5a628d08316bfd301e32d6fa/bin/jsonschema/LICENSE) | -| re2 | [BSD 3-clause](https://github.com/google/re2/blob/13ebb377c6ad763ca61d12dd6f88b1126bd0b911/LICENSE) | -| replxx | [BSD 3-clause](https://github.com/ClickHouse-Extras/replxx/blob/c81be6c68b146f15f2096b7ef80e3f21fe27004c/LICENSE.md) | -| rocksdb | [BSD 3-clause](https://github.com/ClickHouse-Extras/rocksdb/blob/b6480c69bf3ab6e298e0d019a07fd4f69029b26a/LICENSE.leveldb) | -| s2geometry | [Apache](https://github.com/ClickHouse-Extras/s2geometry/blob/20ea540d81f4575a3fc0aea585aac611bcd03ede/LICENSE) | -| sentry-native | [MIT](https://github.com/ClickHouse-Extras/sentry-native/blob/94644e92f0a3ff14bd35ed902a8622a2d15f7be4/LICENSE) | -| simdjson | [Apache](https://github.com/simdjson/simdjson/blob/8df32cea3359cb30120795da6020b3b73da01d38/LICENSE) | -| snappy | [Public Domain](https://github.com/google/snappy/blob/3f194acb57e0487531c96b97af61dcbd025a78a3/COPYING) | -| sparsehash-c11 | [BSD 3-clause](https://github.com/sparsehash/sparsehash-c11/blob/cf0bffaa456f23bc4174462a789b90f8b6f5f42f/LICENSE) | -| stats | [Apache](https://github.com/kthohr/stats/blob/b6dd459c10a88c7ea04693c007e9e35820c5d9ad/LICENSE) | -| thrift | [Apache](https://github.com/apache/thrift/blob/010ccf0a0c7023fea0f6bf4e4078ebdff7e61982/LICENSE) | -| unixodbc | [LGPL](https://github.com/ClickHouse-Extras/UnixODBC/blob/b0ad30f7f6289c12b76f04bfb9d466374bb32168/COPYING) | -| xz | [Public Domain](https://github.com/xz-mirror/xz/blob/869b9d1b4edd6df07f819d360d306251f8147353/COPYING) | -| zlib-ng | [zLib](https://github.com/ClickHouse-Extras/zlib-ng/blob/6a5e93b9007782115f7f7e5235dedc81c4f1facb/LICENSE.md) | -| zstd | [BSD](https://github.com/facebook/zstd/blob/a488ba114ec17ea1054b9057c26a046fc122b3b6/LICENSE) | - -The list of third-party libraries can be obtained by the following query: +ClickHouse utilizes third-party libraries for different purposes, e.g., to connect to other databases, to decode (encode) data during load (save) from (to) disk or to implement certain specialized SQL functions. To be independent of the available libraries in the target system, each third-party library is imported as a Git submodule into ClickHouse's source tree and compiled and linked with ClickHouse. A list of third-party libraries and their licenses can be obtained by the following query: ``` sql SELECT library_name, license_type, license_path FROM system.licenses ORDER BY library_name COLLATE 'en'; ``` +(Note that the listed libraries are the ones located in the `contrib/` directory of the ClickHouse repository. Depending on the build options, some of of the libraries may have not been compiled, and as a result, their functionality may not be available at runtime. + [Example](https://play.clickhouse.com/play?user=play#U0VMRUNUIGxpYnJhcnlfbmFtZSwgbGljZW5zZV90eXBlLCBsaWNlbnNlX3BhdGggRlJPTSBzeXN0ZW0ubGljZW5zZXMgT1JERVIgQlkgbGlicmFyeV9uYW1lIENPTExBVEUgJ2VuJw==) ## Adding new third-party libraries and maintaining patches in third-party libraries {#adding-third-party-libraries} diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 38d99430193..77ddae6a756 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -276,3 +276,23 @@ Testing will commence as soon as ClickHouse employees label your PR with a tag The system will prepare ClickHouse binary builds for your pull request individually. To retrieve these builds click the “Details” link next to “ClickHouse build check” entry in the list of checks. There you will find direct links to the built .deb packages of ClickHouse which you can deploy even on your production servers (if you have no fear). Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page, you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways. + +## Faster builds for development: Split build configuration {#split-build} + +ClickHouse is normally statically linked into a single static `clickhouse` binary with minimal dependencies. This is convenient for distribution, but it means that for every change the entire binary needs to be re-linked, which is slow and inconvenient for development. As an alternative, you can instead build dynamically linked shared libraries and separate binaries `clickhouse-server`, `clickhouse-client` etc., allowing for faster incremental builds. To use it, add the following flags to your `cmake` invocation: +``` +-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 +``` + +Note that the split build has several drawbacks: +* There is no single `clickhouse` binary, and you have to run `clickhouse-server`, `clickhouse-client`, etc. +* Risk of segfault if you run any of the programs while rebuilding the project. +* You cannot run the integration tests since they only work a single complete binary. +* You can't easily copy the binaries elsewhere. Instead of moving a single binary you'll need to copy all binaries and libraries. + +If you are not interested in functionality provided by third-party libraries, you can further speed up the build using `cmake` options +``` +-DENABLE_LIBRARIES=0 -DENABLE_EMBEDDED_COMPILER=0 +``` + +In case of problems with any of the development options, you are on your own! From ee19c218f379042c0525b0eb7ad4bedcf30c7a20 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Jul 2022 14:24:18 +0200 Subject: [PATCH 237/627] Teach ClickHouse's spellcheck "ClickHouse's" --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a7d16d08b2b..cc22b712c62 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -20,6 +20,7 @@ CamelCase CapnProto CentOS ClickHouse +ClickHouse's Config Contrib Ctrl From f701b58dd0560ae2610254ea5b7a67c4094bc0f7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 5 Jul 2022 14:24:58 +0200 Subject: [PATCH 238/627] Update check_name in performance comparsion upload --- docker/test/performance-comparison/compare.sh | 8 ++++---- tests/ci/performance_comparison_check.py | 17 ++++++++++------- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index f4b4ca31437..5c61a257717 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1204,7 +1204,7 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') as select $PR_TO_TEST :: UInt32 AS pull_request_number, '$SHA_TO_TEST' :: LowCardinality(String) AS commit_sha, - 'Performance' :: LowCardinality(String) AS check_name, + '${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME:-Performance}' :: LowCardinality(String) AS check_name, '$(sed -n 's/.*/\1/p' report.html)' :: LowCardinality(String) AS check_status, -- TODO toDateTime() can't parse output of 'date', so no time for now. (($(date +%s) - $CHPC_CHECK_START_TIMESTAMP) * 1000) :: UInt64 AS check_duration_ms, @@ -1226,15 +1226,15 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') select '' test_name, '$(sed -n 's/.*/\1/p' report.html)' test_status, 0 test_duration_ms, - 'https://clickhouse-test-reports.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#fail1' report_url + 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#fail1' report_url union all select test || ' #' || toString(query_index), 'slower' test_status, 0 test_duration_ms, - 'https://clickhouse-test-reports.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#changes-in-performance.' + 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#changes-in-performance.' || test || '.' || toString(query_index) report_url from queries where changed_fail != 0 and diff > 0 union all select test || ' #' || toString(query_index), 'unstable' test_status, 0 test_duration_ms, - 'https://clickhouse-test-reports.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#unstable-queries.' + 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#unstable-queries.' || test || '.' || toString(query_index) report_url from queries where unstable_fail != 0 ) diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index c2d3597b6c8..baf2593130a 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -126,6 +126,14 @@ if __name__ == "__main__": logging.info("Check is already finished according to github status, exiting") sys.exit(0) + check_name_prefix = ( + check_name_with_group.lower() + .replace(" ", "_") + .replace("(", "_") + .replace(")", "_") + .replace(",", "_") + ) + docker_image = get_image_with_version(reports_path, IMAGE_NAME) # with RamDrive(ramdrive_path, ramdrive_size): @@ -161,6 +169,8 @@ if __name__ == "__main__": "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL": f"{database_url}:9440", "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER": database_username, "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD": database_password, + "CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME": check_name_with_group, + "CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX": check_name_prefix, } ) @@ -186,13 +196,6 @@ if __name__ == "__main__": "runlog.log": run_log_path, } - check_name_prefix = ( - check_name_with_group.lower() - .replace(" ", "_") - .replace("(", "_") - .replace(")", "_") - .replace(",", "_") - ) s3_prefix = f"{pr_info.number}/{pr_info.sha}/{check_name_prefix}/" s3_helper = S3Helper("https://s3.amazonaws.com") uploaded = {} # type: Dict[str, str] From 9e17ae365b751f997e4ef7b72ea7ea29c84a7387 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 5 Jul 2022 14:28:20 +0200 Subject: [PATCH 239/627] Deactivate mutations_finalizing_task during shutdown --- src/Storages/StorageReplicatedMergeTree.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ae9f7640f66..51f0d41a196 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4229,6 +4229,7 @@ void StorageReplicatedMergeTree::shutdown() fetcher.blocker.cancelForever(); merger_mutator.merges_blocker.cancelForever(); parts_mover.moves_blocker.cancelForever(); + mutations_finalizing_task->deactivate(); stopBeingLeader(); restarting_thread.shutdown(); From 43d35eec1b540c3cbc679e662a82b1b98ff92793 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 5 Jul 2022 09:39:52 +0200 Subject: [PATCH 240/627] Write unfinished mutations to backup. --- src/Access/ReplicatedAccessStorage.cpp | 8 +- src/Backups/BackupCoordinationDistributed.cpp | 176 +++++++++++++----- src/Backups/BackupCoordinationDistributed.h | 19 +- src/Backups/BackupCoordinationHelpers.cpp | 144 ++++++++++---- src/Backups/BackupCoordinationHelpers.h | 30 ++- src/Backups/BackupCoordinationLocal.cpp | 43 +++-- src/Backups/BackupCoordinationLocal.h | 22 ++- src/Backups/IBackupCoordination.h | 24 ++- src/Storages/MergeTree/MergeTreeData.cpp | 18 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeMutationEntry.cpp | 13 ++ .../MergeTree/MergeTreeMutationEntry.h | 3 + .../ReplicatedMergeTreeMutationEntry.cpp | 21 +++ .../ReplicatedMergeTreeMutationEntry.h | 3 + src/Storages/StorageMergeTree.cpp | 30 +++ src/Storages/StorageMergeTree.h | 5 + src/Storages/StorageReplicatedMergeTree.cpp | 33 +++- .../test_backup_restore_new/test.py | 37 ++++ .../test_backup_restore_on_cluster/test.py | 56 +++++- 19 files changed, 542 insertions(+), 145 deletions(-) diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index 3c750c0ff4b..ed9ee16bef1 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -637,9 +637,8 @@ void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_col backup_entries_collector.getContext()->getAccessControl()); auto backup_coordination = backup_entries_collector.getBackupCoordination(); - backup_coordination->addReplicatedAccessPath(zookeeper_path, backup_entry_with_path.first); String current_host_id = backup_entries_collector.getBackupSettings().host_id; - backup_coordination->setReplicatedAccessHost(zookeeper_path, current_host_id); + backup_coordination->addReplicatedAccessPath(zookeeper_path, current_host_id, backup_entry_with_path.first); backup_entries_collector.addPostTask( [backup_entry = backup_entry_with_path.second, @@ -648,10 +647,7 @@ void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_col &backup_entries_collector, backup_coordination] { - if (current_host_id != backup_coordination->getReplicatedAccessHost(zookeeper_path)) - return; - - for (const String & path : backup_coordination->getReplicatedAccessPaths(zookeeper_path)) + for (const String & path : backup_coordination->getReplicatedAccessPaths(zookeeper_path, current_host_id)) backup_entries_collector.addBackupEntry(path, backup_entry); }); } diff --git a/src/Backups/BackupCoordinationDistributed.cpp b/src/Backups/BackupCoordinationDistributed.cpp index 5b932229e71..caa7c7aaaa5 100644 --- a/src/Backups/BackupCoordinationDistributed.cpp +++ b/src/Backups/BackupCoordinationDistributed.cpp @@ -27,6 +27,7 @@ namespace using SizeAndChecksum = IBackupCoordination::SizeAndChecksum; using FileInfo = IBackupCoordination::FileInfo; using PartNameAndChecksum = IBackupCoordination::PartNameAndChecksum; + using MutationInfo = IBackupCoordination::MutationInfo; struct ReplicatedPartNames { @@ -63,6 +64,41 @@ namespace } }; + struct ReplicatedMutations + { + std::vector mutations; + String table_name_for_logs; + + static String serialize(const std::vector & mutations_, const String & table_name_for_logs_) + { + WriteBufferFromOwnString out; + writeBinary(mutations_.size(), out); + for (const auto & mutation : mutations_) + { + writeBinary(mutation.id, out); + writeBinary(mutation.entry, out); + } + writeBinary(table_name_for_logs_, out); + return out.str(); + } + + static ReplicatedMutations deserialize(const String & str) + { + ReadBufferFromString in{str}; + ReplicatedMutations res; + size_t num; + readBinary(num, in); + res.mutations.resize(num); + for (size_t i = 0; i != num; ++i) + { + readBinary(res.mutations[i].id, in); + readBinary(res.mutations[i].entry, in); + } + readBinary(res.table_name_for_logs, in); + return res; + } + }; + String serializeFileInfo(const FileInfo & info) { WriteBufferFromOwnString out; @@ -144,8 +180,8 @@ void BackupCoordinationDistributed::createRootNodes() zookeeper->createAncestors(zookeeper_path); zookeeper->createIfNotExists(zookeeper_path, ""); zookeeper->createIfNotExists(zookeeper_path + "/repl_part_names", ""); + zookeeper->createIfNotExists(zookeeper_path + "/repl_mutations", ""); zookeeper->createIfNotExists(zookeeper_path + "/repl_data_paths", ""); - zookeeper->createIfNotExists(zookeeper_path + "/repl_access_host", ""); zookeeper->createIfNotExists(zookeeper_path + "/repl_access_paths", ""); zookeeper->createIfNotExists(zookeeper_path + "/file_names", ""); zookeeper->createIfNotExists(zookeeper_path + "/file_infos", ""); @@ -183,8 +219,8 @@ void BackupCoordinationDistributed::addReplicatedPartNames( { { std::lock_guard lock{mutex}; - if (replicated_part_names) - throw Exception(ErrorCodes::LOGICAL_ERROR, "addPartNames() must not be called after getPartNames()"); + if (replicated_parts_and_mutations) + throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedPartNames() must not be called after preparing"); } auto zookeeper = get_zookeeper(); @@ -197,8 +233,74 @@ void BackupCoordinationDistributed::addReplicatedPartNames( Strings BackupCoordinationDistributed::getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const { std::lock_guard lock{mutex}; - prepareReplicatedPartNames(); - return replicated_part_names->getPartNames(table_shared_id, replica_name); + prepareReplicatedPartsAndMutations(); + return replicated_parts_and_mutations->getPartNames(table_shared_id, replica_name); +} + +void BackupCoordinationDistributed::addReplicatedMutations( + const String & table_shared_id, + const String & table_name_for_logs, + const String & replica_name, + const std::vector & mutations) +{ + { + std::lock_guard lock{mutex}; + if (replicated_parts_and_mutations) + throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedMutations() must not be called after preparing"); + } + + auto zookeeper = get_zookeeper(); + String path = zookeeper_path + "/repl_mutations/" + escapeForFileName(table_shared_id); + zookeeper->createIfNotExists(path, ""); + path += "/" + escapeForFileName(replica_name); + zookeeper->create(path, ReplicatedMutations::serialize(mutations, table_name_for_logs), zkutil::CreateMode::Persistent); +} + +std::vector BackupCoordinationDistributed::getReplicatedMutations(const String & table_shared_id, const String & replica_name) const +{ + std::lock_guard lock{mutex}; + prepareReplicatedPartsAndMutations(); + return replicated_parts_and_mutations->getMutations(table_shared_id, replica_name); +} + + +void BackupCoordinationDistributed::prepareReplicatedPartsAndMutations() const +{ + if (replicated_parts_and_mutations) + return; + + replicated_parts_and_mutations.emplace(); + auto zookeeper = get_zookeeper(); + + { + String path = zookeeper_path + "/repl_part_names"; + for (const String & escaped_table_zk_path : zookeeper->getChildren(path)) + { + String table_zk_path = unescapeForFileName(escaped_table_zk_path); + String path2 = path + "/" + escaped_table_zk_path; + for (const String & escaped_replica_name : zookeeper->getChildren(path2)) + { + String replica_name = unescapeForFileName(escaped_replica_name); + auto part_names = ReplicatedPartNames::deserialize(zookeeper->get(path2 + "/" + escaped_replica_name)); + replicated_parts_and_mutations->addPartNames(table_zk_path, part_names.table_name_for_logs, replica_name, part_names.part_names_and_checksums); + } + } + } + + { + String path = zookeeper_path + "/repl_mutations"; + for (const String & escaped_table_zk_path : zookeeper->getChildren(path)) + { + String table_zk_path = unescapeForFileName(escaped_table_zk_path); + String path2 = path + "/" + escaped_table_zk_path; + for (const String & escaped_replica_name : zookeeper->getChildren(path2)) + { + String replica_name = unescapeForFileName(escaped_replica_name); + auto mutations = ReplicatedMutations::deserialize(zookeeper->get(path2 + "/" + escaped_replica_name)); + replicated_parts_and_mutations->addMutations(table_zk_path, mutations.table_name_for_logs, replica_name, mutations.mutations); + } + } + } } @@ -225,53 +327,14 @@ Strings BackupCoordinationDistributed::getReplicatedDataPaths(const String & tab } -void BackupCoordinationDistributed::prepareReplicatedPartNames() const -{ - if (replicated_part_names) - return; - - replicated_part_names.emplace(); - auto zookeeper = get_zookeeper(); - String path = zookeeper_path + "/repl_part_names"; - for (const String & escaped_table_zk_path : zookeeper->getChildren(path)) - { - String table_zk_path = unescapeForFileName(escaped_table_zk_path); - String path2 = path + "/" + escaped_table_zk_path; - for (const String & escaped_replica_name : zookeeper->getChildren(path2)) - { - String replica_name = unescapeForFileName(escaped_replica_name); - auto part_names = ReplicatedPartNames::deserialize(zookeeper->get(path2 + "/" + escaped_replica_name)); - replicated_part_names->addPartNames(table_zk_path, part_names.table_name_for_logs, replica_name, part_names.part_names_and_checksums); - } - } -} - - -void BackupCoordinationDistributed::addReplicatedAccessPath(const String & access_zk_path, const String & file_path) +void BackupCoordinationDistributed::addReplicatedAccessPath(const String & access_zk_path, const String & host_id, const String & file_path) { auto zookeeper = get_zookeeper(); String path = zookeeper_path + "/repl_access_paths/" + escapeForFileName(access_zk_path); zookeeper->createIfNotExists(path, ""); - path += "/" + escapeForFileName(file_path); - zookeeper->createIfNotExists(path, ""); -} + zookeeper->createIfNotExists(path + "/" + escapeForFileName(file_path), ""); -Strings BackupCoordinationDistributed::getReplicatedAccessPaths(const String & access_zk_path) const -{ - auto zookeeper = get_zookeeper(); - String path = zookeeper_path + "/repl_access_paths/" + escapeForFileName(access_zk_path); - Strings children = zookeeper->getChildren(path); - Strings file_paths; - file_paths.reserve(children.size()); - for (const String & child : children) - file_paths.push_back(unescapeForFileName(child)); - return file_paths; -} - -void BackupCoordinationDistributed::setReplicatedAccessHost(const String & access_zk_path, const String & host_id) -{ - auto zookeeper = get_zookeeper(); - String path = zookeeper_path + "/repl_access_host/" + escapeForFileName(access_zk_path); + path += "/host"; auto code = zookeeper->tryCreate(path, host_id, zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) throw zkutil::KeeperException(code, path); @@ -280,11 +343,24 @@ void BackupCoordinationDistributed::setReplicatedAccessHost(const String & acces zookeeper->set(path, host_id); } -String BackupCoordinationDistributed::getReplicatedAccessHost(const String & access_zk_path) const +Strings BackupCoordinationDistributed::getReplicatedAccessPaths(const String & access_zk_path, const String & host_id) const { auto zookeeper = get_zookeeper(); - String path = zookeeper_path + "/repl_access_host/" + escapeForFileName(access_zk_path); - return zookeeper->get(path); + + String path = zookeeper_path + "/repl_access_paths/" + escapeForFileName(access_zk_path); + String path2 = path + "/host"; + if (zookeeper->get(path2) != host_id) + return {}; + + Strings children = zookeeper->getChildren(path); + Strings file_paths; + file_paths.reserve(children.size()); + for (const String & child : children) + { + if (child != "host") + file_paths.push_back(unescapeForFileName(child)); + } + return file_paths; } diff --git a/src/Backups/BackupCoordinationDistributed.h b/src/Backups/BackupCoordinationDistributed.h index 813132bd0b8..c991d273be0 100644 --- a/src/Backups/BackupCoordinationDistributed.h +++ b/src/Backups/BackupCoordinationDistributed.h @@ -26,14 +26,19 @@ public: Strings getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const override; + void addReplicatedMutations( + const String & table_shared_id, + const String & table_name_for_logs, + const String & replica_name, + const std::vector & mutations) override; + + std::vector getReplicatedMutations(const String & table_shared_id, const String & replica_name) const override; + void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; Strings getReplicatedDataPaths(const String & table_shared_id) const override; - void addReplicatedAccessPath(const String & access_zk_path, const String & file_path) override; - Strings getReplicatedAccessPaths(const String & access_zk_path) const override; - - void setReplicatedAccessHost(const String & access_zk_path, const String & host_id) override; - String getReplicatedAccessHost(const String & access_zk_path) const override; + void addReplicatedAccessPath(const String & access_zk_path, const String & host_id, const String & file_path) override; + Strings getReplicatedAccessPaths(const String & access_zk_path, const String & host_id) const override; void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override; void updateFileInfo(const FileInfo & file_info) override; @@ -53,7 +58,7 @@ public: private: void createRootNodes(); void removeAllNodes(); - void prepareReplicatedPartNames() const; + void prepareReplicatedPartsAndMutations() const; const String zookeeper_path; const zkutil::GetZooKeeper get_zookeeper; @@ -61,7 +66,7 @@ private: BackupCoordinationStatusSync status_sync; mutable std::mutex mutex; - mutable std::optional replicated_part_names; + mutable std::optional replicated_parts_and_mutations; }; } diff --git a/src/Backups/BackupCoordinationHelpers.cpp b/src/Backups/BackupCoordinationHelpers.cpp index 7f570ba9c85..715031c2a37 100644 --- a/src/Backups/BackupCoordinationHelpers.cpp +++ b/src/Backups/BackupCoordinationHelpers.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -26,18 +27,15 @@ namespace }; } +using MutationInfo = IBackupCoordination::MutationInfo; -class BackupCoordinationReplicatedPartNames::CoveredPartsFinder + +class BackupCoordinationReplicatedPartsAndMutations::CoveredPartsFinder { public: explicit CoveredPartsFinder(const String & table_name_for_logs_) : table_name_for_logs(table_name_for_logs_) {} - void addPartName(const String & new_part_name, const std::shared_ptr & replica_name) - { - addPartName(MergeTreePartInfo::fromPartName(new_part_name, MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING), replica_name); - } - - void addPartName(MergeTreePartInfo && new_part_info, const std::shared_ptr & replica_name) + void addPartInfo(MergeTreePartInfo && new_part_info, const std::shared_ptr & replica_name) { auto new_min_block = new_part_info.min_block; auto new_max_block = new_part_info.max_block; @@ -83,8 +81,7 @@ public: { throw Exception( ErrorCodes::CANNOT_BACKUP_TABLE, - "Intersected parts detected in the table {}: {} on replica {} and {} on replica {}. It should be investigated", - table_name_for_logs, + "Intersected parts detected: {} on replica {} and {} on replica {}", part.info.getPartName(), *part.replica_name, new_part_info.getPartName(), @@ -153,19 +150,21 @@ private: }; -BackupCoordinationReplicatedPartNames::BackupCoordinationReplicatedPartNames() = default; -BackupCoordinationReplicatedPartNames::~BackupCoordinationReplicatedPartNames() = default; +BackupCoordinationReplicatedPartsAndMutations::BackupCoordinationReplicatedPartsAndMutations() = default; +BackupCoordinationReplicatedPartsAndMutations::~BackupCoordinationReplicatedPartsAndMutations() = default; -void BackupCoordinationReplicatedPartNames::addPartNames( +void BackupCoordinationReplicatedPartsAndMutations::addPartNames( const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) { - if (part_names_prepared) - throw Exception(ErrorCodes::LOGICAL_ERROR, "addPartNames() must not be called after getPartNames()"); + if (prepared) + throw Exception(ErrorCodes::LOGICAL_ERROR, "addPartNames() must not be called after preparing"); auto & table_info = table_infos[table_shared_id]; + table_info.table_name_for_logs = table_name_for_logs; + if (!table_info.covered_parts_finder) table_info.covered_parts_finder = std::make_unique(table_name_for_logs); @@ -175,10 +174,10 @@ void BackupCoordinationReplicatedPartNames::addPartNames( { const auto & part_name = part_name_and_checksum.part_name; const auto & checksum = part_name_and_checksum.checksum; - auto it = table_info.parts_replicas.find(part_name); - if (it == table_info.parts_replicas.end()) + auto it = table_info.replicas_by_part_name.find(part_name); + if (it == table_info.replicas_by_part_name.end()) { - it = table_info.parts_replicas.emplace(part_name, PartReplicas{}).first; + it = table_info.replicas_by_part_name.emplace(part_name, PartReplicas{}).first; it->second.checksum = checksum; } else @@ -202,43 +201,122 @@ void BackupCoordinationReplicatedPartNames::addPartNames( /// `replica_names` should be ordered because we need this vector to be in the same order on every replica. replica_names.insert( std::upper_bound(replica_names.begin(), replica_names.end(), replica_name_ptr, LessReplicaName{}), replica_name_ptr); - - table_info.covered_parts_finder->addPartName(part_name, replica_name_ptr); } } -Strings BackupCoordinationReplicatedPartNames::getPartNames(const String & table_shared_id, const String & replica_name) const +Strings BackupCoordinationReplicatedPartsAndMutations::getPartNames(const String & table_shared_id, const String & replica_name) const { - preparePartNames(); + prepare(); + auto it = table_infos.find(table_shared_id); if (it == table_infos.end()) return {}; - const auto & replicas_parts = it->second.replicas_parts; - auto it2 = replicas_parts.find(replica_name); - if (it2 == replicas_parts.end()) + + const auto & part_names_by_replica_name = it->second.part_names_by_replica_name; + auto it2 = part_names_by_replica_name.find(replica_name); + if (it2 == part_names_by_replica_name.end()) return {}; + return it2->second; } -void BackupCoordinationReplicatedPartNames::preparePartNames() const +void BackupCoordinationReplicatedPartsAndMutations::addMutations( + const String & table_shared_id, + const String & table_name_for_logs, + const String & replica_name, + const std::vector & mutations) { - if (part_names_prepared) + if (prepared) + throw Exception(ErrorCodes::LOGICAL_ERROR, "addMutations() must not be called after preparing"); + + auto & table_info = table_infos[table_shared_id]; + table_info.table_name_for_logs = table_name_for_logs; + for (const auto & [mutation_id, mutation_entry] : mutations) + table_info.mutations.emplace(mutation_id, mutation_entry); + + /// std::max() because the calculation must give the same result being repeated on a different replica. + table_info.replica_name_to_store_mutations = std::max(table_info.replica_name_to_store_mutations, replica_name); +} + +std::vector +BackupCoordinationReplicatedPartsAndMutations::getMutations(const String & table_shared_id, const String & replica_name) const +{ + prepare(); + + auto it = table_infos.find(table_shared_id); + if (it == table_infos.end()) + return {}; + + const auto & table_info = it->second; + if (table_info.replica_name_to_store_mutations != replica_name) + return {}; + + std::vector res; + for (const auto & [mutation_id, mutation_entry] : table_info.mutations) + res.emplace_back(MutationInfo{mutation_id, mutation_entry}); + return res; +} + +void BackupCoordinationReplicatedPartsAndMutations::prepare() const +{ + if (prepared) return; size_t counter = 0; for (const auto & table_info : table_infos | boost::adaptors::map_values) { - for (const auto & [part_name, part_replicas] : table_info.parts_replicas) + try { - if (table_info.covered_parts_finder->isCoveredByAnotherPart(part_name)) - continue; - size_t chosen_index = (counter++) % part_replicas.replica_names.size(); - const auto & chosen_replica_name = *part_replicas.replica_names[chosen_index]; - table_info.replicas_parts[chosen_replica_name].push_back(part_name); + /// Remove parts covered by other parts. + for (const auto & [part_name, part_replicas] : table_info.replicas_by_part_name) + { + auto part_info = MergeTreePartInfo::fromPartName(part_name, MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING); + + auto & min_data_versions_by_partition = table_info.min_data_versions_by_partition; + auto it2 = min_data_versions_by_partition.find(part_info.partition_id); + if (it2 == min_data_versions_by_partition.end()) + min_data_versions_by_partition[part_info.partition_id] = part_info.getDataVersion(); + else + it2->second = std::min(it2->second, part_info.getDataVersion()); + + table_info.covered_parts_finder->addPartInfo(std::move(part_info), part_replicas.replica_names[0]); + } + + for (const auto & [part_name, part_replicas] : table_info.replicas_by_part_name) + { + if (table_info.covered_parts_finder->isCoveredByAnotherPart(part_name)) + continue; + size_t chosen_index = (counter++) % part_replicas.replica_names.size(); + const auto & chosen_replica_name = *part_replicas.replica_names[chosen_index]; + table_info.part_names_by_replica_name[chosen_replica_name].push_back(part_name); + } + + /// Remove finished or unrelated mutations. + std::unordered_map unfinished_mutations; + for (const auto & [mutation_id, mutation_entry_str] : table_info.mutations) + { + auto mutation_entry = ReplicatedMergeTreeMutationEntry::parse(mutation_entry_str, mutation_id); + std::map new_block_numbers; + for (const auto & [partition_id, block_number] : mutation_entry.block_numbers) + { + auto it = table_info.min_data_versions_by_partition.find(partition_id); + if ((it != table_info.min_data_versions_by_partition.end()) && (it->second < block_number)) + new_block_numbers[partition_id] = block_number; + } + mutation_entry.block_numbers = std::move(new_block_numbers); + if (!mutation_entry.block_numbers.empty()) + unfinished_mutations[mutation_id] = mutation_entry.toString(); + } + table_info.mutations = unfinished_mutations; + } + catch (Exception & e) + { + e.addMessage("While checking data of table {}", table_info.table_name_for_logs); + throw; } } - part_names_prepared = true; + prepared = true; } diff --git a/src/Backups/BackupCoordinationHelpers.h b/src/Backups/BackupCoordinationHelpers.h index 2e9e4b3cbde..ad6168c9e8c 100644 --- a/src/Backups/BackupCoordinationHelpers.h +++ b/src/Backups/BackupCoordinationHelpers.h @@ -11,11 +11,11 @@ namespace DB { /// Helper designed to be used in an implementation of the IBackupCoordination interface in the part related to replicated tables. -class BackupCoordinationReplicatedPartNames +class BackupCoordinationReplicatedPartsAndMutations { public: - BackupCoordinationReplicatedPartNames(); - ~BackupCoordinationReplicatedPartNames(); + BackupCoordinationReplicatedPartsAndMutations(); + ~BackupCoordinationReplicatedPartsAndMutations(); using PartNameAndChecksum = IBackupCoordination::PartNameAndChecksum; @@ -29,13 +29,25 @@ public: const String & replica_name, const std::vector & part_names_and_checksums); + using MutationInfo = IBackupCoordination::MutationInfo; + + /// Adds information about mutations of a replicated table. + void addMutations( + const String & table_shared_id, + const String & table_name_for_logs, + const String & replica_name, + const std::vector & mutations); + /// Returns the names of the parts which a specified replica of a replicated table should put to the backup. /// This is the same list as it was added by call of the function addPartNames() but without duplications and without /// parts covered by another parts. Strings getPartNames(const String & table_shared_id, const String & replica_name) const; + /// Returns all mutations of a replicated table which are not finished for some data parts added by addReplicatedPartNames(). + std::vector getMutations(const String & table_shared_id, const String & replica_name) const; + private: - void preparePartNames() const; + void prepare() const; class CoveredPartsFinder; @@ -47,13 +59,17 @@ private: struct TableInfo { - std::map parts_replicas; /// Should be ordered because we need this map to be in the same order on every replica. - mutable std::unordered_map */, Strings> replicas_parts; + String table_name_for_logs; + std::map replicas_by_part_name; /// Should be ordered because we need this map to be in the same order on every replica. + mutable std::unordered_map */, Strings> part_names_by_replica_name; std::unique_ptr covered_parts_finder; + mutable std::unordered_map min_data_versions_by_partition; + mutable std::unordered_map mutations; + String replica_name_to_store_mutations; }; std::map table_infos; /// Should be ordered because we need this map to be in the same order on every replica. - mutable bool part_names_prepared = false; + mutable bool prepared = false; }; diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index a7d5602ca30..fadd15ea935 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -30,13 +30,26 @@ Strings BackupCoordinationLocal::setStatusAndWaitFor(const String &, const Strin void BackupCoordinationLocal::addReplicatedPartNames(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) { std::lock_guard lock{mutex}; - replicated_part_names.addPartNames(table_shared_id, table_name_for_logs, replica_name, part_names_and_checksums); + replicated_parts_and_mutations.addPartNames(table_shared_id, table_name_for_logs, replica_name, part_names_and_checksums); } Strings BackupCoordinationLocal::getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const { std::lock_guard lock{mutex}; - return replicated_part_names.getPartNames(table_shared_id, replica_name); + return replicated_parts_and_mutations.getPartNames(table_shared_id, replica_name); +} + + +void BackupCoordinationLocal::addReplicatedMutations(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) +{ + std::lock_guard lock{mutex}; + replicated_parts_and_mutations.addMutations(table_shared_id, table_name_for_logs, replica_name, mutations); +} + +std::vector BackupCoordinationLocal::getReplicatedMutations(const String & table_shared_id, const String & replica_name) const +{ + std::lock_guard lock{mutex}; + return replicated_parts_and_mutations.getMutations(table_shared_id, replica_name); } @@ -56,34 +69,26 @@ Strings BackupCoordinationLocal::getReplicatedDataPaths(const String & table_sha } -void BackupCoordinationLocal::addReplicatedAccessPath(const String & access_zk_path, const String & file_path) +void BackupCoordinationLocal::addReplicatedAccessPath(const String & access_zk_path, const String & host_id, const String & file_path) { std::lock_guard lock{mutex}; - replicated_access_paths[access_zk_path].push_back(file_path); + auto & ref = replicated_access_paths[access_zk_path]; + ref.file_paths.push_back(file_path); + ref.host_to_store_access = host_id; } -Strings BackupCoordinationLocal::getReplicatedAccessPaths(const String & access_zk_path) const +Strings BackupCoordinationLocal::getReplicatedAccessPaths(const String & access_zk_path, const String & host_id) const { std::lock_guard lock{mutex}; auto it = replicated_access_paths.find(access_zk_path); if (it == replicated_access_paths.end()) return {}; - return it->second; -} -void BackupCoordinationLocal::setReplicatedAccessHost(const String & access_zk_path, const String & host_id) -{ - std::lock_guard lock{mutex}; - replicated_access_hosts[access_zk_path] = host_id; -} - -String BackupCoordinationLocal::getReplicatedAccessHost(const String & access_zk_path) const -{ - std::lock_guard lock{mutex}; - auto it = replicated_access_hosts.find(access_zk_path); - if (it == replicated_access_hosts.end()) + const auto & ref = it->second; + if (ref.host_to_store_access != host_id) return {}; - return it->second; + + return ref.file_paths; } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index dcd6505a438..38e3fc846de 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -27,14 +27,15 @@ public: const std::vector & part_names_and_checksums) override; Strings getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const override; + void addReplicatedMutations(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, + const std::vector & mutations) override; + std::vector getReplicatedMutations(const String & table_shared_id, const String & replica_name) const override; + void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; Strings getReplicatedDataPaths(const String & table_shared_id) const override; - void addReplicatedAccessPath(const String & access_zk_path, const String & file_path) override; - Strings getReplicatedAccessPaths(const String & access_zk_path) const override; - - void setReplicatedAccessHost(const String & access_zk_path, const String & host_id) override; - String getReplicatedAccessHost(const String & access_zk_path) const override; + void addReplicatedAccessPath(const String & access_zk_path, const String & host_id, const String & file_path) override; + Strings getReplicatedAccessPaths(const String & access_zk_path, const String & host_id) const override; void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override; void updateFileInfo(const FileInfo & file_info) override; @@ -51,11 +52,16 @@ public: Strings getAllArchiveSuffixes() const override; private: + struct ReplicatedAccessPath + { + Strings file_paths; + String host_to_store_access; + }; + mutable std::mutex mutex; - BackupCoordinationReplicatedPartNames replicated_part_names TSA_GUARDED_BY(mutex); + BackupCoordinationReplicatedPartsAndMutations replicated_parts_and_mutations TSA_GUARDED_BY(mutex); std::unordered_map replicated_data_paths TSA_GUARDED_BY(mutex); - std::unordered_map replicated_access_paths TSA_GUARDED_BY(mutex); - std::unordered_map replicated_access_hosts TSA_GUARDED_BY(mutex); + std::unordered_map replicated_access_paths TSA_GUARDED_BY(mutex); std::map file_names TSA_GUARDED_BY(mutex); /// Should be ordered alphabetically, see listFiles(). For empty files we assume checksum = 0. std::map file_infos TSA_GUARDED_BY(mutex); /// Information about files. Without empty files. Strings archive_suffixes TSA_GUARDED_BY(mutex); diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index b4c5c7b3d88..d899385d042 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -6,6 +6,8 @@ namespace DB { +struct ReplicatedMergeTreeMutationEntry; + /// Keeps information about files contained in a backup. class IBackupCoordination @@ -36,6 +38,18 @@ public: /// parts covered by another parts. virtual Strings getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const = 0; + struct MutationInfo + { + String id; + String entry; + }; + + /// Adds information about mutations of a replicated table. + virtual void addReplicatedMutations(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) = 0; + + /// Returns all mutations of a replicated table which are not finished for some data parts added by addReplicatedPartNames(). + virtual std::vector getReplicatedMutations(const String & table_shared_id, const String & replica_name) const = 0; + /// Adds a data path in backup for a replicated table. /// Multiple replicas of the replicated table call this function and then all the added paths can be returned by call of the function /// getReplicatedDataPaths(). @@ -45,13 +59,9 @@ public: virtual Strings getReplicatedDataPaths(const String & table_shared_id) const = 0; /// Adds a path to access.txt file keeping access entities of a ReplicatedAccessStorage. - virtual void addReplicatedAccessPath(const String & access_zk_path, const String & file_path) = 0; - virtual Strings getReplicatedAccessPaths(const String & access_zk_path) const = 0; - - /// Sets the host id of a host storing access entities of a ReplicatedAccessStorage to backup. - virtual void setReplicatedAccessHost(const String & access_zk_path, const String & host) = 0; - virtual String getReplicatedAccessHost(const String & access_zk_path) const = 0; - + virtual void addReplicatedAccessPath(const String & access_zk_path, const String & host_id, const String & file_path) = 0; + virtual Strings getReplicatedAccessPaths(const String & access_zk_path, const String & host_id) const = 0; + struct FileInfo { String file_name; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2ed68cef972..29146ef9833 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -72,6 +72,7 @@ #include #include +#include #include #include @@ -3981,17 +3982,19 @@ Pipe MergeTreeData::alterPartition( void MergeTreeData::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) { - backup_entries_collector.addBackupEntries(backupParts(backup_entries_collector.getContext(), data_path_in_backup, partitions)); -} + auto local_context = backup_entries_collector.getContext(); -BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const String & data_path_in_backup, const std::optional & partitions) const -{ DataPartsVector data_parts; if (partitions) data_parts = getVisibleDataPartsVectorInPartitions(local_context, getPartitionIDsFromQuery(*partitions, local_context)); else data_parts = getVisibleDataPartsVector(local_context); + backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup)); +} + +BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup) +{ BackupEntries backup_entries; std::map> temp_dirs; @@ -4018,6 +4021,9 @@ BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const void MergeTreeData::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) { auto backup = restorer.getBackup(); + if (!backup->hasFiles(data_path_in_backup)) + return; + if (!restorer.isNonEmptyTableAllowed() && getTotalActiveSizeInBytes() && backup->hasFiles(data_path_in_backup)) restorer.throwTableIsNotEmpty(getStorageID()); @@ -4091,6 +4097,8 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const auto backup = restorer.getBackup(); Strings part_names = backup->listFiles(data_path_in_backup); + boost::remove_erase(part_names, "mutations"); + auto restored_parts_holder = std::make_shared(std::static_pointer_cast(shared_from_this()), backup, part_names.size()); @@ -4102,7 +4110,7 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version); if (!part_info) { - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File name {} is not the name of a part", + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File name {} is not a part's name", String{data_path_in_backup_fs / part_name}); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 9aa14367f80..91d8b87151e 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1226,7 +1226,7 @@ protected: bool movePartsToSpace(const DataPartsVector & parts, SpacePtr space); /// Makes backup entries to backup the parts of this table. - BackupEntries backupParts(const ContextPtr & local_context, const String & data_path_in_backup, const std::optional & partitions) const; + static BackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup); class RestoredPartsHolder; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index bfa9129bd53..cad749ced21 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -167,4 +168,16 @@ MergeTreeMutationEntry::~MergeTreeMutationEntry() } } +std::shared_ptr MergeTreeMutationEntry::backup() const +{ + WriteBufferFromOwnString out; + out << "block number: " << block_number << "\n"; + + out << "commands: "; + commands.writeText(out); + out << "\n"; + + return std::make_shared(out.str()); +} + } diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 3d4c4d0c4a1..04297f2852a 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -9,6 +9,7 @@ namespace DB { +class IBackupEntry; /// A mutation entry for non-replicated MergeTree storage engines. /// Stores information about mutation in file mutation_*.txt. @@ -48,6 +49,8 @@ struct MergeTreeMutationEntry void writeCSN(CSN csn_); + std::shared_ptr backup() const; + static String versionToFileName(UInt64 block_number_); static UInt64 tryParseFileName(const String & file_name_); static UInt64 parseFileName(const String & file_name_); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp index c617befe9c4..080066c1dff 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -78,4 +79,24 @@ ReplicatedMergeTreeMutationEntry ReplicatedMergeTreeMutationEntry::parse(const S return res; } + +std::shared_ptr ReplicatedMergeTreeMutationEntry::backup() const +{ + WriteBufferFromOwnString out; + out << "block numbers count: " << block_numbers.size() << "\n"; + + for (const auto & kv : block_numbers) + { + const String & partition_id = kv.first; + Int64 number = kv.second; + out << partition_id << "\t" << number << "\n"; + } + + out << "commands: "; + commands.writeText(out); + out << "\n"; + + return std::make_shared(out.str()); +} + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h index e1c7bcb9042..8c359a57279 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h @@ -12,6 +12,7 @@ namespace DB class ReadBuffer; class WriteBuffer; +class IBackupEntry; /// Mutation entry in /mutations path in zookeeper. This record contains information about blocks /// in patitions. We will mutatate all parts with left number less than this numbers. @@ -48,6 +49,8 @@ struct ReplicatedMergeTreeMutationEntry int alter_version = -1; bool isAlterMutation() const { return alter_version != -1; } + + std::shared_ptr backup() const; }; using ReplicatedMergeTreeMutationEntryPtr = std::shared_ptr; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 5fe7214194a..2fe9beca9cf 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -1795,6 +1796,35 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ } +void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) +{ + auto local_context = backup_entries_collector.getContext(); + + DataPartsVector data_parts; + if (partitions) + data_parts = getVisibleDataPartsVectorInPartitions(local_context, getPartitionIDsFromQuery(*partitions, local_context)); + else + data_parts = getVisibleDataPartsVector(local_context); + + Int64 min_data_version = std::numeric_limits::max(); + for (const auto & data_part : data_parts) + min_data_version = std::min(min_data_version, data_part->info.getDataVersion()); + + backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup)); + backup_entries_collector.addBackupEntries(backupMutations(min_data_version + 1, data_path_in_backup)); +} + + +BackupEntries StorageMergeTree::backupMutations(UInt64 version, const String & data_path_in_backup) const +{ + fs::path mutations_path_in_backup = fs::path{data_path_in_backup} / "mutations"; + BackupEntries backup_entries; + for (auto it = current_mutations_by_version.lower_bound(version); it != current_mutations_by_version.end(); ++it) + backup_entries.emplace_back(mutations_path_in_backup / fmt::format("{:010}.txt", it->first), it->second.backup()); + return backup_entries; +} + + void StorageMergeTree::attachRestoredParts(MutableDataPartsVector && parts) { for (auto part : parts) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 582962551d0..fb9478c5eca 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -92,6 +92,9 @@ public: CancellationCode killMutation(const String & mutation_id) override; + /// Makes backup entries to backup the data of the storage. + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void drop() override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; @@ -255,6 +258,8 @@ private: void startBackgroundMovesIfNeeded() override; + BackupEntries backupMutations(UInt64 version, const String & data_path_in_backup) const; + /// Attaches restored parts to the storage. void attachRestoredParts(MutableDataPartsVector && parts) override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2ee1e233688..563ea31a7b8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8188,7 +8188,15 @@ void StorageReplicatedMergeTree::backupData( /// First we generate backup entries in the same way as an ordinary MergeTree does. /// But then we don't add them to the BackupEntriesCollector right away, /// because we need to coordinate them with other replicas (other replicas can have better parts). - auto backup_entries = backupParts(backup_entries_collector.getContext(), "", partitions); + auto local_context = backup_entries_collector.getContext(); + + DataPartsVector data_parts; + if (partitions) + data_parts = getVisibleDataPartsVectorInPartitions(local_context, getPartitionIDsFromQuery(*partitions, local_context)); + else + data_parts = getVisibleDataPartsVector(local_context); + + auto backup_entries = backupParts(data_parts, ""); auto coordination = backup_entries_collector.getBackupCoordination(); String shared_id = getTableSharedID(); @@ -8232,6 +8240,20 @@ void StorageReplicatedMergeTree::backupData( /// Send our list of part names to the coordination (to compare with other replicas). coordination->addReplicatedPartNames(shared_id, getStorageID().getFullTableName(), getReplicaName(), part_names_with_hashes); + /// Send a list of mutations to the coordination too (we need to find the mutations which are not finished for added part names). + { + std::vector mutation_infos; + auto zookeeper = getZooKeeper(); + Strings mutation_ids = zookeeper->getChildren(fs::path(zookeeper_path) / "mutations"); + mutation_infos.reserve(mutation_ids.size()); + for (const auto & mutation_id : mutation_ids) + { + mutation_infos.emplace_back( + IBackupCoordination::MutationInfo{mutation_id, zookeeper->get(fs::path(zookeeper_path) / "mutations" / mutation_id)}); + } + coordination->addReplicatedMutations(shared_id, getStorageID().getFullTableName(), getReplicaName(), mutation_infos); + } + /// This task will be executed after all replicas have collected their parts and the coordination is ready to /// give us the final list of parts to add to the BackupEntriesCollector. auto post_collecting_task = [shared_id, @@ -8258,7 +8280,16 @@ void StorageReplicatedMergeTree::backupData( for (const auto & data_path : data_paths_fs) backup_entries_collector.addBackupEntry(data_path / relative_path, backup_entry); } + + auto mutation_infos = coordination->getReplicatedMutations(shared_id, replica_name); + for (const auto & mutation_info : mutation_infos) + { + auto backup_entry = ReplicatedMergeTreeMutationEntry::parse(mutation_info.entry, mutation_info.id).backup(); + for (const auto & data_path : data_paths_fs) + backup_entries_collector.addBackupEntry(data_path / "mutations" / (mutation_info.id + ".txt"), backup_entry); + } }; + backup_entries_collector.addPostTask(post_collecting_task); } diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index a930ddac7df..3c638efe7cf 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -69,6 +69,15 @@ def new_session_id(): return "Session #" + str(session_id_counter) +def has_mutation_in_backup(mutation_id, backup_name, database, table): + return os.path.exists( + os.path.join( + get_path_to_backup(backup_name), + f"data/{database}/{table}/mutations/{mutation_id}.txt", + ) + ) + + @pytest.mark.parametrize( "engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"] ) @@ -829,3 +838,31 @@ def test_restore_partition(): assert instance.query("SELECT * FROM test.table ORDER BY x") == TSV( [[2, "2"], [3, "3"], [12, "12"], [13, "13"], [22, "22"], [23, "23"]] ) + + +def test_mutation(): + create_and_fill_table(engine="MergeTree ORDER BY tuple()", n=5) + + instance.query( + "INSERT INTO test.table SELECT number, toString(number) FROM numbers(5, 5)" + ) + + instance.query( + "INSERT INTO test.table SELECT number, toString(number) FROM numbers(10, 5)" + ) + + instance.query("ALTER TABLE test.table UPDATE x=x+1 WHERE 1") + instance.query("ALTER TABLE test.table UPDATE x=x+1+sleep(1) WHERE 1") + instance.query("ALTER TABLE test.table UPDATE x=x+1+sleep(2) WHERE 1") + + backup_name = new_backup_name() + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + assert not has_mutation_in_backup("0000000004", backup_name, "test", "table") + assert has_mutation_in_backup("0000000005", backup_name, "test", "table") + assert has_mutation_in_backup("0000000006", backup_name, "test", "table") + assert not has_mutation_in_backup("0000000007", backup_name, "test", "table") + + instance.query("DROP TABLE test.table") + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 02f855cf766..1f65f190922 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -79,7 +79,7 @@ def new_backup_name(): def get_path_to_backup(backup_name): name = backup_name.split(",")[1].strip("')/ ") - return os.path.join(instance.cluster.instances_dir, "backups", name) + return os.path.join(node1.cluster.instances_dir, "backups", name) def test_replicated_table(): @@ -652,3 +652,57 @@ def test_table_in_replicated_database_with_not_synced_def(): assert node2.query( "SELECT name, type FROM system.columns WHERE database='mydb' AND table='tbl'" ) == TSV([["x", "String"], ["y", "String"]]) + + +def has_mutation_in_backup(mutation_id, backup_name, database, table): + return ( + os.path.exists( + os.path.join( + get_path_to_backup(backup_name), + f"shards/1/replicas/1/data/{database}/{table}/mutations/{mutation_id}.txt", + ) + ) + or os.path.exists( + os.path.join( + get_path_to_backup(backup_name), + f"shards/1/replicas/2/data/{database}/{table}/mutations/{mutation_id}.txt", + ) + ) + or os.path.exists( + os.path.join( + get_path_to_backup(backup_name), + f"shards/1/replicas/3/data/{database}/{table}/mutations/{mutation_id}.txt", + ) + ) + ) + + +def test_mutation(): + node1.query( + "CREATE TABLE tbl ON CLUSTER 'cluster' (" + "x UInt8, y String" + ") ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}')" + "ORDER BY tuple()" + ) + + node1.query("INSERT INTO tbl SELECT number, toString(number) FROM numbers(5)") + + node2.query("INSERT INTO tbl SELECT number, toString(number) FROM numbers(5, 5)") + + node1.query("INSERT INTO tbl SELECT number, toString(number) FROM numbers(10, 5)") + + node1.query("ALTER TABLE tbl UPDATE x=x+1 WHERE 1") + node1.query("ALTER TABLE tbl UPDATE x=x+1+sleep(1) WHERE 1") + node1.query("ALTER TABLE tbl UPDATE x=x+1+sleep(2) WHERE 1") + + backup_name = new_backup_name() + node1.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}") + + assert not has_mutation_in_backup("0000000000", backup_name, "default", "tbl") + assert has_mutation_in_backup("0000000001", backup_name, "default", "tbl") + assert has_mutation_in_backup("0000000002", backup_name, "default", "tbl") + assert not has_mutation_in_backup("0000000003", backup_name, "default", "tbl") + + node1.query("DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") + + node1.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") From bdc21737d5c7b091842a2912ea4fbd16a18bf934 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 5 Jul 2022 16:15:00 +0200 Subject: [PATCH 241/627] MergeTree merge disable batch optimization --- src/Interpreters/SortedBlocksWriter.cpp | 12 +- .../Algorithms/MergingSortedAlgorithm.cpp | 156 ++++++++++++++++-- .../Algorithms/MergingSortedAlgorithm.h | 6 + .../Merges/MergingSortedTransform.cpp | 2 + .../Merges/MergingSortedTransform.h | 1 + .../QueryPlan/ReadFromMergeTree.cpp | 5 +- src/Processors/QueryPlan/SortingStep.cpp | 3 + .../Transforms/MergeSortingTransform.cpp | 1 + .../gtest_blocks_size_merging_streams.cpp | 4 +- src/Storages/MergeTree/MergeTask.cpp | 2 +- 10 files changed, 171 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index 3caf144d9a8..c2a6f513224 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -136,7 +136,8 @@ SortedBlocksWriter::TmpFilePtr SortedBlocksWriter::flush(const BlocksList & bloc pipeline.getHeader(), pipeline.getNumStreams(), sort_description, - rows_in_block); + rows_in_block, + SortingQueueStrategy::Default); pipeline.addTransform(std::move(transform)); } @@ -190,7 +191,8 @@ SortedBlocksWriter::PremergedFiles SortedBlocksWriter::premerge() pipeline.getHeader(), pipeline.getNumStreams(), sort_description, - rows_in_block); + rows_in_block, + SortingQueueStrategy::Default); pipeline.addTransform(std::move(transform)); } @@ -222,7 +224,8 @@ SortedBlocksWriter::SortedFiles SortedBlocksWriter::finishMerge(std::function; - queue = QueueType(cursors); - }); + queue_variants.callOnVariant([&](auto & queue) + { + using QueueType = std::decay_t; + queue = QueueType(cursors); + }); + } + else + { + queue_variants.callOnBatchVariant([&](auto & queue) + { + using QueueType = std::decay_t; + queue = QueueType(cursors); + }); + } } void MergingSortedAlgorithm::consume(Input & input, size_t source_num) @@ -82,14 +95,34 @@ void MergingSortedAlgorithm::consume(Input & input, size_t source_num) current_inputs[source_num].swap(input); cursors[source_num].reset(current_inputs[source_num].chunk.getColumns(), header); - queue_variants.callOnBatchVariant([&](auto & queue) + if (sorting_queue_strategy == SortingQueueStrategy::Default) { - queue.push(cursors[source_num]); - }); + queue_variants.callOnVariant([&](auto & queue) + { + queue.push(cursors[source_num]); + }); + } + else + { + queue_variants.callOnBatchVariant([&](auto & queue) + { + queue.push(cursors[source_num]); + }); + } } IMergingAlgorithm::Status MergingSortedAlgorithm::merge() { + if (sorting_queue_strategy == SortingQueueStrategy::Default) + { + IMergingAlgorithm::Status result = queue_variants.callOnVariant([&](auto & queue) + { + return mergeImpl(queue); + }); + + return result; + } + IMergingAlgorithm::Status result = queue_variants.callOnBatchVariant([&](auto & queue) { return mergeBatchImpl(queue); @@ -98,6 +131,100 @@ IMergingAlgorithm::Status MergingSortedAlgorithm::merge() return result; } +template +IMergingAlgorithm::Status MergingSortedAlgorithm::mergeImpl(TSortingHeap & queue) +{ + /// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size` + while (queue.isValid()) + { + if (merged_data.hasEnoughRows()) + return Status(merged_data.pull()); + + auto current = queue.current(); + + if (current.impl->isLast() && current_inputs[current.impl->order].skip_last_row) + { + /// Get the next block from the corresponding source, if there is one. + queue.removeTop(); + return Status(current.impl->order); + } + + if (current.impl->isFirst() + && !current_inputs[current.impl->order].skip_last_row /// Ignore optimization if last row should be skipped. + && (queue.size() == 1 + || (queue.size() >= 2 && current.totallyLessOrEquals(queue.nextChild())))) + { + /** This is special optimization if current cursor is totally less than next cursor. + * We want to insert current cursor chunk directly in merged data. + * + * First if merged_data is not empty we need to flush it. + * We will get into the same condition on next mergeBatch call. + * + * Then we can insert chunk directly in merged data. + */ + + if (merged_data.mergedRows() != 0) + return Status(merged_data.pull()); + + /// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl) + size_t source_num = current.impl->order; + + auto chunk_num_rows = current_inputs[source_num].chunk.getNumRows(); + + UInt64 total_merged_rows_after_insertion = merged_data.mergedRows() + chunk_num_rows; + bool limit_reached = limit && total_merged_rows_after_insertion >= limit; + + if (limit && total_merged_rows_after_insertion > limit) + chunk_num_rows -= total_merged_rows_after_insertion - limit; + + merged_data.insertChunk(std::move(current_inputs[source_num].chunk), chunk_num_rows); + current_inputs[source_num].chunk = Chunk(); + + /// Write order of rows for other columns this data will be used in gather stream + if (out_row_sources_buf) + { + RowSourcePart row_source(source_num); + for (size_t i = 0; i < chunk_num_rows; ++i) + out_row_sources_buf->write(row_source.data); + } + + /// We will get the next block from the corresponding source, if there is one. + queue.removeTop(); + + auto status = Status(merged_data.pull(), limit_reached); + + if (!limit_reached) + status.required_source = source_num; + + return status; + } + + merged_data.insertRow(current->all_columns, current->getRow(), current->rows); + + if (out_row_sources_buf) + { + RowSourcePart row_source(current.impl->order); + out_row_sources_buf->write(row_source.data); + } + + if (limit && merged_data.totalMergedRows() >= limit) + return Status(merged_data.pull(), true); + + if (!current->isLast()) + { + queue.next(); + } + else + { + /// We will get the next block from the corresponding source, if there is one. + queue.removeTop(); + return Status(current.impl->order); + } + } + + return Status(merged_data.pull(), true); +} + template IMergingAlgorithm::Status MergingSortedAlgorithm::mergeBatchImpl(TSortingQueue & queue) @@ -134,14 +261,22 @@ IMergingAlgorithm::Status MergingSortedAlgorithm::mergeBatchImpl(TSortingQueue & } bool limit_reached = false; - if (limit && merged_rows + updated_batch_size > limit) + + if (limit && merged_rows + updated_batch_size >= limit && !batch_skip_last_row) + { + updated_batch_size -= merged_rows + updated_batch_size - limit; + limit_reached = true; + } + else if (limit && merged_rows + updated_batch_size > limit) { batch_skip_last_row = false; updated_batch_size -= merged_rows + updated_batch_size - limit; limit_reached = true; } - if (unlikely(current.impl->isFirst() && current.impl->isLast(initial_batch_size))) + if (unlikely(current.impl->isFirst() && + current.impl->isLast(initial_batch_size) && + !current_inputs[current.impl->order].skip_last_row)) { /** This is special optimization if current cursor is totally less than next cursor. * We want to insert current cursor chunk directly in merged data. @@ -167,9 +302,6 @@ IMergingAlgorithm::Status MergingSortedAlgorithm::mergeBatchImpl(TSortingQueue & out_row_sources_buf->write(row_source.data); } - if (limit_reached) - break; - /// We will get the next block from the corresponding source, if there is one. queue.removeTop(); diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h index 9e517120f38..cf32e5fd4dd 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h @@ -18,6 +18,7 @@ public: size_t num_inputs, SortDescription description_, size_t max_block_size, + SortingQueueStrategy sorting_queue_strategy_, UInt64 limit_ = 0, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false); @@ -47,10 +48,15 @@ private: /// Chunks currently being merged. Inputs current_inputs; + SortingQueueStrategy sorting_queue_strategy; + SortCursorImpls cursors; SortQueueVariants queue_variants; + template + Status mergeImpl(TSortingQueue & queue); + template Status mergeBatchImpl(TSortingQueue & queue); diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index c244388464f..4cb74ffc71e 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -12,6 +12,7 @@ MergingSortedTransform::MergingSortedTransform( size_t num_inputs, SortDescription description_, size_t max_block_size, + SortingQueueStrategy sorting_queue_strategy, UInt64 limit_, WriteBuffer * out_row_sources_buf_, bool quiet_, @@ -23,6 +24,7 @@ MergingSortedTransform::MergingSortedTransform( num_inputs, std::move(description_), max_block_size, + sorting_queue_strategy, limit_, out_row_sources_buf_, use_average_block_sizes) diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index 93bd36d8aec..16e3e2791ee 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -16,6 +16,7 @@ public: size_t num_inputs, SortDescription description, size_t max_block_size, + SortingQueueStrategy sorting_queue_strategy, UInt64 limit_ = 0, WriteBuffer * out_row_sources_buf_ = nullptr, bool quiet_ = false, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index f377709a899..ad2a7485825 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -555,7 +555,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( pipe.getHeader(), pipe.numOutputPorts(), sort_description, - max_block_size); + max_block_size, + SortingQueueStrategy::Batch); pipe.addTransform(std::move(transform)); } @@ -583,7 +584,7 @@ static void addMergingFinal( { case MergeTreeData::MergingParams::Ordinary: return std::make_shared(header, num_outputs, - sort_description, max_block_size); + sort_description, max_block_size, SortingQueueStrategy::Batch); case MergeTreeData::MergingParams::Collapsing: return std::make_shared(header, num_outputs, diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 89a5ecb4cfa..46588ada225 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -124,6 +124,7 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build pipeline.getNumStreams(), prefix_description, max_block_size, + SortingQueueStrategy::Batch, limit_for_merging); pipeline.addTransform(std::move(transform)); @@ -212,6 +213,7 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build pipeline.getNumStreams(), result_description, max_block_size, + SortingQueueStrategy::Batch, limit); pipeline.addTransform(std::move(transform)); @@ -226,6 +228,7 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build pipeline.getNumStreams(), result_description, max_block_size, + SortingQueueStrategy::Batch, limit); pipeline.addTransform(std::move(transform)); diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index 0c4615e9273..7c0422584c9 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -203,6 +203,7 @@ void MergeSortingTransform::consume(Chunk chunk) 0, description, max_merged_block_size, + SortingQueueStrategy::Batch, limit, nullptr, quiet, diff --git a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp index f9eca5f1ee0..2fa5873544f 100644 --- a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp @@ -83,7 +83,7 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) EXPECT_EQ(pipe.numOutputPorts(), 3); auto transform = std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), sort_description, - DEFAULT_MERGE_BLOCK_SIZE, false, nullptr, false, true); + DEFAULT_MERGE_BLOCK_SIZE, SortingQueueStrategy::Batch, 0, nullptr, false, true); pipe.addTransform(std::move(transform)); @@ -125,7 +125,7 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) EXPECT_EQ(pipe.numOutputPorts(), 3); auto transform = std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), sort_description, - DEFAULT_MERGE_BLOCK_SIZE, false, nullptr, false, true); + DEFAULT_MERGE_BLOCK_SIZE, SortingQueueStrategy::Batch, 0, nullptr, false, true); pipe.addTransform(std::move(transform)); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index f16d22f553a..7426b384394 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -855,7 +855,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() { case MergeTreeData::MergingParams::Ordinary: merged_transform = std::make_shared( - header, pipes.size(), sort_description, merge_block_size, 0, ctx->rows_sources_write_buf.get(), true, ctx->blocks_are_granules_size); + header, pipes.size(), sort_description, merge_block_size, SortingQueueStrategy::Default, 0, ctx->rows_sources_write_buf.get(), true, ctx->blocks_are_granules_size); break; case MergeTreeData::MergingParams::Collapsing: From 52e2fa158a382670f69a3b3206e33fc1336b8f8d Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 5 Jul 2022 10:20:55 -0400 Subject: [PATCH 242/627] small edits --- docker/server/README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/server/README.md b/docker/server/README.md index f331b802c37..66e82d0bba1 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -1,8 +1,8 @@ -# ClickHouse Server Docker Image +# ClickHouse Server Docker Imagex ## What is ClickHouse? -ClickHouse is an open-source column-oriented database management system that allows generating of analytical data reports in real-time. +ClickHouse is an open-source column-oriented database management system that allows the generation of analytical data reports in real-time. ClickHouse manages extremely large volumes of data stably and sustainably. It currently powers [Yandex.Metrica](https://metrica.yandex.com/), the world’s [second-largest](http://w3techs.com/technologies/overview/traffic_analysis/all) web analytics platform, with over 13 trillion database records and over 20 billion events a day, generating customized reports on-the-fly, directly from non-aggregated data. This system was successfully implemented at [CERN’s LHCb experiment](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) to store and process metadata on 10bn events with over 1000 attributes per event registered in 2011. @@ -107,14 +107,14 @@ ClickHouse configuration is represented with a file "config.xml" ([documentation docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 -v /path/to/your/config.xml:/etc/clickhouse-server/config.xml clickhouse/clickhouse-server ``` -### Start server as custom user +### Start server as a custom user ```bash # $(pwd)/data/clickhouse should exist and be owned by current user docker run --rm --user ${UID}:${GID} --name some-clickhouse-server --ulimit nofile=262144:262144 -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server ``` -When you use the image with mounting local directories inside you probably would like to not mess your directory tree with files owner and permissions. Then you could use `--user` argument. In this case, you should mount every necessary directory (`/var/lib/clickhouse` and `/var/log/clickhouse-server`) inside the container. Otherwise, the image will complain and not start. +When you use the image with local directories mounted, you probably would like to specify the user to maintain the proper file ownership. Use the `--user` argument and mount `/var/lib/clickhouse` and `/var/log/clickhouse-server` inside the container. Otherwise, the image will complain and not start. ### Start server from root (useful in case of userns enabled) From 09ff006ddb75d34ab99c64ff22ad5a9a64017a59 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 5 Jul 2022 11:27:58 -0400 Subject: [PATCH 243/627] small edits --- docker/server/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/README.md b/docker/server/README.md index 66e82d0bba1..17eaa1a4d8b 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -1,4 +1,4 @@ -# ClickHouse Server Docker Imagex +# ClickHouse Server Docker Image ## What is ClickHouse? From 2f2fe9cffbff4f9676590630611af31c6fb1577e Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 5 Jul 2022 11:39:38 -0400 Subject: [PATCH 244/627] small edits --- docker/server/README.md | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docker/server/README.md b/docker/server/README.md index 17eaa1a4d8b..3639a432b1e 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -65,7 +65,7 @@ echo 'SELECT version()' | curl 'http://localhost:8123/' --data-binary @- ### Volumes -Typically you may want to mount the following folders inside your container to archieve persistency: +Typically you may want to mount the following folders inside your container to achieve persistency: * `/var/lib/clickhouse/` - main folder where ClickHouse stores the data * `/var/log/clickhouse-server/` - logs @@ -87,7 +87,7 @@ You may also want to mount: ClickHouse has some advanced functionality, which requires enabling several [Linux capabilities](https://man7.org/linux/man-pages/man7/capabilities.7.html). -It is optional and can be enabled using the following [docker command-line arguments](https://docs.docker.com/engine/reference/run/#runtime-privilege-and-linux-capabilities): +These are optional and can be enabled using the following [docker command-line arguments](https://docs.docker.com/engine/reference/run/#runtime-privilege-and-linux-capabilities): ```bash docker run -d \ @@ -114,7 +114,7 @@ docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 -v /pa docker run --rm --user ${UID}:${GID} --name some-clickhouse-server --ulimit nofile=262144:262144 -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server ``` -When you use the image with local directories mounted, you probably would like to specify the user to maintain the proper file ownership. Use the `--user` argument and mount `/var/lib/clickhouse` and `/var/log/clickhouse-server` inside the container. Otherwise, the image will complain and not start. +When you use the image with local directories mounted, you probably want to specify the user to maintain the proper file ownership. Use the `--user` argument and mount `/var/lib/clickhouse` and `/var/log/clickhouse-server` inside the container. Otherwise, the image will complain and not start. ### Start server from root (useful in case of userns enabled) @@ -132,7 +132,7 @@ docker run --rm -e CLICKHOUSE_DB=my_database -e CLICKHOUSE_USER=username -e CLIC ## How to extend this image -If you would like to do additional initialization in an image derived from this one, add one or more `*.sql`, `*.sql.gz`, or `*.sh` scripts under `/docker-entrypoint-initdb.d`. After the entrypoint calls `initdb` it will run any `*.sql` files, run any executable `*.sh` scripts, and source any non-executable `*.sh` scripts found in that directory to do further initialization before starting the service. +To perform additional initialization in an image derived from this one, add one or more `*.sql`, `*.sql.gz`, or `*.sh` scripts under `/docker-entrypoint-initdb.d`. After the entrypoint calls `initdb`, it will run any `*.sql` files, run any executable `*.sh` scripts, and source any non-executable `*.sh` scripts found in that directory to do further initialization before starting the service. Also, you can provide environment variables `CLICKHOUSE_USER` & `CLICKHOUSE_PASSWORD` that will be used for clickhouse-client during initialization. For example, to add an additional user and database, add the following to `/docker-entrypoint-initdb.d/init-db.sh`: @@ -150,3 +150,4 @@ EOSQL ## License View [license information](https://github.com/ClickHouse/ClickHouse/blob/master/LICENSE) for the software contained in this image. + From 5f42c08561d1056d96f51f4585250d0e45a4d905 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 5 Jul 2022 11:42:42 -0400 Subject: [PATCH 245/627] small edits --- docker/server/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/README.md b/docker/server/README.md index 3639a432b1e..2ff08620658 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -4,7 +4,7 @@ ClickHouse is an open-source column-oriented database management system that allows the generation of analytical data reports in real-time. -ClickHouse manages extremely large volumes of data stably and sustainably. It currently powers [Yandex.Metrica](https://metrica.yandex.com/), the world’s [second-largest](http://w3techs.com/technologies/overview/traffic_analysis/all) web analytics platform, with over 13 trillion database records and over 20 billion events a day, generating customized reports on-the-fly, directly from non-aggregated data. This system was successfully implemented at [CERN’s LHCb experiment](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) to store and process metadata on 10bn events with over 1000 attributes per event registered in 2011. +ClickHouse manages extremely large volumes of data. It currently powers [Yandex.Metrica](https://metrica.yandex.com/), the world’s [second-largest](http://w3techs.com/technologies/overview/traffic_analysis/all) web analytics platform, with over 13 trillion database records and over 20 billion events a day, generating customized reports on-the-fly, directly from non-aggregated data. This system was successfully implemented at [CERN’s LHCb experiment](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) to store and process metadata on 10bn events with over 1000 attributes per event registered in 2011. For more information and documentation see https://clickhouse.com/. From 98cd92017e2cfcf3bcea47ee7a6691e74615ecf3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 5 Jul 2022 16:50:23 +0300 Subject: [PATCH 246/627] Fix waiting of shared lock after exclusive lock failure When WRITE lock attemp fails (exclusive lock for ALTER/DELETE), and there are multiple READ locks (shared lock for SELECT/INSERT), i.e. one INSERT is in progress and one SELECT is queued after ALTER/DELETE started but before it fails, this SELECT will wait until INSERT will finishes. This happens because in case of WRITE lock failure it does not notify the next READ lock that can be acquired. Signed-off-by: Azat Khuzhin --- src/Common/RWLock.cpp | 20 ++++++--- src/Common/RWLock.h | 3 +- .../0_stateless/02352_rwlock.reference | 1 + tests/queries/0_stateless/02352_rwlock.sh | 43 +++++++++++++++++++ 4 files changed, 60 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02352_rwlock.reference create mode 100755 tests/queries/0_stateless/02352_rwlock.sh diff --git a/src/Common/RWLock.cpp b/src/Common/RWLock.cpp index 92eda0c21aa..cf6d2dd7bab 100644 --- a/src/Common/RWLock.cpp +++ b/src/Common/RWLock.cpp @@ -189,9 +189,10 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c /// Rollback(SM1): nothrow if (it_group->requests == 0) { - (type == Read ? readers_queue : writers_queue).erase(it_group); + /// When WRITE lock fails, we need to notify next read that is waiting, + /// to avoid handing request, hence next=true. + dropOwnerGroupAndPassOwnership(it_group, /* next= */ true); } - return nullptr; } } @@ -211,7 +212,7 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c /// Methods std::list<>::emplace_back() and std::unordered_map<>::emplace() provide strong exception safety /// We only need to roll back the changes to these objects: owner_queries and the readers/writers queue if (it_group->requests == 0) - dropOwnerGroupAndPassOwnership(it_group); /// Rollback(SM1): nothrow + dropOwnerGroupAndPassOwnership(it_group, /* next= */ false); /// Rollback(SM1): nothrow throw; } @@ -259,11 +260,11 @@ void RWLockImpl::unlock(GroupsContainer::iterator group_it, const String & query /// If we are the last remaining referrer, remove this QNode and notify the next one if (--group_it->requests == 0) /// SM: nothrow - dropOwnerGroupAndPassOwnership(group_it); + dropOwnerGroupAndPassOwnership(group_it, /* next= */ false); } -void RWLockImpl::dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it) noexcept +void RWLockImpl::dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it, bool next) noexcept { rdlock_owner = readers_queue.end(); wrlock_owner = writers_queue.end(); @@ -287,7 +288,14 @@ void RWLockImpl::dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_ /// Prepare next phase if (!readers_queue.empty()) { - rdlock_owner = readers_queue.begin(); + if (next && readers_queue.size() > 1) + { + rdlock_owner = std::next(readers_queue.begin()); + } + else + { + rdlock_owner = readers_queue.begin(); + } } else { diff --git a/src/Common/RWLock.h b/src/Common/RWLock.h index e54fcaea433..cb4cf7f9200 100644 --- a/src/Common/RWLock.h +++ b/src/Common/RWLock.h @@ -90,6 +90,7 @@ private: RWLockImpl() = default; void unlock(GroupsContainer::iterator group_it, const String & query_id) noexcept; - void dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it) noexcept; + /// @param next - notify next after begin, used on writer lock failures + void dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it, bool next) noexcept; }; } diff --git a/tests/queries/0_stateless/02352_rwlock.reference b/tests/queries/0_stateless/02352_rwlock.reference new file mode 100644 index 00000000000..cd8584b2e5c --- /dev/null +++ b/tests/queries/0_stateless/02352_rwlock.reference @@ -0,0 +1 @@ +WRITE locking attempt on "default_ordinary.data_02352" has timed out diff --git a/tests/queries/0_stateless/02352_rwlock.sh b/tests/queries/0_stateless/02352_rwlock.sh new file mode 100755 index 00000000000..8054be81781 --- /dev/null +++ b/tests/queries/0_stateless/02352_rwlock.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +# Test that ensures that WRITE lock failure notifies READ. +# In other words to ensure that after WRITE lock failure (DROP), +# READ lock (SELECT) available instantly. + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function wait_query_by_id_started() +{ + local query_id=$1 && shift + # wait for query to be started + while [ "$($CLICKHOUSE_CLIENT "$@" -q "select count() from system.processes where query_id = '$query_id'")" -ne 1 ]; do + sleep 0.1 + done +} + +# to avoid removal via separate thread +$CLICKHOUSE_CLIENT -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_ordinary Engine=Ordinary" --allow_deprecated_database_ordinary=1 +$CLICKHOUSE_CLIENT -q "CREATE TABLE ${CLICKHOUSE_DATABASE}_ordinary.data_02352 (key Int) Engine=Null()" + +query_id="insert-$(random_str 10)" +# 20 seconds sleep +$CLICKHOUSE_CLIENT --query_id "$query_id" -q "INSERT INTO ${CLICKHOUSE_DATABASE}_ordinary.data_02352 SELECT sleepEachRow(1) FROM numbers(20) GROUP BY number" & +wait_query_by_id_started "$query_id" + +query_id="drop-$(random_str 10)" +# 10 second wait +$CLICKHOUSE_CLIENT --query_id "$query_id" -q "DROP TABLE ${CLICKHOUSE_DATABASE}_ordinary.data_02352 SYNC" --lock_acquire_timeout 10 > >(grep -m1 -o 'WRITE locking attempt on ".*" has timed out') 2>&1 & +wait_query_by_id_started "$query_id" + +# NOTE: we need to run SELECT after DROP and +# if the bug is there, then the query will wait 20 seconds (INSERT), instead of 10 (DROP) and will fail +# +# 11 seconds wait (DROP + 1 second lag) +$CLICKHOUSE_CLIENT -q "SELECT * FROM ${CLICKHOUSE_DATABASE}_ordinary.data_02352" --lock_acquire_timeout 11 + +# wait DROP and INSERT +wait + +$CLICKHOUSE_CLIENT -q "DROP DATABASE ${CLICKHOUSE_DATABASE}_ordinary" From f9204315b5c95c69dc3907d9d8d475f8a1adc11f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 5 Jul 2022 19:03:20 +0200 Subject: [PATCH 247/627] Store columns.txt in backups for the Memory table engine too. --- src/Backups/IBackupEntriesBatch.cpp | 37 ------ src/Backups/IBackupEntriesBatch.h | 29 ----- src/Backups/IBackupEntriesLazyBatch.cpp | 77 ++++++++++++ src/Backups/IBackupEntriesLazyBatch.h | 30 +++++ src/Storages/StorageMemory.cpp | 159 +++++++++++++----------- 5 files changed, 193 insertions(+), 139 deletions(-) delete mode 100644 src/Backups/IBackupEntriesBatch.cpp delete mode 100644 src/Backups/IBackupEntriesBatch.h create mode 100644 src/Backups/IBackupEntriesLazyBatch.cpp create mode 100644 src/Backups/IBackupEntriesLazyBatch.h diff --git a/src/Backups/IBackupEntriesBatch.cpp b/src/Backups/IBackupEntriesBatch.cpp deleted file mode 100644 index 34a91668023..00000000000 --- a/src/Backups/IBackupEntriesBatch.cpp +++ /dev/null @@ -1,37 +0,0 @@ -#include -#include - - -namespace DB -{ - -class IBackupEntriesBatch::BackupEntryFromBatch : public IBackupEntry -{ -public: - BackupEntryFromBatch(const std::shared_ptr & generator_, size_t index_) : batch(generator_), index(index_) - { - assert(batch); - } - - UInt64 getSize() const override { return batch->getSize(index); } - std::optional getChecksum() const override { return batch->getChecksum(index); } - std::unique_ptr getReadBuffer() const override { return batch->getReadBuffer(index); } - -private: - const std::shared_ptr batch; - const size_t index; -}; - - -BackupEntries IBackupEntriesBatch::getBackupEntries() -{ - BackupEntries res; - res.reserve(entry_names.size()); - for (size_t i = 0; i != entry_names.size(); ++i) - { - res.emplace_back(entry_names[i], std::make_unique(shared_from_this(), i)); - } - return res; -} - -} diff --git a/src/Backups/IBackupEntriesBatch.h b/src/Backups/IBackupEntriesBatch.h deleted file mode 100644 index 7fceb793c00..00000000000 --- a/src/Backups/IBackupEntriesBatch.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -/// Helper class designed to generate multiple backup entries from one source. -class IBackupEntriesBatch : public std::enable_shared_from_this -{ -public: - BackupEntries getBackupEntries(); - - virtual ~IBackupEntriesBatch() = default; - -protected: - IBackupEntriesBatch(const Strings & entry_names_) : entry_names(entry_names_) {} - - virtual std::unique_ptr getReadBuffer(size_t index) = 0; - virtual UInt64 getSize(size_t index) = 0; - virtual std::optional getChecksum(size_t) { return {}; } - -private: - class BackupEntryFromBatch; - const Strings entry_names; -}; - -} diff --git a/src/Backups/IBackupEntriesLazyBatch.cpp b/src/Backups/IBackupEntriesLazyBatch.cpp new file mode 100644 index 00000000000..5fb3a4cb3c0 --- /dev/null +++ b/src/Backups/IBackupEntriesLazyBatch.cpp @@ -0,0 +1,77 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + +class IBackupEntriesLazyBatch::BackupEntryFromBatch : public IBackupEntry +{ +public: + BackupEntryFromBatch(const std::shared_ptr & batch_, size_t index_) : batch(batch_), index(index_) { } + + UInt64 getSize() const override { return getInternalBackupEntry()->getSize(); } + std::optional getChecksum() const override { return getInternalBackupEntry()->getChecksum(); } + std::unique_ptr getReadBuffer() const override { return getInternalBackupEntry()->getReadBuffer(); } + +private: + BackupEntryPtr getInternalBackupEntry() const + { + std::lock_guard lock{mutex}; + if (!entry) + { + batch->generateIfNecessary(); + entry = batch->entries[index].second; + } + return entry; + } + + const std::shared_ptr batch; + const size_t index; + mutable std::mutex mutex; + mutable BackupEntryPtr entry; +}; + + +BackupEntries IBackupEntriesLazyBatch::getBackupEntries() +{ + BackupEntries res; + size_t size = getSize(); + res.reserve(size); + for (size_t i = 0; i != size; ++i) + { + res.emplace_back(getName(i), std::make_unique(shared_from_this(), i)); + } + return res; +} + +void IBackupEntriesLazyBatch::generateIfNecessary() +{ + std::lock_guard lock{mutex}; + if (generated) + return; + + entries = generate(); + + if (entries.size() != getSize()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup entries were generated incorrectly"); + + for (size_t i = 0; i != entries.size(); ++i) + { + if (entries[i].first != getName(i)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup entries were generated incorrectly"); + } + + generated = true; +} + +IBackupEntriesLazyBatch::~IBackupEntriesLazyBatch() = default; + +} diff --git a/src/Backups/IBackupEntriesLazyBatch.h b/src/Backups/IBackupEntriesLazyBatch.h new file mode 100644 index 00000000000..36f9805ff30 --- /dev/null +++ b/src/Backups/IBackupEntriesLazyBatch.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Helper class designed to generate multiple backup entries from one source. +class IBackupEntriesLazyBatch : public std::enable_shared_from_this +{ +public: + BackupEntries getBackupEntries(); + virtual ~IBackupEntriesLazyBatch(); + +protected: + virtual size_t getSize() const = 0; + virtual const String & getName(size_t i) const = 0; + virtual BackupEntries generate() = 0; + +private: + void generateIfNecessary(); + + class BackupEntryFromBatch; + std::mutex mutex; + BackupEntries entries; + bool generated = false; +}; + +} diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index f9e8b03e9f4..bf71d539cc2 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -24,8 +24,10 @@ #include #include #include +#include +#include #include -#include +#include #include #include #include @@ -382,104 +384,116 @@ void StorageMemory::truncate( namespace { - class MemoryBackupEntriesBatch : public IBackupEntriesBatch, boost::noncopyable + class MemoryBackup : public IBackupEntriesLazyBatch, boost::noncopyable { public: - MemoryBackupEntriesBatch( + MemoryBackup( const StorageMetadataPtr & metadata_snapshot_, const std::shared_ptr blocks_, const String & data_path_in_backup, const DiskPtr & temp_disk_, UInt64 max_compress_block_size_) - : IBackupEntriesBatch( - {fs::path{data_path_in_backup} / "data.bin", - fs::path{data_path_in_backup} / "index.mrk", - fs::path{data_path_in_backup} / "sizes.json"}) - , metadata_snapshot(metadata_snapshot_) + : metadata_snapshot(metadata_snapshot_) , blocks(blocks_) , temp_disk(temp_disk_) , max_compress_block_size(max_compress_block_size_) { + fs::path data_path_in_backup_fs = data_path_in_backup; + data_bin_pos = file_paths.size(); + file_paths.emplace_back(data_path_in_backup_fs / "data.bin"); + index_mrk_pos= file_paths.size(); + file_paths.emplace_back(data_path_in_backup_fs / "index.mrk"); + columns_txt_pos = file_paths.size(); + file_paths.emplace_back(data_path_in_backup_fs / "columns.txt"); + count_txt_pos = file_paths.size(); + file_paths.emplace_back(data_path_in_backup_fs / "count.txt"); + sizes_json_pos = file_paths.size(); + file_paths.emplace_back(data_path_in_backup_fs / "sizes.json"); } private: - static constexpr const size_t kDataBinPos = 0; - static constexpr const size_t kIndexMrkPos = 1; - static constexpr const size_t kSizesJsonPos = 2; - static constexpr const size_t kSize = 3; - - void initialize() + size_t getSize() const override { - std::call_once(initialized_flag, [this]() + return file_paths.size(); + } + + const String & getName(size_t i) const override + { + return file_paths[i]; + } + + BackupEntries generate() override + { + BackupEntries backup_entries; + backup_entries.resize(file_paths.size()); + + temp_dir_owner.emplace(temp_disk); + fs::path temp_dir = temp_dir_owner->getPath(); + temp_disk->createDirectories(temp_dir); + + /// Writing data.bin + IndexForNativeFormat index; { - temp_dir_owner.emplace(temp_disk); - fs::path temp_dir = temp_dir_owner->getPath(); - temp_disk->createDirectories(temp_dir); + auto data_file_path = temp_dir / fs::path{file_paths[data_bin_pos]}.filename(); + auto data_out_compressed = temp_disk->writeFile(data_file_path); + CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size}; + NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index}; + for (const auto & block : *blocks) + block_out.write(block); + backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared(temp_disk, data_file_path)}; + } - /// Writing data.bin - constexpr char data_file_name[] = "data.bin"; - auto data_file_path = temp_dir / data_file_name; - IndexForNativeFormat index; + /// Writing index.mrk + { + auto index_mrk_path = temp_dir / fs::path{file_paths[index_mrk_pos]}.filename(); + auto index_mrk_out_compressed = temp_disk->writeFile(index_mrk_path); + CompressedWriteBuffer index_mrk_out{*index_mrk_out_compressed}; + index.write(index_mrk_out); + backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared(temp_disk, index_mrk_path)}; + } + + /// Writing columns.txt + { + auto columns_desc = metadata_snapshot->getColumns().getAllPhysical().toString(); + backup_entries[columns_txt_pos] = {file_paths[columns_txt_pos], std::make_shared(columns_desc)}; + } + + /// Writing count.txt + { + size_t num_rows = 0; + for (const auto & block : *blocks) + num_rows += block.rows(); + backup_entries[count_txt_pos] = {file_paths[count_txt_pos], std::make_shared(toString(num_rows))}; + } + + /// Writing sizes.json + { + auto sizes_json_path = temp_dir / fs::path{file_paths[sizes_json_pos]}.filename(); + FileChecker file_checker{temp_disk, sizes_json_path}; + for (size_t i = 0; i != file_paths.size(); ++i) { - auto data_out_compressed = temp_disk->writeFile(data_file_path); - CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size}; - NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index}; - for (const auto & block : *blocks) - block_out.write(block); + if (i == sizes_json_pos) + continue; + file_checker.update(temp_dir / fs::path{file_paths[i]}.filename()); } - - /// Writing index.mrk - constexpr char index_file_name[] = "index.mrk"; - auto index_file_path = temp_dir / index_file_name; - { - auto index_out_compressed = temp_disk->writeFile(index_file_path); - CompressedWriteBuffer index_out{*index_out_compressed}; - index.write(index_out); - } - - /// Writing sizes.json - constexpr char sizes_file_name[] = "sizes.json"; - auto sizes_file_path = temp_dir / sizes_file_name; - FileChecker file_checker{temp_disk, sizes_file_path}; - file_checker.update(data_file_path); - file_checker.update(index_file_path); file_checker.save(); + backup_entries[sizes_json_pos] = {file_paths[sizes_json_pos], std::make_shared(temp_disk, sizes_json_path)}; + } - file_paths[kDataBinPos] = data_file_path; - file_sizes[kDataBinPos] = file_checker.getFileSize(data_file_path); + /// We don't need to keep `blocks` any longer. + blocks.reset(); + metadata_snapshot.reset(); - file_paths[kIndexMrkPos] = index_file_path; - file_sizes[kIndexMrkPos] = file_checker.getFileSize(index_file_path); - - file_paths[kSizesJsonPos] = sizes_file_path; - file_sizes[kSizesJsonPos] = temp_disk->getFileSize(sizes_file_path); - - /// We don't need to keep `blocks` any longer. - blocks.reset(); - metadata_snapshot.reset(); - }); - } - - std::unique_ptr getReadBuffer(size_t index) override - { - initialize(); - return temp_disk->readFile(file_paths[index]); - } - - UInt64 getSize(size_t index) override - { - initialize(); - return file_sizes[index]; + return backup_entries; } StorageMetadataPtr metadata_snapshot; std::shared_ptr blocks; DiskPtr temp_disk; - UInt64 max_compress_block_size; - std::once_flag initialized_flag; std::optional temp_dir_owner; - std::array file_paths; - std::array file_sizes; + UInt64 max_compress_block_size; + Strings file_paths; + size_t data_bin_pos, index_mrk_pos, columns_txt_pos, count_txt_pos, sizes_json_pos; }; } @@ -488,8 +502,7 @@ void StorageMemory::backupData(BackupEntriesCollector & backup_entries_collector auto temp_disk = backup_entries_collector.getContext()->getTemporaryVolume()->getDisk(0); auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size; backup_entries_collector.addBackupEntries( - std::make_shared( - getInMemoryMetadataPtr(), data.get(), data_path_in_backup, temp_disk, max_compress_block_size) + std::make_shared(getInMemoryMetadataPtr(), data.get(), data_path_in_backup, temp_disk, max_compress_block_size) ->getBackupEntries()); } From e9b7f52874d5127191b9bb4bbe6a58dbf6344e41 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 5 Jul 2022 17:36:45 +0000 Subject: [PATCH 248/627] another one --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 11bea5ad3bd..85c4ea261a0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -306,7 +306,7 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr span.attributes.push_back(Tuple{"db.statement", elem.query}); span.attributes.push_back(Tuple{"clickhouse.query_id", elem.client_info.current_query_id}); span.attributes.push_back(Tuple{"clickhouse.exception", elem.exception}); - span.attributes.push_back(Tuple{"clickhouse.exception_code", elem.exception_code}); + span.attributes.push_back(Tuple{"clickhouse.exception_code", toString(elem.exception_code)}); if (!context->query_trace_context.tracestate.empty()) { span.attributes.push_back(Tuple{"clickhouse.tracestate", context->query_trace_context.tracestate}); From 1ac46c5e488c2632dafc8837694d9ecddb5f1fdc Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 5 Jul 2022 20:57:01 +0200 Subject: [PATCH 249/627] Fix making backups containing multiple ACL tables. --- src/Access/ReplicatedAccessStorage.cpp | 5 +- src/Backups/BackupCoordinationDistributed.cpp | 61 +++++++++++-------- src/Backups/BackupCoordinationDistributed.h | 6 +- src/Backups/BackupCoordinationHelpers.cpp | 28 +++++++++ src/Backups/BackupCoordinationHelpers.h | 26 ++++++++ src/Backups/BackupCoordinationLocal.cpp | 18 ++---- src/Backups/BackupCoordinationLocal.h | 6 +- src/Backups/IBackupCoordination.h | 6 +- 8 files changed, 106 insertions(+), 50 deletions(-) diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index ed9ee16bef1..c7aec75265c 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -638,16 +638,17 @@ void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_col auto backup_coordination = backup_entries_collector.getBackupCoordination(); String current_host_id = backup_entries_collector.getBackupSettings().host_id; - backup_coordination->addReplicatedAccessPath(zookeeper_path, current_host_id, backup_entry_with_path.first); + backup_coordination->addReplicatedAccessFilePath(zookeeper_path, type, current_host_id, backup_entry_with_path.first); backup_entries_collector.addPostTask( [backup_entry = backup_entry_with_path.second, zookeeper_path = zookeeper_path, + type, current_host_id, &backup_entries_collector, backup_coordination] { - for (const String & path : backup_coordination->getReplicatedAccessPaths(zookeeper_path, current_host_id)) + for (const String & path : backup_coordination->getReplicatedAccessFilePaths(zookeeper_path, type, current_host_id)) backup_entries_collector.addBackupEntry(path, backup_entry); }); } diff --git a/src/Backups/BackupCoordinationDistributed.cpp b/src/Backups/BackupCoordinationDistributed.cpp index caa7c7aaaa5..9f2afff11ed 100644 --- a/src/Backups/BackupCoordinationDistributed.cpp +++ b/src/Backups/BackupCoordinationDistributed.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -182,7 +183,7 @@ void BackupCoordinationDistributed::createRootNodes() zookeeper->createIfNotExists(zookeeper_path + "/repl_part_names", ""); zookeeper->createIfNotExists(zookeeper_path + "/repl_mutations", ""); zookeeper->createIfNotExists(zookeeper_path + "/repl_data_paths", ""); - zookeeper->createIfNotExists(zookeeper_path + "/repl_access_paths", ""); + zookeeper->createIfNotExists(zookeeper_path + "/repl_access", ""); zookeeper->createIfNotExists(zookeeper_path + "/file_names", ""); zookeeper->createIfNotExists(zookeeper_path + "/file_infos", ""); zookeeper->createIfNotExists(zookeeper_path + "/archive_suffixes", ""); @@ -327,40 +328,48 @@ Strings BackupCoordinationDistributed::getReplicatedDataPaths(const String & tab } -void BackupCoordinationDistributed::addReplicatedAccessPath(const String & access_zk_path, const String & host_id, const String & file_path) +void BackupCoordinationDistributed::addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) { auto zookeeper = get_zookeeper(); - String path = zookeeper_path + "/repl_access_paths/" + escapeForFileName(access_zk_path); + String path = zookeeper_path + "/repl_access/" + escapeForFileName(access_zk_path); zookeeper->createIfNotExists(path, ""); - zookeeper->createIfNotExists(path + "/" + escapeForFileName(file_path), ""); - - path += "/host"; - auto code = zookeeper->tryCreate(path, host_id, zkutil::CreateMode::Persistent); - if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); - - if (code == Coordination::Error::ZNODEEXISTS) - zookeeper->set(path, host_id); + path += "/" + AccessEntityTypeInfo::get(access_entity_type).name; + zookeeper->createIfNotExists(path, ""); + path += "/" + host_id; + zookeeper->createIfNotExists(path, file_path); } -Strings BackupCoordinationDistributed::getReplicatedAccessPaths(const String & access_zk_path, const String & host_id) const +Strings BackupCoordinationDistributed::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const { + std::lock_guard lock{mutex}; + prepareReplicatedAccess(); + return replicated_access->getFilePaths(access_zk_path, access_entity_type, host_id); +} + +void BackupCoordinationDistributed::prepareReplicatedAccess() const +{ + if (replicated_access) + return; + + replicated_access.emplace(); auto zookeeper = get_zookeeper(); - - String path = zookeeper_path + "/repl_access_paths/" + escapeForFileName(access_zk_path); - String path2 = path + "/host"; - if (zookeeper->get(path2) != host_id) - return {}; - - Strings children = zookeeper->getChildren(path); - Strings file_paths; - file_paths.reserve(children.size()); - for (const String & child : children) + + String path = zookeeper_path + "/repl_access/"; + for (const String & escaped_access_zk_path : zookeeper->getChildren(path)) { - if (child != "host") - file_paths.push_back(unescapeForFileName(child)); + String access_zk_path = unescapeForFileName(escaped_access_zk_path); + String path2 = path + "/" + escaped_access_zk_path; + for (const String & type_str : zookeeper->getChildren(path2)) + { + AccessEntityType type = AccessEntityTypeInfo::parseType(type_str); + String path3 = path2 + "/" + type_str; + for (const String & host_id : zookeeper->getChildren(path3)) + { + String file_path = zookeeper->get(path3 + "/" + host_id); + replicated_access->addFilePath(access_zk_path, type, host_id, file_path); + } + } } - return file_paths; } diff --git a/src/Backups/BackupCoordinationDistributed.h b/src/Backups/BackupCoordinationDistributed.h index c991d273be0..feb2cdeeb97 100644 --- a/src/Backups/BackupCoordinationDistributed.h +++ b/src/Backups/BackupCoordinationDistributed.h @@ -37,8 +37,8 @@ public: void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; Strings getReplicatedDataPaths(const String & table_shared_id) const override; - void addReplicatedAccessPath(const String & access_zk_path, const String & host_id, const String & file_path) override; - Strings getReplicatedAccessPaths(const String & access_zk_path, const String & host_id) const override; + void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) override; + Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const override; void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override; void updateFileInfo(const FileInfo & file_info) override; @@ -59,6 +59,7 @@ private: void createRootNodes(); void removeAllNodes(); void prepareReplicatedPartsAndMutations() const; + void prepareReplicatedAccess() const; const String zookeeper_path; const zkutil::GetZooKeeper get_zookeeper; @@ -67,6 +68,7 @@ private: mutable std::mutex mutex; mutable std::optional replicated_parts_and_mutations; + mutable std::optional replicated_access; }; } diff --git a/src/Backups/BackupCoordinationHelpers.cpp b/src/Backups/BackupCoordinationHelpers.cpp index 715031c2a37..7f8bdb32c24 100644 --- a/src/Backups/BackupCoordinationHelpers.cpp +++ b/src/Backups/BackupCoordinationHelpers.cpp @@ -320,6 +320,34 @@ void BackupCoordinationReplicatedPartsAndMutations::prepare() const } +/// Helper designed to be used in an implementation of the IBackupCoordination interface in the part related to replicated access storages. +BackupCoordinationReplicatedAccess::BackupCoordinationReplicatedAccess() = default; +BackupCoordinationReplicatedAccess::~BackupCoordinationReplicatedAccess() = default; + +void BackupCoordinationReplicatedAccess::addFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) +{ + auto & ref = file_paths_by_zk_path[std::make_pair(access_zk_path, access_entity_type)]; + ref.file_paths.emplace(file_path); + + /// std::max() because the calculation must give the same result being repeated on a different replica. + ref.host_to_store_access = std::max(ref.host_to_store_access, host_id); +} + +Strings BackupCoordinationReplicatedAccess::getFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const +{ + auto it = file_paths_by_zk_path.find(std::make_pair(access_zk_path, access_entity_type)); + if (it == file_paths_by_zk_path.end()) + return {}; + + auto & file_paths = it->second; + if (file_paths.host_to_store_access != host_id) + return {}; + + Strings res{file_paths.file_paths.begin(), file_paths.file_paths.end()}; + return res; +} + + /// Helps to wait until all hosts come to a specified stage. BackupCoordinationStatusSync::BackupCoordinationStatusSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_) : zookeeper_path(zookeeper_path_) diff --git a/src/Backups/BackupCoordinationHelpers.h b/src/Backups/BackupCoordinationHelpers.h index ad6168c9e8c..9203426a96a 100644 --- a/src/Backups/BackupCoordinationHelpers.h +++ b/src/Backups/BackupCoordinationHelpers.h @@ -9,6 +9,8 @@ namespace DB { +enum class AccessEntityType; + /// Helper designed to be used in an implementation of the IBackupCoordination interface in the part related to replicated tables. class BackupCoordinationReplicatedPartsAndMutations @@ -73,6 +75,30 @@ private: }; +/// Helper designed to be used in an implementation of the IBackupCoordination interface in the part related to replicated access storages. +class BackupCoordinationReplicatedAccess +{ +public: + BackupCoordinationReplicatedAccess(); + ~BackupCoordinationReplicatedAccess(); + + /// Adds a path to access.txt file keeping access entities of a ReplicatedAccessStorage. + void addFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path); + Strings getFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const; + +private: + using ZkPathAndEntityType = std::pair; + + struct FilePathsAndHost + { + std::unordered_set file_paths; + String host_to_store_access; + }; + + std::map file_paths_by_zk_path; +}; + + /// Helps to wait until all hosts come to a specified stage. class BackupCoordinationStatusSync { diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index fadd15ea935..8315de95775 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -69,26 +69,16 @@ Strings BackupCoordinationLocal::getReplicatedDataPaths(const String & table_sha } -void BackupCoordinationLocal::addReplicatedAccessPath(const String & access_zk_path, const String & host_id, const String & file_path) +void BackupCoordinationLocal::addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) { std::lock_guard lock{mutex}; - auto & ref = replicated_access_paths[access_zk_path]; - ref.file_paths.push_back(file_path); - ref.host_to_store_access = host_id; + replicated_access.addFilePath(access_zk_path, access_entity_type, host_id, file_path); } -Strings BackupCoordinationLocal::getReplicatedAccessPaths(const String & access_zk_path, const String & host_id) const +Strings BackupCoordinationLocal::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const { std::lock_guard lock{mutex}; - auto it = replicated_access_paths.find(access_zk_path); - if (it == replicated_access_paths.end()) - return {}; - - const auto & ref = it->second; - if (ref.host_to_store_access != host_id) - return {}; - - return ref.file_paths; + return replicated_access.getFilePaths(access_zk_path, access_entity_type, host_id); } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 38e3fc846de..35218c426dc 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -34,8 +34,8 @@ public: void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; Strings getReplicatedDataPaths(const String & table_shared_id) const override; - void addReplicatedAccessPath(const String & access_zk_path, const String & host_id, const String & file_path) override; - Strings getReplicatedAccessPaths(const String & access_zk_path, const String & host_id) const override; + void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) override; + Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const override; void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override; void updateFileInfo(const FileInfo & file_info) override; @@ -61,7 +61,7 @@ private: mutable std::mutex mutex; BackupCoordinationReplicatedPartsAndMutations replicated_parts_and_mutations TSA_GUARDED_BY(mutex); std::unordered_map replicated_data_paths TSA_GUARDED_BY(mutex); - std::unordered_map replicated_access_paths TSA_GUARDED_BY(mutex); + BackupCoordinationReplicatedAccess replicated_access TSA_GUARDED_BY(mutex); std::map file_names TSA_GUARDED_BY(mutex); /// Should be ordered alphabetically, see listFiles(). For empty files we assume checksum = 0. std::map file_infos TSA_GUARDED_BY(mutex); /// Information about files. Without empty files. Strings archive_suffixes TSA_GUARDED_BY(mutex); diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index d899385d042..2c4781d32da 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -6,7 +6,7 @@ namespace DB { -struct ReplicatedMergeTreeMutationEntry; +enum class AccessEntityType; /// Keeps information about files contained in a backup. @@ -59,8 +59,8 @@ public: virtual Strings getReplicatedDataPaths(const String & table_shared_id) const = 0; /// Adds a path to access.txt file keeping access entities of a ReplicatedAccessStorage. - virtual void addReplicatedAccessPath(const String & access_zk_path, const String & host_id, const String & file_path) = 0; - virtual Strings getReplicatedAccessPaths(const String & access_zk_path, const String & host_id) const = 0; + virtual void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) = 0; + virtual Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const = 0; struct FileInfo { From 6903e6be6c7e95361ae423d54c329b103e4e3bb7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jul 2022 21:18:29 +0200 Subject: [PATCH 250/627] Add README, 53% --- benchmark/README.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/benchmark/README.md b/benchmark/README.md index bfbac6f36d2..a668c8602c9 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -79,7 +79,13 @@ https://datasets.clickhouse.com/hits_compatible/hits.parquet https://datasets.clickhouse.com/hits_compatible/athena/hits.parquet https://datasets.clickhouse.com/hits_compatible/athena_partitioned/hits_{0..99}.parquet -To correctly compare insertion time, the dataset should be downloaded and decompressed before loading. The dataset should be loaded as a single file in the most straightforward way. Splitting the dataset for parallel loading is not recommended, as it will make comparisons more difficult. Splitting the dataset is possible if the system cannot eat it as a whole due to its limitations. +The format of the source data can be selected up to convenience. + +To correctly compare the insertion time, the dataset should be downloaded and decompressed before loading. The dataset should be loaded as a single file in the most straightforward way. Splitting the dataset for parallel loading is not recommended, as it will make comparisons more difficult. Splitting the dataset is possible if the system cannot eat it as a whole due to its limitations. + +You should not wait for cool down after data loading or running OPTIMIZE / VACUUM before the main benchmark queries unless it is strictly required for the system. + +The used storage size can be measured without accounting for temporary data if there is temporary data that will be removed in the background. The built-in introspection capabilities can be used to measure the storage size, or it can be measured by checking the used space in the filesystem. ### Indexing @@ -97,7 +103,7 @@ If a system is of a "multidimensional OLAP" kind, so always or implicitly doing If the system contains a cache for query results, it should be disabled. -If the system performs caching for source data, it is ok. If the cache can be flushed, it should be flushed before the first run of every query. +If the system performs caching for source data (buffer pools and similar), it is ok. If the cache or buffer pools can be flushed, it should be flushed before the first run of every query. If the system contains a cache for intermediate data, it should be disabled if this cache is located near the end of the query execution pipeline, thus similar to a query result cache. From 198cb0ad4372c60f27eb2fded22c9c9a9824145b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jul 2022 21:30:49 +0200 Subject: [PATCH 251/627] Add README, 54% --- benchmark/README.md | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/benchmark/README.md b/benchmark/README.md index a668c8602c9..9a4a2271cb0 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -72,16 +72,18 @@ Fine-tuning and optimization for the benchmark are not recommended but allowed. The dataset is available in `CSV`, `TSV`, `JSONlines` and `Parquet` formats by the following links: -https://datasets.clickhouse.com/hits_compatible/hits.csv.gz -https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz -https://datasets.clickhouse.com/hits_compatible/hits.json.gz -https://datasets.clickhouse.com/hits_compatible/hits.parquet -https://datasets.clickhouse.com/hits_compatible/athena/hits.parquet -https://datasets.clickhouse.com/hits_compatible/athena_partitioned/hits_{0..99}.parquet +- https://datasets.clickhouse.com/hits_compatible/hits.csv.gz +- https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz +- https://datasets.clickhouse.com/hits_compatible/hits.json.gz +- https://datasets.clickhouse.com/hits_compatible/hits.parquet The format of the source data can be selected up to convenience. -To correctly compare the insertion time, the dataset should be downloaded and decompressed before loading. The dataset should be loaded as a single file in the most straightforward way. Splitting the dataset for parallel loading is not recommended, as it will make comparisons more difficult. Splitting the dataset is possible if the system cannot eat it as a whole due to its limitations. +Additional sources for stateless table engines are provided: +- https://datasets.clickhouse.com/hits_compatible/athena/hits.parquet (the same parquet file in its own subdirectory) +- https://datasets.clickhouse.com/hits_compatible/athena_partitioned/hits_{0..99}.parquet (100 files) + +To correctly compare the insertion time, the dataset should be downloaded and decompressed before loading (if it's using external compression; the parquet file includes internal compression and can be loaded as is). The dataset should be loaded as a single file in the most straightforward way. Splitting the dataset for parallel loading is not recommended, as it will make comparisons more difficult. Splitting the dataset is possible if the system cannot eat it as a whole due to its limitations. You should not wait for cool down after data loading or running OPTIMIZE / VACUUM before the main benchmark queries unless it is strictly required for the system. From 99de1ee109db67238741aa7970b5ab092dbaef19 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 5 Jul 2022 20:55:19 +0000 Subject: [PATCH 252/627] Simple cleanup: interpreters and parsers --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.h | 4 ++-- src/Interpreters/InterpreterSelectWithUnionQuery.h | 4 ++-- src/Parsers/ASTAlterQuery.h | 2 +- src/Parsers/ASTCreateQuery.h | 2 +- src/Parsers/ASTDropQuery.h | 2 +- src/Parsers/ASTInsertQuery.h | 2 +- src/Parsers/ASTRenameQuery.h | 2 +- src/Parsers/ASTSelectIntersectExceptQuery.h | 2 +- src/Parsers/ASTSelectQuery.h | 2 +- src/Parsers/ASTSelectWithUnionQuery.h | 2 +- src/Parsers/ASTSystemQuery.h | 2 +- src/Parsers/Access/ASTGrantQuery.h | 2 +- 13 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 8d0c4dee023..057af1fbc24 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1539,7 +1539,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai if (optimize_read_in_order) { - for (auto & child : select_query->orderBy()->children) + for (const auto & child : select_query->orderBy()->children) { auto actions_dag = std::make_shared(columns_after_join); getRootActions(child, only_types, actions_dag); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index e6bd81b93fe..902d9f64058 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -92,12 +92,12 @@ public: BlockIO execute() override; /// Builds QueryPlan for current query. - virtual void buildQueryPlan(QueryPlan & query_plan) override; + void buildQueryPlan(QueryPlan & query_plan) override; bool ignoreLimits() const override { return options.ignore_limits; } bool ignoreQuota() const override { return options.ignore_quota; } - virtual void ignoreWithTotals() override; + void ignoreWithTotals() override; ASTPtr getQuery() const { return query_ptr; } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 0e59e1e01ad..ff763ec6490 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -31,7 +31,7 @@ public: ~InterpreterSelectWithUnionQuery() override; /// Builds QueryPlan for current query. - virtual void buildQueryPlan(QueryPlan & query_plan) override; + void buildQueryPlan(QueryPlan & query_plan) override; BlockIO execute() override; @@ -43,7 +43,7 @@ public: ContextPtr context_, bool is_subquery = false); - virtual void ignoreWithTotals() override; + void ignoreWithTotals() override; bool supportsTransactions() const override { return true; } diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 956f07811ae..4a8c9c14ea9 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -248,7 +248,7 @@ public: return removeOnCluster(clone(), params.default_database); } - virtual QueryKind getQueryKind() const override { return QueryKind::Alter; } + QueryKind getQueryKind() const override { return QueryKind::Alter; } protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index f9f57183a64..f3729b1523f 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -120,7 +120,7 @@ public: bool isView() const { return is_ordinary_view || is_materialized_view || is_live_view || is_window_view; } - virtual QueryKind getQueryKind() const override { return QueryKind::Create; } + QueryKind getQueryKind() const override { return QueryKind::Create; } protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/src/Parsers/ASTDropQuery.h b/src/Parsers/ASTDropQuery.h index b4c96353a09..05515ba4005 100644 --- a/src/Parsers/ASTDropQuery.h +++ b/src/Parsers/ASTDropQuery.h @@ -45,7 +45,7 @@ public: return removeOnCluster(clone(), params.default_database); } - virtual QueryKind getQueryKind() const override { return QueryKind::Drop; } + QueryKind getQueryKind() const override { return QueryKind::Drop; } protected: void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; diff --git a/src/Parsers/ASTInsertQuery.h b/src/Parsers/ASTInsertQuery.h index db9262ea794..43780e27114 100644 --- a/src/Parsers/ASTInsertQuery.h +++ b/src/Parsers/ASTInsertQuery.h @@ -66,7 +66,7 @@ public: return res; } - virtual QueryKind getQueryKind() const override { return QueryKind::Insert; } + QueryKind getQueryKind() const override { return QueryKind::Insert; } protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/src/Parsers/ASTRenameQuery.h b/src/Parsers/ASTRenameQuery.h index ee7cad2d38a..723f680b492 100644 --- a/src/Parsers/ASTRenameQuery.h +++ b/src/Parsers/ASTRenameQuery.h @@ -65,7 +65,7 @@ public: return query_ptr; } - virtual QueryKind getQueryKind() const override { return QueryKind::Rename; } + QueryKind getQueryKind() const override { return QueryKind::Rename; } protected: void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.h b/src/Parsers/ASTSelectIntersectExceptQuery.h index fa574b46c8d..c95944a0c35 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.h +++ b/src/Parsers/ASTSelectIntersectExceptQuery.h @@ -22,7 +22,7 @@ public: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - virtual QueryKind getQueryKind() const override { return QueryKind::SelectIntersectExcept; } + QueryKind getQueryKind() const override { return QueryKind::SelectIntersectExcept; } ASTs getListOfSelects() const; diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index 084627b57a1..1a02717db8d 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -140,7 +140,7 @@ public: void setFinal(); - virtual QueryKind getQueryKind() const override { return QueryKind::Select; } + QueryKind getQueryKind() const override { return QueryKind::Select; } protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index bd45dd7fc05..457a3361b1e 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -17,7 +17,7 @@ public: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - virtual QueryKind getQueryKind() const override { return QueryKind::Select; } + QueryKind getQueryKind() const override { return QueryKind::Select; } SelectUnionMode union_mode; diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 6bcd5f090d0..f2ef7a7a47a 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -114,7 +114,7 @@ public: return removeOnCluster(clone(), params.default_database); } - virtual QueryKind getQueryKind() const override { return QueryKind::System; } + QueryKind getQueryKind() const override { return QueryKind::System; } protected: diff --git a/src/Parsers/Access/ASTGrantQuery.h b/src/Parsers/Access/ASTGrantQuery.h index d2df3f6cf6e..8c7df3cd57e 100644 --- a/src/Parsers/Access/ASTGrantQuery.h +++ b/src/Parsers/Access/ASTGrantQuery.h @@ -34,6 +34,6 @@ public: void replaceEmptyDatabase(const String & current_database); void replaceCurrentUserTag(const String & current_user_name) const; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } - virtual QueryKind getQueryKind() const override { return is_revoke ? QueryKind::Revoke : QueryKind::Grant; } + QueryKind getQueryKind() const override { return is_revoke ? QueryKind::Revoke : QueryKind::Grant; } }; } From fd62494a7789527b8905c699526e7345f50c42da Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 5 Jul 2022 15:31:46 +0000 Subject: [PATCH 253/627] Fix: ORDER BY with braces inefficient execution --- src/Interpreters/ExpressionAnalyzer.cpp | 3 +- src/Interpreters/InterpreterSelectQuery.cpp | 51 ++++++++++++++++++--- src/Interpreters/InterpreterSelectQuery.h | 2 +- src/Interpreters/RewriteOrderByVisitor.cpp | 46 +++++++++++++++++++ src/Interpreters/RewriteOrderByVisitor.hpp | 34 ++++++++++++++ src/Interpreters/TreeRewriter.cpp | 6 +++ src/Parsers/ASTSelectWithUnionQuery.h | 2 +- src/Processors/QueryPlan/SortingStep.cpp | 19 +++++++- 8 files changed, 151 insertions(+), 12 deletions(-) create mode 100644 src/Interpreters/RewriteOrderByVisitor.cpp create mode 100644 src/Interpreters/RewriteOrderByVisitor.hpp diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 8d0c4dee023..76cace4e8e1 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1539,7 +1539,8 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai if (optimize_read_in_order) { - for (auto & child : select_query->orderBy()->children) + const ASTPtr & order_by = select_query->orderBy(); + for (const auto & child : order_by->children) { auto actions_dag = std::make_shared(columns_after_join); getRootActions(child, only_types, actions_dag); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 28438a86e47..8d396366688 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -886,27 +886,64 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or return descr; } -SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query, ContextPtr context_) +SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query, const ContextPtr & context_) { + /// Expected AST from query.orderBy() + /// (1) ... ORDER BY a,b + /// │ ExpressionList (children 2) │ + /// │ OrderByElement (children 1) │ + /// │ Identifier CounterID │ + /// │ OrderByElement (children 1) │ + /// │ Identifier EventDate │ + /// + /// (2) ... ORDER BY (a, b) + /// │ ExpressionList (children 1) │ + /// │ OrderByElement (children 1) │ + /// │ Function tuple (children 1) │ + /// │ ExpressionList (children 2) │ + /// │ Identifier CounterID │ + /// │ Identifier EventDate │ + /// SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); - for (const auto & elem : query.orderBy()->children) - { - String name = elem->children.front()->getColumnName(); - const auto & order_by_elem = elem->as(); + auto add_sort_column_desc = [&](const ASTOrderByElement & order_by_elem, const String & column) + { std::shared_ptr collator; if (order_by_elem.collation) collator = std::make_shared(order_by_elem.collation->as().value.get()); + if (order_by_elem.with_fill) { FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context_); - order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc); + order_descr.emplace_back(column, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc); } else - order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator); + order_descr.emplace_back(column, order_by_elem.direction, order_by_elem.nulls_direction, collator); + }; + + for (const auto & elem : query.orderBy()->children) + { + const auto & order_by_elem = elem->as(); + /// case (2) + if (const auto* func = order_by_elem.children.front()->as()) + { + if (const auto* expr_list = func->children.front()->as()) + { + for(const auto& identifier : expr_list->children) + add_sort_column_desc(order_by_elem, identifier->getColumnName()); + } + } + /// case (1) + else + { + add_sort_column_desc(order_by_elem, elem->children.front()->getColumnName()); + } } + if (order_descr.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to build sort description for ORDER BY"); + order_descr.compile_sort_description = context_->getSettingsRef().compile_sort_description; order_descr.min_count_to_compile_sort_description = context_->getSettingsRef().min_count_to_compile_sort_description; diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index e6bd81b93fe..00cc9a9c67d 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -128,7 +128,7 @@ public: /// It will set shard_num and shard_count to the client_info void setProperClientInfo(size_t replica_num, size_t replica_count); - static SortDescription getSortDescription(const ASTSelectQuery & query, ContextPtr context); + static SortDescription getSortDescription(const ASTSelectQuery & query, const ContextPtr & context); static UInt64 getLimitForSorting(const ASTSelectQuery & query, ContextPtr context); private: diff --git a/src/Interpreters/RewriteOrderByVisitor.cpp b/src/Interpreters/RewriteOrderByVisitor.cpp new file mode 100644 index 00000000000..67788a61e71 --- /dev/null +++ b/src/Interpreters/RewriteOrderByVisitor.cpp @@ -0,0 +1,46 @@ +#include +#include +#include +#include +#include + +namespace DB { + +void RewriteOrderBy::visit(ASTPtr & ast, Data &) +{ + auto * query = ast->as(); + if (!query) + return; + + auto order_by = query->orderBy(); + if (!order_by) + return; + + const auto * expr_list = order_by->as(); + if (!expr_list) + return; + + const auto * order_by_elem = expr_list->children.front()->as(); + if (!order_by_elem) + return; + + const auto * func = order_by_elem->children.front()->as(); + if (!func || func->name != "tuple") + return; + + if (const auto * inner_list = func->children.front()->as()) + { + auto new_order_by = std::make_shared(); + for (const auto & identifier : inner_list->children) + { + auto clone = std::make_shared(*order_by_elem); + clone->children.emplace_back(identifier); + new_order_by->children.emplace_back(clone); + } + if (!new_order_by->children.empty()) + query->setExpression(ASTSelectQuery::Expression::ORDER_BY, std::move(new_order_by)); + } +} + +} + diff --git a/src/Interpreters/RewriteOrderByVisitor.hpp b/src/Interpreters/RewriteOrderByVisitor.hpp new file mode 100644 index 00000000000..e76f47d7a77 --- /dev/null +++ b/src/Interpreters/RewriteOrderByVisitor.hpp @@ -0,0 +1,34 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// rewrite (1) to (2) +/// (1) ... ORDER BY (a, b) +/// │ ExpressionList (children 1) │ +/// │ OrderByElement (children 1) │ +/// │ Function tuple (children 1) │ +/// │ ExpressionList (children 2) │ +/// │ Identifier CounterID │ +/// │ Identifier EventDate │ +/// (2) ... ORDER BY a,b +/// │ ExpressionList (children 2) │ +/// │ OrderByElement (children 1) │ +/// │ Identifier CounterID │ +/// │ OrderByElement (children 1) │ +/// │ Identifier EventDate │ +/// +class RewriteOrderBy +{ +public: + struct Data {}; + static void visit(ASTPtr & ast, Data &); + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } +}; + +using RewriteOrderByVisitor = InDepthNodeVisitor; +} + diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 9a4a956f3dc..aa1f79e3cbc 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include @@ -34,6 +35,7 @@ #include #include #include +#include #include #include @@ -1326,6 +1328,10 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( !select_query->sampleSize() && !select_query->sampleOffset() && !select_query->final() && (tables_with_columns.size() < 2 || isLeft(result.analyzed_join->kind())); + // remove outer braces in order by + RewriteOrderByVisitor::Data data; + RewriteOrderByVisitor(data).visit(query); + return std::make_shared(result); } diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index bd45dd7fc05..457a3361b1e 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -17,7 +17,7 @@ public: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - virtual QueryKind getQueryKind() const override { return QueryKind::Select; } + QueryKind getQueryKind() const override { return QueryKind::Select; } SelectUnionMode union_mode; diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 89a5ecb4cfa..b4742c9444e 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -159,6 +159,22 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build } else if (type == Type::Full) { + if (input_streams.back().sort_mode == DataStream::SortMode::Chunk + && input_streams.back().sort_description == result_description) + { + assert(pipeline.getNumStreams() > 1); + + auto transform = std::make_shared( + pipeline.getHeader(), + pipeline.getNumStreams(), + result_description, + max_block_size, + limit); + + pipeline.addTransform(std::move(transform)); + return; + } + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) @@ -176,8 +192,7 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - auto transform = std::make_shared(header, limits); - return transform; + return std::make_shared(header, limits); }); bool increase_sort_description_compile_attempts = true; From 5a01c27eb1770eb28254c2e8cb11e303ace70c28 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 5 Jul 2022 21:05:10 +0000 Subject: [PATCH 254/627] Fix: clear children in order by element during rewriting - remove unnecessary code for this PR --- src/Interpreters/ExpressionAnalyzer.cpp | 3 +- src/Interpreters/InterpreterSelectQuery.cpp | 47 +++------------------ src/Interpreters/RewriteOrderByVisitor.cpp | 6 ++- src/Interpreters/RewriteOrderByVisitor.hpp | 5 ++- src/Interpreters/TreeRewriter.cpp | 1 - src/Parsers/ASTSelectWithUnionQuery.h | 2 +- src/Processors/QueryPlan/SortingStep.cpp | 19 +-------- 7 files changed, 18 insertions(+), 65 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 76cace4e8e1..8d0c4dee023 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1539,8 +1539,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai if (optimize_read_in_order) { - const ASTPtr & order_by = select_query->orderBy(); - for (const auto & child : order_by->children) + for (auto & child : select_query->orderBy()->children) { auto actions_dag = std::make_shared(columns_after_join); getRootActions(child, only_types, actions_dag); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8d396366688..f3dbb19d41e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -888,27 +888,14 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query, const ContextPtr & context_) { - /// Expected AST from query.orderBy() - /// (1) ... ORDER BY a,b - /// │ ExpressionList (children 2) │ - /// │ OrderByElement (children 1) │ - /// │ Identifier CounterID │ - /// │ OrderByElement (children 1) │ - /// │ Identifier EventDate │ - /// - /// (2) ... ORDER BY (a, b) - /// │ ExpressionList (children 1) │ - /// │ OrderByElement (children 1) │ - /// │ Function tuple (children 1) │ - /// │ ExpressionList (children 2) │ - /// │ Identifier CounterID │ - /// │ Identifier EventDate │ - /// SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); - auto add_sort_column_desc = [&](const ASTOrderByElement & order_by_elem, const String & column) + for (const auto & elem : query.orderBy()->children) { + const String & column_name = elem->children.front()->getColumnName(); + const auto & order_by_elem = elem->as(); + std::shared_ptr collator; if (order_by_elem.collation) collator = std::make_shared(order_by_elem.collation->as().value.get()); @@ -916,34 +903,12 @@ SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery if (order_by_elem.with_fill) { FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context_); - order_descr.emplace_back(column, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc); + order_descr.emplace_back(column_name, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc); } else - order_descr.emplace_back(column, order_by_elem.direction, order_by_elem.nulls_direction, collator); - }; - - for (const auto & elem : query.orderBy()->children) - { - const auto & order_by_elem = elem->as(); - /// case (2) - if (const auto* func = order_by_elem.children.front()->as()) - { - if (const auto* expr_list = func->children.front()->as()) - { - for(const auto& identifier : expr_list->children) - add_sort_column_desc(order_by_elem, identifier->getColumnName()); - } - } - /// case (1) - else - { - add_sort_column_desc(order_by_elem, elem->children.front()->getColumnName()); - } + order_descr.emplace_back(column_name, order_by_elem.direction, order_by_elem.nulls_direction, collator); } - if (order_descr.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to build sort description for ORDER BY"); - order_descr.compile_sort_description = context_->getSettingsRef().compile_sort_description; order_descr.min_count_to_compile_sort_description = context_->getSettingsRef().min_count_to_compile_sort_description; diff --git a/src/Interpreters/RewriteOrderByVisitor.cpp b/src/Interpreters/RewriteOrderByVisitor.cpp index 67788a61e71..634b2ce2290 100644 --- a/src/Interpreters/RewriteOrderByVisitor.cpp +++ b/src/Interpreters/RewriteOrderByVisitor.cpp @@ -4,7 +4,8 @@ #include #include -namespace DB { +namespace DB +{ void RewriteOrderBy::visit(ASTPtr & ast, Data &) { @@ -33,7 +34,10 @@ void RewriteOrderBy::visit(ASTPtr & ast, Data &) auto new_order_by = std::make_shared(); for (const auto & identifier : inner_list->children) { + // clone w/o children auto clone = std::make_shared(*order_by_elem); + clone->children.clear(); + clone->children.emplace_back(identifier); new_order_by->children.emplace_back(clone); } diff --git a/src/Interpreters/RewriteOrderByVisitor.hpp b/src/Interpreters/RewriteOrderByVisitor.hpp index e76f47d7a77..a3857e046ca 100644 --- a/src/Interpreters/RewriteOrderByVisitor.hpp +++ b/src/Interpreters/RewriteOrderByVisitor.hpp @@ -5,8 +5,9 @@ namespace DB { - -/// rewrite (1) to (2) +/// +/// Remove outer braces in ORDER BY +/// For example, rewrite (1) to (2) /// (1) ... ORDER BY (a, b) /// │ ExpressionList (children 1) │ /// │ OrderByElement (children 1) │ diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index aa1f79e3cbc..b57433fb0c6 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -35,7 +35,6 @@ #include #include #include -#include #include #include diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 457a3361b1e..bd45dd7fc05 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -17,7 +17,7 @@ public: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - QueryKind getQueryKind() const override { return QueryKind::Select; } + virtual QueryKind getQueryKind() const override { return QueryKind::Select; } SelectUnionMode union_mode; diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index b4742c9444e..89a5ecb4cfa 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -159,22 +159,6 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build } else if (type == Type::Full) { - if (input_streams.back().sort_mode == DataStream::SortMode::Chunk - && input_streams.back().sort_description == result_description) - { - assert(pipeline.getNumStreams() > 1); - - auto transform = std::make_shared( - pipeline.getHeader(), - pipeline.getNumStreams(), - result_description, - max_block_size, - limit); - - pipeline.addTransform(std::move(transform)); - return; - } - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) @@ -192,7 +176,8 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return std::make_shared(header, limits); + auto transform = std::make_shared(header, limits); + return transform; }); bool increase_sort_description_compile_attempts = true; From 946a9b8c634b16b282e0bc163df768dae3c5e2e6 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 6 Jul 2022 09:26:02 +0800 Subject: [PATCH 255/627] update codes --- src/Storages/Hive/StorageHive.cpp | 10 ---------- src/Storages/Hive/StorageHive.h | 1 - 2 files changed, 11 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 6e52b002bf0..169aeea4507 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -922,16 +922,6 @@ void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPt } } -void StorageHive::alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder & /*alter_lock_holder*/) -{ - auto table_id = getStorageID(); - auto metadata_snapshot = getInMemoryMetadataPtr(); - StorageInMemoryMetadata new_metadata = *metadata_snapshot; - params.apply(new_metadata, local_context); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); - setInMemoryMetadata(new_metadata); -} - std::optional StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const { diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 6accc9fc33c..efc744c6a9f 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -71,7 +71,6 @@ public: std::optional totalRows(const Settings & settings) const override; std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const override; void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override; - void alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder & alter_lock_holder) override; private: using FileFormat = IHiveFile::FileFormat; From 19365597ebc479941a516f0c56f2f5a2cb51bc9f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 5 Jul 2022 23:37:01 -0400 Subject: [PATCH 256/627] move Simple Check to python --- .github/workflows/pr-pass.yml | 21 --------------------- .github/workflows/pull_request.yml | 11 ----------- tests/ci/fast_test_check.py | 14 ++++++++++++-- tests/ci/pr_info.py | 1 + tests/ci/run_check.py | 4 ++++ tests/ci/style_check.py | 14 ++++++++++++-- 6 files changed, 29 insertions(+), 36 deletions(-) delete mode 100644 .github/workflows/pr-pass.yml diff --git a/.github/workflows/pr-pass.yml b/.github/workflows/pr-pass.yml deleted file mode 100644 index 2fbb2fe9e0d..00000000000 --- a/.github/workflows/pr-pass.yml +++ /dev/null @@ -1,21 +0,0 @@ -### Сomplementary to PullRequestCI (pull_request.yml) - should pass -name: PRPassCI -on: - pull_request: - types: - - synchronize - - reopened - - opened - branches: - - master - # Сomplementary paths - paths: - - 'docker/docs/**' - - 'docs/**' - - 'website/**' -jobs: - SimpleCheck: - runs-on: ubuntu-latest - steps: - - name: Pass - run: exit 0 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 8301faa149d..9cd8fd6f49d 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -179,17 +179,6 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - SimpleCheck: - needs: [StyleCheck, FastTest] - runs-on: ubuntu-latest - if: ${{ always() }} - steps: - - name: Allow simple fail - if: ${{ contains(github.event.pull_request.labels.*.name, 'skip simple check') }} - run: exit 0 - - name: Simple failed - if: ${{ (needs.StyleCheck.result != 'success' || needs.FastTest.result != 'success') && !contains(github.event.pull_request.labels.*.name, 'skip simple check') }} - run: exit 1 CompatibilityCheck: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index acc07ccfb23..490b9aa83aa 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -8,8 +8,9 @@ import sys from github import Github -from env_helper import CACHES_PATH, TEMP_PATH -from pr_info import FORCE_TESTS_LABEL, PRInfo +from commit_status_helper import get_commit +from env_helper import CACHES_PATH, TEMP_PATH, GITHUB_SERVER_URL, GITHUB_REPOSITORY +from pr_info import FORCE_TESTS_LABEL, PRInfo, SKIP_SIMPLE_CHECK_LABEL from s3_helper import S3Helper from get_robot_token import get_best_robot_token from upload_result_helper import upload_results @@ -219,4 +220,13 @@ if __name__ == "__main__": if FORCE_TESTS_LABEL in pr_info.labels and state != "error": print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") else: + if SKIP_SIMPLE_CHECK_LABEL not in pr_info.labels: + url = ( + f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/" + "blob/master/.github/PULL_REQUEST_TEMPLATE.md?plain=1" + ) + commit = get_commit(gh, pr_info.sha) + commit.create_status( + context="Simple Check", description=f"{NAME} failed", state="failed", target_url=url + ) sys.exit(1) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index a06c5a75b0f..319cdb70d77 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -15,6 +15,7 @@ from env_helper import ( ) FORCE_TESTS_LABEL = "force tests" +SKIP_SIMPLE_CHECK_LABEL = "skip simple check" DIFF_IN_DOCUMENTATION_EXT = [ ".html", diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 87139c5bb8a..49bb226d1b1 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -223,6 +223,10 @@ if __name__ == "__main__": if pr_labels_to_remove: remove_labels(gh, pr_info, pr_labels_to_remove) + commit.create_status( + context="Simple Check", description="Skipped", state="success", target_url=GITHUB_RUN_URL + ) + if description_error: print( "::error ::Cannot run, PR description does not match the template: " diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 198db86ae6a..e41758bb5e1 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -7,9 +7,10 @@ import sys from github import Github -from env_helper import RUNNER_TEMP, GITHUB_WORKSPACE +from commit_status_helper import get_commit +from env_helper import RUNNER_TEMP, GITHUB_WORKSPACE, GITHUB_REPOSITORY, GITHUB_SERVER_URL from s3_helper import S3Helper -from pr_info import PRInfo +from pr_info import PRInfo, SKIP_SIMPLE_CHECK_LABEL from get_robot_token import get_best_robot_token from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version @@ -120,4 +121,13 @@ if __name__ == "__main__": ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) if state == "error": + if SKIP_SIMPLE_CHECK_LABEL not in pr_info.labels: + url = ( + f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/" + "blob/master/.github/PULL_REQUEST_TEMPLATE.md?plain=1" + ) + commit = get_commit(gh, pr_info.sha) + commit.create_status( + context="Simple Check", description=f"{NAME} failed", state="failed", target_url=url + ) sys.exit(1) From d89ba2e5d98600e1bf682b987339f132b8a6b6cb Mon Sep 17 00:00:00 2001 From: HeenaBansal2009 Date: Tue, 5 Jul 2022 21:18:39 -0700 Subject: [PATCH 257/627] Review Comments --- src/Common/parseGlobs.cpp | 25 +++++++----- .../gtest_makeRegexpPatternFromGlobs.cpp | 27 +++++++++++-- .../02297_regex_parsing_file_names.reference | 1 + .../02297_regex_parsing_file_names.sh | 38 +++++++++++++++++++ 4 files changed, 78 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/02297_regex_parsing_file_names.reference create mode 100755 tests/queries/0_stateless/02297_regex_parsing_file_names.sh diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index 9e18a9c0780..8e9195f9842 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -32,7 +32,7 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob } std::string escaped_with_globs = buf_for_escaping.str(); - static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without {}*, + static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without "{", "}", "*" and "," re2::StringPiece input(escaped_with_globs); re2::StringPiece matched; std::ostringstream oss_for_replacing; // STYLE_CHECK_ALLOW_STD_STRING_STREAM @@ -50,25 +50,32 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob char point; ReadBufferFromString buf_range(buffer); buf_range >> range_begin >> point >> point >> range_end; - bool leading_zeros = buffer[0] == '0'; size_t range_begin_width = buffer.find('.'); size_t range_end_width = buffer.size() - buffer.find_last_of('.') - 1; - //Scenarios {0..10} {0..999} - size_t num_len = 0; + bool leading_zeros = buffer[0] == '0'; + size_t output_width = 0; + + if (range_begin > range_end) //Descending Sequence {20..15} {9..01} + { + std::swap(range_begin,range_end); + leading_zeros = buffer[buffer.find_last_of('.')+1]=='0'; + std::swap(range_begin_width,range_end_width); + } if (range_begin_width == 1 && leading_zeros) - num_len = 1; - //Scenarios {00..99} {00..099} + output_width = 1; ///Special Case: {0..10} {0..999} else - num_len = range_begin_width < range_end_width ? range_end_width : range_begin_width; + output_width = std::max(range_begin_width, range_end_width); + if (leading_zeros) - oss_for_replacing << std::setfill('0') << std::setw(num_len); + oss_for_replacing << std::setfill('0') << std::setw(output_width); oss_for_replacing << range_begin; + for (size_t i = range_begin + 1; i <= range_end; ++i) { oss_for_replacing << '|'; if (leading_zeros) - oss_for_replacing << std::setfill('0') << std::setw(num_len); + oss_for_replacing << std::setfill('0') << std::setw(output_width); oss_for_replacing << i; } } diff --git a/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp b/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp index 833a0a6ba73..fda3a6ee1c8 100644 --- a/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp +++ b/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp @@ -8,21 +8,40 @@ using namespace DB; TEST(Common, makeRegexpPatternFromGlobs) { + EXPECT_EQ(makeRegexpPatternFromGlobs("?"), "[^/]"); EXPECT_EQ(makeRegexpPatternFromGlobs("*"), "[^/]*"); EXPECT_EQ(makeRegexpPatternFromGlobs("/?"), "/[^/]"); EXPECT_EQ(makeRegexpPatternFromGlobs("/*"), "/[^/]*"); EXPECT_EQ(makeRegexpPatternFromGlobs("*_{{a,b,c,d}}/?.csv"), "[^/]*_\\{(a|b|c|d)\\}/[^/]\\.csv"); - EXPECT_EQ(makeRegexpPatternFromGlobs("f{01..9}"), "f(01|02|03|04|05|06|07|08|09)"); - EXPECT_EQ(makeRegexpPatternFromGlobs("f{001..0009}"), "f(0001|0002|0003|0004|0005|0006|0007|0008|0009)"); + /* Regex Parsing for {..} can have three possible cases + 1) The left range width == the right range width + 2) The left range width > the right range width + 3) The left range width < the right range width + */ + // Ascending Sequences + EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..9}"), "f(1|2|3|4|5|6|7|8|9)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{0..10}"), "f(0|1|2|3|4|5|6|7|8|9|10)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{10..20}"), "f(10|11|12|13|14|15|16|17|18|19|20)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{00..10}"), "f(00|01|02|03|04|05|06|07|08|09|10)"); - EXPECT_EQ(makeRegexpPatternFromGlobs("f{000..9}"), "f(000|001|002|003|004|005|006|007|008|009)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{0001..0009}"), "f(0001|0002|0003|0004|0005|0006|0007|0008|0009)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{01..9}"), "f(01|02|03|04|05|06|07|08|09)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{000..9}"), "f(000|001|002|003|004|005|006|007|008|009)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{95..103}"), "f(95|96|97|98|99|100|101|102|103)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{99..109}"), "f(99|100|101|102|103|104|105|106|107|108|109)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{001..0009}"), "f(0001|0002|0003|0004|0005|0006|0007|0008|0009)"); + // Descending Sequences + EXPECT_EQ(makeRegexpPatternFromGlobs("f{20..15}"), "f(15|16|17|18|19|20)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{200..199}"), "f(199|200)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{0009..0001}"), "f(0001|0002|0003|0004|0005|0006|0007|0008|0009)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{100..90}"), "f(90|91|92|93|94|95|96|97|98|99|100)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{103..95}"), "f(95|96|97|98|99|100|101|102|103)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{9..01}"), "f(01|02|03|04|05|06|07|08|09)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{9..000}"), "f(000|001|002|003|004|005|006|007|008|009)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..2}{1..2}"), "f(1|2)(1|2)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..1}{1..1}"), "f(1)(1)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{0..0}{0..0}"), "f(0)(0)"); EXPECT_EQ(makeRegexpPatternFromGlobs("file{1..5}"),"file(1|2|3|4|5)"); EXPECT_EQ(makeRegexpPatternFromGlobs("file{1,2,3}"),"file(1|2|3)"); EXPECT_EQ(makeRegexpPatternFromGlobs("{1,2,3}blabla{a.x,b.x,c.x}smth[]_else{aa,bb}?*"), "(1|2|3)blabla(a\\.x|b\\.x|c\\.x)smth\\[\\]_else(aa|bb)[^/][^/]*"); -} \ No newline at end of file +} diff --git a/tests/queries/0_stateless/02297_regex_parsing_file_names.reference b/tests/queries/0_stateless/02297_regex_parsing_file_names.reference new file mode 100644 index 00000000000..b4de3947675 --- /dev/null +++ b/tests/queries/0_stateless/02297_regex_parsing_file_names.reference @@ -0,0 +1 @@ +11 diff --git a/tests/queries/0_stateless/02297_regex_parsing_file_names.sh b/tests/queries/0_stateless/02297_regex_parsing_file_names.sh new file mode 100755 index 00000000000..2db4ae8044c --- /dev/null +++ b/tests/queries/0_stateless/02297_regex_parsing_file_names.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Data preparation. + +# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: +# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" +CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/ + +echo '{"obj": "aaa", "id": 1, "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_0.json +echo '{"id": 2, "obj": "bbb", "s": "bar"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_1.json +echo '{"id": 3, "obj": "ccc", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_2.json +echo '{"id": 4, "obj": "ddd", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_3.json +echo '{"id": 5, "obj": "eee", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_4.json +echo '{"id": 6, "obj": "fff", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_5.json +echo '{"id": 7, "obj": "ggg", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_6.json +echo '{"id": 8, "obj": "hhh", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_7.json +echo '{"id": 9, "obj": "iii", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_8.json +echo '{"id": 10, "obj":"jjj", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_9.json +echo '{"id": 11, "obj": "kkk", "s": "foo"}'>> ${CLICKHOUSE_USER_FILES_PATH}/file_10.json + + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_regex" + +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_regex (id UInt64, obj String, s String) ENGINE =File(JSONEachRow)" ; + + +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_regex SELECT * FROM file('file_{0..10}.json','JSONEachRow')"; +${CLICKHOUSE_CLIENT} -q "SELECT count() from t_regex" + +rm -rf ${CLICKHOUSE_USER_FILES_PATH}/file_*.json; +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_regex" From 69d1495672bd917a416623bce8c97cd2f023ae66 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 6 Jul 2022 00:22:50 -0400 Subject: [PATCH 258/627] style fix --- tests/ci/fast_test_check.py | 8 +++++--- tests/ci/run_check.py | 5 ++++- tests/ci/style_check.py | 15 +++++++++++---- 3 files changed, 20 insertions(+), 8 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 490b9aa83aa..e66317da84b 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -8,14 +8,13 @@ import sys from github import Github -from commit_status_helper import get_commit from env_helper import CACHES_PATH, TEMP_PATH, GITHUB_SERVER_URL, GITHUB_REPOSITORY from pr_info import FORCE_TESTS_LABEL, PRInfo, SKIP_SIMPLE_CHECK_LABEL from s3_helper import S3Helper from get_robot_token import get_best_robot_token from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version -from commit_status_helper import post_commit_status +from commit_status_helper import post_commit_status, get_commit from clickhouse_helper import ( ClickHouseHelper, mark_flaky_tests, @@ -227,6 +226,9 @@ if __name__ == "__main__": ) commit = get_commit(gh, pr_info.sha) commit.create_status( - context="Simple Check", description=f"{NAME} failed", state="failed", target_url=url + context="Simple Check", + description=f"{NAME} failed", + state="failed", + target_url=url ) sys.exit(1) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 49bb226d1b1..ed57362effb 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -224,7 +224,10 @@ if __name__ == "__main__": remove_labels(gh, pr_info, pr_labels_to_remove) commit.create_status( - context="Simple Check", description="Skipped", state="success", target_url=GITHUB_RUN_URL + context="Simple Check", + description="Skipped", + state="success", + target_url=GITHUB_RUN_URL ) if description_error: diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index e41758bb5e1..62d3f5fd4ba 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -7,14 +7,18 @@ import sys from github import Github -from commit_status_helper import get_commit -from env_helper import RUNNER_TEMP, GITHUB_WORKSPACE, GITHUB_REPOSITORY, GITHUB_SERVER_URL +from env_helper import ( + RUNNER_TEMP, + GITHUB_WORKSPACE, + GITHUB_REPOSITORY, + GITHUB_SERVER_URL +) from s3_helper import S3Helper from pr_info import PRInfo, SKIP_SIMPLE_CHECK_LABEL from get_robot_token import get_best_robot_token from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version -from commit_status_helper import post_commit_status +from commit_status_helper import post_commit_status, get_commit from clickhouse_helper import ( ClickHouseHelper, mark_flaky_tests, @@ -128,6 +132,9 @@ if __name__ == "__main__": ) commit = get_commit(gh, pr_info.sha) commit.create_status( - context="Simple Check", description=f"{NAME} failed", state="failed", target_url=url + context="Simple Check", + description=f"{NAME} failed", + state="failed", + target_url=url ) sys.exit(1) From d6269c04bb9cb4d25bbb01b360219c63880d3949 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 6 Jul 2022 00:40:51 -0400 Subject: [PATCH 259/627] style fix --- tests/ci/fast_test_check.py | 2 +- tests/ci/run_check.py | 2 +- tests/ci/style_check.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index e66317da84b..ce5a4195ceb 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -229,6 +229,6 @@ if __name__ == "__main__": context="Simple Check", description=f"{NAME} failed", state="failed", - target_url=url + target_url=url, ) sys.exit(1) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index ed57362effb..b6d654c7bed 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -227,7 +227,7 @@ if __name__ == "__main__": context="Simple Check", description="Skipped", state="success", - target_url=GITHUB_RUN_URL + target_url=GITHUB_RUN_URL, ) if description_error: diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 62d3f5fd4ba..84ed9e5a124 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -11,7 +11,7 @@ from env_helper import ( RUNNER_TEMP, GITHUB_WORKSPACE, GITHUB_REPOSITORY, - GITHUB_SERVER_URL + GITHUB_SERVER_URL, ) from s3_helper import S3Helper from pr_info import PRInfo, SKIP_SIMPLE_CHECK_LABEL @@ -135,6 +135,6 @@ if __name__ == "__main__": context="Simple Check", description=f"{NAME} failed", state="failed", - target_url=url + target_url=url, ) sys.exit(1) From af1c3d35fdc35d821bb7316d382b196ea1512df8 Mon Sep 17 00:00:00 2001 From: xinhuitian Date: Wed, 6 Jul 2022 13:32:21 +0800 Subject: [PATCH 260/627] make key-value zh doc more readable --- docs/zh/faq/use-cases/key-value.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/zh/faq/use-cases/key-value.md b/docs/zh/faq/use-cases/key-value.md index 4a611512586..79c221d30c8 100644 --- a/docs/zh/faq/use-cases/key-value.md +++ b/docs/zh/faq/use-cases/key-value.md @@ -5,12 +5,12 @@ sidebar_position: 101 --- # 我能把 ClickHouse 当做Key-value 键值存储来使用吗? {#can-i-use-clickhouse-as-a-key-value-storage}. -简短的回答是 **不能** 。关键值的工作量是在列表中的最高位置时,**不能**{.text-danger}使用ClickHouse的情况。它是一个[OLAP](../../faq/general/olap.md)系统,毕竟有很多优秀的键值存储系统在那里。 +简短的回答是 **不能** 。键值类型负载是ClickHouse最**不适合**的多种场景之一。ClickHouse 毕竟只是一个[OLAP](../../faq/general/olap.md)系统,对于这类负载来说,目前还是有很多优秀的键值存储系统可供选择。 -然而,可能在某些情况下,使用ClickHouse进行类似键值的查询仍然是有意义的。通常,是一些低预算的产品,主要的工作负载是分析性的,很适合ClickHouse,但也有一些次要的过程需要一个键值模式,请求吞吐量不是很高,没有严格的延迟要求。如果你有无限的预算,你会为这样的次要工作负载安装一个次要的键值数据库,但实际上,多维护一个存储系统(监控、备份等)会有额外的成本,这可能是值得避免的。 +然而,在某些情况下,使用ClickHouse进行类似键值的查询仍然是有意义的。通常,一些主要的工作负载是分析性的比较适合使用Clickhouse低预算的产品中,也有一些次要的操作是需要使用键值模式的,同时这些操作的请求吞吐量不会很高,也没有严格的延迟要求。如果你有无限的预算,你会为这样的次要工作负载安装一个次要的键值数据库,但实际上,多维护一个存储系统(监控、备份等)会有额外的成本,这是可以考虑避免的。 -如果你决定违背建议,对ClickHouse运行一些类似键值的查询,这里有一些提示。 +如果你决定不遵从这些建议,想要使用ClickHouse运行一些类似键值的查询,那么这里有一些提示。 -- ClickHouse中点查询昂贵的关键原因是其稀疏的主索引[MergeTree表引擎家族](../../engines/table-engines/mergetree-family/mergetree.md)。这个索引不能指向每一行具体的数据,相反,它指向每N行,系统必须从邻近的N行扫描到所需的行,沿途读取过多的数据。在一个键值场景中,通过`index_granularity`的设置来减少N的值可能是有用的。 -- ClickHouse将每一列保存在一组单独的文件中,所以要组装一个完整的行,它需要通过这些文件中的每一个。它们的数量随着列数的增加而线性增加,所以在键值场景中,可能值得避免使用许多列,并将所有的有效数据放在一个单一的`String`列中,并以某种序列化格式(如JSON、Protobuf或任何有效的格式)进行编码。 +- ClickHouse中点查询开销大的关键原因是MergeTree表引擎家族[MergeTree表引擎家族](../../engines/table-engines/mergetree-family/mergetree.md)采用的稀疏主索引。这个索引不能指向每一行具体的数据,相反,它指向每N行,系统必须从邻近的N行扫描到所需的行,沿途读取过多的数据。在一个键值场景中,通过`index_granularity`的设置来减少N的值可能是有用的。 +- ClickHouse将每一列保存在一组单独的文件中,所以要组装一个完整的行,它需要访问文件组中的每一个文件。访问数据数量会随着列数的增加而线性增加,所以在键值场景中,需要避免使用许多列,并将所有的有效数据放在一个单一的`String`列中,并以某种序列化格式(如JSON、Protobuf或任何有效的格式)进行编码。 - 还有一种方法,使用[Join](../../engines/table-engines/special/join.md)表引擎代替正常的`MergeTree`表和[joinGet](../../sql-reference/functions/other-functions.md#joinget) 函数来检索数据。它可以提供更好的查询性能,但可能有一些可用性和可靠性问题。下面是一个[使用实例](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00800_versatile_storage_join.sql#L49-L51)。 From a4ea7a39aa503a83d23c92070199c52cc988a115 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 5 Jul 2022 17:02:25 +0300 Subject: [PATCH 261/627] Fix alignment in PreLoad.cmake Signed-off-by: Azat Khuzhin --- PreLoad.cmake | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/PreLoad.cmake b/PreLoad.cmake index 8ef93a7aa51..2040d147da8 100644 --- a/PreLoad.cmake +++ b/PreLoad.cmake @@ -87,8 +87,7 @@ if (OS MATCHES "Linux" set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-aarch64.cmake" CACHE INTERNAL "") elseif (ARCH MATCHES "^(ppc64le.*|PPC64LE.*)") set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-ppc64le.cmake" CACHE INTERNAL "") -else () + else () message (FATAL_ERROR "Unsupported architecture: ${ARCH}") endif () - endif() From dc28dfe57e836d87a3fdaf354aaab4c42cb66da4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 5 Jul 2022 17:02:34 +0300 Subject: [PATCH 262/627] Do not override compiler if it had been already set Before this patch: cmake -DCMAKE_C_COMPILER=foo .. # will use foo cmake .. # will use clang After: cmake -DCMAKE_C_COMPILER=foo .. # will use foo cmake .. # will use foo Signed-off-by: Azat Khuzhin --- PreLoad.cmake | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/PreLoad.cmake b/PreLoad.cmake index 2040d147da8..95f65b85f7f 100644 --- a/PreLoad.cmake +++ b/PreLoad.cmake @@ -62,9 +62,10 @@ execute_process(COMMAND uname -m OUTPUT_VARIABLE ARCH) # By default, prefer clang on Linux # But note, that you still may change the compiler with -DCMAKE_C_COMPILER/-DCMAKE_CXX_COMPILER. if (OS MATCHES "Linux" - # some build systems may use CC/CXX env variables AND "$ENV{CC}" STREQUAL "" - AND "$ENV{CXX}" STREQUAL "") + AND "$ENV{CXX}" STREQUAL "" + AND NOT DEFINED CMAKE_C_COMPILER + AND NOT DEFINED CMAKE_CXX_COMPILER) find_program(CLANG_PATH clang) if (CLANG_PATH) set(CMAKE_C_COMPILER "clang" CACHE INTERNAL "") From 5d7ad46f6a55bcb2ffe2a9b0f67ef86ab141d5f2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 6 Jul 2022 11:09:31 +0200 Subject: [PATCH 263/627] Move files and write comments. --- src/Backups/BackupCoordinationLocal.cpp | 15 +- src/Backups/BackupCoordinationLocal.h | 15 +- ...buted.cpp => BackupCoordinationRemote.cpp} | 187 ++++++++------- ...stributed.h => BackupCoordinationRemote.h} | 16 +- .../BackupCoordinationReplicatedAccess.cpp | 33 +++ .../BackupCoordinationReplicatedAccess.h | 49 ++++ ...=> BackupCoordinationReplicatedTables.cpp} | 215 +++--------------- ...h => BackupCoordinationReplicatedTables.h} | 96 ++++---- src/Backups/BackupCoordinationStatusSync.cpp | 161 +++++++++++++ src/Backups/BackupCoordinationStatusSync.h | 36 +++ src/Backups/BackupEntriesCollector.cpp | 3 +- src/Backups/BackupImpl.cpp | 2 +- src/Backups/BackupsWorker.cpp | 8 +- src/Backups/IBackupCoordination.h | 8 +- src/Backups/IRestoreCoordination.h | 7 +- src/Backups/RestoreCoordinationLocal.h | 1 + ...uted.cpp => RestoreCoordinationRemote.cpp} | 24 +- ...tributed.h => RestoreCoordinationRemote.h} | 10 +- src/Backups/RestorerFromBackup.cpp | 3 +- 19 files changed, 503 insertions(+), 386 deletions(-) rename src/Backups/{BackupCoordinationDistributed.cpp => BackupCoordinationRemote.cpp} (77%) rename src/Backups/{BackupCoordinationDistributed.h => BackupCoordinationRemote.h} (81%) create mode 100644 src/Backups/BackupCoordinationReplicatedAccess.cpp create mode 100644 src/Backups/BackupCoordinationReplicatedAccess.h rename src/Backups/{BackupCoordinationHelpers.cpp => BackupCoordinationReplicatedTables.cpp} (59%) rename src/Backups/{BackupCoordinationHelpers.h => BackupCoordinationReplicatedTables.h} (54%) create mode 100644 src/Backups/BackupCoordinationStatusSync.cpp create mode 100644 src/Backups/BackupCoordinationStatusSync.h rename src/Backups/{RestoreCoordinationDistributed.cpp => RestoreCoordinationRemote.cpp} (66%) rename src/Backups/{RestoreCoordinationDistributed.h => RestoreCoordinationRemote.h} (82%) diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 8315de95775..b1525060f7d 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -30,42 +30,39 @@ Strings BackupCoordinationLocal::setStatusAndWaitFor(const String &, const Strin void BackupCoordinationLocal::addReplicatedPartNames(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) { std::lock_guard lock{mutex}; - replicated_parts_and_mutations.addPartNames(table_shared_id, table_name_for_logs, replica_name, part_names_and_checksums); + replicated_tables.addPartNames(table_shared_id, table_name_for_logs, replica_name, part_names_and_checksums); } Strings BackupCoordinationLocal::getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const { std::lock_guard lock{mutex}; - return replicated_parts_and_mutations.getPartNames(table_shared_id, replica_name); + return replicated_tables.getPartNames(table_shared_id, replica_name); } void BackupCoordinationLocal::addReplicatedMutations(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) { std::lock_guard lock{mutex}; - replicated_parts_and_mutations.addMutations(table_shared_id, table_name_for_logs, replica_name, mutations); + replicated_tables.addMutations(table_shared_id, table_name_for_logs, replica_name, mutations); } std::vector BackupCoordinationLocal::getReplicatedMutations(const String & table_shared_id, const String & replica_name) const { std::lock_guard lock{mutex}; - return replicated_parts_and_mutations.getMutations(table_shared_id, replica_name); + return replicated_tables.getMutations(table_shared_id, replica_name); } void BackupCoordinationLocal::addReplicatedDataPath(const String & table_shared_id, const String & data_path) { std::lock_guard lock{mutex}; - replicated_data_paths[table_shared_id].push_back(data_path); + replicated_tables.addDataPath(table_shared_id, data_path); } Strings BackupCoordinationLocal::getReplicatedDataPaths(const String & table_shared_id) const { std::lock_guard lock{mutex}; - auto it = replicated_data_paths.find(table_shared_id); - if (it == replicated_data_paths.end()) - return {}; - return it->second; + return replicated_tables.getDataPaths(table_shared_id); } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 35218c426dc..afe7638339d 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -1,7 +1,8 @@ #pragma once #include -#include +#include +#include #include #include #include @@ -12,7 +13,7 @@ namespace Poco { class Logger; } namespace DB { -/// Stores backup contents information in memory. +/// Implementation of the IBackupCoordination interface performing coordination in memory. class BackupCoordinationLocal : public IBackupCoordination { public: @@ -52,15 +53,8 @@ public: Strings getAllArchiveSuffixes() const override; private: - struct ReplicatedAccessPath - { - Strings file_paths; - String host_to_store_access; - }; - mutable std::mutex mutex; - BackupCoordinationReplicatedPartsAndMutations replicated_parts_and_mutations TSA_GUARDED_BY(mutex); - std::unordered_map replicated_data_paths TSA_GUARDED_BY(mutex); + BackupCoordinationReplicatedTables replicated_tables TSA_GUARDED_BY(mutex); BackupCoordinationReplicatedAccess replicated_access TSA_GUARDED_BY(mutex); std::map file_names TSA_GUARDED_BY(mutex); /// Should be ordered alphabetically, see listFiles(). For empty files we assume checksum = 0. std::map file_infos TSA_GUARDED_BY(mutex); /// Information about files. Without empty files. @@ -68,5 +62,4 @@ private: size_t current_archive_suffix TSA_GUARDED_BY(mutex) = 0; }; - } diff --git a/src/Backups/BackupCoordinationDistributed.cpp b/src/Backups/BackupCoordinationRemote.cpp similarity index 77% rename from src/Backups/BackupCoordinationDistributed.cpp rename to src/Backups/BackupCoordinationRemote.cpp index 9f2afff11ed..b9ce43291c5 100644 --- a/src/Backups/BackupCoordinationDistributed.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -165,7 +165,7 @@ namespace constexpr size_t NUM_ATTEMPTS = 10; } -BackupCoordinationDistributed::BackupCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_) +BackupCoordinationRemote::BackupCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) , status_sync(zookeeper_path_ + "/status", get_zookeeper_, &Poco::Logger::get("BackupCoordination")) @@ -173,9 +173,9 @@ BackupCoordinationDistributed::BackupCoordinationDistributed(const String & zook createRootNodes(); } -BackupCoordinationDistributed::~BackupCoordinationDistributed() = default; +BackupCoordinationRemote::~BackupCoordinationRemote() = default; -void BackupCoordinationDistributed::createRootNodes() +void BackupCoordinationRemote::createRootNodes() { auto zookeeper = get_zookeeper(); zookeeper->createAncestors(zookeeper_path); @@ -189,30 +189,30 @@ void BackupCoordinationDistributed::createRootNodes() zookeeper->createIfNotExists(zookeeper_path + "/archive_suffixes", ""); } -void BackupCoordinationDistributed::removeAllNodes() +void BackupCoordinationRemote::removeAllNodes() { auto zookeeper = get_zookeeper(); zookeeper->removeRecursive(zookeeper_path); } -void BackupCoordinationDistributed::setStatus(const String & current_host, const String & new_status, const String & message) +void BackupCoordinationRemote::setStatus(const String & current_host, const String & new_status, const String & message) { status_sync.set(current_host, new_status, message); } -Strings BackupCoordinationDistributed::setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) +Strings BackupCoordinationRemote::setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) { return status_sync.setAndWait(current_host, new_status, message, all_hosts); } -Strings BackupCoordinationDistributed::setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) +Strings BackupCoordinationRemote::setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) { return status_sync.setAndWaitFor(current_host, new_status, message, all_hosts, timeout_ms); } -void BackupCoordinationDistributed::addReplicatedPartNames( +void BackupCoordinationRemote::addReplicatedPartNames( const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, @@ -220,7 +220,7 @@ void BackupCoordinationDistributed::addReplicatedPartNames( { { std::lock_guard lock{mutex}; - if (replicated_parts_and_mutations) + if (replicated_tables) throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedPartNames() must not be called after preparing"); } @@ -231,14 +231,14 @@ void BackupCoordinationDistributed::addReplicatedPartNames( zookeeper->create(path, ReplicatedPartNames::serialize(part_names_and_checksums, table_name_for_logs), zkutil::CreateMode::Persistent); } -Strings BackupCoordinationDistributed::getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const +Strings BackupCoordinationRemote::getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const { std::lock_guard lock{mutex}; - prepareReplicatedPartsAndMutations(); - return replicated_parts_and_mutations->getPartNames(table_shared_id, replica_name); + prepareReplicatedTables(); + return replicated_tables->getPartNames(table_shared_id, replica_name); } -void BackupCoordinationDistributed::addReplicatedMutations( +void BackupCoordinationRemote::addReplicatedMutations( const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, @@ -246,7 +246,7 @@ void BackupCoordinationDistributed::addReplicatedMutations( { { std::lock_guard lock{mutex}; - if (replicated_parts_and_mutations) + if (replicated_tables) throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedMutations() must not be called after preparing"); } @@ -257,57 +257,23 @@ void BackupCoordinationDistributed::addReplicatedMutations( zookeeper->create(path, ReplicatedMutations::serialize(mutations, table_name_for_logs), zkutil::CreateMode::Persistent); } -std::vector BackupCoordinationDistributed::getReplicatedMutations(const String & table_shared_id, const String & replica_name) const +std::vector BackupCoordinationRemote::getReplicatedMutations(const String & table_shared_id, const String & replica_name) const { std::lock_guard lock{mutex}; - prepareReplicatedPartsAndMutations(); - return replicated_parts_and_mutations->getMutations(table_shared_id, replica_name); + prepareReplicatedTables(); + return replicated_tables->getMutations(table_shared_id, replica_name); } -void BackupCoordinationDistributed::prepareReplicatedPartsAndMutations() const -{ - if (replicated_parts_and_mutations) - return; - - replicated_parts_and_mutations.emplace(); - auto zookeeper = get_zookeeper(); - - { - String path = zookeeper_path + "/repl_part_names"; - for (const String & escaped_table_zk_path : zookeeper->getChildren(path)) - { - String table_zk_path = unescapeForFileName(escaped_table_zk_path); - String path2 = path + "/" + escaped_table_zk_path; - for (const String & escaped_replica_name : zookeeper->getChildren(path2)) - { - String replica_name = unescapeForFileName(escaped_replica_name); - auto part_names = ReplicatedPartNames::deserialize(zookeeper->get(path2 + "/" + escaped_replica_name)); - replicated_parts_and_mutations->addPartNames(table_zk_path, part_names.table_name_for_logs, replica_name, part_names.part_names_and_checksums); - } - } - } - - { - String path = zookeeper_path + "/repl_mutations"; - for (const String & escaped_table_zk_path : zookeeper->getChildren(path)) - { - String table_zk_path = unescapeForFileName(escaped_table_zk_path); - String path2 = path + "/" + escaped_table_zk_path; - for (const String & escaped_replica_name : zookeeper->getChildren(path2)) - { - String replica_name = unescapeForFileName(escaped_replica_name); - auto mutations = ReplicatedMutations::deserialize(zookeeper->get(path2 + "/" + escaped_replica_name)); - replicated_parts_and_mutations->addMutations(table_zk_path, mutations.table_name_for_logs, replica_name, mutations.mutations); - } - } - } -} - - -void BackupCoordinationDistributed::addReplicatedDataPath( +void BackupCoordinationRemote::addReplicatedDataPath( const String & table_shared_id, const String & data_path) { + { + std::lock_guard lock{mutex}; + if (replicated_tables) + throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedDataPath() must not be called after preparing"); + } + auto zookeeper = get_zookeeper(); String path = zookeeper_path + "/repl_data_paths/" + escapeForFileName(table_shared_id); zookeeper->createIfNotExists(path, ""); @@ -315,21 +281,76 @@ void BackupCoordinationDistributed::addReplicatedDataPath( zookeeper->createIfNotExists(path, ""); } -Strings BackupCoordinationDistributed::getReplicatedDataPaths(const String & table_shared_id) const +Strings BackupCoordinationRemote::getReplicatedDataPaths(const String & table_shared_id) const { - auto zookeeper = get_zookeeper(); - String path = zookeeper_path + "/repl_data_paths/" + escapeForFileName(table_shared_id); - Strings children = zookeeper->getChildren(path); - Strings data_paths; - data_paths.reserve(children.size()); - for (const String & child : children) - data_paths.push_back(unescapeForFileName(child)); - return data_paths; + std::lock_guard lock{mutex}; + prepareReplicatedTables(); + return replicated_tables->getDataPaths(table_shared_id); } -void BackupCoordinationDistributed::addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) +void BackupCoordinationRemote::prepareReplicatedTables() const { + if (replicated_tables) + return; + + replicated_tables.emplace(); + auto zookeeper = get_zookeeper(); + + { + String path = zookeeper_path + "/repl_part_names"; + for (const String & escaped_table_shared_id : zookeeper->getChildren(path)) + { + String table_shared_id = unescapeForFileName(escaped_table_shared_id); + String path2 = path + "/" + escaped_table_shared_id; + for (const String & escaped_replica_name : zookeeper->getChildren(path2)) + { + String replica_name = unescapeForFileName(escaped_replica_name); + auto part_names = ReplicatedPartNames::deserialize(zookeeper->get(path2 + "/" + escaped_replica_name)); + replicated_tables->addPartNames(table_shared_id, part_names.table_name_for_logs, replica_name, part_names.part_names_and_checksums); + } + } + } + + { + String path = zookeeper_path + "/repl_mutations"; + for (const String & escaped_table_shared_id : zookeeper->getChildren(path)) + { + String table_shared_id = unescapeForFileName(escaped_table_shared_id); + String path2 = path + "/" + escaped_table_shared_id; + for (const String & escaped_replica_name : zookeeper->getChildren(path2)) + { + String replica_name = unescapeForFileName(escaped_replica_name); + auto mutations = ReplicatedMutations::deserialize(zookeeper->get(path2 + "/" + escaped_replica_name)); + replicated_tables->addMutations(table_shared_id, mutations.table_name_for_logs, replica_name, mutations.mutations); + } + } + } + + { + String path = zookeeper_path + "/repl_data_paths"; + for (const String & escaped_table_shared_id : zookeeper->getChildren(path)) + { + String table_shared_id = unescapeForFileName(escaped_table_shared_id); + String path2 = path + "/" + escaped_table_shared_id; + for (const String & escaped_data_path : zookeeper->getChildren(path2)) + { + String data_path = unescapeForFileName(escaped_data_path); + replicated_tables->addDataPath(table_shared_id, data_path); + } + } + } +} + + +void BackupCoordinationRemote::addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) +{ + { + std::lock_guard lock{mutex}; + if (replicated_access) + throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedAccessFilePath() must not be called after preparing"); + } + auto zookeeper = get_zookeeper(); String path = zookeeper_path + "/repl_access/" + escapeForFileName(access_zk_path); zookeeper->createIfNotExists(path, ""); @@ -339,14 +360,14 @@ void BackupCoordinationDistributed::addReplicatedAccessFilePath(const String & a zookeeper->createIfNotExists(path, file_path); } -Strings BackupCoordinationDistributed::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const +Strings BackupCoordinationRemote::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const { std::lock_guard lock{mutex}; prepareReplicatedAccess(); return replicated_access->getFilePaths(access_zk_path, access_entity_type, host_id); } -void BackupCoordinationDistributed::prepareReplicatedAccess() const +void BackupCoordinationRemote::prepareReplicatedAccess() const { if (replicated_access) return; @@ -354,7 +375,7 @@ void BackupCoordinationDistributed::prepareReplicatedAccess() const replicated_access.emplace(); auto zookeeper = get_zookeeper(); - String path = zookeeper_path + "/repl_access/"; + String path = zookeeper_path + "/repl_access"; for (const String & escaped_access_zk_path : zookeeper->getChildren(path)) { String access_zk_path = unescapeForFileName(escaped_access_zk_path); @@ -373,7 +394,7 @@ void BackupCoordinationDistributed::prepareReplicatedAccess() const } -void BackupCoordinationDistributed::addFileInfo(const FileInfo & file_info, bool & is_data_file_required) +void BackupCoordinationRemote::addFileInfo(const FileInfo & file_info, bool & is_data_file_required) { auto zookeeper = get_zookeeper(); @@ -395,7 +416,7 @@ void BackupCoordinationDistributed::addFileInfo(const FileInfo & file_info, bool is_data_file_required = (code == Coordination::Error::ZOK) && (file_info.size > file_info.base_size); } -void BackupCoordinationDistributed::updateFileInfo(const FileInfo & file_info) +void BackupCoordinationRemote::updateFileInfo(const FileInfo & file_info) { if (!file_info.size) return; /// we don't keep FileInfos for empty files, nothing to update @@ -417,7 +438,7 @@ void BackupCoordinationDistributed::updateFileInfo(const FileInfo & file_info) } } -std::vector BackupCoordinationDistributed::getAllFileInfos() const +std::vector BackupCoordinationRemote::getAllFileInfos() const { auto zookeeper = get_zookeeper(); std::vector file_infos; @@ -435,7 +456,7 @@ std::vector BackupCoordinationDistributed::getAllFileInfos() const return file_infos; } -Strings BackupCoordinationDistributed::listFiles(const String & directory, bool recursive) const +Strings BackupCoordinationRemote::listFiles(const String & directory, bool recursive) const { auto zookeeper = get_zookeeper(); Strings escaped_names = zookeeper->getChildren(zookeeper_path + "/file_names"); @@ -468,7 +489,7 @@ Strings BackupCoordinationDistributed::listFiles(const String & directory, bool return elements; } -bool BackupCoordinationDistributed::hasFiles(const String & directory) const +bool BackupCoordinationRemote::hasFiles(const String & directory) const { auto zookeeper = get_zookeeper(); Strings escaped_names = zookeeper->getChildren(zookeeper_path + "/file_names"); @@ -487,7 +508,7 @@ bool BackupCoordinationDistributed::hasFiles(const String & directory) const return false; } -std::optional BackupCoordinationDistributed::getFileInfo(const String & file_name) const +std::optional BackupCoordinationRemote::getFileInfo(const String & file_name) const { auto zookeeper = get_zookeeper(); String size_and_checksum; @@ -501,7 +522,7 @@ std::optional BackupCoordinationDistributed::getFileInfo(const String return file_info; } -std::optional BackupCoordinationDistributed::getFileInfo(const SizeAndChecksum & size_and_checksum) const +std::optional BackupCoordinationRemote::getFileInfo(const SizeAndChecksum & size_and_checksum) const { auto zookeeper = get_zookeeper(); String file_info_str; @@ -510,7 +531,7 @@ std::optional BackupCoordinationDistributed::getFileInfo(const SizeAnd return deserializeFileInfo(file_info_str); } -std::optional BackupCoordinationDistributed::getFileSizeAndChecksum(const String & file_name) const +std::optional BackupCoordinationRemote::getFileSizeAndChecksum(const String & file_name) const { auto zookeeper = get_zookeeper(); String size_and_checksum; @@ -519,7 +540,7 @@ std::optional BackupCoordinationDistributed::getFileSizeAndChec return deserializeSizeAndChecksum(size_and_checksum); } -String BackupCoordinationDistributed::getNextArchiveSuffix() +String BackupCoordinationRemote::getNextArchiveSuffix() { auto zookeeper = get_zookeeper(); String path = zookeeper_path + "/archive_suffixes/a"; @@ -530,7 +551,7 @@ String BackupCoordinationDistributed::getNextArchiveSuffix() return formatArchiveSuffix(extractCounterFromSequentialNodeName(path_created)); } -Strings BackupCoordinationDistributed::getAllArchiveSuffixes() const +Strings BackupCoordinationRemote::getAllArchiveSuffixes() const { auto zookeeper = get_zookeeper(); Strings node_names = zookeeper->getChildren(zookeeper_path + "/archive_suffixes"); @@ -539,7 +560,7 @@ Strings BackupCoordinationDistributed::getAllArchiveSuffixes() const return node_names; } -void BackupCoordinationDistributed::drop() +void BackupCoordinationRemote::drop() { removeAllNodes(); } diff --git a/src/Backups/BackupCoordinationDistributed.h b/src/Backups/BackupCoordinationRemote.h similarity index 81% rename from src/Backups/BackupCoordinationDistributed.h rename to src/Backups/BackupCoordinationRemote.h index feb2cdeeb97..e5ecf87d7bf 100644 --- a/src/Backups/BackupCoordinationDistributed.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -1,18 +1,20 @@ #pragma once #include -#include +#include +#include +#include namespace DB { -/// Stores backup temporary information in Zookeeper, used to perform BACKUP ON CLUSTER. -class BackupCoordinationDistributed : public IBackupCoordination +/// Implementation of the IBackupCoordination interface performing coordination via ZooKeeper. It's necessary for "BACKUP ON CLUSTER". +class BackupCoordinationRemote : public IBackupCoordination { public: - BackupCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_); - ~BackupCoordinationDistributed() override; + BackupCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_); + ~BackupCoordinationRemote() override; void setStatus(const String & current_host, const String & new_status, const String & message) override; Strings setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) override; @@ -58,7 +60,7 @@ public: private: void createRootNodes(); void removeAllNodes(); - void prepareReplicatedPartsAndMutations() const; + void prepareReplicatedTables() const; void prepareReplicatedAccess() const; const String zookeeper_path; @@ -67,7 +69,7 @@ private: BackupCoordinationStatusSync status_sync; mutable std::mutex mutex; - mutable std::optional replicated_parts_and_mutations; + mutable std::optional replicated_tables; mutable std::optional replicated_access; }; diff --git a/src/Backups/BackupCoordinationReplicatedAccess.cpp b/src/Backups/BackupCoordinationReplicatedAccess.cpp new file mode 100644 index 00000000000..3211e4258fa --- /dev/null +++ b/src/Backups/BackupCoordinationReplicatedAccess.cpp @@ -0,0 +1,33 @@ +#include + + +namespace DB +{ + +BackupCoordinationReplicatedAccess::BackupCoordinationReplicatedAccess() = default; +BackupCoordinationReplicatedAccess::~BackupCoordinationReplicatedAccess() = default; + +void BackupCoordinationReplicatedAccess::addFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) +{ + auto & ref = file_paths_by_zk_path[std::make_pair(access_zk_path, access_entity_type)]; + ref.file_paths.emplace(file_path); + + /// std::max() because the calculation must give the same result being repeated on a different replica. + ref.host_to_store_access = std::max(ref.host_to_store_access, host_id); +} + +Strings BackupCoordinationReplicatedAccess::getFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const +{ + auto it = file_paths_by_zk_path.find(std::make_pair(access_zk_path, access_entity_type)); + if (it == file_paths_by_zk_path.end()) + return {}; + + auto & file_paths = it->second; + if (file_paths.host_to_store_access != host_id) + return {}; + + Strings res{file_paths.file_paths.begin(), file_paths.file_paths.end()}; + return res; +} + +} diff --git a/src/Backups/BackupCoordinationReplicatedAccess.h b/src/Backups/BackupCoordinationReplicatedAccess.h new file mode 100644 index 00000000000..93ae0203be9 --- /dev/null +++ b/src/Backups/BackupCoordinationReplicatedAccess.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +enum class AccessEntityType; + +/// This class is used by hosts to coordinate the access entities of ReplicatedAccessStorage they're writing to a backup. +/// It's designed to make all hosts save the same access entities to the backup even in case the ReplicatedAccessStorage changes +/// while the backup is being produced. This is important to make RESTORE more predicitible. +/// +/// For example, let's consider three replicas having a ReplicatedAccessStorage on them. +/// This class ensures that the following files in the backup are the same: +/// /shards/1/replicas/1/data/system/users/access01.txt +/// /shards/1/replicas/2/data/system/users/access01.txt +/// /shards/1/replicas/3/data/system/users/access01.txt +/// +/// To implement that this class chooses one host to write access entities for all the hosts so in fact all those files +/// in the example above are written by the same host. + +class BackupCoordinationReplicatedAccess +{ +public: + BackupCoordinationReplicatedAccess(); + ~BackupCoordinationReplicatedAccess(); + + /// Adds a path to access*.txt file keeping access entities of a ReplicatedAccessStorage. + void addFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path); + + /// Returns all paths added by addFilePath() if `host_id` is a host chosen to store access. + Strings getFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const; + +private: + using ZkPathAndEntityType = std::pair; + + struct FilePathsAndHost + { + std::unordered_set file_paths; + String host_to_store_access; + }; + + std::map file_paths_by_zk_path; +}; + +} diff --git a/src/Backups/BackupCoordinationHelpers.cpp b/src/Backups/BackupCoordinationReplicatedTables.cpp similarity index 59% rename from src/Backups/BackupCoordinationHelpers.cpp rename to src/Backups/BackupCoordinationReplicatedTables.cpp index 7f8bdb32c24..3c12e5b062f 100644 --- a/src/Backups/BackupCoordinationHelpers.cpp +++ b/src/Backups/BackupCoordinationReplicatedTables.cpp @@ -1,10 +1,7 @@ -#include +#include #include #include #include -#include -#include -#include #include @@ -14,7 +11,6 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_BACKUP_TABLE; - extern const int FAILED_TO_SYNC_BACKUP_OR_RESTORE; extern const int LOGICAL_ERROR; } @@ -30,7 +26,7 @@ namespace using MutationInfo = IBackupCoordination::MutationInfo; -class BackupCoordinationReplicatedPartsAndMutations::CoveredPartsFinder +class BackupCoordinationReplicatedTables::CoveredPartsFinder { public: explicit CoveredPartsFinder(const String & table_name_for_logs_) : table_name_for_logs(table_name_for_logs_) {} @@ -150,10 +146,10 @@ private: }; -BackupCoordinationReplicatedPartsAndMutations::BackupCoordinationReplicatedPartsAndMutations() = default; -BackupCoordinationReplicatedPartsAndMutations::~BackupCoordinationReplicatedPartsAndMutations() = default; +BackupCoordinationReplicatedTables::BackupCoordinationReplicatedTables() = default; +BackupCoordinationReplicatedTables::~BackupCoordinationReplicatedTables() = default; -void BackupCoordinationReplicatedPartsAndMutations::addPartNames( +void BackupCoordinationReplicatedTables::addPartNames( const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, @@ -204,7 +200,7 @@ void BackupCoordinationReplicatedPartsAndMutations::addPartNames( } } -Strings BackupCoordinationReplicatedPartsAndMutations::getPartNames(const String & table_shared_id, const String & replica_name) const +Strings BackupCoordinationReplicatedTables::getPartNames(const String & table_shared_id, const String & replica_name) const { prepare(); @@ -220,7 +216,7 @@ Strings BackupCoordinationReplicatedPartsAndMutations::getPartNames(const String return it2->second; } -void BackupCoordinationReplicatedPartsAndMutations::addMutations( +void BackupCoordinationReplicatedTables::addMutations( const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, @@ -239,7 +235,7 @@ void BackupCoordinationReplicatedPartsAndMutations::addMutations( } std::vector -BackupCoordinationReplicatedPartsAndMutations::getMutations(const String & table_shared_id, const String & replica_name) const +BackupCoordinationReplicatedTables::getMutations(const String & table_shared_id, const String & replica_name) const { prepare(); @@ -257,7 +253,24 @@ BackupCoordinationReplicatedPartsAndMutations::getMutations(const String & table return res; } -void BackupCoordinationReplicatedPartsAndMutations::prepare() const +void BackupCoordinationReplicatedTables::addDataPath(const String & table_shared_id, const String & data_path) +{ + auto & table_info = table_infos[table_shared_id]; + table_info.data_paths.emplace(data_path); +} + +Strings BackupCoordinationReplicatedTables::getDataPaths(const String & table_shared_id) const +{ + auto it = table_infos.find(table_shared_id); + if (it == table_infos.end()) + return {}; + + const auto & table_info = it->second; + return Strings{table_info.data_paths.begin(), table_info.data_paths.end()}; +} + + +void BackupCoordinationReplicatedTables::prepare() const { if (prepared) return; @@ -319,180 +332,4 @@ void BackupCoordinationReplicatedPartsAndMutations::prepare() const prepared = true; } - -/// Helper designed to be used in an implementation of the IBackupCoordination interface in the part related to replicated access storages. -BackupCoordinationReplicatedAccess::BackupCoordinationReplicatedAccess() = default; -BackupCoordinationReplicatedAccess::~BackupCoordinationReplicatedAccess() = default; - -void BackupCoordinationReplicatedAccess::addFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) -{ - auto & ref = file_paths_by_zk_path[std::make_pair(access_zk_path, access_entity_type)]; - ref.file_paths.emplace(file_path); - - /// std::max() because the calculation must give the same result being repeated on a different replica. - ref.host_to_store_access = std::max(ref.host_to_store_access, host_id); -} - -Strings BackupCoordinationReplicatedAccess::getFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const -{ - auto it = file_paths_by_zk_path.find(std::make_pair(access_zk_path, access_entity_type)); - if (it == file_paths_by_zk_path.end()) - return {}; - - auto & file_paths = it->second; - if (file_paths.host_to_store_access != host_id) - return {}; - - Strings res{file_paths.file_paths.begin(), file_paths.file_paths.end()}; - return res; -} - - -/// Helps to wait until all hosts come to a specified stage. -BackupCoordinationStatusSync::BackupCoordinationStatusSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_) - : zookeeper_path(zookeeper_path_) - , get_zookeeper(get_zookeeper_) - , log(log_) -{ - createRootNodes(); -} - -void BackupCoordinationStatusSync::createRootNodes() -{ - auto zookeeper = get_zookeeper(); - zookeeper->createAncestors(zookeeper_path); - zookeeper->createIfNotExists(zookeeper_path, ""); -} - -void BackupCoordinationStatusSync::set(const String & current_host, const String & new_status, const String & message) -{ - setImpl(current_host, new_status, message, {}, {}); -} - -Strings BackupCoordinationStatusSync::setAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) -{ - return setImpl(current_host, new_status, message, all_hosts, {}); -} - -Strings BackupCoordinationStatusSync::setAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) -{ - return setImpl(current_host, new_status, message, all_hosts, timeout_ms); -} - -Strings BackupCoordinationStatusSync::setImpl(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, const std::optional & timeout_ms) -{ - /// Put new status to ZooKeeper. - auto zookeeper = get_zookeeper(); - zookeeper->createIfNotExists(zookeeper_path + "/" + current_host + "|" + new_status, message); - - if (all_hosts.empty() || (new_status == kErrorStatus)) - return {}; - - if ((all_hosts.size() == 1) && (all_hosts.front() == current_host)) - return {message}; - - /// Wait for other hosts. - - Strings ready_hosts_results; - ready_hosts_results.resize(all_hosts.size()); - - std::map /* index in `ready_hosts_results` */> unready_hosts; - for (size_t i = 0; i != all_hosts.size(); ++i) - unready_hosts[all_hosts[i]].push_back(i); - - std::optional host_with_error; - std::optional error_message; - - /// Process ZooKeeper's nodes and set `all_hosts_ready` or `unready_host` or `error_message`. - auto process_zk_nodes = [&](const Strings & zk_nodes) - { - for (const String & zk_node : zk_nodes) - { - if (zk_node.starts_with("remove_watch-")) - continue; - - size_t separator_pos = zk_node.find('|'); - if (separator_pos == String::npos) - throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Unexpected zk node {}", zookeeper_path + "/" + zk_node); - String host = zk_node.substr(0, separator_pos); - String status = zk_node.substr(separator_pos + 1); - if (status == kErrorStatus) - { - host_with_error = host; - error_message = zookeeper->get(zookeeper_path + "/" + zk_node); - return; - } - auto it = unready_hosts.find(host); - if ((it != unready_hosts.end()) && (status == new_status)) - { - String result = zookeeper->get(zookeeper_path + "/" + zk_node); - for (size_t i : it->second) - ready_hosts_results[i] = result; - unready_hosts.erase(it); - } - } - }; - - /// Wait until all hosts are ready or an error happens or time is out. - std::atomic watch_set = false; - std::condition_variable watch_triggered_event; - - auto watch_callback = [&](const Coordination::WatchResponse &) - { - watch_set = false; /// After it's triggered it's not set until we call getChildrenWatch() again. - watch_triggered_event.notify_all(); - }; - - auto watch_triggered = [&] { return !watch_set; }; - - bool use_timeout = timeout_ms.has_value(); - std::chrono::milliseconds timeout{timeout_ms.value_or(0)}; - std::chrono::steady_clock::time_point start_time = std::chrono::steady_clock::now(); - std::chrono::steady_clock::duration elapsed; - std::mutex dummy_mutex; - - while (!unready_hosts.empty() && !error_message) - { - watch_set = true; - Strings nodes = zookeeper->getChildrenWatch(zookeeper_path, nullptr, watch_callback); - process_zk_nodes(nodes); - - if (!unready_hosts.empty() && !error_message) - { - LOG_TRACE(log, "Waiting for host {}", unready_hosts.begin()->first); - std::unique_lock dummy_lock{dummy_mutex}; - if (use_timeout) - { - elapsed = std::chrono::steady_clock::now() - start_time; - if ((elapsed > timeout) || !watch_triggered_event.wait_for(dummy_lock, timeout - elapsed, watch_triggered)) - break; - } - else - watch_triggered_event.wait(dummy_lock, watch_triggered); - } - } - - if (watch_set) - { - /// Remove watch by triggering it. - zookeeper->create(zookeeper_path + "/remove_watch-", "", zkutil::CreateMode::EphemeralSequential); - std::unique_lock dummy_lock{dummy_mutex}; - watch_triggered_event.wait(dummy_lock, watch_triggered); - } - - if (error_message) - throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Error occurred on host {}: {}", *host_with_error, *error_message); - - if (!unready_hosts.empty()) - { - throw Exception( - ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, - "Waited for host {} too long ({})", - unready_hosts.begin()->first, - to_string(elapsed)); - } - - return ready_hosts_results; -} - } diff --git a/src/Backups/BackupCoordinationHelpers.h b/src/Backups/BackupCoordinationReplicatedTables.h similarity index 54% rename from src/Backups/BackupCoordinationHelpers.h rename to src/Backups/BackupCoordinationReplicatedTables.h index 9203426a96a..eb7b2e15e5a 100644 --- a/src/Backups/BackupCoordinationHelpers.h +++ b/src/Backups/BackupCoordinationReplicatedTables.h @@ -1,23 +1,40 @@ #pragma once #include -#include -#include #include +#include #include +#include namespace DB { -enum class AccessEntityType; +/// Replicas used this class to coordinate how they're writing replicated tables to a backup. +/// "BACKUP ON CLUSTER" can be executed on multiple hosts and parts of replicated tables on those hosts could be slightly different +/// at any specific moment. This class is designed so that inside the backup all replicas would contain all the parts +/// no matter if the replication queues of those tables are fast or slow. +/// This is important to make RESTORE more correct and not dependent on random things like how fast the replicas doing RESTORE +/// comparing to each other or how many replicas will be when RESTORE will be executed. +/// +/// Example 1: Let's consider two replicas of a table, and let the first replica contain part all_1_1_0 and the second replica contain +/// all_2_2_0. The files in the backup will look like this: +/// /shards/1/replicas/1/data/mydb/mytable/all_1_1_0 +/// /shards/1/replicas/1/data/mydb/mytable/all_2_2_0 +/// /shards/1/replicas/2/data/mydb/mytable/all_1_1_0 +/// /shards/1/replicas/2/data/mydb/mytable/all_2_2_0 +/// +/// Example 2: Let's consider two replicas again, and let the first replica contain parts all_1_1_0 and all_2_2_0 and +/// the second replica contain part all_1_2_1 (i.e. the second replica have those parts merged). +/// In this case the files in the backup will look like this: +/// /shards/1/replicas/1/data/mydb/mytable/all_1_2_1 +/// /shards/1/replicas/2/data/mydb/mytable/all_1_2_1 -/// Helper designed to be used in an implementation of the IBackupCoordination interface in the part related to replicated tables. -class BackupCoordinationReplicatedPartsAndMutations +class BackupCoordinationReplicatedTables { public: - BackupCoordinationReplicatedPartsAndMutations(); - ~BackupCoordinationReplicatedPartsAndMutations(); + BackupCoordinationReplicatedTables(); + ~BackupCoordinationReplicatedTables(); using PartNameAndChecksum = IBackupCoordination::PartNameAndChecksum; @@ -31,6 +48,11 @@ public: const String & replica_name, const std::vector & part_names_and_checksums); + /// Returns the names of the parts which a specified replica of a replicated table should put to the backup. + /// This is the same list as it was added by call of the function addPartNames() but without duplications and without + /// parts covered by another parts. + Strings getPartNames(const String & table_shared_id, const String & replica_name) const; + using MutationInfo = IBackupCoordination::MutationInfo; /// Adds information about mutations of a replicated table. @@ -40,14 +62,17 @@ public: const String & replica_name, const std::vector & mutations); - /// Returns the names of the parts which a specified replica of a replicated table should put to the backup. - /// This is the same list as it was added by call of the function addPartNames() but without duplications and without - /// parts covered by another parts. - Strings getPartNames(const String & table_shared_id, const String & replica_name) const; - /// Returns all mutations of a replicated table which are not finished for some data parts added by addReplicatedPartNames(). std::vector getMutations(const String & table_shared_id, const String & replica_name) const; + /// Adds a data path in backup for a replicated table. + /// Multiple replicas of the replicated table call this function and then all the added paths can be returned by call of the function + /// getDataPaths(). + void addDataPath(const String & table_shared_id, const String & data_path); + + /// Returns all the data paths in backup added for a replicated table (see also addReplicatedDataPath()). + Strings getDataPaths(const String & table_shared_id) const; + private: void prepare() const; @@ -68,56 +93,11 @@ private: mutable std::unordered_map min_data_versions_by_partition; mutable std::unordered_map mutations; String replica_name_to_store_mutations; + std::unordered_set data_paths; }; std::map table_infos; /// Should be ordered because we need this map to be in the same order on every replica. mutable bool prepared = false; }; - -/// Helper designed to be used in an implementation of the IBackupCoordination interface in the part related to replicated access storages. -class BackupCoordinationReplicatedAccess -{ -public: - BackupCoordinationReplicatedAccess(); - ~BackupCoordinationReplicatedAccess(); - - /// Adds a path to access.txt file keeping access entities of a ReplicatedAccessStorage. - void addFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path); - Strings getFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const; - -private: - using ZkPathAndEntityType = std::pair; - - struct FilePathsAndHost - { - std::unordered_set file_paths; - String host_to_store_access; - }; - - std::map file_paths_by_zk_path; -}; - - -/// Helps to wait until all hosts come to a specified stage. -class BackupCoordinationStatusSync -{ -public: - BackupCoordinationStatusSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_); - - void set(const String & current_host, const String & new_status, const String & message); - Strings setAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts); - Strings setAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms); - - static constexpr const char * kErrorStatus = "error"; - -private: - void createRootNodes(); - Strings setImpl(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, const std::optional & timeout_ms); - - String zookeeper_path; - zkutil::GetZooKeeper get_zookeeper; - Poco::Logger * log; -}; - } diff --git a/src/Backups/BackupCoordinationStatusSync.cpp b/src/Backups/BackupCoordinationStatusSync.cpp new file mode 100644 index 00000000000..7e1198869dd --- /dev/null +++ b/src/Backups/BackupCoordinationStatusSync.cpp @@ -0,0 +1,161 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FAILED_TO_SYNC_BACKUP_OR_RESTORE; +} + + +BackupCoordinationStatusSync::BackupCoordinationStatusSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_) + : zookeeper_path(zookeeper_path_) + , get_zookeeper(get_zookeeper_) + , log(log_) +{ + createRootNodes(); +} + +void BackupCoordinationStatusSync::createRootNodes() +{ + auto zookeeper = get_zookeeper(); + zookeeper->createAncestors(zookeeper_path); + zookeeper->createIfNotExists(zookeeper_path, ""); +} + +void BackupCoordinationStatusSync::set(const String & current_host, const String & new_status, const String & message) +{ + setImpl(current_host, new_status, message, {}, {}); +} + +Strings BackupCoordinationStatusSync::setAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) +{ + return setImpl(current_host, new_status, message, all_hosts, {}); +} + +Strings BackupCoordinationStatusSync::setAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) +{ + return setImpl(current_host, new_status, message, all_hosts, timeout_ms); +} + +Strings BackupCoordinationStatusSync::setImpl(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, const std::optional & timeout_ms) +{ + /// Put new status to ZooKeeper. + auto zookeeper = get_zookeeper(); + zookeeper->createIfNotExists(zookeeper_path + "/" + current_host + "|" + new_status, message); + + if (all_hosts.empty() || (new_status == kErrorStatus)) + return {}; + + if ((all_hosts.size() == 1) && (all_hosts.front() == current_host)) + return {message}; + + /// Wait for other hosts. + + Strings ready_hosts_results; + ready_hosts_results.resize(all_hosts.size()); + + std::map /* index in `ready_hosts_results` */> unready_hosts; + for (size_t i = 0; i != all_hosts.size(); ++i) + unready_hosts[all_hosts[i]].push_back(i); + + std::optional host_with_error; + std::optional error_message; + + /// Process ZooKeeper's nodes and set `all_hosts_ready` or `unready_host` or `error_message`. + auto process_zk_nodes = [&](const Strings & zk_nodes) + { + for (const String & zk_node : zk_nodes) + { + if (zk_node.starts_with("remove_watch-")) + continue; + + size_t separator_pos = zk_node.find('|'); + if (separator_pos == String::npos) + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Unexpected zk node {}", zookeeper_path + "/" + zk_node); + String host = zk_node.substr(0, separator_pos); + String status = zk_node.substr(separator_pos + 1); + if (status == kErrorStatus) + { + host_with_error = host; + error_message = zookeeper->get(zookeeper_path + "/" + zk_node); + return; + } + auto it = unready_hosts.find(host); + if ((it != unready_hosts.end()) && (status == new_status)) + { + String result = zookeeper->get(zookeeper_path + "/" + zk_node); + for (size_t i : it->second) + ready_hosts_results[i] = result; + unready_hosts.erase(it); + } + } + }; + + /// Wait until all hosts are ready or an error happens or time is out. + std::atomic watch_set = false; + std::condition_variable watch_triggered_event; + + auto watch_callback = [&](const Coordination::WatchResponse &) + { + watch_set = false; /// After it's triggered it's not set until we call getChildrenWatch() again. + watch_triggered_event.notify_all(); + }; + + auto watch_triggered = [&] { return !watch_set; }; + + bool use_timeout = timeout_ms.has_value(); + std::chrono::milliseconds timeout{timeout_ms.value_or(0)}; + std::chrono::steady_clock::time_point start_time = std::chrono::steady_clock::now(); + std::chrono::steady_clock::duration elapsed; + std::mutex dummy_mutex; + + while (!unready_hosts.empty() && !error_message) + { + watch_set = true; + Strings nodes = zookeeper->getChildrenWatch(zookeeper_path, nullptr, watch_callback); + process_zk_nodes(nodes); + + if (!unready_hosts.empty() && !error_message) + { + LOG_TRACE(log, "Waiting for host {}", unready_hosts.begin()->first); + std::unique_lock dummy_lock{dummy_mutex}; + if (use_timeout) + { + elapsed = std::chrono::steady_clock::now() - start_time; + if ((elapsed > timeout) || !watch_triggered_event.wait_for(dummy_lock, timeout - elapsed, watch_triggered)) + break; + } + else + watch_triggered_event.wait(dummy_lock, watch_triggered); + } + } + + if (watch_set) + { + /// Remove watch by triggering it. + zookeeper->create(zookeeper_path + "/remove_watch-", "", zkutil::CreateMode::EphemeralSequential); + std::unique_lock dummy_lock{dummy_mutex}; + watch_triggered_event.wait(dummy_lock, watch_triggered); + } + + if (error_message) + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Error occurred on host {}: {}", *host_with_error, *error_message); + + if (!unready_hosts.empty()) + { + throw Exception( + ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, + "Waited for host {} too long ({})", + unready_hosts.begin()->first, + to_string(elapsed)); + } + + return ready_hosts_results; +} + +} diff --git a/src/Backups/BackupCoordinationStatusSync.h b/src/Backups/BackupCoordinationStatusSync.h new file mode 100644 index 00000000000..638bd81610d --- /dev/null +++ b/src/Backups/BackupCoordinationStatusSync.h @@ -0,0 +1,36 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Used to coordinate hosts so all hosts would come to a specific status at around the same time. +class BackupCoordinationStatusSync +{ +public: + BackupCoordinationStatusSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_); + + /// Sets the status of the current host and signal other hosts if there were other hosts waiting for that. + void set(const String & current_host, const String & new_status, const String & message); + + /// Sets the status of the current host and waits until all hosts come to the same status. + /// The function returns the messages all hosts set when they come to the required status. + Strings setAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts); + + /// Almost the same as setAndWait() but this one stops waiting and throws an exception after a specific amount of time. + Strings setAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms); + + static constexpr const char * kErrorStatus = "error"; + +private: + void createRootNodes(); + Strings setImpl(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, const std::optional & timeout_ms); + + String zookeeper_path; + zkutil::GetZooKeeper get_zookeeper; + Poco::Logger * log; +}; + +} diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 90f9486a0df..8c96f24311c 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include @@ -47,7 +46,7 @@ namespace constexpr const char * kWritingBackupStatus = "writing backup"; /// Error status. - constexpr const char * kErrorStatus = BackupCoordinationStatusSync::kErrorStatus; + constexpr const char * kErrorStatus = IBackupCoordination::kErrorStatus; /// Uppercases the first character of a passed string. String toUpperFirst(const String & str) diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 09c1cd5e9b5..5d4dba748e3 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 635b2810941..85a997e24d3 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -5,9 +5,9 @@ #include #include #include -#include +#include #include -#include +#include #include #include #include @@ -120,7 +120,7 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c if (!backup_settings.coordination_zk_path.empty()) { - backup_coordination = std::make_shared( + backup_coordination = std::make_shared( backup_settings.coordination_zk_path, [global_context = context_in_use->getGlobalContext()] { return global_context->getZooKeeper(); }); } @@ -291,7 +291,7 @@ UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr conte if (!restore_settings.coordination_zk_path.empty()) { - restore_coordination = std::make_shared( + restore_coordination = std::make_shared( restore_settings.coordination_zk_path, [global_context = context_in_use->getGlobalContext()] { return global_context->getZooKeeper(); }); } diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 2c4781d32da..0658aac346d 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -8,8 +8,10 @@ namespace DB { enum class AccessEntityType; - -/// Keeps information about files contained in a backup. +/// Replicas use this class to coordinate what they're writing to a backup while executing BACKUP ON CLUSTER. +/// There are two implementation of this interface: BackupCoordinationLocal and BackupCoordinationRemote. +/// BackupCoordinationLocal is used while executing BACKUP without ON CLUSTER and performs coordination in memory. +/// BackupCoordinationRemote is used while executing BACKUP with ON CLUSTER and performs coordination via ZooKeeper. class IBackupCoordination { public: @@ -20,6 +22,8 @@ public: virtual Strings setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & other_hosts) = 0; virtual Strings setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & other_hosts, UInt64 timeout_ms) = 0; + static constexpr const char * kErrorStatus = "error"; + struct PartNameAndChecksum { String part_name; diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index ba76a6e0c99..87cec38cbb8 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -7,7 +7,10 @@ namespace DB { using DatabaseAndTableName = std::pair; -/// Keeps information about files contained in a backup. +/// Replicas use this class to coordinate what they're reading from a backup while executing RESTORE ON CLUSTER. +/// There are two implementation of this interface: RestoreCoordinationLocal and RestoreCoordinationRemote. +/// RestoreCoordinationLocal is used while executing RESTORE without ON CLUSTER and performs coordination in memory. +/// RestoreCoordinationRemote is used while executing RESTORE with ON CLUSTER and performs coordination via ZooKeeper. class IRestoreCoordination { public: @@ -18,6 +21,8 @@ public: virtual Strings setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & other_hosts) = 0; virtual Strings setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & other_hosts, UInt64 timeout_ms) = 0; + static constexpr const char * kErrorStatus = "error"; + /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. virtual bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) = 0; diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index 68624481a7a..2dcc71ded1a 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -11,6 +11,7 @@ namespace Poco { class Logger; } namespace DB { +/// Implementation of the IRestoreCoordination interface performing coordination in memory. class RestoreCoordinationLocal : public IRestoreCoordination { public: diff --git a/src/Backups/RestoreCoordinationDistributed.cpp b/src/Backups/RestoreCoordinationRemote.cpp similarity index 66% rename from src/Backups/RestoreCoordinationDistributed.cpp rename to src/Backups/RestoreCoordinationRemote.cpp index 8cbaa01810d..db995fdd25b 100644 --- a/src/Backups/RestoreCoordinationDistributed.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -6,7 +6,7 @@ namespace DB { -RestoreCoordinationDistributed::RestoreCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_) +RestoreCoordinationRemote::RestoreCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) , status_sync(zookeeper_path_ + "/status", get_zookeeper_, &Poco::Logger::get("RestoreCoordination")) @@ -14,9 +14,9 @@ RestoreCoordinationDistributed::RestoreCoordinationDistributed(const String & zo createRootNodes(); } -RestoreCoordinationDistributed::~RestoreCoordinationDistributed() = default; +RestoreCoordinationRemote::~RestoreCoordinationRemote() = default; -void RestoreCoordinationDistributed::createRootNodes() +void RestoreCoordinationRemote::createRootNodes() { auto zookeeper = get_zookeeper(); zookeeper->createAncestors(zookeeper_path); @@ -26,22 +26,22 @@ void RestoreCoordinationDistributed::createRootNodes() zookeeper->createIfNotExists(zookeeper_path + "/repl_access_storages_acquired", ""); } -void RestoreCoordinationDistributed::setStatus(const String & current_host, const String & new_status, const String & message) +void RestoreCoordinationRemote::setStatus(const String & current_host, const String & new_status, const String & message) { status_sync.set(current_host, new_status, message); } -Strings RestoreCoordinationDistributed::setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) +Strings RestoreCoordinationRemote::setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) { return status_sync.setAndWait(current_host, new_status, message, all_hosts); } -Strings RestoreCoordinationDistributed::setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) +Strings RestoreCoordinationRemote::setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) { return status_sync.setAndWaitFor(current_host, new_status, message, all_hosts, timeout_ms); } -bool RestoreCoordinationDistributed::acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) +bool RestoreCoordinationRemote::acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) { auto zookeeper = get_zookeeper(); @@ -56,7 +56,7 @@ bool RestoreCoordinationDistributed::acquireCreatingTableInReplicatedDatabase(co return (code == Coordination::Error::ZOK); } -bool RestoreCoordinationDistributed::acquireInsertingDataIntoReplicatedTable(const String & table_zk_path) +bool RestoreCoordinationRemote::acquireInsertingDataIntoReplicatedTable(const String & table_zk_path) { auto zookeeper = get_zookeeper(); @@ -68,7 +68,7 @@ bool RestoreCoordinationDistributed::acquireInsertingDataIntoReplicatedTable(con return (code == Coordination::Error::ZOK); } -bool RestoreCoordinationDistributed::acquireReplicatedAccessStorage(const String & access_storage_zk_path) +bool RestoreCoordinationRemote::acquireReplicatedAccessStorage(const String & access_storage_zk_path) { auto zookeeper = get_zookeeper(); @@ -80,13 +80,13 @@ bool RestoreCoordinationDistributed::acquireReplicatedAccessStorage(const String return (code == Coordination::Error::ZOK); } -void RestoreCoordinationDistributed::removeAllNodes() +void RestoreCoordinationRemote::removeAllNodes() { auto zookeeper = get_zookeeper(); zookeeper->removeRecursive(zookeeper_path); } -void RestoreCoordinationDistributed::drop() +void RestoreCoordinationRemote::drop() { removeAllNodes(); } diff --git a/src/Backups/RestoreCoordinationDistributed.h b/src/Backups/RestoreCoordinationRemote.h similarity index 82% rename from src/Backups/RestoreCoordinationDistributed.h rename to src/Backups/RestoreCoordinationRemote.h index 52b961cf0ef..4960a9e187a 100644 --- a/src/Backups/RestoreCoordinationDistributed.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -1,18 +1,18 @@ #pragma once #include -#include +#include namespace DB { -/// Stores restore temporary information in Zookeeper, used to perform RESTORE ON CLUSTER. -class RestoreCoordinationDistributed : public IRestoreCoordination +/// Implementation of the IRestoreCoordination interface performing coordination via ZooKeeper. It's necessary for "RESTORE ON CLUSTER". +class RestoreCoordinationRemote : public IRestoreCoordination { public: - RestoreCoordinationDistributed(const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper); - ~RestoreCoordinationDistributed() override; + RestoreCoordinationRemote(const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper); + ~RestoreCoordinationRemote() override; /// Sets the current status and waits for other hosts to come to this status too. If status starts with "error:" it'll stop waiting on all the hosts. void setStatus(const String & current_host, const String & new_status, const String & message) override; diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 24fe2001f1f..ea00be5f514 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -54,7 +53,7 @@ namespace constexpr const char * kInsertingDataToTablesStatus = "inserting data to tables"; /// Error status. - constexpr const char * kErrorStatus = BackupCoordinationStatusSync::kErrorStatus; + constexpr const char * kErrorStatus = IRestoreCoordination::kErrorStatus; /// Uppercases the first character of a passed string. String toUpperFirst(const String & str) From a0f6f6f82d499cdd8f3b6b3ecd8133490f7f1259 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Jul 2022 10:23:05 +0000 Subject: [PATCH 264/627] Fix: rewrite only outer braces --- src/Interpreters/RewriteOrderByVisitor.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/RewriteOrderByVisitor.cpp b/src/Interpreters/RewriteOrderByVisitor.cpp index 634b2ce2290..694dec84b7a 100644 --- a/src/Interpreters/RewriteOrderByVisitor.cpp +++ b/src/Interpreters/RewriteOrderByVisitor.cpp @@ -13,7 +13,7 @@ void RewriteOrderBy::visit(ASTPtr & ast, Data &) if (!query) return; - auto order_by = query->orderBy(); + const ASTPtr & order_by = query->orderBy(); if (!order_by) return; @@ -21,6 +21,9 @@ void RewriteOrderBy::visit(ASTPtr & ast, Data &) if (!expr_list) return; + if (expr_list->children.size() != 1) + return; + const auto * order_by_elem = expr_list->children.front()->as(); if (!order_by_elem) return; From 86d1c319f40e7a35a79f1ec5dc10e91e91f00fae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Jul 2022 12:23:45 +0200 Subject: [PATCH 265/627] Remove unnecessary log --- src/Storages/MergeTree/DataPartStorageOnDisk.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index fc4a4554304..2cc28d020bb 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -861,7 +861,6 @@ std::string DataPartStorageBuilderOnDisk::getRelativePath() const void DataPartStorageBuilderOnDisk::createDirectories() { - LOG_INFO(&Poco::Logger::get("DEBUG"), "CREATING DIRECTORY {}", (fs::path(root_path) / part_dir).string()); transaction->createDirectories(fs::path(root_path) / part_dir); } From 9df843c26419d034c4afe4d410a23a2ffde109ef Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 6 Jul 2022 10:49:18 +0000 Subject: [PATCH 266/627] Update version_date.tsv and changelogs after v22.6.3.35-stable --- docs/changelogs/v22.6.3.35-stable.md | 34 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 35 insertions(+) create mode 100644 docs/changelogs/v22.6.3.35-stable.md diff --git a/docs/changelogs/v22.6.3.35-stable.md b/docs/changelogs/v22.6.3.35-stable.md new file mode 100644 index 00000000000..584aeafc48e --- /dev/null +++ b/docs/changelogs/v22.6.3.35-stable.md @@ -0,0 +1,34 @@ +--- +sidebar_position: 1 +sidebar_label: 2022 +--- + +# 2022 Changelog + +### ClickHouse release v22.6.3.35-stable FIXME as compared to v22.6.2.12-stable + +#### Bug Fix +* Backported in [#38812](https://github.com/ClickHouse/ClickHouse/issues/38812): Fix crash when executing GRANT ALL ON *.* with ON CLUSTER. It was broken in https://github.com/ClickHouse/ClickHouse/pull/35767. This closes [#38618](https://github.com/ClickHouse/ClickHouse/issues/38618). [#38674](https://github.com/ClickHouse/ClickHouse/pull/38674) ([Vitaly Baranov](https://github.com/vitlibar)). + +#### Build/Testing/Packaging Improvement +* Backported in [#38883](https://github.com/ClickHouse/ClickHouse/issues/38883): Add `clickhouse-diagnostics` binary to the packages. [#38647](https://github.com/ClickHouse/ClickHouse/pull/38647) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#38690](https://github.com/ClickHouse/ClickHouse/issues/38690): Fix incorrect columns order in subqueries of UNION (in case of duplicated columns in subselects may produce incorrect result). [#37887](https://github.com/ClickHouse/ClickHouse/pull/37887) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#38500](https://github.com/ClickHouse/ClickHouse/issues/38500): Do not allow recursive usage of OvercommitTracker during logging. Fixes [#37794](https://github.com/ClickHouse/ClickHouse/issues/37794) cc @tavplubix @davenger. [#38246](https://github.com/ClickHouse/ClickHouse/pull/38246) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#38712](https://github.com/ClickHouse/ClickHouse/issues/38712): Fix incorrect result of distributed queries with `DISTINCT` and `LIMIT`. Fixes [#38282](https://github.com/ClickHouse/ClickHouse/issues/38282). [#38371](https://github.com/ClickHouse/ClickHouse/pull/38371) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#38594](https://github.com/ClickHouse/ClickHouse/issues/38594): Fix parts removal (will be left forever if they had not been removed on server shutdown) after incorrect server shutdown. [#38486](https://github.com/ClickHouse/ClickHouse/pull/38486) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#38597](https://github.com/ClickHouse/ClickHouse/issues/38597): Fix table creation to avoid replication issues with pre-22.4 replicas. [#38541](https://github.com/ClickHouse/ClickHouse/pull/38541) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#38687](https://github.com/ClickHouse/ClickHouse/issues/38687): Now it's possible to start a clickhouse-server and attach/detach tables even for tables with the incorrect values of IPv4/IPv6 representation. Proper fix for issue [#35156](https://github.com/ClickHouse/ClickHouse/issues/35156). [#38590](https://github.com/ClickHouse/ClickHouse/pull/38590) ([alesapin](https://github.com/alesapin)). +* Backported in [#38665](https://github.com/ClickHouse/ClickHouse/issues/38665): Adapt some more nodes to avoid issues with pre-22.4 replicas. [#38627](https://github.com/ClickHouse/ClickHouse/pull/38627) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#38778](https://github.com/ClickHouse/ClickHouse/issues/38778): `rankCorr` function will work correctly if some arguments are NaNs. This closes [#38396](https://github.com/ClickHouse/ClickHouse/issues/38396). [#38722](https://github.com/ClickHouse/ClickHouse/pull/38722) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#38782](https://github.com/ClickHouse/ClickHouse/issues/38782): Fix use-after-free for Map combinator that leads to incorrect result. [#38748](https://github.com/ClickHouse/ClickHouse/pull/38748) ([Azat Khuzhin](https://github.com/azat)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix test for system table count in diag tool [#38236](https://github.com/ClickHouse/ClickHouse/pull/38236) ([Dale McDiarmid](https://github.com/gingerwizard)). +* Integration tests volume [#38291](https://github.com/ClickHouse/ClickHouse/pull/38291) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update docker-compose to try get rid of v1 errors [#38394](https://github.com/ClickHouse/ClickHouse/pull/38394) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix backports diff [#38703](https://github.com/ClickHouse/ClickHouse/pull/38703) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 64e2050f683..c6f32ae3171 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v22.6.3.35-stable 2022-07-06 v22.6.2.12-stable 2022-06-29 v22.6.1.1985-stable 2022-06-16 v22.5.1.2079-stable 2022-05-19 From ffac598029e8ece79cb0f4eb353f61fb8ad9ecf8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Jul 2022 12:45:48 +0200 Subject: [PATCH 267/627] Fix settings profile with seconds unit --- src/Core/SettingsFields.cpp | 3 ++- .../0_stateless/02294_fp_seconds_profile.reference | 2 ++ .../queries/0_stateless/02294_fp_seconds_profile.sql | 12 ++++++++++++ 3 files changed, 16 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02294_fp_seconds_profile.reference create mode 100644 tests/queries/0_stateless/02294_fp_seconds_profile.sql diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index d35865d35c6..51ffdd10190 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -192,7 +192,8 @@ namespace } template <> -SettingFieldSeconds::SettingFieldTimespan(const Field & f) : SettingFieldTimespan(float64AsSecondsToTimespan(fieldToNumber(f))) +SettingFieldSeconds::SettingFieldTimespan(const Field & f) + : SettingFieldTimespan(Poco::Timespan{float64AsSecondsToTimespan(fieldToNumber(f))}) { } diff --git a/tests/queries/0_stateless/02294_fp_seconds_profile.reference b/tests/queries/0_stateless/02294_fp_seconds_profile.reference new file mode 100644 index 00000000000..f765104a6b8 --- /dev/null +++ b/tests/queries/0_stateless/02294_fp_seconds_profile.reference @@ -0,0 +1,2 @@ +CREATE SETTINGS PROFILE `02294_profile1` SETTINGS timeout_before_checking_execution_speed = 3. TO default +CREATE SETTINGS PROFILE `02294_profile2` SETTINGS max_execution_time = 0.5 TO default diff --git a/tests/queries/0_stateless/02294_fp_seconds_profile.sql b/tests/queries/0_stateless/02294_fp_seconds_profile.sql new file mode 100644 index 00000000000..db1e298a6a9 --- /dev/null +++ b/tests/queries/0_stateless/02294_fp_seconds_profile.sql @@ -0,0 +1,12 @@ +-- Tags: no-parallel +-- Bug: https://github.com/ClickHouse/ClickHouse/issues/38863 + +DROP SETTINGS PROFILE IF EXISTS 02294_profile1, 02294_profile2; + +CREATE SETTINGS PROFILE 02294_profile1 SETTINGS timeout_before_checking_execution_speed = 3 TO default; +SHOW CREATE SETTINGS PROFILE 02294_profile1; + +CREATE SETTINGS PROFILE 02294_profile2 SETTINGS max_execution_time = 0.5 TO default; +SHOW CREATE SETTINGS PROFILE 02294_profile2; + +DROP SETTINGS PROFILE IF EXISTS 02294_profile1, 02294_profile2; From 4410caf31dce49e209de9972f0ad7aab44343d81 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Jul 2022 13:58:46 +0200 Subject: [PATCH 268/627] Retry docker buildx commands with progressive sleep in between --- tests/ci/docker_server.py | 68 ++++++++++++++++++++++----------------- 1 file changed, 39 insertions(+), 29 deletions(-) diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 8d44bace891..a54a8989565 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -6,6 +6,7 @@ import json import logging import subprocess import sys +import time from os import path as p, makedirs from typing import List, Tuple @@ -115,6 +116,34 @@ def parse_args() -> argparse.Namespace: return parser.parse_args() +def retry_popen(cmd: str) -> int: + max_retries = 5 + for retry in range(max_retries): + # From time to time docker build may failed. Curl issues, or even push + # It will sleep progressively 5, 15, 30 and 50 seconds between retries + progressive_sleep = 5 * sum(i + 1 for i in range(retry)) + if progressive_sleep: + logging.warning( + "The following command failed, sleep %s before retry: %s", + progressive_sleep, + cmd, + ) + time.sleep(progressive_sleep) + with subprocess.Popen( + cmd, + shell=True, + stderr=subprocess.STDOUT, + stdout=subprocess.PIPE, + universal_newlines=True, + ) as process: + for line in process.stdout: # type: ignore + print(line, end="") + retcode = process.wait() + if retcode == 0: + return 0 + return retcode + + def auto_release_type(version: ClickHouseVersion, release_type: str) -> str: if release_type != "auto": return release_type @@ -240,41 +269,22 @@ def build_and_push_image( ) cmd = " ".join(cmd_args) logging.info("Building image %s:%s for arch %s: %s", image.repo, tag, arch, cmd) - with subprocess.Popen( - cmd, - shell=True, - stderr=subprocess.STDOUT, - stdout=subprocess.PIPE, - universal_newlines=True, - ) as process: - for line in process.stdout: # type: ignore - print(line, end="") - retcode = process.wait() - if retcode != 0: - result.append((f"{image.repo}:{tag}-{arch}", "FAIL")) - return result - result.append((f"{image.repo}:{tag}-{arch}", "OK")) - with open(metadata_path, "rb") as m: - metadata = json.load(m) - digests.append(metadata["containerimage.digest"]) + if retry_popen(cmd) != 0: + result.append((f"{image.repo}:{tag}-{arch}", "FAIL")) + return result + result.append((f"{image.repo}:{tag}-{arch}", "OK")) + with open(metadata_path, "rb") as m: + metadata = json.load(m) + digests.append(metadata["containerimage.digest"]) if push: cmd = ( "docker buildx imagetools create " f"--tag {image.repo}:{tag} {' '.join(digests)}" ) logging.info("Pushing merged %s:%s image: %s", image.repo, tag, cmd) - with subprocess.Popen( - cmd, - shell=True, - stderr=subprocess.STDOUT, - stdout=subprocess.PIPE, - universal_newlines=True, - ) as process: - for line in process.stdout: # type: ignore - print(line, end="") - retcode = process.wait() - if retcode != 0: - result.append((f"{image.repo}:{tag}", "FAIL")) + if retry_popen(cmd) != 0: + result.append((f"{image.repo}:{tag}", "FAIL")) + return result else: logging.info( "Merging is available only on push, separate %s images are created", From cad876fc4292fc8bf21515884070979c939087bf Mon Sep 17 00:00:00 2001 From: San Date: Wed, 6 Jul 2022 22:04:53 +1000 Subject: [PATCH 269/627] Update SECURITY.md --- SECURITY.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/SECURITY.md b/SECURITY.md index 467bdb51466..81d2fd18fb2 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -37,7 +37,7 @@ The following versions of ClickHouse server are currently being supported with s We're extremely grateful for security researchers and users that report vulnerabilities to the ClickHouse Open Source Community. All reports are thoroughly investigated by developers. -To report a potential vulnerability in ClickHouse please send the details about it to [security@clickhouse.com](mailto:security@clickhouse.com). +To report a potential vulnerability in ClickHouse please send the details about it to [security@clickhouse.com](mailto:security@clickhouse.com). We do not offer any financial rewards for reporting issues to us using this method. Alternatively, you can also submit your findings through our public bug bounty program hosted by [Bugcrowd](https://bugcrowd.com/clickhouse) and be rewarded for it as per the program scope and rules of engagement. ### When Should I Report a Vulnerability? From 059c5c0f0a84a42b5ac0d889b3f9ee16e2f4df50 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 5 Jul 2022 22:47:00 +0000 Subject: [PATCH 270/627] Simple cleanup: context ptr usage in interpreters --- src/Interpreters/IInterpreterUnionOrSelectQuery.h | 4 ++-- src/Interpreters/InterpreterSelectQuery.cpp | 14 +++++++------- src/Interpreters/InterpreterSelectQuery.h | 14 +++++++------- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index 1af9a25614e..98e0432f3d5 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -12,12 +12,12 @@ namespace DB class IInterpreterUnionOrSelectQuery : public IInterpreter { public: - IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, ContextPtr context_, const SelectQueryOptions & options_) + IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, const ContextPtr & context_, const SelectQueryOptions & options_) : IInterpreterUnionOrSelectQuery(query_ptr_, Context::createCopy(context_), options_) { } - IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_, const SelectQueryOptions & options_) + IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, const ContextMutablePtr & context_, const SelectQueryOptions & options_) : query_ptr(query_ptr_) , context(context_) , options(options_) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 28438a86e47..c8c0c44e21f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -159,7 +159,7 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextPtr context_, + const ContextPtr & context_, const SelectQueryOptions & options_, const Names & required_result_column_names_) : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, required_result_column_names_) @@ -167,7 +167,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextMutablePtr context_, + const ContextMutablePtr & context_, const SelectQueryOptions & options_, const Names & required_result_column_names_) : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, required_result_column_names_) @@ -175,7 +175,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextPtr context_, + const ContextPtr & context_, Pipe input_pipe_, const SelectQueryOptions & options_) : InterpreterSelectQuery(query_ptr_, context_, std::move(input_pipe_), nullptr, options_.copy().noSubquery()) @@ -183,7 +183,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextPtr context_, + const ContextPtr & context_, const StoragePtr & storage_, const StorageMetadataPtr & metadata_snapshot_, const SelectQueryOptions & options_) @@ -192,7 +192,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextPtr context_, + const ContextPtr & context_, const SelectQueryOptions & options_, SubqueriesForSets subquery_for_sets_, PreparedSets prepared_sets_) @@ -282,7 +282,7 @@ static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextPtr context_, + const ContextPtr & context_, std::optional input_pipe_, const StoragePtr & storage_, const SelectQueryOptions & options_, @@ -304,7 +304,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextMutablePtr context_, + const ContextMutablePtr & context_, std::optional input_pipe_, const StoragePtr & storage_, const SelectQueryOptions & options_, diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 902d9f64058..703fb588080 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -52,27 +52,27 @@ public: InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextPtr context_, + const ContextPtr & context_, const SelectQueryOptions &, const Names & required_result_column_names_ = Names{}); InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextMutablePtr context_, + const ContextMutablePtr & context_, const SelectQueryOptions &, const Names & required_result_column_names_ = Names{}); /// Read data not from the table specified in the query, but from the prepared pipe `input`. InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextPtr context_, + const ContextPtr & context_, Pipe input_pipe_, const SelectQueryOptions & = {}); /// Read data not from the table specified in the query, but from the specified `storage_`. InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextPtr context_, + const ContextPtr & context_, const StoragePtr & storage_, const StorageMetadataPtr & metadata_snapshot_ = nullptr, const SelectQueryOptions & = {}); @@ -81,7 +81,7 @@ public: /// TODO: Find a general way of sharing sets among different interpreters, such as subqueries. InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextPtr context_, + const ContextPtr & context_, const SelectQueryOptions &, SubqueriesForSets subquery_for_sets_, PreparedSets prepared_sets_); @@ -134,7 +134,7 @@ public: private: InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextPtr context_, + const ContextPtr & context_, std::optional input_pipe, const StoragePtr & storage_, const SelectQueryOptions &, @@ -145,7 +145,7 @@ private: InterpreterSelectQuery( const ASTPtr & query_ptr_, - ContextMutablePtr context_, + const ContextMutablePtr & context_, std::optional input_pipe, const StoragePtr & storage_, const SelectQueryOptions &, From a3a94a1c090f08d90be5a33f336ca095d1ca7ab0 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Jul 2022 14:32:17 +0200 Subject: [PATCH 271/627] Avoid false positive red sanitizer asserts check in stress test --- docker/test/stress/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index b3ee4e62f20..13b39d61a4c 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -228,7 +228,7 @@ clickhouse-client --query "SELECT 'Server successfully started', 'OK'" >> /test_ # Sanitizer asserts grep -Fa "==================" /var/log/clickhouse-server/stderr.log | grep -v "in query:" >> /test_output/tmp grep -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp -zgrep -Fav "ASan doesn't fully support makecontext/swapcontext functions" /test_output/tmp > /dev/null \ +zgrep -Fav -e "ASan doesn't fully support makecontext/swapcontext functions" -e "DB::Exception" /test_output/tmp > /dev/null \ && echo -e 'Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'No sanitizer asserts\tOK' >> /test_output/test_results.tsv rm -f /test_output/tmp @@ -368,7 +368,7 @@ then # Sanitizer asserts zgrep -Fa "==================" /var/log/clickhouse-server/stderr.log >> /test_output/tmp zgrep -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp - zgrep -Fav "ASan doesn't fully support makecontext/swapcontext functions" /test_output/tmp > /dev/null \ + zgrep -Fav -e "ASan doesn't fully support makecontext/swapcontext functions" -e "DB::Exception" /test_output/tmp > /dev/null \ && echo -e 'Backward compatibility check: Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No sanitizer asserts\tOK' >> /test_output/test_results.tsv rm -f /test_output/tmp From 9ca08f8fdafbc002cc4639d97c0260b14c2bca97 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Jul 2022 12:59:25 +0000 Subject: [PATCH 272/627] Continue: ContextPtr -> const ContextPtr & in parameters --- src/Interpreters/InterpreterSelectQuery.cpp | 20 +++++++++---------- src/Interpreters/InterpreterSelectQuery.h | 6 +++--- .../evaluateConstantExpression.cpp | 8 ++++---- src/Interpreters/evaluateConstantExpression.h | 8 ++++---- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c8c0c44e21f..42a53fb7138 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -206,7 +206,7 @@ InterpreterSelectQuery::~InterpreterSelectQuery() = default; /** There are no limits on the maximum size of the result for the subquery. * Since the result of the query is not the result of the entire query. */ -static ContextPtr getSubqueryContext(ContextPtr context) +static ContextPtr getSubqueryContext(const ContextPtr & context) { auto subquery_context = Context::createCopy(context); Settings subquery_settings = context->getSettings(); @@ -239,7 +239,7 @@ static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & table /// Checks that the current user has the SELECT privilege. static void checkAccessRightsForSelect( - ContextPtr context, + const ContextPtr & context, const StorageID & table_id, const StorageMetadataPtr & table_metadata, const TreeRewriterResult & syntax_analyzer_result) @@ -816,7 +816,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() return analysis_result.final_projection->getResultColumns(); } -static Field getWithFillFieldValue(const ASTPtr & node, ContextPtr context) +static Field getWithFillFieldValue(const ASTPtr & node, const ContextPtr & context) { auto [field, type] = evaluateConstantExpression(node, context); @@ -826,7 +826,7 @@ static Field getWithFillFieldValue(const ASTPtr & node, ContextPtr context) return field; } -static std::pair> getWithFillStep(const ASTPtr & node, ContextPtr context) +static std::pair> getWithFillStep(const ASTPtr & node, const ContextPtr & context) { auto [field, type] = evaluateConstantExpression(node, context); @@ -839,7 +839,7 @@ static std::pair> getWithFillStep(const ASTPt throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); } -static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, ContextPtr context) +static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, const ContextPtr & context) { FillColumnDescription descr; @@ -886,7 +886,7 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or return descr; } -SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query, ContextPtr context_) +SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query, const ContextPtr & context_) { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); @@ -998,7 +998,7 @@ static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & quer return order_descr; } -static UInt64 getLimitUIntValue(const ASTPtr & node, ContextPtr context, const std::string & expr) +static UInt64 getLimitUIntValue(const ASTPtr & node, const ContextPtr & context, const std::string & expr) { const auto & [field, type] = evaluateConstantExpression(node, context); @@ -1016,7 +1016,7 @@ static UInt64 getLimitUIntValue(const ASTPtr & node, ContextPtr context, const s } -static std::pair getLimitLengthAndOffset(const ASTSelectQuery & query, ContextPtr context) +static std::pair getLimitLengthAndOffset(const ASTSelectQuery & query, const ContextPtr & context) { UInt64 length = 0; UInt64 offset = 0; @@ -1033,7 +1033,7 @@ static std::pair getLimitLengthAndOffset(const ASTSelectQuery & } -UInt64 InterpreterSelectQuery::getLimitForSorting(const ASTSelectQuery & query, ContextPtr context_) +UInt64 InterpreterSelectQuery::getLimitForSorting(const ASTSelectQuery & query, const ContextPtr & context_) { /// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY, neither ARRAY JOIN. if (!query.distinct && !query.limitBy() && !query.limit_with_ties && !query.arrayJoinExpressionList().first && query.limitLength()) @@ -1631,7 +1631,7 @@ static void executeMergeAggregatedImpl( } void InterpreterSelectQuery::addEmptySourceToQueryPlan( - QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, ContextPtr context_) + QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, const ContextPtr & context_) { Pipe pipe(std::make_shared(source_header)); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 703fb588080..a95ff00bc0d 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -112,7 +112,7 @@ public: bool hasAggregation() const { return query_analyzer->hasAggregation(); } static void addEmptySourceToQueryPlan( - QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, ContextPtr context_); + QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, const ContextPtr & context_); Names getRequiredColumns() { return required_columns; } @@ -128,8 +128,8 @@ public: /// It will set shard_num and shard_count to the client_info void setProperClientInfo(size_t replica_num, size_t replica_count); - static SortDescription getSortDescription(const ASTSelectQuery & query, ContextPtr context); - static UInt64 getLimitForSorting(const ASTSelectQuery & query, ContextPtr context); + static SortDescription getSortDescription(const ASTSelectQuery & query, const ContextPtr & context); + static UInt64 getLimitForSorting(const ASTSelectQuery & query, const ContextPtr & context); private: InterpreterSelectQuery( diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 039d79ed445..a5cdbf78070 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -42,7 +42,7 @@ static std::pair> getFieldAndDataTypeFro return {res, type}; } -std::pair> evaluateConstantExpression(const ASTPtr & node, ContextPtr context) +std::pair> evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context) { if (ASTLiteral * literal = node->as()) return getFieldAndDataTypeFromLiteral(literal); @@ -105,7 +105,7 @@ std::pair> evaluateConstantExpression(co } -ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, ContextPtr context) +ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const ContextPtr & context) { /// If it's already a literal. if (node->as()) @@ -113,7 +113,7 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, ContextPtr conte return std::make_shared(evaluateConstantExpression(node, context).first); } -ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, ContextPtr context) +ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, const ContextPtr & context) { if (const auto * id = node->as()) return std::make_shared(id->name()); @@ -121,7 +121,7 @@ ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, Cont return evaluateConstantExpressionAsLiteral(node, context); } -ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, ContextPtr context) +ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, const ContextPtr & context) { ASTPtr res = evaluateConstantExpressionOrIdentifierAsLiteral(node, context); auto & literal = res->as(); diff --git a/src/Interpreters/evaluateConstantExpression.h b/src/Interpreters/evaluateConstantExpression.h index cf01e834fcd..91f3ac5dffd 100644 --- a/src/Interpreters/evaluateConstantExpression.h +++ b/src/Interpreters/evaluateConstantExpression.h @@ -22,25 +22,25 @@ using ExpressionActionsPtr = std::shared_ptr; * Throws exception if it's not a constant expression. * Quite suboptimal. */ -std::pair> evaluateConstantExpression(const ASTPtr & node, ContextPtr context); +std::pair> evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context); /** Evaluate constant expression and returns ASTLiteral with its value. */ -ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, ContextPtr context); +ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const ContextPtr & context); /** Evaluate constant expression and returns ASTLiteral with its value. * Also, if AST is identifier, then return string literal with its name. * Useful in places where some name may be specified as identifier, or as result of a constant expression. */ -ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, ContextPtr context); +ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, const ContextPtr & context); /** The same as evaluateConstantExpressionOrIdentifierAsLiteral(...), * but if result is an empty string, replace it with current database name * or default database name. */ -ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, ContextPtr context); +ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, const ContextPtr & context); /** Try to fold condition to countable set of constant values. * @param node a condition that we try to fold. From 653e7cbeaf115585bb24daba4a1a4f11e8d7569f Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 6 Jul 2022 14:15:24 +0000 Subject: [PATCH 273/627] Add compression level --- src/Client/ClientBase.cpp | 12 ++++++- src/Parsers/ASTQueryWithOutput.h | 1 + src/Parsers/ParserQueryWithOutput.cpp | 3 ++ .../02353_compression_level.reference | 18 ++++++++++ .../0_stateless/02353_compression_level.sh | 34 +++++++++++++++++++ 5 files changed, 67 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02353_compression_level.reference create mode 100755 tests/queries/0_stateless/02353_compression_level.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8230c97f49c..13056c65449 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -525,16 +525,26 @@ try out_file = out_file_node.value.safeGet(); std::string compression_method; + UInt64 compression_level = 3; if (query_with_output->compression) { const auto & compression_method_node = query_with_output->compression->as(); compression_method = compression_method_node.value.safeGet(); + + if (query_with_output->compression_level) + { + const auto & compression_level_node = query_with_output->compression_level->as(); + bool res = compression_level_node.value.tryGet(compression_level); + + if (!res || compression_level < 1 || compression_level > 9) + throw Exception("Invalid compression level, must be positive integer in range 1-9", ErrorCodes::BAD_ARGUMENTS); + } } out_file_buf = wrapWriteBufferWithCompressionMethod( std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), chooseCompressionMethod(out_file, compression_method), - /* compression level = */ 3 + compression_level ); // We are writing to file, so default format is the same as in non-interactive mode. diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index 10852467a76..a34826d128c 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -18,6 +18,7 @@ public: ASTPtr format; ASTPtr settings_ast; ASTPtr compression; + ASTPtr compression_level; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const final; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index a6af19fc670..f3c67791309 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -96,6 +96,9 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserStringLiteral compression; if (!compression.parse(pos, query_with_output.compression, expected)) return false; + + ParserNumber compression_level; + compression_level.parse(pos, query_with_output.compression_level, expected); } query_with_output.children.push_back(query_with_output.out_file); diff --git a/tests/queries/0_stateless/02353_compression_level.reference b/tests/queries/0_stateless/02353_compression_level.reference new file mode 100644 index 00000000000..0ab28a4efa5 --- /dev/null +++ b/tests/queries/0_stateless/02353_compression_level.reference @@ -0,0 +1,18 @@ +6000 5999 13 +6000 5999 13 +Ok +6000 5999 13 +6000 5999 13 +Ok +6000 5999 13 +6000 5999 13 +Ok +6000 5999 13 +6000 5999 13 +Ok +6000 5999 13 +6000 5999 13 +Ok +6000 5999 13 +6000 5999 13 +Ok diff --git a/tests/queries/0_stateless/02353_compression_level.sh b/tests/queries/0_stateless/02353_compression_level.sh new file mode 100755 index 00000000000..364479063a2 --- /dev/null +++ b/tests/queries/0_stateless/02353_compression_level.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel +# Tag no-fasttest: depends on brotli and bzip2 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +WORKING_FOLDER_02353="${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}" + +rm -rf "${WORKING_FOLDER_02353}" +mkdir "${WORKING_FOLDER_02353}" + +for m in gz br xz zst lz4 bz2 +do + ${CLICKHOUSE_CLIENT} --query "SELECT number, 'Hello, world!' FROM numbers(6000) INTO OUTFILE '${WORKING_FOLDER_02353}/${m}_1.${m}' COMPRESSION '${m}' 1" + ${CLICKHOUSE_CLIENT} --query "SELECT number, 'Hello, world!' FROM numbers(6000) INTO OUTFILE '${WORKING_FOLDER_02353}/${m}_9.${m}' COMPRESSION '${m}' 9" + + ${CLICKHOUSE_CLIENT} --query "SELECT count(), max(x), avg(length(s)) FROM file('${WORKING_FOLDER_02353}/${m}_1.${m}', 'TabSeparated', 'x UInt32, s String')" + ${CLICKHOUSE_CLIENT} --query "SELECT count(), max(x), avg(length(s)) FROM file('${WORKING_FOLDER_02353}/${m}_9.${m}', 'TabSeparated', 'x UInt32, s String')" + + level_1=$(du -b ${WORKING_FOLDER_02353}/${m}_1.${m} | awk '{print $1}') + level_9=$(du -b ${WORKING_FOLDER_02353}/${m}_9.${m} | awk '{print $1}') + + if [ "$level_1" != "$level_9" ]; then + echo "Ok" + else + echo "Failed" + fi + +done + +rm -rf "${WORKING_FOLDER_02353}" From f1bec4ef21666a1782af46b430bb13eff5b77e83 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Jul 2022 14:21:59 +0000 Subject: [PATCH 274/627] EXPLAIN AST option after_rewrite --- src/Interpreters/InterpreterExplainQuery.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 4d2628d8b26..e0de29fe415 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -146,12 +146,14 @@ namespace struct QueryASTSettings { bool graph = false; + bool after_rewrite = false; constexpr static char name[] = "AST"; std::unordered_map> boolean_settings = { {"graph", graph}, + {"after_rewrite", after_rewrite} }; }; @@ -278,6 +280,12 @@ QueryPipeline InterpreterExplainQuery::executeImpl() case ASTExplainQuery::ParsedAST: { auto settings = checkAndGetSettings(ast.getSettings()); + if (settings.after_rewrite) + { + ExplainAnalyzedSyntaxVisitor::Data data(getContext()); + ExplainAnalyzedSyntaxVisitor(data).visit(query); + } + if (settings.graph) dumpASTInDotFormat(*ast.getExplainedQuery(), buf); else From fa8eb355998ce60cb1ab1aa6705c901576302d6a Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 30 Mar 2022 10:07:09 +0000 Subject: [PATCH 275/627] Pipeline for full sorting merge join --- src/Core/SettingsEnums.cpp | 3 +- src/Core/SettingsEnums.h | 1 + src/Interpreters/ExpressionAnalyzer.cpp | 11 ++- src/Interpreters/FullSortingMergeJoin.h | 52 +++++++++++++ src/Interpreters/HashJoin.h | 13 ++++ src/Interpreters/IJoin.h | 27 ++++++- src/Interpreters/InterpreterSelectQuery.cpp | 42 ++++++++-- src/Interpreters/TableJoin.h | 5 +- src/Interpreters/TreeRewriter.cpp | 2 + src/Processors/Merges/IMergingTransform.cpp | 19 +++++ src/Processors/Merges/IMergingTransform.h | 18 +++++ src/Processors/Port.cpp | 7 +- src/Processors/QueryPlan/JoinStep.cpp | 4 + .../Transforms/MergeJoinTransform.cpp | 30 +++++++ .../Transforms/MergeJoinTransform.h | 78 +++++++++++++++++++ src/QueryPipeline/QueryPipelineBuilder.cpp | 52 +++++++++++++ src/QueryPipeline/QueryPipelineBuilder.h | 16 ++++ 17 files changed, 369 insertions(+), 11 deletions(-) create mode 100644 src/Interpreters/FullSortingMergeJoin.h create mode 100644 src/Processors/Transforms/MergeJoinTransform.cpp create mode 100644 src/Processors/Transforms/MergeJoinTransform.h diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 66d38337243..3585cffb8ec 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -36,7 +36,8 @@ IMPLEMENT_SETTING_MULTI_ENUM(JoinAlgorithm, ErrorCodes::UNKNOWN_JOIN, {"partial_merge", JoinAlgorithm::PARTIAL_MERGE}, {"prefer_partial_merge", JoinAlgorithm::PREFER_PARTIAL_MERGE}, {"parallel_hash", JoinAlgorithm::PARALLEL_HASH}, - {"direct", JoinAlgorithm::DIRECT}}) + {"direct", JoinAlgorithm::DIRECT}, + {"full_sorting_merge", JoinAlgorithm::FULL_SORTING_MERGE}}) IMPLEMENT_SETTING_ENUM(TotalsMode, ErrorCodes::UNKNOWN_TOTALS_MODE, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index bb9311a808b..6b9ff8277b1 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -44,6 +44,7 @@ enum class JoinAlgorithm PREFER_PARTIAL_MERGE, PARALLEL_HASH, DIRECT, + FULL_SORTING_MERGE, }; DECLARE_SETTING_MULTI_ENUM(JoinAlgorithm) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 66d852e31c1..944f41a249d 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include @@ -1087,8 +1088,16 @@ static std::shared_ptr chooseJoinAlgorithm(std::shared_ptr ana return std::make_shared(analyzed_join, right_sample_block); } else if (analyzed_join->forceMergeJoin() || (analyzed_join->preferMergeJoin() && allow_merge_join)) + { return std::make_shared(analyzed_join, right_sample_block); - return std::make_shared(analyzed_join, right_sample_block); + } + else if (analyzed_join->forceFullSortingMergeJoin()) + { + if (analyzed_join->getClauses().size() != 1) + throw Exception("Full sorting merge join is supported only for single-condition joins", ErrorCodes::NOT_IMPLEMENTED); + return std::make_shared(*analyzed_join); + } + return std::make_shared(analyzed_join, sample_block); } static std::unique_ptr buildJoinedPlan( diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h new file mode 100644 index 00000000000..cc1e85abdaf --- /dev/null +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -0,0 +1,52 @@ +#include +#include +#include + +namespace DB +{ + +/// Dummy class, actual joining is done by MergeTransform +class FullSortingMergeJoin : public IJoin +{ +public: + explicit FullSortingMergeJoin(TableJoin & table_join_) + : table_join(table_join_) + { + LOG_TRACE(&Poco::Logger::get("FullSortingMergeJoin"), "Will use full sorting merge join"); + } + + const TableJoin & getTableJoin() const override { return table_join; } + + bool addJoinedBlock(const Block & /* block */, bool /* check_limits */) override { __builtin_unreachable(); } + + void checkTypesOfKeys(const Block & /* block */) const override + { + } + + /// Used just to get result header + void joinBlock(Block & block, std::shared_ptr & /* not_processed */) override + { + UNUSED(block); + /// ... + } + + void setTotals(const Block & /* block */) override { __builtin_unreachable(); } + const Block & getTotals() const override { __builtin_unreachable(); } + + size_t getTotalRowCount() const override { __builtin_unreachable(); } + size_t getTotalByteCount() const override { __builtin_unreachable(); } + bool alwaysReturnsEmptySet() const override { __builtin_unreachable(); } + + std::shared_ptr + getNonJoinedBlocks(const Block & /* left_sample_block */, const Block & /* result_sample_block */, UInt64 /* max_block_size */) const override + { + __builtin_unreachable(); + } + + virtual JoinPipelineType pipelineType() const override { return JoinPipelineType::YShaped; } + +private: + TableJoin & table_join; +}; + +} diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index ae6b059696c..cc0c3f25d20 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -173,6 +173,19 @@ public: bool isFilled() const override { return from_storage_join || data->type == Type::DICT; } + /** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. + */ + void setTotals(const Block & block) override { totals = block; } + const Block & getTotals() const override { return totals; } + + virtual JoinPipelineType pipelineType() const override + { + bool is_filled = from_storage_join || data->type == Type::DICT; + if (is_filled) + return JoinPipelineType::Filled; + return JoinPipelineType::RShaped; + } + /** For RIGHT and FULL JOINs. * A stream that will contain default values from left table, joined with rows from right table, that was not joined before. * Use only after all calls to joinBlock was done. diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 2cc4ce175ba..9874c357c52 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -6,16 +6,39 @@ #include #include #include +#include namespace DB { +class Block; + struct ExtraBlock; using ExtraBlockPtr = std::shared_ptr; class TableJoin; class NotJoinedBlocks; +enum class JoinPipelineType +{ + /* + * Right stream processed first. + * The pipeline is not sorted. + */ + RShaped, + + /* + * Only left stream is processed, right is already filled. + */ + Filled, + + /* + * The pipeline is created from the left and right streams processed with merging transform. + * The pipeline is sorted. + */ + YShaped, +}; + class IJoin { public: @@ -48,7 +71,8 @@ public: /// StorageJoin/Dictionary is already filled. No need to call addJoinedBlock. /// Different query plan is used for such joins. - virtual bool isFilled() const { return false; } + virtual bool isFilled() const { return pipelineType() == JoinPipelineType::Filled; } + virtual JoinPipelineType pipelineType() const { return JoinPipelineType::RShaped; } // That can run FillingRightJoinSideTransform parallelly virtual bool supportParallelJoin() const { return false; } @@ -60,6 +84,7 @@ private: Block totals; }; + using JoinPtr = std::shared_ptr; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 28438a86e47..fb63be2aae2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -83,9 +83,10 @@ #include #include #include +#include +#include #include #include -#include namespace DB @@ -1347,6 +1348,35 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( + plan.getCurrentDataStream(), + order_descr, + settings.max_block_size, + 0 /* LIMIT */, + SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), + settings.max_bytes_before_remerge_sort, + settings.remerge_sort_lowered_memory_bytes_ratio, + settings.max_bytes_before_external_sort, + this->context->getTemporaryVolume(), + settings.min_free_disk_space_for_temporary_data); + sorting_step->setStepDescription("Sort before JOIN"); + plan.addStep(std::move(sorting_step)); + }; + + if (expressions.join->pipelineType() == JoinPipelineType::YShaped) + { + const auto & join_clause = expressions.join->getTableJoin().getOnlyClause(); + add_sorting(query_plan, join_clause.key_names_left); + add_sorting(*joined_plan, join_clause.key_names_right); + } + QueryPlanStepPtr join_step = std::make_unique( query_plan.getCurrentDataStream(), joined_plan->getCurrentDataStream(), @@ -1355,7 +1385,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

setStepDescription("JOIN"); + join_step->setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType())); std::vector plans; plans.emplace_back(std::make_unique(std::move(query_plan))); plans.emplace_back(std::move(joined_plan)); @@ -1994,7 +2024,8 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); - /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY, WITH TIES but LIMIT is specified, and limit + offset < max_block_size, + /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, JOIN, LIMIT BY, WITH TIES + * but LIMIT is specified, and limit + offset < max_block_size, * then as the block size we will use limit + offset (not to read more from the table than requested), * and also set the number of threads to 1. */ @@ -2006,13 +2037,14 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc && !query.having() && !query.orderBy() && !query.limitBy() - && query.limitLength() + && !query.join() && !query_analyzer->hasAggregation() && !query_analyzer->hasWindow() + && query.limitLength() && limit_length <= std::numeric_limits::max() - limit_offset && limit_length + limit_offset < max_block_size) { - max_block_size = std::max(UInt64{1}, limit_length + limit_offset); + max_block_size = std::max(1, limit_length + limit_offset); max_threads_execute_query = max_streams = 1; } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index ff8c62100b6..e44a0657da3 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -199,6 +199,10 @@ public: bool preferMergeJoin() const { return join_algorithm == MultiEnum(JoinAlgorithm::PREFER_PARTIAL_MERGE); } bool forceMergeJoin() const { return join_algorithm == MultiEnum(JoinAlgorithm::PARTIAL_MERGE); } + + bool allowParallelHashJoin() const; + bool forceFullSortingMergeJoin() const { return join_algorithm.isSet(JoinAlgorithm::FULL_SORTING_MERGE); } + bool forceHashJoin() const { /// HashJoin always used for DictJoin @@ -206,7 +210,6 @@ public: || join_algorithm == MultiEnum(JoinAlgorithm::HASH) || join_algorithm == MultiEnum(JoinAlgorithm::PARALLEL_HASH); } - bool allowParallelHashJoin() const; bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(table_join.kind); } bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(table_join.kind); } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 9a4a956f3dc..73ec7fb1d7a 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -27,6 +28,7 @@ #include #include +#include #include #include #include diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index 226f55b3e92..57e7ce20124 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -21,6 +21,25 @@ IMergingTransformBase::IMergingTransformBase( { } +static InputPorts createPorts(const Blocks & blocks) +{ + InputPorts ports; + for (const auto & block : blocks) + ports.emplace_back(block); + return ports; +} + +IMergingTransformBase::IMergingTransformBase( + const Blocks & input_headers, + const Block & output_header, + bool have_all_inputs_, + UInt64 limit_hint_) + : IProcessor(createPorts(input_headers), {output_header}) + , have_all_inputs(have_all_inputs_) + , limit_hint(limit_hint_) +{ +} + void IMergingTransformBase::onNewInput() { throw Exception("onNewInput is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 144c47c96f5..62cfc784ab6 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -19,6 +19,12 @@ public: bool have_all_inputs_, UInt64 limit_hint_); + IMergingTransformBase( + const Blocks & input_headers, + const Block & output_header, + bool have_all_inputs_, + UInt64 limit_hint_); + OutputPort & getOutputPort() { return outputs.front(); } /// Methods to add additional input port. It is possible to do only before the first call of `prepare`. @@ -82,6 +88,18 @@ public: { } + template + IMergingTransform( + const Blocks & input_headers, + const Block & output_header, + bool have_all_inputs_, + UInt64 limit_hint_, + Args && ... args) + : IMergingTransformBase(input_headers, output_header, have_all_inputs_, limit_hint_) + , algorithm(std::forward(args) ...) + { + } + void work() override { if (!state.init_chunks.empty()) diff --git a/src/Processors/Port.cpp b/src/Processors/Port.cpp index e3ce31e1ac8..86431dbc6e6 100644 --- a/src/Processors/Port.cpp +++ b/src/Processors/Port.cpp @@ -10,8 +10,11 @@ namespace ErrorCodes void connect(OutputPort & output, InputPort & input) { - if (input.state || output.state) - throw Exception("Port is already connected", ErrorCodes::LOGICAL_ERROR); + if (input.state) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Port is already connected, (header: [{}])", input.header.dumpStructure()); + + if (output.state) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Port is already connected, (header: [{}])", output.header.dumpStructure()); auto out_name = output.getProcessor().getName(); auto in_name = input.getProcessor().getName(); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 209d91af4d8..0ed23be9855 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -32,6 +33,9 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (pipelines.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); + if (join->pipelineType() == JoinPipelineType::YShaped) + return QueryPipelineBuilder::joinPipelines2(std::move(pipelines[0]), std::move(pipelines[1]), max_block_size, &processors); + return QueryPipelineBuilder::joinPipelines(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, max_streams, keep_left_read_in_order, &processors); } diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp new file mode 100644 index 00000000000..6ea36035c19 --- /dev/null +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -0,0 +1,30 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +MergeJoinTransform::MergeJoinTransform( + const Blocks & input_headers, + const Block & output_header, + UInt64 limit_hint) + : IMergingTransform(input_headers, output_header, true, limit_hint) + , log(&Poco::Logger::get("MergeJoinTransform")) +{ + LOG_TRACE(log, "Will use MergeJoinTransform"); +} + +void MergeJoinTransform::onFinish() +{ + LOG_TRACE(log, "MergeJoinTransform::onFinish"); +} + + +} diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h new file mode 100644 index 00000000000..fc50a8b0620 --- /dev/null +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -0,0 +1,78 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + + +namespace Poco { class Logger; } + +namespace DB +{ + +/* + * This class is used to join chunks from two sorted streams. + * It is used in MergeJoinTransform. + */ +class MergeJoinAlgorithm final : public IMergingAlgorithm +{ +public: + MergeJoinAlgorithm() + : log(&Poco::Logger::get("MergeJoinAlgorithm")) + { + } + + virtual void initialize(Inputs inputs) override + { + LOG_DEBUG(log, "MergeJoinAlgorithm initialize, number of inputs: {}", inputs.size()); + } + + virtual void consume(Input & input, size_t source_num) override + { + LOG_DEBUG(log, "Consume from {}", source_num); + if (source_num == 0) + { + chunk = std::move(input.chunk); + } + } + + virtual Status merge() override + { + LOG_DEBUG(log, "merge (chunk: {})", bool(chunk)); + + if (chunk) + return Status(std::move(chunk), true); + + return Status(0); + } + +private: + Inputs current_inputs; + + Chunk chunk; + Poco::Logger * log; +}; + +class MergeJoinTransform final : public IMergingTransform +{ +public: + MergeJoinTransform( + const Blocks & input_headers, + const Block & output_header, + UInt64 limit_hint = 0); + + String getName() const override { return "MergeJoinTransform"; } + +protected: + void onFinish() override; + UInt64 elapsed_ns = 0; + + Poco::Logger * log; +}; + + +} diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 7eccb1c70f5..c3a2835c6c1 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -9,6 +9,8 @@ #include #include #include +#include +#include #include #include #include @@ -298,6 +300,56 @@ QueryPipelineBuilder QueryPipelineBuilder::unitePipelines( return pipeline; } +QueryPipelineBuilderPtr QueryPipelineBuilder::mergePipelines( + QueryPipelineBuilderPtr left, + QueryPipelineBuilderPtr right, + ProcessorPtr transform, + Processors * collected_processors) +{ + if (transform->getOutputs().size() != 1) + throw Exception("Merge transform must have exactly 1 output", ErrorCodes::LOGICAL_ERROR); + + connect(*left->pipe.output_ports.front(), transform->getInputs().front()); + connect(*right->pipe.output_ports.front(), transform->getInputs().back()); + + if (collected_processors) + collected_processors->emplace_back(transform); + + left->pipe.output_ports.front() = &transform->getOutputs().front(); + left->pipe.processors.emplace_back(transform); + + left->pipe.processors.insert(left->pipe.processors.end(), right->pipe.processors.begin(), right->pipe.processors.end()); + left->pipe.holder = std::move(right->pipe.holder); + left->pipe.header = left->pipe.output_ports.front()->getHeader(); + left->pipe.max_parallel_streams = std::max(left->pipe.max_parallel_streams, right->pipe.max_parallel_streams); + return left; +} + +std::unique_ptr QueryPipelineBuilder::joinPipelines2( + std::unique_ptr left, + std::unique_ptr right, + size_t max_block_size, + Processors * collected_processors) +{ + UNUSED(max_block_size); + + left->checkInitializedAndNotCompleted(); + right->checkInitializedAndNotCompleted(); + + left->pipe.dropExtremes(); + right->pipe.dropExtremes(); + + if (left->pipe.output_ports.size() != 1 || right->pipe.output_ports.size() != 1) + throw Exception("Join is supported only for pipelines with one output port", ErrorCodes::LOGICAL_ERROR); + + Blocks inputs = {left->getHeader(), right->getHeader()}; + Block output = left->getHeader(); + auto joining = std::make_shared(inputs, output); + + auto result = mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); + return result; +} + std::unique_ptr QueryPipelineBuilder::joinPipelines( std::unique_ptr left, std::unique_ptr right, diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 77ca3f05031..efba419327b 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -30,6 +30,9 @@ struct ExpressionActionsSettings; class IJoin; using JoinPtr = std::shared_ptr; +class QueryPipelineBuilder; +using QueryPipelineBuilderPtr = std::unique_ptr; + class QueryPipelineBuilder { public: @@ -97,6 +100,13 @@ public: size_t max_threads_limit = 0, Processors * collected_processors = nullptr); + + static QueryPipelineBuilderPtr mergePipelines( + QueryPipelineBuilderPtr left, + QueryPipelineBuilderPtr right, + ProcessorPtr transform, + Processors * collected_processors); + /// Join two pipelines together using JoinPtr. /// If collector is used, it will collect only newly-added processors, but not processors from pipelines. static std::unique_ptr joinPipelines( @@ -108,6 +118,12 @@ public: bool keep_left_read_in_order, Processors * collected_processors = nullptr); + static std::unique_ptr joinPipelines2( + std::unique_ptr left, + std::unique_ptr right, + size_t max_block_size, + Processors * collected_processors = nullptr); + /// Add other pipeline and execute it before current one. /// Pipeline must have empty header, it should not generate any chunk. /// This is used for CreatingSets. From 2a0e497ceddf72bfa1e53665e3f6dd225cf38acd Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 1 Apr 2022 18:16:38 +0000 Subject: [PATCH 276/627] Rename enumerators in JoinPipelineType --- src/Interpreters/HashJoin.h | 4 ++-- src/Interpreters/IJoin.h | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index cc0c3f25d20..8402a220f21 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -182,8 +182,8 @@ public: { bool is_filled = from_storage_join || data->type == Type::DICT; if (is_filled) - return JoinPipelineType::Filled; - return JoinPipelineType::RShaped; + return JoinPipelineType::FilledRight; + return JoinPipelineType::FillRightFirst; } /** For RIGHT and FULL JOINs. diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 9874c357c52..83c1eb31277 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -25,12 +25,12 @@ enum class JoinPipelineType * Right stream processed first. * The pipeline is not sorted. */ - RShaped, + FillRightFirst, /* * Only left stream is processed, right is already filled. */ - Filled, + FilledRight, /* * The pipeline is created from the left and right streams processed with merging transform. @@ -71,8 +71,8 @@ public: /// StorageJoin/Dictionary is already filled. No need to call addJoinedBlock. /// Different query plan is used for such joins. - virtual bool isFilled() const { return pipelineType() == JoinPipelineType::Filled; } - virtual JoinPipelineType pipelineType() const { return JoinPipelineType::RShaped; } + virtual bool isFilled() const { return pipelineType() == JoinPipelineType::FilledRight; } + virtual JoinPipelineType pipelineType() const { return JoinPipelineType::FillRightFirst; } // That can run FillingRightJoinSideTransform parallelly virtual bool supportParallelJoin() const { return false; } From 4a161959643d8a8c3768942a896ae0a1f9f883f7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 1 Apr 2022 18:20:58 +0000 Subject: [PATCH 277/627] Calculate output header for full sorting merge join --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/FullSortingMergeJoin.h | 14 +++-- src/Processors/QueryPlan/JoinStep.cpp | 2 +- .../Transforms/MergeJoinTransform.cpp | 61 +++++++++++++++++++ .../Transforms/MergeJoinTransform.h | 33 ++-------- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 +- src/QueryPipeline/QueryPipelineBuilder.h | 1 + 7 files changed, 80 insertions(+), 37 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 944f41a249d..2f07ecf8dc2 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1095,7 +1095,7 @@ static std::shared_ptr chooseJoinAlgorithm(std::shared_ptr ana { if (analyzed_join->getClauses().size() != 1) throw Exception("Full sorting merge join is supported only for single-condition joins", ErrorCodes::NOT_IMPLEMENTED); - return std::make_shared(*analyzed_join); + return std::make_shared(*analyzed_join, sample_block); } return std::make_shared(analyzed_join, sample_block); } diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index cc1e85abdaf..5bc764ce375 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -9,8 +9,9 @@ namespace DB class FullSortingMergeJoin : public IJoin { public: - explicit FullSortingMergeJoin(TableJoin & table_join_) + explicit FullSortingMergeJoin(TableJoin & table_join_, const Block & right_sample_block_) : table_join(table_join_) + , right_sample_block(right_sample_block_) { LOG_TRACE(&Poco::Logger::get("FullSortingMergeJoin"), "Will use full sorting merge join"); } @@ -26,12 +27,13 @@ public: /// Used just to get result header void joinBlock(Block & block, std::shared_ptr & /* not_processed */) override { - UNUSED(block); - /// ... + for (const auto & col : right_sample_block) + block.insert(col); + block = materializeBlock(block).cloneEmpty(); } - void setTotals(const Block & /* block */) override { __builtin_unreachable(); } - const Block & getTotals() const override { __builtin_unreachable(); } + void setTotals(const Block & block) override { totals = block; } + const Block & getTotals() const override { return totals; } size_t getTotalRowCount() const override { __builtin_unreachable(); } size_t getTotalByteCount() const override { __builtin_unreachable(); } @@ -47,6 +49,8 @@ public: private: TableJoin & table_join; + Block right_sample_block; + Block totals; }; } diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 0ed23be9855..4835fc08078 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -34,7 +34,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); if (join->pipelineType() == JoinPipelineType::YShaped) - return QueryPipelineBuilder::joinPipelines2(std::move(pipelines[0]), std::move(pipelines[1]), max_block_size, &processors); + return QueryPipelineBuilder::joinPipelines2(std::move(pipelines[0]), std::move(pipelines[1]), output_stream->header, max_block_size, &processors); return QueryPipelineBuilder::joinPipelines(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, max_streams, keep_left_read_in_order, &processors); } diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 6ea36035c19..dc35cfe2e74 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -11,6 +11,67 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +MergeJoinAlgorithm::MergeJoinAlgorithm() + : log(&Poco::Logger::get("MergeJoinAlgorithm")) +{ +} + +void MergeJoinAlgorithm::initialize(Inputs inputs) +{ + LOG_DEBUG(log, "MergeJoinAlgorithm initialize, number of inputs: {}", inputs.size()); + current_inputs = std::move(inputs); + chunks.resize(current_inputs.size()); + for (size_t i = 0; i < current_inputs.size(); ++i) + { + consume(current_inputs[i], i); + } +} + +static void prepareChunk(Chunk & chunk) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); +} + +void MergeJoinAlgorithm::consume(Input & input, size_t source_num) +{ + LOG_DEBUG(log, "Consume from {}", source_num); + + prepareChunk(input.chunk); + chunks[source_num] = std::move(input.chunk); +} + +IMergingAlgorithm::Status MergeJoinAlgorithm::merge() +{ + if (!chunks[0]) + { + return Status(0); + } + if (!chunks[1]) + { + return Status(1); + } + + size_t rows = std::min(chunks[0].getNumRows(), chunks[1].getNumRows()); + Chunk res; + + for (auto & col : chunks[0].detachColumns()) + { + col->cut(0, rows); + res.addColumn(std::move(col)); + } + for (auto & col : chunks[1].detachColumns()) + { + col->cut(0, rows); + res.addColumn(std::move(col)); + } + return Status(std::move(res), true); +} + MergeJoinTransform::MergeJoinTransform( const Blocks & input_headers, const Block & output_header, diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index fc50a8b0620..95a6f919cd6 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -21,39 +21,16 @@ namespace DB class MergeJoinAlgorithm final : public IMergingAlgorithm { public: - MergeJoinAlgorithm() - : log(&Poco::Logger::get("MergeJoinAlgorithm")) - { - } + MergeJoinAlgorithm(); - virtual void initialize(Inputs inputs) override - { - LOG_DEBUG(log, "MergeJoinAlgorithm initialize, number of inputs: {}", inputs.size()); - } - - virtual void consume(Input & input, size_t source_num) override - { - LOG_DEBUG(log, "Consume from {}", source_num); - if (source_num == 0) - { - chunk = std::move(input.chunk); - } - } - - virtual Status merge() override - { - LOG_DEBUG(log, "merge (chunk: {})", bool(chunk)); - - if (chunk) - return Status(std::move(chunk), true); - - return Status(0); - } + virtual void initialize(Inputs inputs) override; + virtual void consume(Input & input, size_t source_num) override; + virtual Status merge() override; private: Inputs current_inputs; - Chunk chunk; + Chunks chunks; Poco::Logger * log; }; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index c3a2835c6c1..d9725aeafed 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -328,6 +328,7 @@ QueryPipelineBuilderPtr QueryPipelineBuilder::mergePipelines( std::unique_ptr QueryPipelineBuilder::joinPipelines2( std::unique_ptr left, std::unique_ptr right, + const Block & out_header, size_t max_block_size, Processors * collected_processors) { @@ -343,8 +344,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelines2( throw Exception("Join is supported only for pipelines with one output port", ErrorCodes::LOGICAL_ERROR); Blocks inputs = {left->getHeader(), right->getHeader()}; - Block output = left->getHeader(); - auto joining = std::make_shared(inputs, output); + auto joining = std::make_shared(inputs, out_header); auto result = mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); return result; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index efba419327b..3c9aa036851 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -121,6 +121,7 @@ public: static std::unique_ptr joinPipelines2( std::unique_ptr left, std::unique_ptr right, + const Block & out_header, size_t max_block_size, Processors * collected_processors = nullptr); From 8dce97123c24660f222158ab3346430ed6343181 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sun, 3 Apr 2022 15:54:22 +0000 Subject: [PATCH 278/627] wip: any inner full sorting join --- src/Processors/Merges/IMergingTransform.cpp | 4 + src/Processors/Merges/IMergingTransform.h | 9 + src/Processors/QueryPlan/JoinStep.cpp | 5 +- .../Transforms/MergeJoinTransform.cpp | 183 +++++++++++++++--- .../Transforms/MergeJoinTransform.h | 22 ++- src/QueryPipeline/QueryPipelineBuilder.cpp | 7 +- src/QueryPipeline/QueryPipelineBuilder.h | 3 +- 7 files changed, 202 insertions(+), 31 deletions(-) diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index 57e7ce20124..1b847069fea 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -189,6 +189,10 @@ IProcessor::Status IMergingTransformBase::prepare() state.has_input = true; } + else + { + state.no_data = true; + } state.need_data = false; } diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 62cfc784ab6..061750d91e4 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -46,6 +46,7 @@ protected: bool has_input = false; bool is_finished = false; bool need_data = false; + bool no_data = false; size_t next_input_to_read = 0; IMergingAlgorithm::Inputs init_chunks; @@ -98,6 +99,7 @@ public: : IMergingTransformBase(input_headers, output_header, have_all_inputs_, limit_hint_) , algorithm(std::forward(args) ...) { + empty_chunk_on_finish = true; } void work() override @@ -112,6 +114,12 @@ public: algorithm.consume(state.input_chunk, state.next_input_to_read); state.has_input = false; } + else if (state.no_data && empty_chunk_on_finish) + { + IMergingAlgorithm::Input current_input; + algorithm.consume(current_input, state.next_input_to_read); + state.no_data = false; + } IMergingAlgorithm::Status status = algorithm.merge(); @@ -141,6 +149,7 @@ protected: /// Profile info. Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; + bool empty_chunk_on_finish = false; private: using IMergingTransformBase::state; }; diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 4835fc08078..9e6e56ebda3 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -34,7 +34,10 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); if (join->pipelineType() == JoinPipelineType::YShaped) - return QueryPipelineBuilder::joinPipelines2(std::move(pipelines[0]), std::move(pipelines[1]), output_stream->header, max_block_size, &processors); + return QueryPipelineBuilder::joinPipelines2( + std::move(pipelines[0]), std::move(pipelines[1]), + join->getTableJoin(), output_stream->header, + max_block_size, &processors); return QueryPipelineBuilder::joinPipelines(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, max_streams, keep_left_read_in_order, &processors); } diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index dc35cfe2e74..f48436ad83b 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -1,6 +1,12 @@ +#include +#include #include #include #include +#include +#include +#include "Columns/IColumn.h" +#include "Core/SortCursor.h" namespace DB @@ -9,22 +15,68 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int TOO_MANY_ROWS; } -MergeJoinAlgorithm::MergeJoinAlgorithm() - : log(&Poco::Logger::get("MergeJoinAlgorithm")) +namespace { + +constexpr size_t EMPTY_VALUE = std::numeric_limits::max(); + +SortCursorImpl createCursor(const Block & block, const Names & columns) +{ + SortDescription res; + res.reserve(columns.size()); + + for (const auto & name : columns) + res.emplace_back(block.getPositionByName(name)); + + return SortCursorImpl(block.cloneEmpty().getColumns(), res); +} + + +int ALWAYS_INLINE compareCursors(const SortCursor & lhs, const SortCursor & rhs) +{ + for (size_t i = 0; i < lhs->sort_columns_size; ++i) + { + const auto & desc = lhs->desc[i]; + int direction = desc.direction; + int nulls_direction = desc.nulls_direction; + int res = direction * lhs->sort_columns[i]->compareAt(lhs->getRow(), rhs->getRow(), *(rhs.impl->sort_columns[i]), nulls_direction); + if (res != 0) + return res; + } + return 0; +} + +} + +MergeJoinAlgorithm::MergeJoinAlgorithm( + const TableJoin & table_join_, + const Blocks & input_headers) + : table_join(table_join_) + , log(&Poco::Logger::get("MergeJoinAlgorithm")) +{ + if (input_headers.size() != 2) + throw Exception("MergeJoinAlgorithm requires exactly two inputs", ErrorCodes::LOGICAL_ERROR); + + if (table_join.strictness() != ASTTableJoin::Strictness::Any) + throw Exception("MergeJoinAlgorithm is not implemented for strictness != ANY", ErrorCodes::NOT_IMPLEMENTED); + + const auto & join_on = table_join.getOnlyClause(); + + cursors.push_back(createCursor(input_headers[0], join_on.key_names_left)); + cursors.push_back(createCursor(input_headers[1], join_on.key_names_right)); } void MergeJoinAlgorithm::initialize(Inputs inputs) { + if (inputs.size() != 2) + throw Exception("MergeJoinAlgorithm requires exactly two inputs", ErrorCodes::LOGICAL_ERROR); LOG_DEBUG(log, "MergeJoinAlgorithm initialize, number of inputs: {}", inputs.size()); - current_inputs = std::move(inputs); - chunks.resize(current_inputs.size()); - for (size_t i = 0; i < current_inputs.size(); ++i) - { - consume(current_inputs[i], i); - } + current_inputs.resize(inputs.size()); + for (size_t i = 0; i < inputs.size(); ++i) + consume(inputs[i], i); } static void prepareChunk(Chunk & chunk) @@ -39,44 +91,129 @@ static void prepareChunk(Chunk & chunk) void MergeJoinAlgorithm::consume(Input & input, size_t source_num) { - LOG_DEBUG(log, "Consume from {}", source_num); + LOG_DEBUG(log, "Consume from {} chunk: {}", source_num, bool(input.chunk)); + + left_stream_finished = left_stream_finished || (!input.chunk && source_num == 0); + right_stream_finished = right_stream_finished || (!input.chunk && source_num == 1); prepareChunk(input.chunk); - chunks[source_num] = std::move(input.chunk); + + if (input.chunk.getNumRows() >= EMPTY_VALUE) + throw Exception("Too many rows in input", ErrorCodes::TOO_MANY_ROWS); + + current_inputs[source_num] = std::move(input); + if (current_inputs[source_num].chunk) + { + cursors[source_num].reset(current_inputs[source_num].chunk.getColumns(), {}, current_inputs[source_num].permutation); + } } IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { - if (!chunks[0]) + if (!current_inputs[0].chunk && !left_stream_finished) { return Status(0); } - if (!chunks[1]) + + if (!current_inputs[1].chunk && !right_stream_finished) { return Status(1); } - size_t rows = std::min(chunks[0].getNumRows(), chunks[1].getNumRows()); - Chunk res; + if (left_stream_finished || right_stream_finished) + { + return Status({}, true); + } - for (auto & col : chunks[0].detachColumns()) + SortCursor left_cursor(&cursors[0]); + SortCursor right_cursor(&cursors[1]); + + if (!left_cursor->isValid() || left_cursor.totallyLessOrEquals(right_cursor)) { - col->cut(0, rows); - res.addColumn(std::move(col)); + current_inputs[0] = {}; + if (left_stream_finished) + { + return Status({}, true); + } + return Status(0); } - for (auto & col : chunks[1].detachColumns()) + + if (!right_cursor->isValid() || right_cursor.totallyLessOrEquals(left_cursor)) { - col->cut(0, rows); - res.addColumn(std::move(col)); + current_inputs[1] = {}; + if (right_stream_finished) + { + return Status({}, true); + } + return Status(1); } - return Status(std::move(res), true); + + auto left_map = ColumnUInt64::create(); + auto right_map = ColumnUInt64::create(); + + while (left_cursor->isValid() && right_cursor->isValid()) + { + int cmp = compareCursors(left_cursor, right_cursor); + if (cmp == 0) + { + left_map->getData().emplace_back(left_cursor->getPosRef()); + right_map->getData().emplace_back(right_cursor->getPosRef()); + left_cursor->next(); + right_cursor->next(); + } + else if (cmp < 0) + { + left_cursor->next(); + left_map->getData().emplace_back(left_cursor->getPosRef()); + right_map->getData().emplace_back(right_cursor->rows); + } + else + { + right_cursor->next(); + } + } + + while (left_cursor->isValid()) + { + left_map->getData().emplace_back(left_cursor->getPosRef()); + right_map->getData().emplace_back(right_cursor->rows); + left_cursor->next(); + } + + Chunk result; + for (const auto & col : current_inputs[0].chunk.getColumns()) + { + auto tmp_col = col->cloneResized(col->size() + 1); + ColumnPtr new_col = tmp_col->index(*left_map, 0); + result.addColumn(std::move(new_col)); + } + + for (const auto & col : current_inputs[1].chunk.getColumns()) + { + auto tmp_col = col->cloneResized(col->size() + 1); + ColumnPtr new_col = tmp_col->index(*right_map, 0); + result.addColumn(std::move(new_col)); + } + + if (!left_cursor->isValid()) + { + current_inputs[0] = {}; + } + + if (!right_cursor->isValid()) + { + current_inputs[1] = {}; + } + + return Status(std::move(result), left_stream_finished || right_stream_finished); } MergeJoinTransform::MergeJoinTransform( + const TableJoin & table_join, const Blocks & input_headers, const Block & output_header, UInt64 limit_hint) - : IMergingTransform(input_headers, output_header, true, limit_hint) + : IMergingTransform(input_headers, output_header, true, limit_hint, table_join, input_headers) , log(&Poco::Logger::get("MergeJoinTransform")) { LOG_TRACE(log, "Will use MergeJoinTransform"); @@ -84,7 +221,7 @@ MergeJoinTransform::MergeJoinTransform( void MergeJoinTransform::onFinish() { - LOG_TRACE(log, "MergeJoinTransform::onFinish"); + LOG_TRACE(log, "onFinish"); } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 95a6f919cd6..7f562d4c942 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -1,19 +1,24 @@ #pragma once #include +#include #include #include +#include #include #include #include #include - +#include namespace Poco { class Logger; } namespace DB { +class TableJoin; + + /* * This class is used to join chunks from two sorted streams. * It is used in MergeJoinTransform. @@ -21,16 +26,23 @@ namespace DB class MergeJoinAlgorithm final : public IMergingAlgorithm { public: - MergeJoinAlgorithm(); + explicit MergeJoinAlgorithm(const TableJoin & table_join, const Blocks & input_headers); virtual void initialize(Inputs inputs) override; virtual void consume(Input & input, size_t source_num) override; virtual Status merge() override; private: - Inputs current_inputs; + SortDescription left_desc; + SortDescription right_desc; - Chunks chunks; + std::vector current_inputs; + std::vector cursors; + + bool left_stream_finished = false; + bool right_stream_finished = false; + + const TableJoin & table_join; Poco::Logger * log; }; @@ -38,6 +50,7 @@ class MergeJoinTransform final : public IMergingTransform { public: MergeJoinTransform( + const TableJoin & table_join, const Blocks & input_headers, const Block & output_header, UInt64 limit_hint = 0); @@ -51,5 +64,4 @@ protected: Poco::Logger * log; }; - } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index d9725aeafed..a1e7354ad5d 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -18,8 +19,10 @@ #include #include #include +#include #include #include +#include "Core/SortDescription.h" #include #include #include @@ -328,6 +331,7 @@ QueryPipelineBuilderPtr QueryPipelineBuilder::mergePipelines( std::unique_ptr QueryPipelineBuilder::joinPipelines2( std::unique_ptr left, std::unique_ptr right, + const TableJoin & table_join, const Block & out_header, size_t max_block_size, Processors * collected_processors) @@ -344,7 +348,8 @@ std::unique_ptr QueryPipelineBuilder::joinPipelines2( throw Exception("Join is supported only for pipelines with one output port", ErrorCodes::LOGICAL_ERROR); Blocks inputs = {left->getHeader(), right->getHeader()}; - auto joining = std::make_shared(inputs, out_header); + + auto joining = std::make_shared(table_join, inputs, out_header); auto result = mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); return result; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 3c9aa036851..43f99ea7bc1 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -29,6 +29,7 @@ struct ExpressionActionsSettings; class IJoin; using JoinPtr = std::shared_ptr; +class TableJoin; class QueryPipelineBuilder; using QueryPipelineBuilderPtr = std::unique_ptr; @@ -100,7 +101,6 @@ public: size_t max_threads_limit = 0, Processors * collected_processors = nullptr); - static QueryPipelineBuilderPtr mergePipelines( QueryPipelineBuilderPtr left, QueryPipelineBuilderPtr right, @@ -121,6 +121,7 @@ public: static std::unique_ptr joinPipelines2( std::unique_ptr left, std::unique_ptr right, + const TableJoin & table_join, const Block & out_header, size_t max_block_size, Processors * collected_processors = nullptr); From 1b429fc1af043fe428227efcb5c3b87c72582dfe Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 5 Apr 2022 10:12:42 +0000 Subject: [PATCH 279/627] wip: any left/right sorting join --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/FullSortingMergeJoin.h | 6 +- src/Processors/QueryPlan/JoinStep.cpp | 2 +- .../Transforms/MergeJoinTransform.cpp | 224 +++++++++++++----- .../Transforms/MergeJoinTransform.h | 64 ++++- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 +- src/QueryPipeline/QueryPipelineBuilder.h | 2 +- .../0_stateless/01031_new_any_join.sql | 2 + 8 files changed, 239 insertions(+), 67 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 2f07ecf8dc2..7059240e408 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1095,7 +1095,7 @@ static std::shared_ptr chooseJoinAlgorithm(std::shared_ptr ana { if (analyzed_join->getClauses().size() != 1) throw Exception("Full sorting merge join is supported only for single-condition joins", ErrorCodes::NOT_IMPLEMENTED); - return std::make_shared(*analyzed_join, sample_block); + return std::make_shared(analyzed_join, sample_block); } return std::make_shared(analyzed_join, sample_block); } diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 5bc764ce375..7689010f13a 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -9,14 +9,14 @@ namespace DB class FullSortingMergeJoin : public IJoin { public: - explicit FullSortingMergeJoin(TableJoin & table_join_, const Block & right_sample_block_) + explicit FullSortingMergeJoin(std::shared_ptr table_join_, const Block & right_sample_block_) : table_join(table_join_) , right_sample_block(right_sample_block_) { LOG_TRACE(&Poco::Logger::get("FullSortingMergeJoin"), "Will use full sorting merge join"); } - const TableJoin & getTableJoin() const override { return table_join; } + const TableJoin & getTableJoin() const override { return *table_join; } bool addJoinedBlock(const Block & /* block */, bool /* check_limits */) override { __builtin_unreachable(); } @@ -48,7 +48,7 @@ public: virtual JoinPipelineType pipelineType() const override { return JoinPipelineType::YShaped; } private: - TableJoin & table_join; + std::shared_ptr table_join; Block right_sample_block; Block totals; }; diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 9e6e56ebda3..ff3063f01a3 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -36,7 +36,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (join->pipelineType() == JoinPipelineType::YShaped) return QueryPipelineBuilder::joinPipelines2( std::move(pipelines[0]), std::move(pipelines[1]), - join->getTableJoin(), output_stream->header, + join, output_stream->header, max_block_size, &processors); return QueryPipelineBuilder::joinPipelines(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, max_streams, keep_left_read_in_order, &processors); diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index f48436ad83b..37e81ab500b 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -1,12 +1,18 @@ #include +#include #include #include #include #include #include #include +#include +#include "Columns/ColumnsNumber.h" #include "Columns/IColumn.h" #include "Core/SortCursor.h" +#include "Parsers/ASTTablesInSelectQuery.h" +#include "base/defines.h" +#include "base/types.h" namespace DB @@ -23,15 +29,16 @@ namespace constexpr size_t EMPTY_VALUE = std::numeric_limits::max(); -SortCursorImpl createCursor(const Block & block, const Names & columns) + +FullMergeJoinCursor createCursor(const Block & block, const Names & columns) { - SortDescription res; - res.reserve(columns.size()); + SortDescription desc; + desc.reserve(columns.size()); for (const auto & name : columns) - res.emplace_back(block.getPositionByName(name)); + desc.emplace_back(block.getPositionByName(name)); - return SortCursorImpl(block.cloneEmpty().getColumns(), res); + return FullMergeJoinCursor(block.cloneEmpty().getColumns(), desc); } @@ -49,10 +56,27 @@ int ALWAYS_INLINE compareCursors(const SortCursor & lhs, const SortCursor & rhs) return 0; } +void addIndexColumn(const Columns & columns, const IColumn & indices, Chunk & result) +{ + for (const auto & col : columns) + { + if (indices.empty()) + { + result.addColumn(col); + } + else + { + auto tmp_col = col->cloneResized(col->size() + 1); + ColumnPtr new_col = tmp_col->index(indices, 0); + result.addColumn(std::move(new_col)); + } + } +} + } MergeJoinAlgorithm::MergeJoinAlgorithm( - const TableJoin & table_join_, + JoinPtr table_join_, const Blocks & input_headers) : table_join(table_join_) , log(&Poco::Logger::get("MergeJoinAlgorithm")) @@ -60,15 +84,25 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( if (input_headers.size() != 2) throw Exception("MergeJoinAlgorithm requires exactly two inputs", ErrorCodes::LOGICAL_ERROR); - if (table_join.strictness() != ASTTableJoin::Strictness::Any) + if (table_join->getTableJoin().strictness() != ASTTableJoin::Strictness::Any) throw Exception("MergeJoinAlgorithm is not implemented for strictness != ANY", ErrorCodes::NOT_IMPLEMENTED); - const auto & join_on = table_join.getOnlyClause(); + const auto & join_on = table_join->getTableJoin().getOnlyClause(); cursors.push_back(createCursor(input_headers[0], join_on.key_names_left)); cursors.push_back(createCursor(input_headers[1], join_on.key_names_right)); } + +static void createSampleChunk(const Chunk & chunk, Chunk & sample_chunk, size_t size = 0) +{ + const auto & cols = chunk.getColumns(); + for (const auto & col : cols) + { + sample_chunk.addColumn(col->cloneResized(size)); + } +} + void MergeJoinAlgorithm::initialize(Inputs inputs) { if (inputs.size() != 2) @@ -76,7 +110,10 @@ void MergeJoinAlgorithm::initialize(Inputs inputs) LOG_DEBUG(log, "MergeJoinAlgorithm initialize, number of inputs: {}", inputs.size()); current_inputs.resize(inputs.size()); for (size_t i = 0; i < inputs.size(); ++i) + { + createSampleChunk(inputs[i].chunk, sample_chunks.emplace_back()); consume(inputs[i], i); + } } static void prepareChunk(Chunk & chunk) @@ -104,12 +141,85 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) current_inputs[source_num] = std::move(input); if (current_inputs[source_num].chunk) { - cursors[source_num].reset(current_inputs[source_num].chunk.getColumns(), {}, current_inputs[source_num].permutation); + cursors[source_num].getImpl().reset(current_inputs[source_num].chunk.getColumns(), {}, current_inputs[source_num].permutation); + } +} + +static size_t ALWAYS_INLINE rowsLeft(SortCursor cursor) +{ + return cursor->rows - cursor->getPosRef(); +} + +using JoinKind = ASTTableJoin::Kind; + +template +static void leftOrFullAny(SortCursor left_cursor, SortCursor right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) +{ + static_assert(kind == JoinKind::Left || kind == JoinKind::Right || kind == JoinKind::Inner, "Invalid join kind"); + + size_t num_rows = kind == JoinKind::Left ? rowsLeft(left_cursor) : + kind == JoinKind::Right ? rowsLeft(right_cursor) : + std::min(rowsLeft(left_cursor), rowsLeft(right_cursor)); + + constexpr bool is_left_or_inner = kind == JoinKind::Left || kind == JoinKind::Inner; + constexpr bool is_right_or_inner = kind == JoinKind::Right || kind == JoinKind::Inner; + + if constexpr (is_left_or_inner) + right_map.reserve(num_rows); + + if constexpr (is_right_or_inner) + left_map.reserve(num_rows); + + while (left_cursor->isValid() && right_cursor->isValid()) + { + int cmp = compareCursors(left_cursor, right_cursor); + if (cmp == 0) + { + if constexpr (is_left_or_inner) + right_map.emplace_back(right_cursor->getRow()); + + if constexpr (is_right_or_inner) + left_map.emplace_back(left_cursor->getRow()); + + if constexpr (is_left_or_inner) + left_cursor->next(); + + if constexpr (is_right_or_inner) + right_cursor->next(); + + } + else if (cmp < 0) + { + if constexpr (kind == JoinKind::Left) + right_map.emplace_back(right_cursor->rows); + left_cursor->next(); + } + else + { + if constexpr (kind == JoinKind::Right) + left_map.emplace_back(left_cursor->rows); + right_cursor->next(); + } + } + + while (left_cursor->isValid() && kind == JoinKind::Left) + { + right_map.emplace_back(right_cursor->rows); + left_cursor->next(); + } + + while (right_cursor->isValid() && kind == JoinKind::Right) + { + left_map.emplace_back(left_cursor->rows); + right_cursor->next(); } } IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { + if (current_inputs[0].skip_last_row || current_inputs[1].skip_last_row) + throw Exception("MergeJoinAlgorithm does not support skipLastRow", ErrorCodes::LOGICAL_ERROR); + if (!current_inputs[0].chunk && !left_stream_finished) { return Status(0); @@ -120,15 +230,45 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() return Status(1); } - if (left_stream_finished || right_stream_finished) + JoinKind kind = table_join->getTableJoin().kind(); + + if (left_stream_finished && right_stream_finished) { return Status({}, true); } - SortCursor left_cursor(&cursors[0]); - SortCursor right_cursor(&cursors[1]); + if (isInner(kind) && (left_stream_finished || right_stream_finished)) + { + return Status({}, true); + } - if (!left_cursor->isValid() || left_cursor.totallyLessOrEquals(right_cursor)) + auto create_block_with_defaults = [] (const Chunk & lhs, const Chunk & rhs) -> Chunk + { + Chunk result; + size_t num_rows = std::max(lhs.getNumRows(), rhs.getNumRows()); + createSampleChunk(lhs, result, num_rows); + createSampleChunk(rhs, result, num_rows); + return result; + }; + + if (isLeftOrFull(kind) && right_stream_finished) + { + Chunk result = create_block_with_defaults(current_inputs[0].chunk, sample_chunks[1]); + current_inputs[0] = {}; + return Status(std::move(result), left_stream_finished && right_stream_finished); + } + + if (isRightOrFull(kind) && left_stream_finished) + { + Chunk result = create_block_with_defaults(sample_chunks[0], current_inputs[1].chunk); + current_inputs[1] = {}; + return Status(std::move(result), left_stream_finished && right_stream_finished); + } + + SortCursor left_cursor = cursors[0].getCursor(); + SortCursor right_cursor = cursors[1].getCursor(); + + if (!left_cursor->isValid() || (right_cursor->isValid() && left_cursor.totallyLessOrEquals(right_cursor))) { current_inputs[0] = {}; if (left_stream_finished) @@ -138,7 +278,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() return Status(0); } - if (!right_cursor->isValid() || right_cursor.totallyLessOrEquals(left_cursor)) + if (!right_cursor->isValid() || (left_cursor->isValid() && right_cursor.totallyLessOrEquals(left_cursor))) { current_inputs[1] = {}; if (right_stream_finished) @@ -150,50 +290,26 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() auto left_map = ColumnUInt64::create(); auto right_map = ColumnUInt64::create(); - - while (left_cursor->isValid() && right_cursor->isValid()) + if (isInner(kind)) { - int cmp = compareCursors(left_cursor, right_cursor); - if (cmp == 0) - { - left_map->getData().emplace_back(left_cursor->getPosRef()); - right_map->getData().emplace_back(right_cursor->getPosRef()); - left_cursor->next(); - right_cursor->next(); - } - else if (cmp < 0) - { - left_cursor->next(); - left_map->getData().emplace_back(left_cursor->getPosRef()); - right_map->getData().emplace_back(right_cursor->rows); - } - else - { - right_cursor->next(); - } + leftOrFullAny(left_cursor, right_cursor, left_map->getData(), right_map->getData()); } - - while (left_cursor->isValid()) + else if (isLeft(kind)) { - left_map->getData().emplace_back(left_cursor->getPosRef()); - right_map->getData().emplace_back(right_cursor->rows); - left_cursor->next(); + leftOrFullAny(left_cursor, right_cursor, left_map->getData(), right_map->getData()); + } + else if (isRight(kind)) + { + leftOrFullAny(left_cursor, right_cursor, left_map->getData(), right_map->getData()); + } + else + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", table_join->getTableJoin().kind()); } Chunk result; - for (const auto & col : current_inputs[0].chunk.getColumns()) - { - auto tmp_col = col->cloneResized(col->size() + 1); - ColumnPtr new_col = tmp_col->index(*left_map, 0); - result.addColumn(std::move(new_col)); - } - - for (const auto & col : current_inputs[1].chunk.getColumns()) - { - auto tmp_col = col->cloneResized(col->size() + 1); - ColumnPtr new_col = tmp_col->index(*right_map, 0); - result.addColumn(std::move(new_col)); - } + addIndexColumn(current_inputs[0].chunk.getColumns(), *left_map, result); + addIndexColumn(current_inputs[1].chunk.getColumns(), *right_map, result); if (!left_cursor->isValid()) { @@ -205,11 +321,11 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() current_inputs[1] = {}; } - return Status(std::move(result), left_stream_finished || right_stream_finished); + return Status(std::move(result), left_stream_finished && right_stream_finished); } MergeJoinTransform::MergeJoinTransform( - const TableJoin & table_join, + JoinPtr table_join, const Blocks & input_headers, const Block & output_header, UInt64 limit_hint) diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 7f562d4c942..fef2c3fd720 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -4,6 +4,7 @@ #include #include #include +#include "Interpreters/IJoin.h" #include #include #include @@ -16,9 +17,60 @@ namespace Poco { class Logger; } namespace DB { -class TableJoin; +class IJoin; +using JoinPtr = std::shared_ptr; +/* + * Wrapper for SortCursorImpl + * It is used to store information about the current state of the cursor. + */ +class FullMergeJoinCursor +{ +public: + + FullMergeJoinCursor(const Columns & columns, const SortDescription & desc) + : impl(columns, desc) + { + } + + SortCursor getCursor() + { + return SortCursor(&impl); + } + + /* + /// Expects !atEnd() + size_t getEqualLength() const + { + size_t pos = impl.getRow() + 1; + for (; pos < impl.rows; ++pos) + if (!samePrev(pos)) + break; + return pos - impl.getRow(); + } + + /// Expects lhs_pos > 0 + bool ALWAYS_INLINE samePrev(size_t lhs_pos) const + { + for (size_t i = 0; i < impl.sort_columns_size; ++i) + if (impl.sort_columns[i]->compareAt(lhs_pos - 1, lhs_pos, *(impl.sort_columns[i]), 1) != 0) + return false; + return true; + } + */ + + SortCursorImpl & getImpl() + { + return impl; + } + +private: + SortCursorImpl impl; + // bool has_left_nullable = false; + // bool has_right_nullable = false; +}; + /* * This class is used to join chunks from two sorted streams. * It is used in MergeJoinTransform. @@ -26,7 +78,7 @@ class TableJoin; class MergeJoinAlgorithm final : public IMergingAlgorithm { public: - explicit MergeJoinAlgorithm(const TableJoin & table_join, const Blocks & input_headers); + explicit MergeJoinAlgorithm(JoinPtr table_join, const Blocks & input_headers); virtual void initialize(Inputs inputs) override; virtual void consume(Input & input, size_t source_num) override; @@ -37,12 +89,14 @@ private: SortDescription right_desc; std::vector current_inputs; - std::vector cursors; + std::vector cursors; + + std::vector sample_chunks; bool left_stream_finished = false; bool right_stream_finished = false; - const TableJoin & table_join; + JoinPtr table_join; Poco::Logger * log; }; @@ -50,7 +104,7 @@ class MergeJoinTransform final : public IMergingTransform { public: MergeJoinTransform( - const TableJoin & table_join, + JoinPtr table_join, const Blocks & input_headers, const Block & output_header, UInt64 limit_hint = 0); diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index a1e7354ad5d..e127c2823ab 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -331,7 +331,7 @@ QueryPipelineBuilderPtr QueryPipelineBuilder::mergePipelines( std::unique_ptr QueryPipelineBuilder::joinPipelines2( std::unique_ptr left, std::unique_ptr right, - const TableJoin & table_join, + JoinPtr join, const Block & out_header, size_t max_block_size, Processors * collected_processors) @@ -349,7 +349,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelines2( Blocks inputs = {left->getHeader(), right->getHeader()}; - auto joining = std::make_shared(table_join, inputs, out_header); + auto joining = std::make_shared(join, inputs, out_header); auto result = mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); return result; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 43f99ea7bc1..d12de77230a 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -121,7 +121,7 @@ public: static std::unique_ptr joinPipelines2( std::unique_ptr left, std::unique_ptr right, - const TableJoin & table_join, + JoinPtr table_join, const Block & out_header, size_t max_block_size, Processors * collected_processors = nullptr); diff --git a/tests/queries/0_stateless/01031_new_any_join.sql b/tests/queries/0_stateless/01031_new_any_join.sql index 2f2a8b2ad1d..c69a765ebce 100644 --- a/tests/queries/0_stateless/01031_new_any_join.sql +++ b/tests/queries/0_stateless/01031_new_any_join.sql @@ -10,6 +10,8 @@ INSERT INTO t2 (x, s) VALUES (2, 'b1'), (4, 'b3'), (5, 'b6'); SET join_use_nulls = 0; SET any_join_distinct_right_table_keys = 0; +SET join_algorithm = 'full_sorting_merge'; + SELECT 'any left'; SELECT t1.*, t2.* FROM t1 ANY LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; From d34a66c91520815bb6e95616e69c0d086f50af1a Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 5 Apr 2022 10:14:42 +0000 Subject: [PATCH 280/627] wip sorting merge --- src/Interpreters/ExpressionAnalyzer.cpp | 13 +- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/TableJoin.cpp | 44 +++- src/Interpreters/TableJoin.h | 7 +- src/Processors/Merges/IMergingTransform.h | 1 + .../Transforms/MergeJoinTransform.cpp | 198 +++++++++++------- .../Transforms/MergeJoinTransform.h | 47 ++++- 7 files changed, 214 insertions(+), 98 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 7059240e408..9cbf6258199 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1047,10 +1047,13 @@ bool SelectQueryExpressionAnalyzer::appendJoinLeftKeys(ExpressionActionsChain & return true; } -JoinPtr SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, ActionsDAGPtr & converting_join_columns) +JoinPtr SelectQueryExpressionAnalyzer::appendJoin( + ExpressionActionsChain & chain, + ActionsDAGPtr & converting_join_columns) { const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns(); - JoinPtr table_join = makeTableJoin(*syntax->ast_join, left_sample_columns, converting_join_columns); + + JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_join_columns); if (converting_join_columns) { @@ -1060,9 +1063,9 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain ExpressionActionsChain::Step & step = chain.lastStep(columns_after_array_join); chain.steps.push_back(std::make_unique( - syntax->analyzed_join, table_join, step.getResultColumns())); + syntax->analyzed_join, join, step.getResultColumns())); chain.addStep(); - return table_join; + return join; } static ActionsDAGPtr createJoinedBlockActions(ContextPtr context, const TableJoin & analyzed_join) @@ -1199,7 +1202,7 @@ std::shared_ptr tryKeyValueJoin(std::shared_ptr a return std::make_shared(analyzed_join, right_sample_block, storage); } -JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( +JoinPtr SelectQueryExpressionAnalyzer::makeJoin( const ASTTablesInSelectQueryElement & join_element, const ColumnsWithTypeAndName & left_columns, ActionsDAGPtr & left_convert_actions) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 278415f6429..167c3dfd918 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -375,7 +375,7 @@ private: NameSet required_result_columns; SelectQueryOptions query_options; - JoinPtr makeTableJoin( + JoinPtr makeJoin( const ASTTablesInSelectQueryElement & join_element, const ColumnsWithTypeAndName & left_columns, ActionsDAGPtr & left_convert_actions); diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 20842023bab..db75a86fff6 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -507,14 +507,18 @@ static void renameIfNeeded(String & name, const NameToNameMap & renames) } std::pair -TableJoin::createConvertingActions(const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns) +TableJoin::createConvertingActions( + const ColumnsWithTypeAndName & left_sample_columns, + const ColumnsWithTypeAndName & right_sample_columns) { inferJoinKeyCommonType(left_sample_columns, right_sample_columns, !isSpecialStorage()); NameToNameMap left_key_column_rename; NameToNameMap right_key_column_rename; - auto left_converting_actions = applyKeyConvertToTable(left_sample_columns, left_type_map, left_key_column_rename, forceNullableLeft()); - auto right_converting_actions = applyKeyConvertToTable(right_sample_columns, right_type_map, right_key_column_rename, forceNullableRight()); + auto left_converting_actions = applyKeyConvertToTable( + left_sample_columns, left_type_map, left_key_column_rename, forceNullableLeft()); + auto right_converting_actions = applyKeyConvertToTable( + right_sample_columns, right_type_map, right_key_column_rename, forceNullableRight()); { auto log_actions = [](const String & side, const ActionsDAGPtr & dag) @@ -536,7 +540,18 @@ TableJoin::createConvertingActions(const ColumnsWithTypeAndName & left_sample_co else { LOG_DEBUG(&Poco::Logger::get("TableJoin"), "{} JOIN converting actions: empty", side); + return; } + auto format_cols = [](const auto & cols) -> std::string + { + std::vector str_cols; + str_cols.reserve(cols.size()); + for (const auto & col : cols) + str_cols.push_back(fmt::format("'{}': {}", col.name, col.type->getName())); + return fmt::format("[{}]", fmt::join(str_cols, ", ")); + }; + LOG_DEBUG(&Poco::Logger::get("TableJoin"), "{} JOIN converting actions: {} -> {}", + side, format_cols(dag->getRequiredColumns()), format_cols(dag->getResultColumns())); }; log_actions("Left", left_converting_actions); log_actions("Right", right_converting_actions); @@ -646,10 +661,18 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src, if (!has_some_to_do) return nullptr; - return ActionsDAG::makeConvertingActions(cols_src, cols_dst, ActionsDAG::MatchColumnsMode::Name, true, add_new_cols, &key_column_rename); + return ActionsDAG::makeConvertingActions( + /* source= */ cols_src, + /* result= */ cols_dst, + /* mode= */ ActionsDAG::MatchColumnsMode::Name, + /* ignore_constant_values= */ true, + /* add_casted_columns= */ add_new_cols, + /* new_names= */ &key_column_rename); } -static ActionsDAGPtr changeTypesToNullable(const ColumnsWithTypeAndName & cols_src, const NameSet & exception_cols) +static ActionsDAGPtr changeTypesToNullable( + const ColumnsWithTypeAndName & cols_src, + const NameSet & exception_cols) { ColumnsWithTypeAndName cols_dst = cols_src; bool has_some_to_do = false; @@ -664,7 +687,14 @@ static ActionsDAGPtr changeTypesToNullable(const ColumnsWithTypeAndName & cols_s if (!has_some_to_do) return nullptr; - return ActionsDAG::makeConvertingActions(cols_src, cols_dst, ActionsDAG::MatchColumnsMode::Name, true, false, nullptr); + + return ActionsDAG::makeConvertingActions( + /* source= */ cols_src, + /* result= */ cols_dst, + /* mode= */ ActionsDAG::MatchColumnsMode::Name, + /* ignore_constant_values= */ true, + /* add_casted_columns= */ false, + /* new_names= */ nullptr); } ActionsDAGPtr TableJoin::applyKeyConvertToTable( @@ -679,7 +709,7 @@ ActionsDAGPtr TableJoin::applyKeyConvertToTable( /// Create DAG to make columns nullable if needed if (make_nullable) { - /// Do not need to make nullable temporary columns that would be used only as join keys, but now shown to user + /// Do not need to make nullable temporary columns that would be used only as join keys, but not shown to user NameSet cols_not_nullable; for (const auto & t : key_column_rename) cols_not_nullable.insert(t.second); diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index e44a0657da3..45a4106d040 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -156,7 +156,8 @@ private: /// Create converting actions and change key column names if required ActionsDAGPtr applyKeyConvertToTable( - const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, NameToNameMap & key_column_rename, + const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, + NameToNameMap & key_column_rename, bool make_nullable) const; void addKey(const String & left_name, const String & right_name, const ASTPtr & left_ast, const ASTPtr & right_ast = nullptr); @@ -278,7 +279,9 @@ public: /// For `USING` join we will convert key columns inplace and affect into types in the result table /// For `JOIN ON` we will create new columns with converted keys to join by. std::pair - createConvertingActions(const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns); + createConvertingActions( + const ColumnsWithTypeAndName & left_sample_columns, + const ColumnsWithTypeAndName & right_sample_columns); void setAsofInequality(ASOF::Inequality inequality) { asof_inequality = inequality; } ASOF::Inequality getAsofInequality() { return asof_inequality; } diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 061750d91e4..f7178d7b1ae 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 37e81ab500b..09319438c1d 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -7,12 +8,13 @@ #include #include #include -#include "Columns/ColumnsNumber.h" -#include "Columns/IColumn.h" -#include "Core/SortCursor.h" -#include "Parsers/ASTTablesInSelectQuery.h" -#include "base/defines.h" -#include "base/types.h" +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -42,20 +44,89 @@ FullMergeJoinCursor createCursor(const Block & block, const Names & columns) } -int ALWAYS_INLINE compareCursors(const SortCursor & lhs, const SortCursor & rhs) +template +int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos, int null_direction_hint = 1) +{ + if constexpr (has_left_nulls && has_right_nulls) + { + const auto * left_nullable = checkAndGetColumn(left_column); + const auto * right_nullable = checkAndGetColumn(right_column); + + if (left_nullable && right_nullable) + { + int res = left_column.compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); + if (res) + return res; + + /// NULL != NULL case + if (left_column.isNullAt(lhs_pos)) + return null_direction_hint; + + return 0; + } + } + + if constexpr (has_left_nulls) + { + if (const auto * left_nullable = checkAndGetColumn(left_column)) + { + if (left_column.isNullAt(lhs_pos)) + return null_direction_hint; + return left_nullable->getNestedColumn().compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); + } + } + + if constexpr (has_right_nulls) + { + if (const auto * right_nullable = checkAndGetColumn(right_column)) + { + if (right_column.isNullAt(rhs_pos)) + return -null_direction_hint; + return left_column.compareAt(lhs_pos, rhs_pos, right_nullable->getNestedColumn(), null_direction_hint); + } + } + + return left_column.compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); +} + +/// If on_pos == true, compare two columns at specified positions. +/// Otherwise, compare two columns at the current positions, `lpos` and `rpos` are ignored. +template +int ALWAYS_INLINE compareCursors(const Cursor & lhs, const Cursor & rhs, + [[ maybe_unused ]] size_t lpos = 0, + [[ maybe_unused ]] size_t rpos = 0) { for (size_t i = 0; i < lhs->sort_columns_size; ++i) { const auto & desc = lhs->desc[i]; int direction = desc.direction; int nulls_direction = desc.nulls_direction; - int res = direction * lhs->sort_columns[i]->compareAt(lhs->getRow(), rhs->getRow(), *(rhs.impl->sort_columns[i]), nulls_direction); - if (res != 0) - return res; + + int cmp = direction * nullableCompareAt( + *lhs->sort_columns[i], + *rhs->sort_columns[i], + on_pos ? lpos : lhs->getRow(), + on_pos ? rpos : rhs->getRow(), + nulls_direction); + if (cmp != 0) + return cmp; } return 0; } +bool ALWAYS_INLINE totallyLess(const FullMergeJoinCursor & lhs, const FullMergeJoinCursor & rhs) +{ + if (lhs->rows == 0 || rhs->rows == 0) + return false; + + if (!lhs->isValid() || !rhs->isValid()) + return false; + + /// The last row of this cursor is no larger than the first row of the another cursor. + int cmp = compareCursors(lhs, rhs, lhs->rows - 1, 0); + return cmp < 0; +} + void addIndexColumn(const Columns & columns, const IColumn & indices, Chunk & result) { for (const auto & col : columns) @@ -108,7 +179,7 @@ void MergeJoinAlgorithm::initialize(Inputs inputs) if (inputs.size() != 2) throw Exception("MergeJoinAlgorithm requires exactly two inputs", ErrorCodes::LOGICAL_ERROR); LOG_DEBUG(log, "MergeJoinAlgorithm initialize, number of inputs: {}", inputs.size()); - current_inputs.resize(inputs.size()); + for (size_t i = 0; i < inputs.size(); ++i) { createSampleChunk(inputs[i].chunk, sample_chunks.emplace_back()); @@ -130,19 +201,15 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) { LOG_DEBUG(log, "Consume from {} chunk: {}", source_num, bool(input.chunk)); - left_stream_finished = left_stream_finished || (!input.chunk && source_num == 0); - right_stream_finished = right_stream_finished || (!input.chunk && source_num == 1); + if (!input.chunk) + cursors[source_num].completeAll(); prepareChunk(input.chunk); if (input.chunk.getNumRows() >= EMPTY_VALUE) throw Exception("Too many rows in input", ErrorCodes::TOO_MANY_ROWS); - current_inputs[source_num] = std::move(input); - if (current_inputs[source_num].chunk) - { - cursors[source_num].getImpl().reset(current_inputs[source_num].chunk.getColumns(), {}, current_inputs[source_num].permutation); - } + cursors[source_num].setInput(std::move(input)); } static size_t ALWAYS_INLINE rowsLeft(SortCursor cursor) @@ -215,92 +282,77 @@ static void leftOrFullAny(SortCursor left_cursor, SortCursor right_cursor, Padde } } +static Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs) +{ + Chunk result; + size_t num_rows = std::max(lhs.getNumRows(), rhs.getNumRows()); + createSampleChunk(lhs, result, num_rows); + createSampleChunk(rhs, result, num_rows); + return result; +} + IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { - if (current_inputs[0].skip_last_row || current_inputs[1].skip_last_row) - throw Exception("MergeJoinAlgorithm does not support skipLastRow", ErrorCodes::LOGICAL_ERROR); - - if (!current_inputs[0].chunk && !left_stream_finished) - { + if (!cursors[0].isValid() && !cursors[0].fullyCompleted()) return Status(0); - } - if (!current_inputs[1].chunk && !right_stream_finished) - { + if (!cursors[1].isValid() && !cursors[1].fullyCompleted()) return Status(1); - } JoinKind kind = table_join->getTableJoin().kind(); - if (left_stream_finished && right_stream_finished) + if (cursors[0].fullyCompleted() && cursors[1].fullyCompleted()) { return Status({}, true); } - if (isInner(kind) && (left_stream_finished || right_stream_finished)) + if (isInner(kind) && (cursors[0].fullyCompleted() || cursors[1].fullyCompleted())) { + LOG_DEBUG(log, "{}:{} ", __FILE__, __LINE__); return Status({}, true); } - auto create_block_with_defaults = [] (const Chunk & lhs, const Chunk & rhs) -> Chunk + if (cursors[0].fullyCompleted() && isRightOrFull(kind)) { - Chunk result; - size_t num_rows = std::max(lhs.getNumRows(), rhs.getNumRows()); - createSampleChunk(lhs, result, num_rows); - createSampleChunk(rhs, result, num_rows); - return result; - }; - - if (isLeftOrFull(kind) && right_stream_finished) - { - Chunk result = create_block_with_defaults(current_inputs[0].chunk, sample_chunks[1]); - current_inputs[0] = {}; - return Status(std::move(result), left_stream_finished && right_stream_finished); + Chunk result = createBlockWithDefaults(sample_chunks[0], cursors[1].moveCurrentChunk()); + return Status(std::move(result)); } - if (isRightOrFull(kind) && left_stream_finished) + if (isLeftOrFull(kind) && cursors[1].fullyCompleted()) { - Chunk result = create_block_with_defaults(sample_chunks[0], current_inputs[1].chunk); - current_inputs[1] = {}; - return Status(std::move(result), left_stream_finished && right_stream_finished); + Chunk result = createBlockWithDefaults(cursors[0].moveCurrentChunk(), sample_chunks[1]); + return Status(std::move(result)); } - SortCursor left_cursor = cursors[0].getCursor(); - SortCursor right_cursor = cursors[1].getCursor(); - - if (!left_cursor->isValid() || (right_cursor->isValid() && left_cursor.totallyLessOrEquals(right_cursor))) + if (!cursors[0]->isValid() || totallyLess(cursors[0], cursors[1])) { - current_inputs[0] = {}; - if (left_stream_finished) + if (cursors[0]->isValid() && isLeft(kind)) { - return Status({}, true); + Chunk result = createBlockWithDefaults(cursors[0].moveCurrentChunk(), sample_chunks[1]); + return Status(std::move(result), false); } + cursors[0].moveCurrentChunk(); + if (cursors[0].fullyCompleted()) + return Status({}, true); return Status(0); } - if (!right_cursor->isValid() || (left_cursor->isValid() && right_cursor.totallyLessOrEquals(left_cursor))) - { - current_inputs[1] = {}; - if (right_stream_finished) - { - return Status({}, true); - } - return Status(1); - } + // if (!cursors[1]->isValid() || totallyLess(cursors[1], cursors[0])) + // ... auto left_map = ColumnUInt64::create(); auto right_map = ColumnUInt64::create(); if (isInner(kind)) { - leftOrFullAny(left_cursor, right_cursor, left_map->getData(), right_map->getData()); + leftOrFullAny(cursors[0].getCursor(), cursors[1].getCursor(), left_map->getData(), right_map->getData()); } else if (isLeft(kind)) { - leftOrFullAny(left_cursor, right_cursor, left_map->getData(), right_map->getData()); + leftOrFullAny(cursors[0].getCursor(), cursors[1].getCursor(), left_map->getData(), right_map->getData()); } else if (isRight(kind)) { - leftOrFullAny(left_cursor, right_cursor, left_map->getData(), right_map->getData()); + leftOrFullAny(cursors[0].getCursor(), cursors[1].getCursor(), left_map->getData(), right_map->getData()); } else { @@ -308,20 +360,10 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() } Chunk result; - addIndexColumn(current_inputs[0].chunk.getColumns(), *left_map, result); - addIndexColumn(current_inputs[1].chunk.getColumns(), *right_map, result); + addIndexColumn(cursors[0].getCurrentChunk().getColumns(), *left_map, result); + addIndexColumn(cursors[1].getCurrentChunk().getColumns(), *right_map, result); - if (!left_cursor->isValid()) - { - current_inputs[0] = {}; - } - - if (!right_cursor->isValid()) - { - current_inputs[1] = {}; - } - - return Status(std::move(result), left_stream_finished && right_stream_finished); + return Status(std::move(result), cursors[0].fullyCompleted() && cursors[1].fullyCompleted()); } MergeJoinTransform::MergeJoinTransform( diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index fef2c3fd720..673724e188c 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -65,8 +65,50 @@ public: return impl; } + Chunk moveCurrentChunk() + { + Chunk res = std::move(current_input.chunk); + current_input = {}; + return res; + } + + const Chunk & getCurrentChunk() const + { + return current_input.chunk; + } + + void setInput(IMergingAlgorithm::Input && input) + { + current_input = std::move(input); + + if (!current_input.chunk) + completeAll(); + + if (current_input.skip_last_row) + throw Exception("MergeJoinAlgorithm does not support skipLastRow", ErrorCodes::LOGICAL_ERROR); + + if (current_input.chunk) + { + impl.reset(current_input.chunk.getColumns(), {}, current_input.permutation); + } + } + + bool isValid() const + { + return current_input.chunk && impl.isValid(); + } + + bool fullyCompleted() const { return fully_completed; } + + void completeAll() { fully_completed = true; } + + SortCursorImpl * operator-> () { return &impl; } + const SortCursorImpl * operator-> () const { return &impl; } + private: SortCursorImpl impl; + IMergingAlgorithm::Input current_input; + bool fully_completed = false; // bool has_left_nullable = false; // bool has_right_nullable = false; }; @@ -88,14 +130,9 @@ private: SortDescription left_desc; SortDescription right_desc; - std::vector current_inputs; std::vector cursors; - std::vector sample_chunks; - bool left_stream_finished = false; - bool right_stream_finished = false; - JoinPtr table_join; Poco::Logger * log; }; From ba787db0bb30f4ff681b0f29b837048fad8d41ed Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Apr 2022 14:36:51 +0000 Subject: [PATCH 281/627] Fix build, small changes --- src/Core/SortCursor.h | 3 + .../Transforms/MergeJoinTransform.cpp | 87 +++++++++++-------- .../Transforms/MergeJoinTransform.h | 20 ++--- 3 files changed, 64 insertions(+), 46 deletions(-) diff --git a/src/Core/SortCursor.h b/src/Core/SortCursor.h index 3b58923c2c9..d3c9a99b8d8 100644 --- a/src/Core/SortCursor.h +++ b/src/Core/SortCursor.h @@ -142,9 +142,12 @@ struct SortCursorImpl bool isLast() const { return pos + 1 >= rows; } bool isLast(size_t size) const { return pos + size >= rows; } bool isValid() const { return pos < rows; } + void next() { ++pos; } void next(size_t size) { pos += size; } + size_t getSize() const { return rows; } + size_t rowsLeft() const { return rows - pos; } /// Prevent using pos instead of getRow() private: diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 09319438c1d..225b6717d47 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -31,19 +31,15 @@ namespace constexpr size_t EMPTY_VALUE = std::numeric_limits::max(); - FullMergeJoinCursor createCursor(const Block & block, const Names & columns) { SortDescription desc; desc.reserve(columns.size()); - for (const auto & name : columns) - desc.emplace_back(block.getPositionByName(name)); - - return FullMergeJoinCursor(block.cloneEmpty().getColumns(), desc); + desc.emplace_back(name); + return FullMergeJoinCursor(block, desc); } - template int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos, int null_direction_hint = 1) { @@ -127,6 +123,15 @@ bool ALWAYS_INLINE totallyLess(const FullMergeJoinCursor & lhs, const FullMergeJ return cmp < 0; } +int ALWAYS_INLINE totallyCompare(const FullMergeJoinCursor & lhs, const FullMergeJoinCursor & rhs) +{ + if (totallyLess(lhs, rhs)) + return -1; + if (totallyLess(rhs, lhs)) + return 1; + return 0; +} + void addIndexColumn(const Columns & columns, const IColumn & indices, Chunk & result) { for (const auto & col : columns) @@ -201,9 +206,6 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) { LOG_DEBUG(log, "Consume from {} chunk: {}", source_num, bool(input.chunk)); - if (!input.chunk) - cursors[source_num].completeAll(); - prepareChunk(input.chunk); if (input.chunk.getNumRows() >= EMPTY_VALUE) @@ -212,11 +214,6 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) cursors[source_num].setInput(std::move(input)); } -static size_t ALWAYS_INLINE rowsLeft(SortCursor cursor) -{ - return cursor->rows - cursor->getPosRef(); -} - using JoinKind = ASTTableJoin::Kind; template @@ -224,9 +221,9 @@ static void leftOrFullAny(SortCursor left_cursor, SortCursor right_cursor, Padde { static_assert(kind == JoinKind::Left || kind == JoinKind::Right || kind == JoinKind::Inner, "Invalid join kind"); - size_t num_rows = kind == JoinKind::Left ? rowsLeft(left_cursor) : - kind == JoinKind::Right ? rowsLeft(right_cursor) : - std::min(rowsLeft(left_cursor), rowsLeft(right_cursor)); + size_t num_rows = kind == JoinKind::Left ? left_cursor->rowsLeft() : + kind == JoinKind::Right ? right_cursor->rowsLeft() : + std::min(left_cursor->rowsLeft(), right_cursor->rowsLeft()); constexpr bool is_left_or_inner = kind == JoinKind::Left || kind == JoinKind::Inner; constexpr bool is_right_or_inner = kind == JoinKind::Right || kind == JoinKind::Inner; @@ -291,27 +288,32 @@ static Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs) return result; } +static bool isFinished(const std::vector & cursors, JoinKind kind) +{ + return (cursors[0].fullyCompleted() && cursors[1].fullyCompleted()) + || ((isLeft(kind) || isInner(kind)) && cursors[0].fullyCompleted()) + || ((isRight(kind) || isInner(kind)) && cursors[1].fullyCompleted()); +} + IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { if (!cursors[0].isValid() && !cursors[0].fullyCompleted()) + { return Status(0); + } if (!cursors[1].isValid() && !cursors[1].fullyCompleted()) + { return Status(1); + } JoinKind kind = table_join->getTableJoin().kind(); - if (cursors[0].fullyCompleted() && cursors[1].fullyCompleted()) + if (isFinished(cursors, kind)) { return Status({}, true); } - if (isInner(kind) && (cursors[0].fullyCompleted() || cursors[1].fullyCompleted())) - { - LOG_DEBUG(log, "{}:{} ", __FILE__, __LINE__); - return Status({}, true); - } - if (cursors[0].fullyCompleted() && isRightOrFull(kind)) { Chunk result = createBlockWithDefaults(sample_chunks[0], cursors[1].moveCurrentChunk()); @@ -324,21 +326,32 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() return Status(std::move(result)); } - if (!cursors[0]->isValid() || totallyLess(cursors[0], cursors[1])) + if (int cmp = totallyCompare(cursors[0], cursors[1]); cmp != 0) { - if (cursors[0]->isValid() && isLeft(kind)) + if (cursors[0]->isValid() && isLeftOrFull(kind)) { - Chunk result = createBlockWithDefaults(cursors[0].moveCurrentChunk(), sample_chunks[1]); - return Status(std::move(result), false); + return Status(createBlockWithDefaults(cursors[0].moveCurrentChunk(), sample_chunks[1])); + } + if (isRightOrFull(kind) && cursors[1]->isValid()) + { + return Status(createBlockWithDefaults(sample_chunks[0], cursors[1].moveCurrentChunk())); } - cursors[0].moveCurrentChunk(); - if (cursors[0].fullyCompleted()) - return Status({}, true); - return Status(0); - } - // if (!cursors[1]->isValid() || totallyLess(cursors[1], cursors[0])) - // ... + if (cmp < 0) + { + cursors[0].moveCurrentChunk(); + return Status(0); + } + + if (cmp > 0) + { + cursors[1].moveCurrentChunk(); + return Status(1); + } + + if (!isInner(kind) && !isLeft(kind) && !isRight(kind) && !isFull(kind)) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for kind {}", kind); + } auto left_map = ColumnUInt64::create(); auto right_map = ColumnUInt64::create(); @@ -359,11 +372,13 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", table_join->getTableJoin().kind()); } + assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); + Chunk result; addIndexColumn(cursors[0].getCurrentChunk().getColumns(), *left_map, result); addIndexColumn(cursors[1].getCurrentChunk().getColumns(), *right_map, result); - return Status(std::move(result), cursors[0].fullyCompleted() && cursors[1].fullyCompleted()); + return Status(std::move(result), isFinished(cursors, kind)); } MergeJoinTransform::MergeJoinTransform( diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 673724e188c..0685bac04fd 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -29,8 +29,9 @@ class FullMergeJoinCursor { public: - FullMergeJoinCursor(const Columns & columns, const SortDescription & desc) - : impl(columns, desc) + FullMergeJoinCursor(const Block & block, const SortDescription & desc) + : impl(block, desc) + , sample_block(block) { } @@ -82,14 +83,14 @@ public: current_input = std::move(input); if (!current_input.chunk) - completeAll(); + fully_completed = true; if (current_input.skip_last_row) throw Exception("MergeJoinAlgorithm does not support skipLastRow", ErrorCodes::LOGICAL_ERROR); if (current_input.chunk) { - impl.reset(current_input.chunk.getColumns(), {}, current_input.permutation); + impl.reset(current_input.chunk.getColumns(), sample_block, current_input.permutation); } } @@ -98,17 +99,19 @@ public: return current_input.chunk && impl.isValid(); } - bool fullyCompleted() const { return fully_completed; } - - void completeAll() { fully_completed = true; } + bool fullyCompleted() const { return !isValid() && fully_completed; } SortCursorImpl * operator-> () { return &impl; } const SortCursorImpl * operator-> () const { return &impl; } private: SortCursorImpl impl; + IMergingAlgorithm::Input current_input; + bool fully_completed = false; + + Block sample_block; // bool has_left_nullable = false; // bool has_right_nullable = false; }; @@ -127,9 +130,6 @@ public: virtual Status merge() override; private: - SortDescription left_desc; - SortDescription right_desc; - std::vector cursors; std::vector sample_chunks; From 075a7065832ce488633429d56afda6caceece7a1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Apr 2022 14:40:17 +0000 Subject: [PATCH 282/627] some tests with full_sorting_merge --- ...erence => 01031_new_any_join.reference.j2} | 3 + ...any_join.sql => 01031_new_any_join.sql.j2} | 7 +- .../02249_full_sort_join.reference | 286 ++++++++++++++++++ .../0_stateless/02249_full_sort_join.sql.j2 | 76 +++++ 4 files changed, 371 insertions(+), 1 deletion(-) rename tests/queries/0_stateless/{01031_new_any_join.reference => 01031_new_any_join.reference.j2} (70%) rename tests/queries/0_stateless/{01031_new_any_join.sql => 01031_new_any_join.sql.j2} (85%) create mode 100644 tests/queries/0_stateless/02249_full_sort_join.reference create mode 100644 tests/queries/0_stateless/02249_full_sort_join.sql.j2 diff --git a/tests/queries/0_stateless/01031_new_any_join.reference b/tests/queries/0_stateless/01031_new_any_join.reference.j2 similarity index 70% rename from tests/queries/0_stateless/01031_new_any_join.reference rename to tests/queries/0_stateless/01031_new_any_join.reference.j2 index 7b08703e422..347d335d479 100644 --- a/tests/queries/0_stateless/01031_new_any_join.reference +++ b/tests/queries/0_stateless/01031_new_any_join.reference.j2 @@ -1,3 +1,5 @@ +{% for join_algorithm in ['hash', 'full_sorting_merge'] -%} +join_algorithm: {{ join_algorithm }} any left 0 a1 0 1 a2 0 @@ -24,3 +26,4 @@ any right (rev) 2 a3 2 b1 3 a4 0 4 a5 4 b3 +{% endfor -%} diff --git a/tests/queries/0_stateless/01031_new_any_join.sql b/tests/queries/0_stateless/01031_new_any_join.sql.j2 similarity index 85% rename from tests/queries/0_stateless/01031_new_any_join.sql rename to tests/queries/0_stateless/01031_new_any_join.sql.j2 index c69a765ebce..00244785c99 100644 --- a/tests/queries/0_stateless/01031_new_any_join.sql +++ b/tests/queries/0_stateless/01031_new_any_join.sql.j2 @@ -10,7 +10,10 @@ INSERT INTO t2 (x, s) VALUES (2, 'b1'), (4, 'b3'), (5, 'b6'); SET join_use_nulls = 0; SET any_join_distinct_right_table_keys = 0; -SET join_algorithm = 'full_sorting_merge'; +{% for join_algorithm in ['hash', 'full_sorting_merge'] -%} + +SET join_algorithm = '{{ join_algorithm }}'; +SELECT 'join_algorithm: {{ join_algorithm }}'; SELECT 'any left'; SELECT t1.*, t2.* FROM t1 ANY LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; @@ -30,5 +33,7 @@ SELECT t1.*, t2.* FROM t1 ANY RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; SELECT 'any right (rev)'; SELECT t1.*, t2.* FROM t2 ANY RIGHT JOIN t1 USING(x) ORDER BY t1.x, t2.x; +{% endfor -%} + DROP TABLE t1; DROP TABLE t2; diff --git a/tests/queries/0_stateless/02249_full_sort_join.reference b/tests/queries/0_stateless/02249_full_sort_join.reference new file mode 100644 index 00000000000..8e394e5fd43 --- /dev/null +++ b/tests/queries/0_stateless/02249_full_sort_join.reference @@ -0,0 +1,286 @@ +simple cases, block size = 1 +ANY INNER +8 8 +2 2 +ANY LEFT +1 0 val0 +9 0 val1 +8 8 val2 +5 0 val3 +9 0 val4 +1 0 val5 +10 0 val6 +1 0 val7 +5 0 val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +0 7 val3 +8 8 val4 +0 4 val5 +0 6 val6 +0 4 val7 +2 2 val8 +2 2 val9 +simple cases, block size = 2 +ANY INNER +8 8 +2 2 +ANY LEFT +1 0 val0 +9 0 val1 +8 8 val2 +5 0 val3 +9 0 val4 +1 0 val5 +10 0 val6 +1 0 val7 +5 0 val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +0 7 val3 +8 8 val4 +0 4 val5 +0 6 val6 +0 4 val7 +2 2 val8 +2 2 val9 +simple cases, block size = 3 +ANY INNER +8 8 +2 2 +ANY LEFT +1 0 val0 +9 0 val1 +8 8 val2 +5 0 val3 +9 0 val4 +1 0 val5 +10 0 val6 +1 0 val7 +5 0 val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +0 7 val3 +8 8 val4 +0 4 val5 +0 6 val6 +0 4 val7 +2 2 val8 +2 2 val9 +simple cases, block size = 4 +ANY INNER +8 8 +2 2 +ANY LEFT +1 0 val0 +9 0 val1 +8 8 val2 +5 0 val3 +9 0 val4 +1 0 val5 +10 0 val6 +1 0 val7 +5 0 val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +0 7 val3 +8 8 val4 +0 4 val5 +0 6 val6 +0 4 val7 +2 2 val8 +2 2 val9 +simple cases, block size = 5 +ANY INNER +8 8 +2 2 +ANY LEFT +1 0 val0 +9 0 val1 +8 8 val2 +5 0 val3 +9 0 val4 +1 0 val5 +10 0 val6 +1 0 val7 +5 0 val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +0 7 val3 +8 8 val4 +0 4 val5 +0 6 val6 +0 4 val7 +2 2 val8 +2 2 val9 +simple cases, block size = 6 +ANY INNER +8 8 +2 2 +ANY LEFT +1 0 val0 +9 0 val1 +8 8 val2 +5 0 val3 +9 0 val4 +1 0 val5 +10 0 val6 +1 0 val7 +5 0 val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +0 7 val3 +8 8 val4 +0 4 val5 +0 6 val6 +0 4 val7 +2 2 val8 +2 2 val9 +simple cases, block size = 7 +ANY INNER +8 8 +2 2 +ANY LEFT +1 0 val0 +9 0 val1 +8 8 val2 +5 0 val3 +9 0 val4 +1 0 val5 +10 0 val6 +1 0 val7 +5 0 val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +0 7 val3 +8 8 val4 +0 4 val5 +0 6 val6 +0 4 val7 +2 2 val8 +2 2 val9 +simple cases, block size = 8 +ANY INNER +8 8 +2 2 +ANY LEFT +1 0 val0 +9 0 val1 +8 8 val2 +5 0 val3 +9 0 val4 +1 0 val5 +10 0 val6 +1 0 val7 +5 0 val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +0 7 val3 +8 8 val4 +0 4 val5 +0 6 val6 +0 4 val7 +2 2 val8 +2 2 val9 +simple cases, block size = 9 +ANY INNER +8 8 +2 2 +ANY LEFT +1 0 val0 +9 0 val1 +8 8 val2 +5 0 val3 +9 0 val4 +1 0 val5 +10 0 val6 +1 0 val7 +5 0 val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +0 7 val3 +8 8 val4 +0 4 val5 +0 6 val6 +0 4 val7 +2 2 val8 +2 2 val9 +simple cases, block size = 10 +ANY INNER +8 8 +2 2 +ANY LEFT +1 0 val0 +9 0 val1 +8 8 val2 +5 0 val3 +9 0 val4 +1 0 val5 +10 0 val6 +1 0 val7 +5 0 val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +0 7 val3 +8 8 val4 +0 4 val5 +0 6 val6 +0 4 val7 +2 2 val8 +2 2 val9 +simple cases: join_use_nulls +ANY INNER +8 8 +2 2 +ANY LEFT +1 \N val0 +9 \N val1 +8 8 val2 +5 \N val3 +9 \N val4 +1 \N val5 +10 \N val6 +1 \N val7 +5 \N val8 +2 2 val9 +ANY RIGHT +2 2 val0 +8 8 val1 +8 8 val2 +\N 7 val3 +8 8 val4 +\N 4 val5 +\N 6 val6 +\N 4 val7 +2 2 val8 +2 2 val9 diff --git a/tests/queries/0_stateless/02249_full_sort_join.sql.j2 b/tests/queries/0_stateless/02249_full_sort_join.sql.j2 new file mode 100644 index 00000000000..d2958666616 --- /dev/null +++ b/tests/queries/0_stateless/02249_full_sort_join.sql.j2 @@ -0,0 +1,76 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (key UInt32, s String) engine = MergeTree ORDER BY key; +CREATE TABLE t2 (key UInt32, s String) engine = MergeTree ORDER BY key; + +{% set table_size = 10 %} + +INSERT INTO t1 + SELECT + sipHash64(number, 'x') % {{ table_size }} + 1 as key, + 'val' || toString(number) as s + FROM numbers_mt({{ table_size }}); + +INSERT INTO t2 + SELECT + sipHash64(number, 'y') % {{ table_size }} + 1 as key, + 'val' || toString(number) as s + FROM numbers_mt({{ table_size }}); + +SET join_algorithm = 'full_sorting_merge'; + +{% for block_size in range (1, table_size + 1) %} + +SET max_block_size = {{ block_size }}; + +SELECT 'simple cases, block size = {{ block_size }}'; + +SELECT 'ANY INNER'; +SELECT t1.key, t2.key FROM t1 +ANY INNER JOIN t2 +ON t1.key == t2.key +ORDER BY t1.s, t2.s +; + +SELECT 'ANY LEFT'; +SELECT t1.key, t2.key, t1.s FROM t1 +ANY LEFT JOIN t2 +ON t1.key == t2.key +ORDER BY t1.s +; + +SELECT 'ANY RIGHT'; +SELECT t1.key, t2.key, t2.s FROM t1 +ANY RIGHT JOIN t2 +ON t1.key == t2.key +ORDER BY t2.s +; + +{% endfor %} + + +SET join_use_nulls = 1; + +SELECT 'simple cases: join_use_nulls'; + +SELECT 'ANY INNER'; +SELECT t1.key, t2.key FROM t1 +ANY INNER JOIN t2 +ON t1.key == t2.key +ORDER BY t1.s, t2.s +; + +SELECT 'ANY LEFT'; +SELECT t1.key, t2.key, t1.s FROM t1 +ANY LEFT JOIN t2 +ON t1.key == t2.key +ORDER BY t1.s +; + +SELECT 'ANY RIGHT'; +SELECT t1.key, t2.key, t2.s FROM t1 +ANY RIGHT JOIN t2 +ON t1.key == t2.key +ORDER BY t2.s +; From 88d8dc5be2e2a5139cf602ff185cd9b50b3bbe16 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 7 Apr 2022 16:14:56 +0000 Subject: [PATCH 283/627] wip full sort any join --- .../Transforms/MergeJoinTransform.cpp | 109 ++++++++++-------- .../Transforms/MergeJoinTransform.h | 40 ++++--- .../02250_full_sort_join_long.reference | 24 ++++ .../02250_full_sort_join_long.sql.j2 | 53 +++++++++ 4 files changed, 162 insertions(+), 64 deletions(-) create mode 100644 tests/queries/0_stateless/02250_full_sort_join_long.reference create mode 100644 tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 225b6717d47..f6d8ca065d6 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -29,7 +29,7 @@ namespace ErrorCodes namespace { -constexpr size_t EMPTY_VALUE = std::numeric_limits::max(); +constexpr size_t EMPTY_VALUE_IDX = std::numeric_limits::max(); FullMergeJoinCursor createCursor(const Block & block, const Names & columns) { @@ -132,18 +132,23 @@ int ALWAYS_INLINE totallyCompare(const FullMergeJoinCursor & lhs, const FullMerg return 0; } -void addIndexColumn(const Columns & columns, const IColumn & indices, Chunk & result) +void addIndexColumn(const Columns & columns, ColumnUInt64 & indices, Chunk & result, size_t start, size_t limit) { for (const auto & col : columns) { if (indices.empty()) { - result.addColumn(col); + result.addColumn(col->cut(start, limit)); } else { + if (limit == 0) + limit = indices.size(); + + assert(limit == indices.size()); + auto tmp_col = col->cloneResized(col->size() + 1); - ColumnPtr new_col = tmp_col->index(indices, 0); + ColumnPtr new_col = tmp_col->index(indices, limit); result.addColumn(std::move(new_col)); } } @@ -170,12 +175,18 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( } -static void createSampleChunk(const Chunk & chunk, Chunk & sample_chunk, size_t size = 0) +static void copyColumnsResized(const Chunk & chunk, size_t start, size_t size, Chunk & result_chunk) { const auto & cols = chunk.getColumns(); for (const auto & col : cols) { - sample_chunk.addColumn(col->cloneResized(size)); + if (!start || start > col->size()) + result_chunk.addColumn(col->cloneResized(size)); + else + { + assert(size <= col->size()); + result_chunk.addColumn(col->cut(start, size)); + } } } @@ -183,11 +194,12 @@ void MergeJoinAlgorithm::initialize(Inputs inputs) { if (inputs.size() != 2) throw Exception("MergeJoinAlgorithm requires exactly two inputs", ErrorCodes::LOGICAL_ERROR); + LOG_DEBUG(log, "MergeJoinAlgorithm initialize, number of inputs: {}", inputs.size()); for (size_t i = 0; i < inputs.size(); ++i) { - createSampleChunk(inputs[i].chunk, sample_chunks.emplace_back()); + copyColumnsResized(inputs[i].chunk, 0, 0, sample_chunks.emplace_back()); consume(inputs[i], i); } } @@ -204,11 +216,11 @@ static void prepareChunk(Chunk & chunk) void MergeJoinAlgorithm::consume(Input & input, size_t source_num) { - LOG_DEBUG(log, "Consume from {} chunk: {}", source_num, bool(input.chunk)); + LOG_DEBUG(log, "TODO: remove. Consume from {} chunk: {}", source_num, bool(input.chunk)); prepareChunk(input.chunk); - if (input.chunk.getNumRows() >= EMPTY_VALUE) + if (input.chunk.getNumRows() >= EMPTY_VALUE_IDX) throw Exception("Too many rows in input", ErrorCodes::TOO_MANY_ROWS); cursors[source_num].setInput(std::move(input)); @@ -217,7 +229,7 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) using JoinKind = ASTTableJoin::Kind; template -static void leftOrFullAny(SortCursor left_cursor, SortCursor right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) +static void anyJoin(FullMergeJoinCursor & left_cursor, FullMergeJoinCursor & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) { static_assert(kind == JoinKind::Left || kind == JoinKind::Right || kind == JoinKind::Inner, "Invalid join kind"); @@ -265,29 +277,30 @@ static void leftOrFullAny(SortCursor left_cursor, SortCursor right_cursor, Padde right_cursor->next(); } } - - while (left_cursor->isValid() && kind == JoinKind::Left) - { - right_map.emplace_back(right_cursor->rows); - left_cursor->next(); - } - - while (right_cursor->isValid() && kind == JoinKind::Right) - { - left_map.emplace_back(left_cursor->rows); - right_cursor->next(); - } } -static Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs) +static Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs, size_t start, size_t num_rows) { Chunk result; - size_t num_rows = std::max(lhs.getNumRows(), rhs.getNumRows()); - createSampleChunk(lhs, result, num_rows); - createSampleChunk(rhs, result, num_rows); + copyColumnsResized(lhs, start, num_rows, result); + copyColumnsResized(rhs, start, num_rows, result); return result; } +static Chunk createBlockWithDefaults(const Chunk & lhs, FullMergeJoinCursor & rhs) +{ + auto res = createBlockWithDefaults(lhs, rhs.getCurrentChunk(), rhs->getRow(), rhs->rowsLeft()); + rhs.reset(); + return res; +} + +static Chunk createBlockWithDefaults(FullMergeJoinCursor & lhs, const Chunk & rhs) +{ + auto res = createBlockWithDefaults(lhs.getCurrentChunk(), rhs, lhs->getRow(), lhs->rowsLeft()); + lhs.reset(); + return res; +} + static bool isFinished(const std::vector & cursors, JoinKind kind) { return (cursors[0].fullyCompleted() && cursors[1].fullyCompleted()) @@ -297,12 +310,12 @@ static bool isFinished(const std::vector & cursors, JoinKin IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { - if (!cursors[0].isValid() && !cursors[0].fullyCompleted()) + if (!cursors[0]->isValid() && !cursors[0].fullyCompleted()) { return Status(0); } - if (!cursors[1].isValid() && !cursors[1].fullyCompleted()) + if (!cursors[1]->isValid() && !cursors[1].fullyCompleted()) { return Status(1); } @@ -316,36 +329,35 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() if (cursors[0].fullyCompleted() && isRightOrFull(kind)) { - Chunk result = createBlockWithDefaults(sample_chunks[0], cursors[1].moveCurrentChunk()); + Chunk result = createBlockWithDefaults(sample_chunks[0], cursors[1]); return Status(std::move(result)); } if (isLeftOrFull(kind) && cursors[1].fullyCompleted()) { - Chunk result = createBlockWithDefaults(cursors[0].moveCurrentChunk(), sample_chunks[1]); + Chunk result = createBlockWithDefaults(cursors[0], sample_chunks[1]); return Status(std::move(result)); } if (int cmp = totallyCompare(cursors[0], cursors[1]); cmp != 0) { - if (cursors[0]->isValid() && isLeftOrFull(kind)) - { - return Status(createBlockWithDefaults(cursors[0].moveCurrentChunk(), sample_chunks[1])); - } - if (isRightOrFull(kind) && cursors[1]->isValid()) - { - return Status(createBlockWithDefaults(sample_chunks[0], cursors[1].moveCurrentChunk())); - } - if (cmp < 0) { - cursors[0].moveCurrentChunk(); + if (cursors[0]->isValid() && isLeftOrFull(kind)) + { + return Status(createBlockWithDefaults(cursors[0], sample_chunks[1])); + } + cursors[0].reset(); return Status(0); } if (cmp > 0) { - cursors[1].moveCurrentChunk(); + if (isRightOrFull(kind) && cursors[1]->isValid()) + { + return Status(createBlockWithDefaults(sample_chunks[0], cursors[1])); + } + cursors[1].reset(); return Status(1); } @@ -355,17 +367,18 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() auto left_map = ColumnUInt64::create(); auto right_map = ColumnUInt64::create(); + std::pair prev_pos = std::make_pair(cursors[0]->getRow(), cursors[1]->getRow()); if (isInner(kind)) { - leftOrFullAny(cursors[0].getCursor(), cursors[1].getCursor(), left_map->getData(), right_map->getData()); + anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); } else if (isLeft(kind)) { - leftOrFullAny(cursors[0].getCursor(), cursors[1].getCursor(), left_map->getData(), right_map->getData()); + anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); } else if (isRight(kind)) { - leftOrFullAny(cursors[0].getCursor(), cursors[1].getCursor(), left_map->getData(), right_map->getData()); + anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); } else { @@ -375,9 +388,9 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); Chunk result; - addIndexColumn(cursors[0].getCurrentChunk().getColumns(), *left_map, result); - addIndexColumn(cursors[1].getCurrentChunk().getColumns(), *right_map, result); - + size_t num_result_rows = std::max(left_map->size(), right_map->size()); + addIndexColumn(cursors[0].getCurrentChunk().getColumns(), *left_map, result, prev_pos.first, num_result_rows); + addIndexColumn(cursors[1].getCurrentChunk().getColumns(), *right_map, result, prev_pos.second, num_result_rows); return Status(std::move(result), isFinished(cursors, kind)); } @@ -394,7 +407,7 @@ MergeJoinTransform::MergeJoinTransform( void MergeJoinTransform::onFinish() { - LOG_TRACE(log, "onFinish"); + LOG_TRACE(log, "TODO: remove onFinish"); } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 0685bac04fd..36ea3cdd256 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -66,11 +66,10 @@ public: return impl; } - Chunk moveCurrentChunk() + void reset() { - Chunk res = std::move(current_input.chunk); current_input = {}; - return res; + resetInternalCursor(); } const Chunk & getCurrentChunk() const @@ -80,31 +79,40 @@ public: void setInput(IMergingAlgorithm::Input && input) { + if (input.skip_last_row) + throw Exception("FullMergeJoinCursor does not support skipLastRow", ErrorCodes::NOT_IMPLEMENTED); + + if (current_input.permutation) + throw DB::Exception("FullMergeJoinCursor: permutation is not supported", ErrorCodes::NOT_IMPLEMENTED); + + current_input = std::move(input); if (!current_input.chunk) fully_completed = true; - if (current_input.skip_last_row) - throw Exception("MergeJoinAlgorithm does not support skipLastRow", ErrorCodes::LOGICAL_ERROR); - - if (current_input.chunk) - { - impl.reset(current_input.chunk.getColumns(), sample_block, current_input.permutation); - } + resetInternalCursor(); } - bool isValid() const - { - return current_input.chunk && impl.isValid(); - } - - bool fullyCompleted() const { return !isValid() && fully_completed; } + bool fullyCompleted() const { return !impl.isValid() && fully_completed; } SortCursorImpl * operator-> () { return &impl; } const SortCursorImpl * operator-> () const { return &impl; } private: + + void resetInternalCursor() + { + if (current_input.chunk) + { + impl.reset(current_input.chunk.getColumns(), sample_block, current_input.permutation); + } + else + { + impl.reset(sample_block.cloneEmpty().getColumns(), sample_block); + } + } + SortCursorImpl impl; IMergingAlgorithm::Input current_input; diff --git a/tests/queries/0_stateless/02250_full_sort_join_long.reference b/tests/queries/0_stateless/02250_full_sort_join_long.reference new file mode 100644 index 00000000000..c334781e589 --- /dev/null +++ b/tests/queries/0_stateless/02250_full_sort_join_long.reference @@ -0,0 +1,24 @@ +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +31625246885215 199622811843 6323501 6323501 399458 +ANY RIGHT +199622811843 316117334104 631994 399458 631994 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +31625246885215 199622811843 6323501 6323501 399458 +ANY RIGHT +199622811843 316117334104 631994 399458 631994 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +31625246885215 199622811843 6323501 6323501 399458 +ANY RIGHT +199622811843 316117334104 631994 399458 631994 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +31625246885215 199622811843 6323501 6323501 399458 +ANY RIGHT +199622811843 316117334104 631994 399458 631994 diff --git a/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 new file mode 100644 index 00000000000..41184d2a198 --- /dev/null +++ b/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 @@ -0,0 +1,53 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (key UInt32, s String) engine = MergeTree ORDER BY key; +CREATE TABLE t2 (key UInt32, s String) engine = MergeTree ORDER BY key; + +{% set ltable_size = 10000000 -%} +{% set rtable_size = 1000000 -%} + +INSERT INTO t1 + SELECT key, any(s) as s FROM ( + SELECT + sipHash64(number, 'x') % {{ ltable_size }} + 1 as key, + 'val' || toString(number) as s + FROM numbers_mt({{ ltable_size }}) + ) group by key +; + +INSERT INTO t2 + SELECT key, any(s) as s FROM ( + SELECT + sipHash64(number, 'y') % {{ rtable_size }} + 1 as key, + 'val' || toString(number) as s + FROM numbers_mt({{ rtable_size }}) + ) group by key +; + +-- SET join_algorithm = 'hash'; +-- SET join_algorithm = 'full_sorting_merge'; + +{% for block_size in [32001, 65505, 65536, range(32001, 65536) | random] %} + +SET max_block_size = {{ block_size }}; + +SELECT 'ANY INNER'; +SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 +ANY INNER JOIN t2 +ON t1.key == t2.key +; + +SELECT 'ANY LEFT'; +SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 +ANY LEFT JOIN t2 +ON t1.key == t2.key +; + +SELECT 'ANY RIGHT'; +SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 +ANY RIGHT JOIN t2 +ON t1.key == t2.key +; + +{% endfor %} From 6d198ff3d78ee5095c48ca2528a8e40be090c72c Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 8 Apr 2022 10:04:52 +0000 Subject: [PATCH 284/627] fix style --- src/Interpreters/FullSortingMergeJoin.h | 2 ++ src/Processors/Transforms/MergeJoinTransform.cpp | 1 + src/Processors/Transforms/MergeJoinTransform.h | 5 +++++ tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 | 3 +-- 4 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 7689010f13a..df9bdfa5fc8 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -1,3 +1,5 @@ +#pragma once + #include #include #include diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index f6d8ca065d6..d780c25ea0e 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int TOO_MANY_ROWS; + extern const int LOGICAL_ERROR; } namespace diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 36ea3cdd256..498a092cf33 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -17,6 +17,11 @@ namespace Poco { class Logger; } namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + class IJoin; using JoinPtr = std::shared_ptr; diff --git a/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 index 41184d2a198..9b7d035b9cd 100644 --- a/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 @@ -25,8 +25,7 @@ INSERT INTO t2 ) group by key ; --- SET join_algorithm = 'hash'; --- SET join_algorithm = 'full_sorting_merge'; +SET join_algorithm = 'full_sorting_merge'; {% for block_size in [32001, 65505, 65536, range(32001, 65536) | random] %} From 0b9d4ee64014fa2e4396bc96e2dba473589e9ac2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 8 Apr 2022 15:05:38 +0000 Subject: [PATCH 285/627] wip sort join same rows --- .../Transforms/MergeJoinTransform.cpp | 154 +++++++++++++++--- .../Transforms/MergeJoinTransform.h | 129 +++++++-------- .../02250_full_sort_join_long.sql.j2 | 6 +- 3 files changed, 193 insertions(+), 96 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index d780c25ea0e..0f9c545793d 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -27,22 +27,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace -{ - constexpr size_t EMPTY_VALUE_IDX = std::numeric_limits::max(); -FullMergeJoinCursor createCursor(const Block & block, const Names & columns) -{ - SortDescription desc; - desc.reserve(columns.size()); - for (const auto & name : columns) - desc.emplace_back(name); - return FullMergeJoinCursor(block, desc); -} - template -int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos, int null_direction_hint = 1) +static int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos, int null_direction_hint = 1) { if constexpr (has_left_nulls && has_right_nulls) { @@ -86,6 +74,111 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, return left_column.compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); } +FullMergeJoinCursor::FullMergeJoinCursor(const Block & block, const SortDescription & desc_) + : impl(block, desc_) + , sample_block(block) +{ +} + +bool ALWAYS_INLINE FullMergeJoinCursor::sameNext() const +{ + if (!impl.isValid() || impl.isLast()) + return false; + + for (size_t i = 0; i < impl.sort_columns_size; ++i) + { + const auto & col = *impl.sort_columns[i]; + int cmp = nullableCompareAt( + col, col, impl.getRow(), impl.getRow() + 1, 0); + if (cmp != 0) + return false; + } + return true; +} + +bool FullMergeJoinCursor::sameUnitlEnd() const +{ + if (!impl.isValid() || impl.isLast()) + return true; + + for (size_t i = 0; i < impl.sort_columns_size; ++i) + { + const auto & col = *impl.sort_columns[i]; + int cmp = nullableCompareAt( + col, col, impl.getRow(), impl.rows - 1, 0); + if (cmp != 0) + return false; + } + + return true; +} + +size_t FullMergeJoinCursor::nextDistinct() +{ + if (sameUnitlEnd()) + return 0; + + size_t start_pos = impl.getRow(); + while (sameNext()) + { + impl.next(); + } + impl.next(); + return impl.getRow() - start_pos; +} + +void FullMergeJoinCursor::reset() +{ + current_input = {}; + resetInternalCursor(); +} + +const Chunk & FullMergeJoinCursor::getCurrentChunk() const +{ + return current_input.chunk; +} + +void FullMergeJoinCursor::setInput(IMergingAlgorithm::Input && input) +{ + if (input.skip_last_row) + throw Exception("FullMergeJoinCursor does not support skipLastRow", ErrorCodes::NOT_IMPLEMENTED); + + if (current_input.permutation) + throw DB::Exception("FullMergeJoinCursor: permutation is not supported", ErrorCodes::NOT_IMPLEMENTED); + + + current_input = std::move(input); + + if (!current_input.chunk) + fully_completed = true; + + resetInternalCursor(); +} + +void FullMergeJoinCursor::resetInternalCursor() +{ + if (current_input.chunk) + { + impl.reset(current_input.chunk.getColumns(), sample_block, current_input.permutation); + } + else + { + impl.reset(sample_block.cloneEmpty().getColumns(), sample_block); + } +} + +namespace +{ + +FullMergeJoinCursor createCursor(const Block & block, const Names & columns) +{ + SortDescription desc; + desc.reserve(columns.size()); + for (const auto & name : columns) + desc.emplace_back(name); + return FullMergeJoinCursor(block, desc); +} + /// If on_pos == true, compare two columns at specified positions. /// Otherwise, compare two columns at the current positions, `lpos` and `rpos` are ignored. template @@ -224,13 +317,16 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) if (input.chunk.getNumRows() >= EMPTY_VALUE_IDX) throw Exception("Too many rows in input", ErrorCodes::TOO_MANY_ROWS); + if (input.chunk) + stat.num_blocks[source_num] += 1; + cursors[source_num].setInput(std::move(input)); } using JoinKind = ASTTableJoin::Kind; template -static void anyJoin(FullMergeJoinCursor & left_cursor, FullMergeJoinCursor & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) +static std::optional anyJoin(FullMergeJoinCursor & left_cursor, FullMergeJoinCursor & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) { static_assert(kind == JoinKind::Left || kind == JoinKind::Right || kind == JoinKind::Inner, "Invalid join kind"); @@ -267,17 +363,24 @@ static void anyJoin(FullMergeJoinCursor & left_cursor, FullMergeJoinCursor & rig } else if (cmp < 0) { + size_t num = left_cursor.nextDistinct(); + if (num == 0) + return 0; + if constexpr (kind == JoinKind::Left) - right_map.emplace_back(right_cursor->rows); - left_cursor->next(); + right_map.resize_fill(right_map.size() + num, right_cursor->rows); } else { + size_t num = right_cursor.nextDistinct(); + if (num == 0) + return 1; + if constexpr (kind == JoinKind::Right) - left_map.emplace_back(left_cursor->rows); - right_cursor->next(); + left_map.resize_fill(left_map.size() + num, left_cursor->rows); } } + return std::nullopt; } static Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs, size_t start, size_t num_rows) @@ -311,6 +414,13 @@ static bool isFinished(const std::vector & cursors, JoinKin IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { + if (required_input.has_value()) + { + size_t r = required_input.value(); + required_input = {}; + return Status(r); + } + if (!cursors[0]->isValid() && !cursors[0].fullyCompleted()) { return Status(0); @@ -371,15 +481,15 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() std::pair prev_pos = std::make_pair(cursors[0]->getRow(), cursors[1]->getRow()); if (isInner(kind)) { - anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); + required_input = anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); } else if (isLeft(kind)) { - anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); + required_input = anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); } else if (isRight(kind)) { - anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); + required_input = anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); } else { @@ -408,7 +518,7 @@ MergeJoinTransform::MergeJoinTransform( void MergeJoinTransform::onFinish() { - LOG_TRACE(log, "TODO: remove onFinish"); + algorithm.onFinish(total_stopwatch.elapsedSeconds()); } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 498a092cf33..32b47dc0b6a 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -26,6 +27,37 @@ class IJoin; using JoinPtr = std::shared_ptr; +class MultiCursor +{ + void next() + { + } + + bool isLast() + { + return false; + } + + bool isValid() + { + return false; + } + +private: + SortDescription desc; + struct CursorWithBlock + { + SortCursorImpl impl; + Chunk input; + }; + + using CursorList = std::list; + using CursorListIt = CursorList::iterator; + + CursorListIt current; + CursorList inputs; +}; + /* * Wrapper for SortCursorImpl * It is used to store information about the current state of the cursor. @@ -34,70 +66,21 @@ class FullMergeJoinCursor { public: - FullMergeJoinCursor(const Block & block, const SortDescription & desc) - : impl(block, desc) - , sample_block(block) - { - } + FullMergeJoinCursor(const Block & block, const SortDescription & desc); - SortCursor getCursor() - { - return SortCursor(&impl); - } + SortCursor getCursor(); - /* - /// Expects !atEnd() - size_t getEqualLength() const - { - size_t pos = impl.getRow() + 1; - for (; pos < impl.rows; ++pos) - if (!samePrev(pos)) - break; - return pos - impl.getRow(); - } + bool sameUnitlEnd() const; - /// Expects lhs_pos > 0 - bool ALWAYS_INLINE samePrev(size_t lhs_pos) const - { - for (size_t i = 0; i < impl.sort_columns_size; ++i) - if (impl.sort_columns[i]->compareAt(lhs_pos - 1, lhs_pos, *(impl.sort_columns[i]), 1) != 0) - return false; - return true; - } - */ + bool ALWAYS_INLINE sameNext() const; - SortCursorImpl & getImpl() - { - return impl; - } + size_t nextDistinct(); - void reset() - { - current_input = {}; - resetInternalCursor(); - } + void reset(); - const Chunk & getCurrentChunk() const - { - return current_input.chunk; - } + const Chunk & getCurrentChunk() const; - void setInput(IMergingAlgorithm::Input && input) - { - if (input.skip_last_row) - throw Exception("FullMergeJoinCursor does not support skipLastRow", ErrorCodes::NOT_IMPLEMENTED); - - if (current_input.permutation) - throw DB::Exception("FullMergeJoinCursor: permutation is not supported", ErrorCodes::NOT_IMPLEMENTED); - - - current_input = std::move(input); - - if (!current_input.chunk) - fully_completed = true; - - resetInternalCursor(); - } + void setInput(IMergingAlgorithm::Input && input); bool fullyCompleted() const { return !impl.isValid() && fully_completed; } @@ -105,23 +88,15 @@ public: const SortCursorImpl * operator-> () const { return &impl; } private: + void resetInternalCursor(); + - void resetInternalCursor() - { - if (current_input.chunk) - { - impl.reset(current_input.chunk.getColumns(), sample_block, current_input.permutation); - } - else - { - impl.reset(sample_block.cloneEmpty().getColumns(), sample_block); - } - } SortCursorImpl impl; IMergingAlgorithm::Input current_input; + bool fully_completed = false; Block sample_block; @@ -142,11 +117,28 @@ public: virtual void consume(Input & input, size_t source_num) override; virtual Status merge() override; + void onFinish(double seconds) + { + LOG_TRACE(log, "Finished pocessing {} left and {} right blocks in {} seconds", + stat.num_blocks[0], + stat.num_blocks[1], + seconds); + } + private: + std::optional required_input = std::nullopt; + std::vector cursors; std::vector sample_chunks; JoinPtr table_join; + + struct Statistic + { + size_t num_blocks[2] = {0, 0}; + }; + Statistic stat; + Poco::Logger * log; }; @@ -163,7 +155,6 @@ public: protected: void onFinish() override; - UInt64 elapsed_ns = 0; Poco::Logger * log; }; diff --git a/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 index 9b7d035b9cd..9d30795e7e3 100644 --- a/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 @@ -8,24 +8,20 @@ CREATE TABLE t2 (key UInt32, s String) engine = MergeTree ORDER BY key; {% set rtable_size = 1000000 -%} INSERT INTO t1 - SELECT key, any(s) as s FROM ( SELECT sipHash64(number, 'x') % {{ ltable_size }} + 1 as key, 'val' || toString(number) as s FROM numbers_mt({{ ltable_size }}) - ) group by key ; INSERT INTO t2 - SELECT key, any(s) as s FROM ( SELECT sipHash64(number, 'y') % {{ rtable_size }} + 1 as key, 'val' || toString(number) as s FROM numbers_mt({{ rtable_size }}) - ) group by key ; -SET join_algorithm = 'full_sorting_merge'; +-- SET join_algorithm = 'full_sorting_merge'; {% for block_size in [32001, 65505, 65536, range(32001, 65536) | random] %} From a2d190edb8d5ecd06f458b03cf1333929eca62f7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 12 Apr 2022 13:23:36 +0000 Subject: [PATCH 286/627] wip MergeJoinTransform --- .../Transforms/MergeJoinTransform.cpp | 547 +++++++++++------- .../Transforms/MergeJoinTransform.h | 182 ++++-- 2 files changed, 458 insertions(+), 271 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 0f9c545793d..e6f3c46e1c6 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -1,5 +1,7 @@ +#include #include #include +#include #include #include #include @@ -15,6 +17,9 @@ #include #include #include +#include +#include "Common/Exception.h" +#include "Core/SettingsEnums.h" namespace DB @@ -28,9 +33,24 @@ namespace ErrorCodes } constexpr size_t EMPTY_VALUE_IDX = std::numeric_limits::max(); +using JoinKind = ASTTableJoin::Kind; + + +namespace +{ + +FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns) +{ + SortDescription desc; + desc.reserve(columns.size()); + for (const auto & name : columns) + desc.emplace_back(name); + return std::make_unique(block, desc); +} + template -static int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos, int null_direction_hint = 1) +int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos, int null_direction_hint = 1) { if constexpr (has_left_nulls && has_right_nulls) { @@ -74,129 +94,20 @@ static int nullableCompareAt(const IColumn & left_column, const IColumn & right_ return left_column.compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); } -FullMergeJoinCursor::FullMergeJoinCursor(const Block & block, const SortDescription & desc_) - : impl(block, desc_) - , sample_block(block) +int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, size_t lpos, + const SortCursorImpl & rhs, size_t rpos) { -} - -bool ALWAYS_INLINE FullMergeJoinCursor::sameNext() const -{ - if (!impl.isValid() || impl.isLast()) - return false; - - for (size_t i = 0; i < impl.sort_columns_size; ++i) + for (size_t i = 0; i < lhs.sort_columns_size; ++i) { - const auto & col = *impl.sort_columns[i]; - int cmp = nullableCompareAt( - col, col, impl.getRow(), impl.getRow() + 1, 0); - if (cmp != 0) - return false; - } - return true; -} - -bool FullMergeJoinCursor::sameUnitlEnd() const -{ - if (!impl.isValid() || impl.isLast()) - return true; - - for (size_t i = 0; i < impl.sort_columns_size; ++i) - { - const auto & col = *impl.sort_columns[i]; - int cmp = nullableCompareAt( - col, col, impl.getRow(), impl.rows - 1, 0); - if (cmp != 0) - return false; - } - - return true; -} - -size_t FullMergeJoinCursor::nextDistinct() -{ - if (sameUnitlEnd()) - return 0; - - size_t start_pos = impl.getRow(); - while (sameNext()) - { - impl.next(); - } - impl.next(); - return impl.getRow() - start_pos; -} - -void FullMergeJoinCursor::reset() -{ - current_input = {}; - resetInternalCursor(); -} - -const Chunk & FullMergeJoinCursor::getCurrentChunk() const -{ - return current_input.chunk; -} - -void FullMergeJoinCursor::setInput(IMergingAlgorithm::Input && input) -{ - if (input.skip_last_row) - throw Exception("FullMergeJoinCursor does not support skipLastRow", ErrorCodes::NOT_IMPLEMENTED); - - if (current_input.permutation) - throw DB::Exception("FullMergeJoinCursor: permutation is not supported", ErrorCodes::NOT_IMPLEMENTED); - - - current_input = std::move(input); - - if (!current_input.chunk) - fully_completed = true; - - resetInternalCursor(); -} - -void FullMergeJoinCursor::resetInternalCursor() -{ - if (current_input.chunk) - { - impl.reset(current_input.chunk.getColumns(), sample_block, current_input.permutation); - } - else - { - impl.reset(sample_block.cloneEmpty().getColumns(), sample_block); - } -} - -namespace -{ - -FullMergeJoinCursor createCursor(const Block & block, const Names & columns) -{ - SortDescription desc; - desc.reserve(columns.size()); - for (const auto & name : columns) - desc.emplace_back(name); - return FullMergeJoinCursor(block, desc); -} - -/// If on_pos == true, compare two columns at specified positions. -/// Otherwise, compare two columns at the current positions, `lpos` and `rpos` are ignored. -template -int ALWAYS_INLINE compareCursors(const Cursor & lhs, const Cursor & rhs, - [[ maybe_unused ]] size_t lpos = 0, - [[ maybe_unused ]] size_t rpos = 0) -{ - for (size_t i = 0; i < lhs->sort_columns_size; ++i) - { - const auto & desc = lhs->desc[i]; + const auto & desc = lhs.desc[i]; int direction = desc.direction; int nulls_direction = desc.nulls_direction; int cmp = direction * nullableCompareAt( - *lhs->sort_columns[i], - *rhs->sort_columns[i], - on_pos ? lpos : lhs->getRow(), - on_pos ? rpos : rhs->getRow(), + *lhs.sort_columns[i], + *rhs.sort_columns[i], + lpos, + rpos, nulls_direction); if (cmp != 0) return cmp; @@ -204,16 +115,21 @@ int ALWAYS_INLINE compareCursors(const Cursor & lhs, const Cursor & rhs, return 0; } +int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImpl & rhs) +{ + return compareCursors(lhs, lhs.getRow(), rhs, rhs.getRow()); +} + + bool ALWAYS_INLINE totallyLess(const FullMergeJoinCursor & lhs, const FullMergeJoinCursor & rhs) { - if (lhs->rows == 0 || rhs->rows == 0) - return false; - - if (!lhs->isValid() || !rhs->isValid()) + if (!lhs.isValid() || !rhs.isValid()) return false; + const auto & lhs_impl = lhs.getCurrent().impl; + const auto & rhs_impl = rhs.getCurrent().impl; /// The last row of this cursor is no larger than the first row of the another cursor. - int cmp = compareCursors(lhs, rhs, lhs->rows - 1, 0); + int cmp = compareCursors(lhs_impl, lhs_impl.rows - 1, rhs_impl, 0); return cmp < 0; } @@ -248,6 +164,174 @@ void addIndexColumn(const Columns & columns, ColumnUInt64 & indices, Chunk & res } } +bool sameNext(const SortCursorImpl & impl) +{ + for (size_t i = 0; i < impl.sort_columns_size; ++i) + { + const auto & col = *impl.sort_columns[i]; + int cmp = nullableCompareAt( + col, col, impl.getRow(), impl.getRow() + 1, 0); + if (cmp != 0) + return false; + } + return true; +} + +size_t nextDistinct(SortCursorImpl & impl) +{ + size_t start_pos = impl.getRow(); + while (sameNext(impl)) + { + impl.next(); + } + impl.next(); + return impl.getRow() - start_pos; +} + +Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs, size_t start, size_t num_rows) +{ + Chunk result; + copyColumnsResized(lhs, start, num_rows, result); + copyColumnsResized(rhs, start, num_rows, result); + return result; +} + +Chunk createBlockWithDefaults(const Chunk & lhs, FullMergeJoinCursor & rhs) +{ + size_t start = rhs.getCurrent().impl.getRow(); + size_t rows_left = rhs.getCurrent().impl.rowsLeft(); + return createBlockWithDefaults(lhs, rhs.detachCurrentChunk(), start, rows_left); +} + +Chunk createBlockWithDefaults(FullMergeJoinCursor & lhs, const Chunk & rhs) +{ + size_t start = lhs.getCurrent().impl.getRow(); + size_t rows_left = lhs.getCurrent().impl.rowsLeft(); + return createBlockWithDefaults(lhs.detachCurrentChunk(), rhs, start, rows_left); +} + +} + +void FullMergeJoinCursor::next() +{ + if (current == inputs.end()) + return; + + if (current->impl.isValid()) + { + current->impl.next(); + return; + } + current++; + + if (current == inputs.end()) + return; + + assert(current->impl.isValid()); +} + +/// The current row of is not equal to the last avaliable row +/// Also returns false if the current row is not valid and we stream wasn't finished. +bool FullMergeJoinCursor::haveAllCurrentRange() const +{ + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} haveAllCurrentRange: " + "size {}, " + "end: {} [{}/{}]" + " isValid: {} isLast {} " + "recieved_all_blocks {}", __FILE__, __LINE__, + inputs.size(), + current == inputs.end(), + current != inputs.end() ? current->impl.getRow() : 0, + current != inputs.end() ? current->impl.rows : 0, + current != inputs.end() ? current->impl.isValid() : false, + current != inputs.end() ? current->impl.isLast() : true, + recieved_all_blocks); + + if (recieved_all_blocks) + return true; + + if (isLast()) + return false; + + assert(current->impl.isValid() && inputs.back().impl.isValid()); + + return compareCursors( + current->impl, current->impl.getRow(), + inputs.back().impl, inputs.back().impl.rows - 1) != 0; +} + +size_t FullMergeJoinCursor::nextDistinct() +{ + while (current != inputs.end() && !current->impl.isValid()) + current++; + + if (recieved_all_blocks) + { + if (!isValid()) + return 0; + return DB::nextDistinct(current->impl); + } + + if (!haveAllCurrentRange()) + return 0; + + size_t skipped_rows_in_blocks = 0; + + while (compareCursors(current->impl, current->impl.getRow(), + current->impl, current->impl.rows - 1) == 0) + { + skipped_rows_in_blocks += current->input.getNumRows(); + current++; + } + + assert(isValid()); + + size_t skipped_rows = DB::nextDistinct(current->impl); + return skipped_rows + skipped_rows_in_blocks; +} + +Chunk FullMergeJoinCursor::detachCurrentChunk() +{ + if (!isValid()) + throw DB::Exception("Cursor is not valid", ErrorCodes::LOGICAL_ERROR); + Chunk res = std::move(current->input); + current++; + dropBlocksUntilCurrent(); + return res; +} + +const ColumnRawPtrs & FullMergeJoinCursor::getSortColumns() const +{ + return current->impl.sort_columns; +} + + +void FullMergeJoinCursor::dropBlocksUntilCurrent() +{ + inputs.erase(inputs.begin(), current); +} + +bool FullMergeJoinCursor::isValid() const +{ + return current != inputs.end() && + (current != std::prev(inputs.end()) || current->impl.isValid()); +} + +bool FullMergeJoinCursor::isLast() const +{ + return current == inputs.end() || + (current == std::prev(inputs.end()) && current->impl.isLast()); +} + +bool FullMergeJoinCursor::fullyCompleted() const +{ + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} end: {} currentIsValid: {} allIsValid {}, result {}", __FILE__, __LINE__, + current == inputs.end(), + current != inputs.end() ? current->impl.isValid() : false, + isValid(), + !isValid() && recieved_all_blocks + ); + return !isValid() && recieved_all_blocks; } MergeJoinAlgorithm::MergeJoinAlgorithm( @@ -259,8 +343,13 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( if (input_headers.size() != 2) throw Exception("MergeJoinAlgorithm requires exactly two inputs", ErrorCodes::LOGICAL_ERROR); - if (table_join->getTableJoin().strictness() != ASTTableJoin::Strictness::Any) - throw Exception("MergeJoinAlgorithm is not implemented for strictness != ANY", ErrorCodes::NOT_IMPLEMENTED); + auto strictness = table_join->getTableJoin().strictness(); + if (strictness != ASTTableJoin::Strictness::Any) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for strictness != ANY"); + + auto kind = table_join->getTableJoin().kind(); + if (!isInner(kind) && !isLeft(kind) && !isRight(kind) && !isFull(kind)) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for kind {}", kind); const auto & join_on = table_join->getTableJoin().getOnlyClause(); @@ -300,6 +389,9 @@ void MergeJoinAlgorithm::initialize(Inputs inputs) static void prepareChunk(Chunk & chunk) { + if (!chunk) + return; + auto num_rows = chunk.getNumRows(); auto columns = chunk.detachColumns(); for (auto & column : columns) @@ -310,29 +402,36 @@ static void prepareChunk(Chunk & chunk) void MergeJoinAlgorithm::consume(Input & input, size_t source_num) { - LOG_DEBUG(log, "TODO: remove. Consume from {} chunk: {}", source_num, bool(input.chunk)); + if (input.skip_last_row) + throw Exception("skip_last_row is not supported", ErrorCodes::NOT_IMPLEMENTED); + + if (input.permutation) + throw DB::Exception("permutation is not supported", ErrorCodes::NOT_IMPLEMENTED); + + LOG_DEBUG(log, "TODO: remove. XXXX Consume from {} chunk: {}", source_num, bool(input.chunk)); - prepareChunk(input.chunk); if (input.chunk.getNumRows() >= EMPTY_VALUE_IDX) throw Exception("Too many rows in input", ErrorCodes::TOO_MANY_ROWS); if (input.chunk) + { stat.num_blocks[source_num] += 1; + stat.num_rows[source_num] += input.chunk.getNumRows(); + } - cursors[source_num].setInput(std::move(input)); + prepareChunk(input.chunk); + cursors[source_num]->addChunk(std::move(input.chunk)); } -using JoinKind = ASTTableJoin::Kind; - template -static std::optional anyJoin(FullMergeJoinCursor & left_cursor, FullMergeJoinCursor & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) +static std::optional anyJoinImpl(SortCursorImpl & left_cursor, SortCursorImpl & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) { static_assert(kind == JoinKind::Left || kind == JoinKind::Right || kind == JoinKind::Inner, "Invalid join kind"); - size_t num_rows = kind == JoinKind::Left ? left_cursor->rowsLeft() : - kind == JoinKind::Right ? right_cursor->rowsLeft() : - std::min(left_cursor->rowsLeft(), right_cursor->rowsLeft()); + size_t num_rows = kind == JoinKind::Left ? left_cursor.rowsLeft() : + kind == JoinKind::Right ? right_cursor.rowsLeft() : + std::min(left_cursor.rowsLeft(), right_cursor.rowsLeft()); constexpr bool is_left_or_inner = kind == JoinKind::Left || kind == JoinKind::Inner; constexpr bool is_right_or_inner = kind == JoinKind::Right || kind == JoinKind::Inner; @@ -343,166 +442,190 @@ static std::optional anyJoin(FullMergeJoinCursor & left_cursor, FullMerg if constexpr (is_right_or_inner) left_map.reserve(num_rows); - while (left_cursor->isValid() && right_cursor->isValid()) + while (left_cursor.isValid() && right_cursor.isValid()) { int cmp = compareCursors(left_cursor, right_cursor); if (cmp == 0) { if constexpr (is_left_or_inner) - right_map.emplace_back(right_cursor->getRow()); + right_map.emplace_back(right_cursor.getRow()); if constexpr (is_right_or_inner) - left_map.emplace_back(left_cursor->getRow()); + left_map.emplace_back(left_cursor.getRow()); if constexpr (is_left_or_inner) - left_cursor->next(); + left_cursor.next(); if constexpr (is_right_or_inner) - right_cursor->next(); + right_cursor.next(); } else if (cmp < 0) { - size_t num = left_cursor.nextDistinct(); + size_t num = nextDistinct(left_cursor); if (num == 0) return 0; if constexpr (kind == JoinKind::Left) - right_map.resize_fill(right_map.size() + num, right_cursor->rows); + right_map.resize_fill(right_map.size() + num, right_cursor.rows); } else { - size_t num = right_cursor.nextDistinct(); + size_t num = nextDistinct(right_cursor); if (num == 0) return 1; if constexpr (kind == JoinKind::Right) - left_map.resize_fill(left_map.size() + num, left_cursor->rows); + left_map.resize_fill(left_map.size() + num, left_cursor.rows); } } + return std::nullopt; } -static Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs, size_t start, size_t num_rows) +static std::optional anyJoinDispatch(const std::vector & cursors, JoinKind kind, PaddedPODArray & left_map, PaddedPODArray & right_map) { + auto & left_cursor = cursors[0]->getCurrentMutable(); + auto & right_cursor = cursors[1]->getCurrentMutable(); + if (isInner(kind)) + { + return anyJoinImpl(left_cursor, right_cursor, left_map, right_map); + } + else if (isLeft(kind)) + { + return anyJoinImpl(left_cursor, right_cursor, left_map, right_map); + } + else if (isRight(kind)) + { + return anyJoinImpl(left_cursor, right_cursor, left_map, right_map); + } + else + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); + } + __builtin_unreachable(); +} + +static bool isFinished(const std::vector & cursors, JoinKind kind) +{ + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} - {} {}", __FILE__, __LINE__, + kind, + cursors[0]->fullyCompleted(), cursors[1]->fullyCompleted() + ); + return (cursors[0]->fullyCompleted() && cursors[1]->fullyCompleted()) + || ((isLeft(kind) || isInner(kind)) && cursors[0]->fullyCompleted()) + || ((isRight(kind) || isInner(kind)) && cursors[1]->fullyCompleted()); +} + +Chunk MergeJoinAlgorithm::anyJoin(JoinKind kind) +{ + auto left_map = ColumnUInt64::create(); + auto right_map = ColumnUInt64::create(); + size_t prev_pos[] = {cursors[0]->getCurrent().impl.getRow(), cursors[1]->getCurrent().impl.getRow()}; + + required_input = anyJoinDispatch(cursors, kind, left_map->getData(), right_map->getData()); + + assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); + + { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} [{}] [{}]", __FILE__, __LINE__, + fmt::join(left_map->getData(), ", "), + fmt::join(right_map->getData(), ", ") + ); + } Chunk result; - copyColumnsResized(lhs, start, num_rows, result); - copyColumnsResized(rhs, start, num_rows, result); + size_t num_result_rows = std::max(left_map->size(), right_map->size()); + addIndexColumn(cursors[0]->getCurrent().input.getColumns(), *left_map, result, prev_pos[0], num_result_rows); + addIndexColumn(cursors[1]->getCurrent().input.getColumns(), *right_map, result, prev_pos[1], num_result_rows); + + if (required_input != 0) + cursors[0]->dropBlocksUntilCurrent(); + + if (required_input != 1) + cursors[1]->dropBlocksUntilCurrent(); + return result; } -static Chunk createBlockWithDefaults(const Chunk & lhs, FullMergeJoinCursor & rhs) -{ - auto res = createBlockWithDefaults(lhs, rhs.getCurrentChunk(), rhs->getRow(), rhs->rowsLeft()); - rhs.reset(); - return res; -} - -static Chunk createBlockWithDefaults(FullMergeJoinCursor & lhs, const Chunk & rhs) -{ - auto res = createBlockWithDefaults(lhs.getCurrentChunk(), rhs, lhs->getRow(), lhs->rowsLeft()); - lhs.reset(); - return res; -} - -static bool isFinished(const std::vector & cursors, JoinKind kind) -{ - return (cursors[0].fullyCompleted() && cursors[1].fullyCompleted()) - || ((isLeft(kind) || isInner(kind)) && cursors[0].fullyCompleted()) - || ((isRight(kind) || isInner(kind)) && cursors[1].fullyCompleted()); -} - IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { + LOG_DEBUG(log, "TODO: remove. XXXX Merge"); + if (required_input.has_value()) { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); size_t r = required_input.value(); required_input = {}; return Status(r); } - if (!cursors[0]->isValid() && !cursors[0].fullyCompleted()) + if (!cursors[0]->haveAllCurrentRange() && !cursors[0]->fullyCompleted()) { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); return Status(0); } - if (!cursors[1]->isValid() && !cursors[1].fullyCompleted()) + if (!cursors[1]->haveAllCurrentRange() && !cursors[1]->fullyCompleted()) { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); return Status(1); } - JoinKind kind = table_join->getTableJoin().kind(); + auto kind = table_join->getTableJoin().kind(); + auto strictness = table_join->getTableJoin().strictness(); if (isFinished(cursors, kind)) { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); return Status({}, true); } - if (cursors[0].fullyCompleted() && isRightOrFull(kind)) + if (cursors[0]->fullyCompleted() && isRightOrFull(kind)) { - Chunk result = createBlockWithDefaults(sample_chunks[0], cursors[1]); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); + + Chunk result = createBlockWithDefaults(sample_chunks[0], *cursors[1]); return Status(std::move(result)); } - if (isLeftOrFull(kind) && cursors[1].fullyCompleted()) + if (isLeftOrFull(kind) && cursors[1]->fullyCompleted()) { - Chunk result = createBlockWithDefaults(cursors[0], sample_chunks[1]); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); + + Chunk result = createBlockWithDefaults(*cursors[0], sample_chunks[1]); return Status(std::move(result)); } - if (int cmp = totallyCompare(cursors[0], cursors[1]); cmp != 0) + assert(!cursors[0]->fullyCompleted() && cursors[0]->isValid() && + !cursors[1]->fullyCompleted() && cursors[1]->isValid()); + + if (int cmp = totallyCompare(*cursors[0], *cursors[1]); cmp != 0) { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); if (cmp < 0) { - if (cursors[0]->isValid() && isLeftOrFull(kind)) - { - return Status(createBlockWithDefaults(cursors[0], sample_chunks[1])); - } - cursors[0].reset(); + if (isLeftOrFull(kind)) + return Status(createBlockWithDefaults(*cursors[0], sample_chunks[1])); + cursors[0]->detachCurrentChunk(); return Status(0); } if (cmp > 0) { - if (isRightOrFull(kind) && cursors[1]->isValid()) - { - return Status(createBlockWithDefaults(sample_chunks[0], cursors[1])); - } - cursors[1].reset(); + if (isRightOrFull(kind)) + return Status(createBlockWithDefaults(sample_chunks[0], *cursors[1])); + cursors[1]->detachCurrentChunk(); return Status(1); } - - if (!isInner(kind) && !isLeft(kind) && !isRight(kind) && !isFull(kind)) - throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for kind {}", kind); } - auto left_map = ColumnUInt64::create(); - auto right_map = ColumnUInt64::create(); - std::pair prev_pos = std::make_pair(cursors[0]->getRow(), cursors[1]->getRow()); - if (isInner(kind)) + if (strictness == ASTTableJoin::Strictness::Any) { - required_input = anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); - } - else if (isLeft(kind)) - { - required_input = anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); - } - else if (isRight(kind)) - { - required_input = anyJoin(cursors[0], cursors[1], left_map->getData(), right_map->getData()); - } - else - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", table_join->getTableJoin().kind()); + Chunk result = anyJoin(kind); + return Status(std::move(result), isFinished(cursors, kind)); } - assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); - - Chunk result; - size_t num_result_rows = std::max(left_map->size(), right_map->size()); - addIndexColumn(cursors[0].getCurrentChunk().getColumns(), *left_map, result, prev_pos.first, num_result_rows); - addIndexColumn(cursors[1].getCurrentChunk().getColumns(), *right_map, result, prev_pos.second, num_result_rows); - return Status(std::move(result), isFinished(cursors, kind)); + throw Exception("Unsupported strictness: " + toString(strictness), ErrorCodes::NOT_IMPLEMENTED); } MergeJoinTransform::MergeJoinTransform( diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 32b47dc0b6a..4bc43ac6386 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -1,10 +1,14 @@ #pragma once +#include #include #include +#include #include #include #include +#include "Columns/IColumn.h" +#include "Core/Field.h" #include "Interpreters/IJoin.h" #include #include @@ -12,6 +16,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -26,84 +31,142 @@ namespace ErrorCodes class IJoin; using JoinPtr = std::shared_ptr; +class FullMergeJoinCursor; -class MultiCursor +using FullMergeJoinCursorPtr = std::unique_ptr; + +/* + * Wrapper for SortCursorImpl + * It is used to keep cursor for list of blocks. + */ +class FullMergeJoinCursor : boost::noncopyable { - void next() - { - } - - bool isLast() - { - return false; - } - - bool isValid() - { - return false; - } - -private: - SortDescription desc; +public: struct CursorWithBlock { - SortCursorImpl impl; + CursorWithBlock(const Block & header, const SortDescription & desc_, Chunk && chunk) + : input(std::move(chunk)) + , impl(header, input.getColumns(), desc_) + { + } + Chunk input; + SortCursorImpl impl; }; + /* + /// Used in any join, instead of storing previous block + struct Row + { + std::vector sort_columns; + + explicit Row(const SortCursorImpl & impl_) + { + assert(impl_.isValid()); + + sort_columns.reserve(impl_.sort_columns.size()); + for (const auto & col : impl_.sort_columns) + { + auto new_col = col->cloneEmpty(); + new_col->insertFrom(*col, impl_.getRow()); + sort_columns.push_back(std::move(new_col)); + } + } + + }; + */ + using CursorList = std::list; using CursorListIt = CursorList::iterator; - CursorListIt current; - CursorList inputs; -}; + explicit FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_) + : sample_block(sample_block_) + , desc(description_) + , current(inputs.end()) + { + } -/* - * Wrapper for SortCursorImpl - * It is used to store information about the current state of the cursor. - */ -class FullMergeJoinCursor -{ -public: - - FullMergeJoinCursor(const Block & block, const SortDescription & desc); - - SortCursor getCursor(); - - bool sameUnitlEnd() const; - - bool ALWAYS_INLINE sameNext() const; + void next(); size_t nextDistinct(); - void reset(); + bool haveAllCurrentRange() const; + bool isValid() const; + bool isLast() const; + bool fullyCompleted() const; - const Chunk & getCurrentChunk() const; + void addChunk(Chunk && chunk) + { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} addChunk {} {} {} {}", __FILE__, __LINE__, + bool(chunk), + chunk.hasRows(), chunk.getNumRows(), + chunk.hasColumns()); - void setInput(IMergingAlgorithm::Input && input); + assert(!recieved_all_blocks); + if (!chunk) + { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} recieved_all_blocks = true", __FILE__, __LINE__); + recieved_all_blocks = true; + return; + } - bool fullyCompleted() const { return !impl.isValid() && fully_completed; } + dropBlocksUntilCurrent(); + inputs.emplace_back(sample_block, desc, std::move(chunk)); - SortCursorImpl * operator-> () { return &impl; } - const SortCursorImpl * operator-> () const { return &impl; } + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} end {} prev {} size {}; [{}/{};{}]", __FILE__, __LINE__, + current == inputs.end(), + current == std::prev(inputs.end()), + inputs.size(), + inputs.back().impl.getRow(), + inputs.back().impl.rows, + inputs.back().input.getNumRows()); + + if (current == inputs.end()) + { + + current = std::prev(inputs.end()); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} end {} prev {} size {}", __FILE__, __LINE__, + current == inputs.end(), + current == std::prev(inputs.end()), + inputs.size()); + + } + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} end {} prev {} size {}", __FILE__, __LINE__, + current == inputs.end(), + current == std::prev(inputs.end()), + inputs.size()); + + } + + Chunk detachCurrentChunk(); + + const CursorWithBlock & getCurrent() const + { + assert(current != inputs.end()); + return *current; + } + + SortCursorImpl & getCurrentMutable() + { + assert(isValid()); + return current->impl; + } + + const ColumnRawPtrs & getSortColumns() const; + void dropBlocksUntilCurrent(); private: - void resetInternalCursor(); - - - - SortCursorImpl impl; - - IMergingAlgorithm::Input current_input; - - - bool fully_completed = false; Block sample_block; - // bool has_left_nullable = false; - // bool has_right_nullable = false; + SortDescription desc; + + CursorList inputs; + CursorListIt current; + + bool recieved_all_blocks = false; }; + /* * This class is used to join chunks from two sorted streams. * It is used in MergeJoinTransform. @@ -119,16 +182,16 @@ public: void onFinish(double seconds) { - LOG_TRACE(log, "Finished pocessing {} left and {} right blocks in {} seconds", - stat.num_blocks[0], - stat.num_blocks[1], - seconds); + LOG_TRACE(log, "Finished pocessing in {} seconds - left: {} blocks, {} rows; right: {} blocks, {} rows", + seconds, stat.num_blocks[0], stat.num_rows[0], stat.num_blocks[1], stat.num_rows[1]); } private: + Chunk anyJoin(ASTTableJoin::Kind kind); + std::optional required_input = std::nullopt; - std::vector cursors; + std::vector cursors; std::vector sample_chunks; JoinPtr table_join; @@ -136,6 +199,7 @@ private: struct Statistic { size_t num_blocks[2] = {0, 0}; + size_t num_rows[2] = {0, 0}; }; Statistic stat; From 26d812ec7243a1d361fb0a7b195715585f7d24ef Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 14 Apr 2022 12:30:34 +0000 Subject: [PATCH 287/627] wip any full sorting merge, rewrite cursor --- src/Parsers/ASTTablesInSelectQuery.h | 19 +- .../Transforms/MergeJoinTransform.cpp | 426 ++++++------- .../Transforms/MergeJoinTransform.h | 175 +++--- .../02249_full_sort_join.reference | 286 --------- .../02250_full_sort_join_long.reference | 23 - .../02269_full_sort_join.reference | 576 ++++++++++++++++++ ...oin.sql.j2 => 02269_full_sort_join.sql.j2} | 23 +- .../02270_full_sort_join_nodistinct.reference | 260 ++++++++ .../02270_full_sort_join_nodistinct.sql.j2 | 41 ++ .../02271_full_sort_join_long.reference | 6 + ...ql.j2 => 02271_full_sort_join_long.sql.j2} | 7 +- 11 files changed, 1182 insertions(+), 660 deletions(-) delete mode 100644 tests/queries/0_stateless/02249_full_sort_join.reference create mode 100644 tests/queries/0_stateless/02269_full_sort_join.reference rename tests/queries/0_stateless/{02249_full_sort_join.sql.j2 => 02269_full_sort_join.sql.j2} (77%) create mode 100644 tests/queries/0_stateless/02270_full_sort_join_nodistinct.reference create mode 100644 tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 create mode 100644 tests/queries/0_stateless/02271_full_sort_join_long.reference rename tests/queries/0_stateless/{02250_full_sort_join_long.sql.j2 => 02271_full_sort_join_long.sql.j2} (82%) diff --git a/src/Parsers/ASTTablesInSelectQuery.h b/src/Parsers/ASTTablesInSelectQuery.h index 2b07d31fb7d..9eddaf3137d 100644 --- a/src/Parsers/ASTTablesInSelectQuery.h +++ b/src/Parsers/ASTTablesInSelectQuery.h @@ -113,16 +113,15 @@ struct ASTTableJoin : public IAST void updateTreeHashImpl(SipHash & hash_state) const override; }; -inline bool isLeft(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Left; } -inline bool isRight(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Right; } -inline bool isInner(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Inner; } -inline bool isFull(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Full; } -inline bool isCrossOrComma(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Comma || kind == ASTTableJoin::Kind::Cross; } -inline bool isRightOrFull(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Right || kind == ASTTableJoin::Kind::Full; } -inline bool isLeftOrFull(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Left || kind == ASTTableJoin::Kind::Full; } -inline bool isInnerOrRight(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Right; } -inline bool isInnerOrLeft(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Left; } - +inline constexpr bool isLeft(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Left; } +inline constexpr bool isRight(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Right; } +inline constexpr bool isInner(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Inner; } +inline constexpr bool isFull(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Full; } +inline constexpr bool isCrossOrComma(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Comma || kind == ASTTableJoin::Kind::Cross; } +inline constexpr bool isRightOrFull(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Right || kind == ASTTableJoin::Kind::Full; } +inline constexpr bool isLeftOrFull(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Left || kind == ASTTableJoin::Kind::Full; } +inline constexpr bool isInnerOrRight(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Right; } +inline constexpr bool isInnerOrLeft(ASTTableJoin::Kind kind) { return kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Left; } /// Specification of ARRAY JOIN. struct ASTArrayJoin : public IAST diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index e6f3c46e1c6..cd53d919e36 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -121,19 +122,17 @@ int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImp } -bool ALWAYS_INLINE totallyLess(const FullMergeJoinCursor & lhs, const FullMergeJoinCursor & rhs) +bool ALWAYS_INLINE totallyLess(SortCursorImpl & lhs, SortCursorImpl & rhs) { if (!lhs.isValid() || !rhs.isValid()) return false; - const auto & lhs_impl = lhs.getCurrent().impl; - const auto & rhs_impl = rhs.getCurrent().impl; /// The last row of this cursor is no larger than the first row of the another cursor. - int cmp = compareCursors(lhs_impl, lhs_impl.rows - 1, rhs_impl, 0); + int cmp = compareCursors(lhs, lhs.rows - 1, rhs, 0); return cmp < 0; } -int ALWAYS_INLINE totallyCompare(const FullMergeJoinCursor & lhs, const FullMergeJoinCursor & rhs) +int ALWAYS_INLINE totallyCompare(SortCursorImpl & lhs, SortCursorImpl & rhs) { if (totallyLess(lhs, rhs)) return -1; @@ -180,158 +179,74 @@ bool sameNext(const SortCursorImpl & impl) size_t nextDistinct(SortCursorImpl & impl) { size_t start_pos = impl.getRow(); - while (sameNext(impl)) + while (!impl.isLast() && sameNext(impl)) { impl.next(); } impl.next(); - return impl.getRow() - start_pos; + + if (impl.isValid()) + return impl.getRow() - start_pos; + return impl.rows - start_pos; } -Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs, size_t start, size_t num_rows) +} + +static void logAll(FullMergeJoinCursor::CursorList & inputs, FullMergeJoinCursor::CursorListIt & current, const String & msg) { - Chunk result; - copyColumnsResized(lhs, start, num_rows, result); - copyColumnsResized(rhs, start, num_rows, result); - return result; -} - -Chunk createBlockWithDefaults(const Chunk & lhs, FullMergeJoinCursor & rhs) -{ - size_t start = rhs.getCurrent().impl.getRow(); - size_t rows_left = rhs.getCurrent().impl.rowsLeft(); - return createBlockWithDefaults(lhs, rhs.detachCurrentChunk(), start, rows_left); -} - -Chunk createBlockWithDefaults(FullMergeJoinCursor & lhs, const Chunk & rhs) -{ - size_t start = lhs.getCurrent().impl.getRow(); - size_t rows_left = lhs.getCurrent().impl.rowsLeft(); - return createBlockWithDefaults(lhs.detachCurrentChunk(), rhs, start, rows_left); -} - -} - -void FullMergeJoinCursor::next() -{ - if (current == inputs.end()) - return; - - if (current->impl.isValid()) + std::vector text; + for (auto it = inputs.begin(); it != inputs.end(); ++it) { - current->impl.next(); - return; + text.push_back(fmt::format("<{} - {}/{}>{}", it->cursor.isValid(), it->cursor.getRow(), it->cursor.rows, + it == current ? "*" : "")); } - current++; - - if (current == inputs.end()) - return; - - assert(current->impl.isValid()); + text.push_back(fmt::format("{}", inputs.end() == current ? "*" : "")); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} [{}]", __FILE__, __LINE__, msg, fmt::join(text, " ")); } -/// The current row of is not equal to the last avaliable row -/// Also returns false if the current row is not valid and we stream wasn't finished. -bool FullMergeJoinCursor::haveAllCurrentRange() const +FullMergeJoinCursor::CursorWithBlock & FullMergeJoinCursor::getCurrent() { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} haveAllCurrentRange: " - "size {}, " - "end: {} [{}/{}]" - " isValid: {} isLast {} " - "recieved_all_blocks {}", __FILE__, __LINE__, - inputs.size(), - current == inputs.end(), - current != inputs.end() ? current->impl.getRow() : 0, - current != inputs.end() ? current->impl.rows : 0, - current != inputs.end() ? current->impl.isValid() : false, - current != inputs.end() ? current->impl.isLast() : true, - recieved_all_blocks); - - if (recieved_all_blocks) - return true; - - if (isLast()) - return false; - - assert(current->impl.isValid() && inputs.back().impl.isValid()); - - return compareCursors( - current->impl, current->impl.getRow(), - inputs.back().impl, inputs.back().impl.rows - 1) != 0; -} - -size_t FullMergeJoinCursor::nextDistinct() -{ - while (current != inputs.end() && !current->impl.isValid()) + while (current != inputs.end() && !current->cursor.isValid()) current++; - if (recieved_all_blocks) - { - if (!isValid()) - return 0; - return DB::nextDistinct(current->impl); - } - - if (!haveAllCurrentRange()) - return 0; - - size_t skipped_rows_in_blocks = 0; - - while (compareCursors(current->impl, current->impl.getRow(), - current->impl, current->impl.rows - 1) == 0) - { - skipped_rows_in_blocks += current->input.getNumRows(); - current++; - } - - assert(isValid()); - - size_t skipped_rows = DB::nextDistinct(current->impl); - return skipped_rows + skipped_rows_in_blocks; + return current != inputs.end() ? *current : empty_cursor; } -Chunk FullMergeJoinCursor::detachCurrentChunk() +void FullMergeJoinCursor::addChunk(Chunk && chunk) { - if (!isValid()) - throw DB::Exception("Cursor is not valid", ErrorCodes::LOGICAL_ERROR); - Chunk res = std::move(current->input); - current++; + assert(!recieved_all_blocks); + if (!chunk) + { + recieved_all_blocks = true; + return; + } + + logAll(inputs, current, "addChunk>>>"); + dropBlocksUntilCurrent(); - return res; -} + logAll(inputs, current, "addChunk~~~"); -const ColumnRawPtrs & FullMergeJoinCursor::getSortColumns() const -{ - return current->impl.sort_columns; -} + inputs.emplace_back(sample_block, desc, std::move(chunk)); + if (current == inputs.end()) + { + current = std::prev(inputs.end()); + } + logAll(inputs, current, "addChunk<<<"); + +} void FullMergeJoinCursor::dropBlocksUntilCurrent() { + while (current != inputs.end() && !current->cursor.isValid()) + current++; + inputs.erase(inputs.begin(), current); } -bool FullMergeJoinCursor::isValid() const +bool FullMergeJoinCursor::fullyCompleted() { - return current != inputs.end() && - (current != std::prev(inputs.end()) || current->impl.isValid()); -} - -bool FullMergeJoinCursor::isLast() const -{ - return current == inputs.end() || - (current == std::prev(inputs.end()) && current->impl.isLast()); -} - -bool FullMergeJoinCursor::fullyCompleted() const -{ - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} end: {} currentIsValid: {} allIsValid {}, result {}", __FILE__, __LINE__, - current == inputs.end(), - current != inputs.end() ? current->impl.isValid() : false, - isValid(), - !isValid() && recieved_all_blocks - ); - return !isValid() && recieved_all_blocks; + return !getCurrent()->isValid() && recieved_all_blocks; } MergeJoinAlgorithm::MergeJoinAlgorithm( @@ -373,6 +288,30 @@ static void copyColumnsResized(const Chunk & chunk, size_t start, size_t size, C } } +static Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs, size_t start, size_t num_rows) +{ + Chunk result; + copyColumnsResized(lhs, start, num_rows, result); + copyColumnsResized(rhs, start, num_rows, result); + return result; +} + +static Chunk createBlockWithDefaults(const Chunk & lhs, FullMergeJoinCursor::CursorWithBlock & rhs) +{ + size_t start = rhs->getRow(); + size_t num_rows = rhs->rowsLeft(); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} [{}/{}]", __FILE__, __LINE__, rhs->getRow(), rhs->rowsLeft()); + return createBlockWithDefaults(lhs, rhs.detach(), start, num_rows); +} + +static Chunk createBlockWithDefaults(FullMergeJoinCursor::CursorWithBlock & lhs, const Chunk & rhs) +{ + size_t start = lhs->getRow(); + size_t num_rows = lhs->rowsLeft(); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} [{}/{}]", __FILE__, __LINE__, lhs->getRow(), lhs->rowsLeft()); + return createBlockWithDefaults(lhs.detach(), rhs, start, num_rows); +} + void MergeJoinAlgorithm::initialize(Inputs inputs) { if (inputs.size() != 2) @@ -425,68 +364,77 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) } template -static std::optional anyJoinImpl(SortCursorImpl & left_cursor, SortCursorImpl & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) +static std::unique_ptr anyJoinImpl(SortCursorImpl & left_cursor, SortCursorImpl & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) { static_assert(kind == JoinKind::Left || kind == JoinKind::Right || kind == JoinKind::Inner, "Invalid join kind"); - size_t num_rows = kind == JoinKind::Left ? left_cursor.rowsLeft() : - kind == JoinKind::Right ? right_cursor.rowsLeft() : + size_t num_rows = isLeft(kind) ? left_cursor.rowsLeft() : + isRight(kind) ? right_cursor.rowsLeft() : std::min(left_cursor.rowsLeft(), right_cursor.rowsLeft()); - constexpr bool is_left_or_inner = kind == JoinKind::Left || kind == JoinKind::Inner; - constexpr bool is_right_or_inner = kind == JoinKind::Right || kind == JoinKind::Inner; - - if constexpr (is_left_or_inner) + if constexpr (isLeft(kind) || isInner(kind)) right_map.reserve(num_rows); - if constexpr (is_right_or_inner) + if constexpr (isRight(kind) || isInner(kind)) left_map.reserve(num_rows); + size_t rpos = std::numeric_limits::max(); + size_t lpos = std::numeric_limits::max(); + assert(left_cursor.isValid() && right_cursor.isValid()); while (left_cursor.isValid() && right_cursor.isValid()) { - int cmp = compareCursors(left_cursor, right_cursor); - if (cmp == 0) + lpos = left_cursor.getRow(); + rpos = right_cursor.getRow(); + + if (int cmp = compareCursors(left_cursor, right_cursor); cmp == 0) { - if constexpr (is_left_or_inner) - right_map.emplace_back(right_cursor.getRow()); + if constexpr (isLeft(kind)) + { + size_t lnum = nextDistinct(left_cursor); + right_map.resize_fill(right_map.size() + lnum, rpos); + } - if constexpr (is_right_or_inner) - left_map.emplace_back(left_cursor.getRow()); - - if constexpr (is_left_or_inner) - left_cursor.next(); - - if constexpr (is_right_or_inner) - right_cursor.next(); + if constexpr (isRight(kind)) + { + size_t rnum = nextDistinct(right_cursor); + left_map.resize_fill(left_map.size() + rnum, lpos); + } + if constexpr (isInner(kind)) + { + nextDistinct(left_cursor); + nextDistinct(right_cursor); + left_map.emplace_back(lpos); + right_map.emplace_back(rpos); + } } else if (cmp < 0) { size_t num = nextDistinct(left_cursor); - if (num == 0) - return 0; - - if constexpr (kind == JoinKind::Left) + if constexpr (isLeft(kind)) right_map.resize_fill(right_map.size() + num, right_cursor.rows); } else { size_t num = nextDistinct(right_cursor); - if (num == 0) - return 1; - - if constexpr (kind == JoinKind::Right) + if constexpr (isRight(kind)) left_map.resize_fill(left_map.size() + num, left_cursor.rows); } } - return std::nullopt; + std::unique_ptr result = std::make_unique(); + + if (left_cursor.isValid()) + result->set(left_cursor, lpos, 0); + + if (right_cursor.isValid()) + result->set(right_cursor, rpos, 1); + + return result; } -static std::optional anyJoinDispatch(const std::vector & cursors, JoinKind kind, PaddedPODArray & left_map, PaddedPODArray & right_map) +static std::unique_ptr anyJoinDispatch(JoinKind kind, SortCursorImpl & left_cursor, SortCursorImpl & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) { - auto & left_cursor = cursors[0]->getCurrentMutable(); - auto & right_cursor = cursors[1]->getCurrentMutable(); if (isInner(kind)) { return anyJoinImpl(left_cursor, right_cursor, left_map, right_map); @@ -506,124 +454,144 @@ static std::optional anyJoinDispatch(const std::vector & cursors, JoinKind kind) +static Chunk handleAnyJoinState(std::unique_ptr & state, FullMergeJoinCursor & cursor, const Chunk & other_sample, JoinKind kind) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} - {} {}", __FILE__, __LINE__, - kind, - cursors[0]->fullyCompleted(), cursors[1]->fullyCompleted() - ); - return (cursors[0]->fullyCompleted() && cursors[1]->fullyCompleted()) - || ((isLeft(kind) || isInner(kind)) && cursors[0]->fullyCompleted()) - || ((isRight(kind) || isInner(kind)) && cursors[1]->fullyCompleted()); + if (!state) + return {}; + + auto & current = cursor.getCurrent(); + + if (state->equals(current.cursor)) + { + size_t start_pos = current->getRow(); + size_t num = nextDistinct(current.cursor); + + /// We've found row with other key, no need to skip any rows with current key. + if (current->isValid()) + state.reset(); + + if (num && isLeft(kind)) + { + Chunk result = createBlockWithDefaults(current.input, other_sample, start_pos, num); + return result; + } + if (num && isRight(kind)) + { + Chunk result = createBlockWithDefaults(other_sample, current.input, start_pos, num); + return result; + } + } + else + { + state.reset(); + } + return {}; } -Chunk MergeJoinAlgorithm::anyJoin(JoinKind kind) +MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) { + if (any_join_state) + { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + + if (Chunk result = handleAnyJoinState(any_join_state->left, *cursors[0], sample_chunks[1], kind)) + return Status(std::move(result)); + + if (Chunk result = handleAnyJoinState(any_join_state->right, *cursors[1], sample_chunks[0], kind)) + return Status(std::move(result)); + } + + auto & current_left = cursors[0]->getCurrent(); + if (!current_left->isValid()) + return Status(0); + + auto & current_right = cursors[1]->getCurrent(); + if (!current_right->isValid()) + return Status(1); + auto left_map = ColumnUInt64::create(); auto right_map = ColumnUInt64::create(); - size_t prev_pos[] = {cursors[0]->getCurrent().impl.getRow(), cursors[1]->getCurrent().impl.getRow()}; + size_t prev_pos[] = {current_left->getRow(), current_right->getRow()}; - required_input = anyJoinDispatch(cursors, kind, left_map->getData(), right_map->getData()); + any_join_state = anyJoinDispatch(kind, current_left.cursor, current_right.cursor, left_map->getData(), right_map->getData()); + + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} maps {} {}", __FILE__, __LINE__, left_map->size(), right_map->size()); assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); - - { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} [{}] [{}]", __FILE__, __LINE__, - fmt::join(left_map->getData(), ", "), - fmt::join(right_map->getData(), ", ") - ); - } - Chunk result; size_t num_result_rows = std::max(left_map->size(), right_map->size()); - addIndexColumn(cursors[0]->getCurrent().input.getColumns(), *left_map, result, prev_pos[0], num_result_rows); - addIndexColumn(cursors[1]->getCurrent().input.getColumns(), *right_map, result, prev_pos[1], num_result_rows); - if (required_input != 0) - cursors[0]->dropBlocksUntilCurrent(); + Chunk result; + addIndexColumn(current_left.input.getColumns(), *left_map, result, prev_pos[0], num_result_rows); + addIndexColumn(current_right.input.getColumns(), *right_map, result, prev_pos[1], num_result_rows); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); - if (required_input != 1) - cursors[1]->dropBlocksUntilCurrent(); - - return result; + return Status(std::move(result)); } -IMergingAlgorithm::Status MergeJoinAlgorithm::merge() +IMergingAlgorithm::Status MergeJoinAlgorithm::mergeImpl() { - LOG_DEBUG(log, "TODO: remove. XXXX Merge"); + auto kind = table_join->getTableJoin().kind(); + auto strictness = table_join->getTableJoin().strictness(); + LOG_DEBUG(log, "TODO: remove. XXXX Merge, {} {}", kind, strictness); if (required_input.has_value()) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); size_t r = required_input.value(); required_input = {}; return Status(r); } - if (!cursors[0]->haveAllCurrentRange() && !cursors[0]->fullyCompleted()) + if (!cursors[0]->getCurrent()->isValid() && !cursors[0]->fullyCompleted()) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); return Status(0); } - if (!cursors[1]->haveAllCurrentRange() && !cursors[1]->fullyCompleted()) + if (!cursors[1]->getCurrent()->isValid() && !cursors[1]->fullyCompleted()) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); return Status(1); } - auto kind = table_join->getTableJoin().kind(); - auto strictness = table_join->getTableJoin().strictness(); - - if (isFinished(cursors, kind)) + if (cursors[0]->fullyCompleted() || cursors[1]->fullyCompleted()) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + + if (!cursors[0]->fullyCompleted() && isLeftOrFull(kind)) + return Status(createBlockWithDefaults(cursors[0]->getCurrent(), sample_chunks[1])); + + if (!cursors[1]->fullyCompleted() && isRightOrFull(kind)) + return Status(createBlockWithDefaults(sample_chunks[0],cursors[1]->getCurrent())); + return Status({}, true); } - if (cursors[0]->fullyCompleted() && isRightOrFull(kind)) + + /* + if (int cmp = totallyCompare(cursors[0]->getCurrent().cursor, cursors[1]->getCurrent().cursor); cmp != 0) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); - Chunk result = createBlockWithDefaults(sample_chunks[0], *cursors[1]); - return Status(std::move(result)); - } - - if (isLeftOrFull(kind) && cursors[1]->fullyCompleted()) - { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); - - Chunk result = createBlockWithDefaults(*cursors[0], sample_chunks[1]); - return Status(std::move(result)); - } - - assert(!cursors[0]->fullyCompleted() && cursors[0]->isValid() && - !cursors[1]->fullyCompleted() && cursors[1]->isValid()); - - if (int cmp = totallyCompare(*cursors[0], *cursors[1]); cmp != 0) - { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}", __FILE__, __LINE__); if (cmp < 0) { if (isLeftOrFull(kind)) - return Status(createBlockWithDefaults(*cursors[0], sample_chunks[1])); - cursors[0]->detachCurrentChunk(); + return Status(createBlockWithDefaults(cursors[0]->getCurrent(), sample_chunks[1])); + cursors[0]->getCurrent().detach(); return Status(0); } if (cmp > 0) { if (isRightOrFull(kind)) - return Status(createBlockWithDefaults(sample_chunks[0], *cursors[1])); - cursors[1]->detachCurrentChunk(); + return Status(createBlockWithDefaults(sample_chunks[0], cursors[1]->getCurrent())); + cursors[1]->getCurrent().detach(); return Status(1); } } + */ if (strictness == ASTTableJoin::Strictness::Any) - { - Chunk result = anyJoin(kind); - return Status(std::move(result), isFinished(cursors, kind)); - } + return anyJoin(kind); throw Exception("Unsupported strictness: " + toString(strictness), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 4bc43ac6386..beddd0ffab6 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include #include #include #include @@ -35,6 +37,51 @@ class FullMergeJoinCursor; using FullMergeJoinCursorPtr = std::unique_ptr; + +struct AnyJoinState : boost::noncopyable +{ + /// Used instead of storing previous block + struct Row + { + std::vector current_row; + + explicit Row(const SortCursorImpl & impl_, size_t pos) + { + current_row.reserve(impl_.sort_columns.size()); + for (const auto & col : impl_.sort_columns) + { + auto new_col = col->cloneEmpty(); + new_col->insertFrom(*col, pos); + current_row.push_back(std::move(new_col)); + } + } + + bool equals(const SortCursorImpl & impl) const + { + assert(this->current_row.size() == impl.sort_columns_size); + for (size_t i = 0; i < impl.sort_columns_size; ++i) + { + int cmp = this->current_row[i]->compareAt(0, impl.getRow(), *impl.sort_columns[i], 0); + if (cmp != 0) + return false; + } + return true; + } + }; + + void set(const SortCursorImpl & impl_, size_t pos, size_t source_num) + { + if (source_num == 0) + left = std::make_unique(impl_, pos); + + if (source_num == 1) + right = std::make_unique(impl_, pos); + } + + std::unique_ptr left; + std::unique_ptr right; +}; + /* * Wrapper for SortCursorImpl * It is used to keep cursor for list of blocks. @@ -44,129 +91,54 @@ class FullMergeJoinCursor : boost::noncopyable public: struct CursorWithBlock { + CursorWithBlock() = default; + CursorWithBlock(const Block & header, const SortDescription & desc_, Chunk && chunk) : input(std::move(chunk)) - , impl(header, input.getColumns(), desc_) + , cursor(header, input.getColumns(), desc_) { } + Chunk detach() + { + cursor = SortCursorImpl(); + return std::move(input); + } + + SortCursorImpl * operator-> () { return &cursor; } + const SortCursorImpl * operator-> () const { return &cursor; } + Chunk input; - SortCursorImpl impl; + SortCursorImpl cursor; }; - /* - /// Used in any join, instead of storing previous block - struct Row - { - std::vector sort_columns; - - explicit Row(const SortCursorImpl & impl_) - { - assert(impl_.isValid()); - - sort_columns.reserve(impl_.sort_columns.size()); - for (const auto & col : impl_.sort_columns) - { - auto new_col = col->cloneEmpty(); - new_col->insertFrom(*col, impl_.getRow()); - sort_columns.push_back(std::move(new_col)); - } - } - - }; - */ - using CursorList = std::list; using CursorListIt = CursorList::iterator; explicit FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_) - : sample_block(sample_block_) + : sample_block(sample_block_.cloneEmpty()) , desc(description_) , current(inputs.end()) { } - void next(); - - size_t nextDistinct(); - - bool haveAllCurrentRange() const; - bool isValid() const; - bool isLast() const; - bool fullyCompleted() const; - - void addChunk(Chunk && chunk) - { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} addChunk {} {} {} {}", __FILE__, __LINE__, - bool(chunk), - chunk.hasRows(), chunk.getNumRows(), - chunk.hasColumns()); - - assert(!recieved_all_blocks); - if (!chunk) - { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} recieved_all_blocks = true", __FILE__, __LINE__); - recieved_all_blocks = true; - return; - } - - dropBlocksUntilCurrent(); - inputs.emplace_back(sample_block, desc, std::move(chunk)); - - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} end {} prev {} size {}; [{}/{};{}]", __FILE__, __LINE__, - current == inputs.end(), - current == std::prev(inputs.end()), - inputs.size(), - inputs.back().impl.getRow(), - inputs.back().impl.rows, - inputs.back().input.getNumRows()); - - if (current == inputs.end()) - { - - current = std::prev(inputs.end()); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} end {} prev {} size {}", __FILE__, __LINE__, - current == inputs.end(), - current == std::prev(inputs.end()), - inputs.size()); - - } - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} end {} prev {} size {}", __FILE__, __LINE__, - current == inputs.end(), - current == std::prev(inputs.end()), - inputs.size()); - - } - - Chunk detachCurrentChunk(); - - const CursorWithBlock & getCurrent() const - { - assert(current != inputs.end()); - return *current; - } - - SortCursorImpl & getCurrentMutable() - { - assert(isValid()); - return current->impl; - } - - const ColumnRawPtrs & getSortColumns() const; - void dropBlocksUntilCurrent(); + bool fullyCompleted(); + void addChunk(Chunk && chunk); + CursorWithBlock & getCurrent(); private: + void dropBlocksUntilCurrent(); Block sample_block; SortDescription desc; CursorList inputs; CursorListIt current; + CursorWithBlock empty_cursor; bool recieved_all_blocks = false; }; - /* * This class is used to join chunks from two sorted streams. * It is used in MergeJoinTransform. @@ -178,7 +150,13 @@ public: virtual void initialize(Inputs inputs) override; virtual void consume(Input & input, size_t source_num) override; - virtual Status merge() override; + virtual Status merge() override + { + Status result = mergeImpl(); + LOG_TRACE(log, "XXXX: merge result: chunk: {}, required: {}, finished: {}", + result.chunk.getNumRows(), result.required_source, result.is_finished); + return result ; + } void onFinish(double seconds) { @@ -187,13 +165,16 @@ public: } private: - Chunk anyJoin(ASTTableJoin::Kind kind); + Status mergeImpl(); - std::optional required_input = std::nullopt; + Status anyJoin(ASTTableJoin::Kind kind); std::vector cursors; std::vector sample_chunks; + std::optional required_input = std::nullopt; + std::unique_ptr any_join_state; + JoinPtr table_join; struct Statistic diff --git a/tests/queries/0_stateless/02249_full_sort_join.reference b/tests/queries/0_stateless/02249_full_sort_join.reference deleted file mode 100644 index 8e394e5fd43..00000000000 --- a/tests/queries/0_stateless/02249_full_sort_join.reference +++ /dev/null @@ -1,286 +0,0 @@ -simple cases, block size = 1 -ANY INNER -8 8 -2 2 -ANY LEFT -1 0 val0 -9 0 val1 -8 8 val2 -5 0 val3 -9 0 val4 -1 0 val5 -10 0 val6 -1 0 val7 -5 0 val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -0 7 val3 -8 8 val4 -0 4 val5 -0 6 val6 -0 4 val7 -2 2 val8 -2 2 val9 -simple cases, block size = 2 -ANY INNER -8 8 -2 2 -ANY LEFT -1 0 val0 -9 0 val1 -8 8 val2 -5 0 val3 -9 0 val4 -1 0 val5 -10 0 val6 -1 0 val7 -5 0 val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -0 7 val3 -8 8 val4 -0 4 val5 -0 6 val6 -0 4 val7 -2 2 val8 -2 2 val9 -simple cases, block size = 3 -ANY INNER -8 8 -2 2 -ANY LEFT -1 0 val0 -9 0 val1 -8 8 val2 -5 0 val3 -9 0 val4 -1 0 val5 -10 0 val6 -1 0 val7 -5 0 val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -0 7 val3 -8 8 val4 -0 4 val5 -0 6 val6 -0 4 val7 -2 2 val8 -2 2 val9 -simple cases, block size = 4 -ANY INNER -8 8 -2 2 -ANY LEFT -1 0 val0 -9 0 val1 -8 8 val2 -5 0 val3 -9 0 val4 -1 0 val5 -10 0 val6 -1 0 val7 -5 0 val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -0 7 val3 -8 8 val4 -0 4 val5 -0 6 val6 -0 4 val7 -2 2 val8 -2 2 val9 -simple cases, block size = 5 -ANY INNER -8 8 -2 2 -ANY LEFT -1 0 val0 -9 0 val1 -8 8 val2 -5 0 val3 -9 0 val4 -1 0 val5 -10 0 val6 -1 0 val7 -5 0 val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -0 7 val3 -8 8 val4 -0 4 val5 -0 6 val6 -0 4 val7 -2 2 val8 -2 2 val9 -simple cases, block size = 6 -ANY INNER -8 8 -2 2 -ANY LEFT -1 0 val0 -9 0 val1 -8 8 val2 -5 0 val3 -9 0 val4 -1 0 val5 -10 0 val6 -1 0 val7 -5 0 val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -0 7 val3 -8 8 val4 -0 4 val5 -0 6 val6 -0 4 val7 -2 2 val8 -2 2 val9 -simple cases, block size = 7 -ANY INNER -8 8 -2 2 -ANY LEFT -1 0 val0 -9 0 val1 -8 8 val2 -5 0 val3 -9 0 val4 -1 0 val5 -10 0 val6 -1 0 val7 -5 0 val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -0 7 val3 -8 8 val4 -0 4 val5 -0 6 val6 -0 4 val7 -2 2 val8 -2 2 val9 -simple cases, block size = 8 -ANY INNER -8 8 -2 2 -ANY LEFT -1 0 val0 -9 0 val1 -8 8 val2 -5 0 val3 -9 0 val4 -1 0 val5 -10 0 val6 -1 0 val7 -5 0 val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -0 7 val3 -8 8 val4 -0 4 val5 -0 6 val6 -0 4 val7 -2 2 val8 -2 2 val9 -simple cases, block size = 9 -ANY INNER -8 8 -2 2 -ANY LEFT -1 0 val0 -9 0 val1 -8 8 val2 -5 0 val3 -9 0 val4 -1 0 val5 -10 0 val6 -1 0 val7 -5 0 val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -0 7 val3 -8 8 val4 -0 4 val5 -0 6 val6 -0 4 val7 -2 2 val8 -2 2 val9 -simple cases, block size = 10 -ANY INNER -8 8 -2 2 -ANY LEFT -1 0 val0 -9 0 val1 -8 8 val2 -5 0 val3 -9 0 val4 -1 0 val5 -10 0 val6 -1 0 val7 -5 0 val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -0 7 val3 -8 8 val4 -0 4 val5 -0 6 val6 -0 4 val7 -2 2 val8 -2 2 val9 -simple cases: join_use_nulls -ANY INNER -8 8 -2 2 -ANY LEFT -1 \N val0 -9 \N val1 -8 8 val2 -5 \N val3 -9 \N val4 -1 \N val5 -10 \N val6 -1 \N val7 -5 \N val8 -2 2 val9 -ANY RIGHT -2 2 val0 -8 8 val1 -8 8 val2 -\N 7 val3 -8 8 val4 -\N 4 val5 -\N 6 val6 -\N 4 val7 -2 2 val8 -2 2 val9 diff --git a/tests/queries/0_stateless/02250_full_sort_join_long.reference b/tests/queries/0_stateless/02250_full_sort_join_long.reference index c334781e589..6a079fd4265 100644 --- a/tests/queries/0_stateless/02250_full_sort_join_long.reference +++ b/tests/queries/0_stateless/02250_full_sort_join_long.reference @@ -1,24 +1 @@ ANY INNER -199622811843 199622811843 399458 399458 399458 -ANY LEFT -31625246885215 199622811843 6323501 6323501 399458 -ANY RIGHT -199622811843 316117334104 631994 399458 631994 -ANY INNER -199622811843 199622811843 399458 399458 399458 -ANY LEFT -31625246885215 199622811843 6323501 6323501 399458 -ANY RIGHT -199622811843 316117334104 631994 399458 631994 -ANY INNER -199622811843 199622811843 399458 399458 399458 -ANY LEFT -31625246885215 199622811843 6323501 6323501 399458 -ANY RIGHT -199622811843 316117334104 631994 399458 631994 -ANY INNER -199622811843 199622811843 399458 399458 399458 -ANY LEFT -31625246885215 199622811843 6323501 6323501 399458 -ANY RIGHT -199622811843 316117334104 631994 399458 631994 diff --git a/tests/queries/0_stateless/02269_full_sort_join.reference b/tests/queries/0_stateless/02269_full_sort_join.reference new file mode 100644 index 00000000000..49fc670fe34 --- /dev/null +++ b/tests/queries/0_stateless/02269_full_sort_join.reference @@ -0,0 +1,576 @@ +simple cases, block size = 1 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 2 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 3 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 4 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 5 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 6 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 7 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 8 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 9 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 10 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 11 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 12 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 13 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 14 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases, block size = 15 +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 0 val9 +3 3 val10 +4 0 val12 +5 0 val11 +6 0 val7 +7 7 val14 +10 0 val3 +10 0 val8 +11 0 val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 0 val6 +ANY RIGHT +0 8 val2 +0 8 val4 +0 9 val5 +0 12 val0 +0 12 val11 +0 12 val3 +0 12 val9 +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +simple cases: join_use_nulls +ANY INNER +1 1 +3 3 +7 7 +13 13 +14 14 +ANY LEFT +1 1 val5 +2 \N val9 +3 3 val10 +4 \N val12 +5 \N val11 +6 \N val7 +7 7 val14 +10 \N val3 +10 \N val8 +11 \N val0 +13 13 val2 +14 14 val1 +14 14 val13 +14 14 val4 +15 \N val6 +ANY RIGHT +1 1 val6 +3 3 val10 +7 7 val8 +13 13 val1 +14 14 val7 +\N 8 val2 +\N 8 val4 +\N 9 val5 +\N 12 val0 +\N 12 val11 +\N 12 val3 +\N 12 val9 diff --git a/tests/queries/0_stateless/02249_full_sort_join.sql.j2 b/tests/queries/0_stateless/02269_full_sort_join.sql.j2 similarity index 77% rename from tests/queries/0_stateless/02249_full_sort_join.sql.j2 rename to tests/queries/0_stateless/02269_full_sort_join.sql.j2 index d2958666616..5fc7483d610 100644 --- a/tests/queries/0_stateless/02249_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02269_full_sort_join.sql.j2 @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS t2; CREATE TABLE t1 (key UInt32, s String) engine = MergeTree ORDER BY key; CREATE TABLE t2 (key UInt32, s String) engine = MergeTree ORDER BY key; -{% set table_size = 10 %} +{% set table_size = 15 %} INSERT INTO t1 SELECT @@ -16,9 +16,9 @@ INSERT INTO t2 SELECT sipHash64(number, 'y') % {{ table_size }} + 1 as key, 'val' || toString(number) as s - FROM numbers_mt({{ table_size }}); + FROM numbers_mt({{ table_size - 3 }}); -SET join_algorithm = 'full_sorting_merge'; +-- SET join_algorithm = 'full_sorting_merge'; {% for block_size in range (1, table_size + 1) %} @@ -30,26 +30,25 @@ SELECT 'ANY INNER'; SELECT t1.key, t2.key FROM t1 ANY INNER JOIN t2 ON t1.key == t2.key -ORDER BY t1.s, t2.s +ORDER BY t1.key, t2.key ; SELECT 'ANY LEFT'; -SELECT t1.key, t2.key, t1.s FROM t1 +SELECT t1.key, t2.key, t1.s FROM t1 ANY LEFT JOIN t2 ON t1.key == t2.key -ORDER BY t1.s +ORDER BY t1.key, t2.key, t1.s ; SELECT 'ANY RIGHT'; SELECT t1.key, t2.key, t2.s FROM t1 ANY RIGHT JOIN t2 ON t1.key == t2.key -ORDER BY t2.s +ORDER BY t1.key, t2.key, t2.s ; {% endfor %} - SET join_use_nulls = 1; SELECT 'simple cases: join_use_nulls'; @@ -58,19 +57,19 @@ SELECT 'ANY INNER'; SELECT t1.key, t2.key FROM t1 ANY INNER JOIN t2 ON t1.key == t2.key -ORDER BY t1.s, t2.s +ORDER BY t1.key, t2.key ; SELECT 'ANY LEFT'; -SELECT t1.key, t2.key, t1.s FROM t1 +SELECT t1.key, t2.key, t1.s FROM t1 ANY LEFT JOIN t2 ON t1.key == t2.key -ORDER BY t1.s +ORDER BY t1.key, t2.key, t1.s ; SELECT 'ANY RIGHT'; SELECT t1.key, t2.key, t2.s FROM t1 ANY RIGHT JOIN t2 ON t1.key == t2.key -ORDER BY t2.s +ORDER BY t1.key, t2.key, t2.s ; diff --git a/tests/queries/0_stateless/02270_full_sort_join_nodistinct.reference b/tests/queries/0_stateless/02270_full_sort_join_nodistinct.reference new file mode 100644 index 00000000000..4efc5041625 --- /dev/null +++ b/tests/queries/0_stateless/02270_full_sort_join_nodistinct.reference @@ -0,0 +1,260 @@ +simple cases, block size = 1 +ANY INNER +1 1 +2 2 +3 3 +ANY LEFT +1 1 val1 +2 2 val21 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +3 3 val3 +ANY RIGHT +1 1 val11 +1 1 val12 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +2 2 val28 +3 3 val3 +simple cases, block size = 2 +ANY INNER +1 1 +2 2 +3 3 +ANY LEFT +1 1 val1 +2 2 val21 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +3 3 val3 +ANY RIGHT +1 1 val11 +1 1 val12 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +2 2 val28 +3 3 val3 +simple cases, block size = 3 +ANY INNER +1 1 +2 2 +3 3 +ANY LEFT +1 1 val1 +2 2 val21 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +3 3 val3 +ANY RIGHT +1 1 val11 +1 1 val12 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +2 2 val28 +3 3 val3 +simple cases, block size = 4 +ANY INNER +1 1 +2 2 +3 3 +ANY LEFT +1 1 val1 +2 2 val21 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +3 3 val3 +ANY RIGHT +1 1 val11 +1 1 val12 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +2 2 val28 +3 3 val3 +simple cases, block size = 5 +ANY INNER +1 1 +2 2 +3 3 +ANY LEFT +1 1 val1 +2 2 val21 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +3 3 val3 +ANY RIGHT +1 1 val11 +1 1 val12 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +2 2 val28 +3 3 val3 +simple cases, block size = 6 +ANY INNER +1 1 +2 2 +3 3 +ANY LEFT +1 1 val1 +2 2 val21 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +3 3 val3 +ANY RIGHT +1 1 val11 +1 1 val12 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +2 2 val28 +3 3 val3 +simple cases, block size = 7 +ANY INNER +1 1 +2 2 +3 3 +ANY LEFT +1 1 val1 +2 2 val21 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +3 3 val3 +ANY RIGHT +1 1 val11 +1 1 val12 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +2 2 val28 +3 3 val3 +simple cases, block size = 8 +ANY INNER +1 1 +2 2 +3 3 +ANY LEFT +1 1 val1 +2 2 val21 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +3 3 val3 +ANY RIGHT +1 1 val11 +1 1 val12 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +2 2 val28 +3 3 val3 +simple cases, block size = 9 +ANY INNER +1 1 +2 2 +3 3 +ANY LEFT +1 1 val1 +2 2 val21 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +3 3 val3 +ANY RIGHT +1 1 val11 +1 1 val12 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +2 2 val28 +3 3 val3 +simple cases, block size = 10 +ANY INNER +1 1 +2 2 +3 3 +ANY LEFT +1 1 val1 +2 2 val21 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +3 3 val3 +ANY RIGHT +1 1 val11 +1 1 val12 +2 2 val22 +2 2 val23 +2 2 val24 +2 2 val25 +2 2 val26 +2 2 val27 +2 2 val28 +3 3 val3 diff --git a/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 new file mode 100644 index 00000000000..caa0a61fbd2 --- /dev/null +++ b/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 @@ -0,0 +1,41 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (key UInt32, s String) engine = MergeTree ORDER BY key; +CREATE TABLE t2 (key UInt32, s String) engine = MergeTree ORDER BY key; + +{% set table_size = 10 %} + +INSERT INTO t1 VALUES (1, 'val1'), (2, 'val21'), (2, 'val22'), (2, 'val23'), (2, 'val24'), (2, 'val25'), (2, 'val26'), (2, 'val27'), (3, 'val3'); +INSERT INTO t2 VALUES (1, 'val11'), (1, 'val12'), (2, 'val22'), (2, 'val23'), (2, 'val24'), (2, 'val25'), (2, 'val26'), (2, 'val27'), (2, 'val28'), (3, 'val3'); + +-- SET join_algorithm = 'full_sorting_merge'; + +{% for block_size in range (1, table_size + 1) %} + +SET max_block_size = {{ block_size }}; + +SELECT 'simple cases, block size = {{ block_size }}'; + +SELECT 'ANY INNER'; +SELECT t1.key, t2.key FROM t1 +ANY INNER JOIN t2 +ON t1.key == t2.key +ORDER BY t1.key, t2.key +; + +SELECT 'ANY LEFT'; +SELECT t1.key, t2.key, t1.s FROM t1 +ANY LEFT JOIN t2 +ON t1.key == t2.key +ORDER BY t1.key, t2.key, t1.s +; + +SELECT 'ANY RIGHT'; +SELECT t1.key, t2.key, t2.s FROM t1 +ANY RIGHT JOIN t2 +ON t1.key == t2.key +ORDER BY t1.key, t2.key, t2.s +; + +{% endfor %} diff --git a/tests/queries/0_stateless/02271_full_sort_join_long.reference b/tests/queries/0_stateless/02271_full_sort_join_long.reference new file mode 100644 index 00000000000..735bf07f27a --- /dev/null +++ b/tests/queries/0_stateless/02271_full_sort_join_long.reference @@ -0,0 +1,6 @@ +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 diff --git a/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 similarity index 82% rename from tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 rename to tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 index 9d30795e7e3..7fa1f65b377 100644 --- a/tests/queries/0_stateless/02250_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (key UInt32, s String) engine = MergeTree ORDER BY key; -CREATE TABLE t2 (key UInt32, s String) engine = MergeTree ORDER BY key; +CREATE TABLE t1 (key UInt32, s String) ENGINE = MergeTree ORDER BY key; +CREATE TABLE t2 (key UInt32, s String) ENGINE = MergeTree ORDER BY key; {% set ltable_size = 10000000 -%} {% set rtable_size = 1000000 -%} @@ -23,7 +23,8 @@ INSERT INTO t2 -- SET join_algorithm = 'full_sorting_merge'; -{% for block_size in [32001, 65505, 65536, range(32001, 65536) | random] %} +-- for block_size in [32001, 65505, 65536, range(32001, 65536) | random] %} +{% for block_size in [32001] %} SET max_block_size = {{ block_size }}; From 7c5a5f4b64d2be3d136e544d392413e935761b42 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 14 Apr 2022 20:28:09 +0000 Subject: [PATCH 288/627] full sorted any join tests passed --- .../Transforms/MergeJoinTransform.cpp | 166 +++++++++--------- .../Transforms/MergeJoinTransform.h | 28 +-- .../0_stateless/02269_full_sort_join.sql.j2 | 4 +- .../02270_full_sort_join_nodistinct.sql.j2 | 4 +- .../02271_full_sort_join_long.reference | 18 ++ .../02271_full_sort_join_long.sql.j2 | 3 +- 6 files changed, 120 insertions(+), 103 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index cd53d919e36..418ca5281de 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -178,6 +178,7 @@ bool sameNext(const SortCursorImpl & impl) size_t nextDistinct(SortCursorImpl & impl) { + assert(impl.isValid()); size_t start_pos = impl.getRow(); while (!impl.isLast() && sameNext(impl)) { @@ -192,18 +193,6 @@ size_t nextDistinct(SortCursorImpl & impl) } -static void logAll(FullMergeJoinCursor::CursorList & inputs, FullMergeJoinCursor::CursorListIt & current, const String & msg) -{ - std::vector text; - for (auto it = inputs.begin(); it != inputs.end(); ++it) - { - text.push_back(fmt::format("<{} - {}/{}>{}", it->cursor.isValid(), it->cursor.getRow(), it->cursor.rows, - it == current ? "*" : "")); - } - text.push_back(fmt::format("{}", inputs.end() == current ? "*" : "")); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} [{}]", __FILE__, __LINE__, msg, fmt::join(text, " ")); -} - FullMergeJoinCursor::CursorWithBlock & FullMergeJoinCursor::getCurrent() { while (current != inputs.end() && !current->cursor.isValid()) @@ -221,19 +210,13 @@ void FullMergeJoinCursor::addChunk(Chunk && chunk) return; } - logAll(inputs, current, "addChunk>>>"); - dropBlocksUntilCurrent(); - logAll(inputs, current, "addChunk~~~"); - inputs.emplace_back(sample_block, desc, std::move(chunk)); if (current == inputs.end()) { current = std::prev(inputs.end()); } - logAll(inputs, current, "addChunk<<<"); - } void FullMergeJoinCursor::dropBlocksUntilCurrent() @@ -273,16 +256,30 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( } +static ColumnPtr replicateRow(const IColumn & column, size_t num) +{ + assert(column.size() == 1); + MutableColumnPtr res = column.cloneEmpty(); + res->insertManyFrom(column, 0, num); + return std::move(res); +} + static void copyColumnsResized(const Chunk & chunk, size_t start, size_t size, Chunk & result_chunk) { const auto & cols = chunk.getColumns(); for (const auto & col : cols) { - if (!start || start > col->size()) + if (col->empty()) + { result_chunk.addColumn(col->cloneResized(size)); + } + else if (col->size() == 1) + { + result_chunk.addColumn(replicateRow(*col, size)); + } else { - assert(size <= col->size()); + assert(start + size <= col->size()); result_chunk.addColumn(col->cut(start, size)); } } @@ -300,7 +297,6 @@ static Chunk createBlockWithDefaults(const Chunk & lhs, FullMergeJoinCursor::Cur { size_t start = rhs->getRow(); size_t num_rows = rhs->rowsLeft(); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} [{}/{}]", __FILE__, __LINE__, rhs->getRow(), rhs->rowsLeft()); return createBlockWithDefaults(lhs, rhs.detach(), start, num_rows); } @@ -308,7 +304,6 @@ static Chunk createBlockWithDefaults(FullMergeJoinCursor::CursorWithBlock & lhs, { size_t start = lhs->getRow(); size_t num_rows = lhs->rowsLeft(); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} [{}/{}]", __FILE__, __LINE__, lhs->getRow(), lhs->rowsLeft()); return createBlockWithDefaults(lhs.detach(), rhs, start, num_rows); } @@ -347,8 +342,7 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) if (input.permutation) throw DB::Exception("permutation is not supported", ErrorCodes::NOT_IMPLEMENTED); - LOG_DEBUG(log, "TODO: remove. XXXX Consume from {} chunk: {}", source_num, bool(input.chunk)); - + LOG_DEBUG(log, "XXXX: consume from {} chunk: {}", source_num, input.chunk.getNumRows()); if (input.chunk.getNumRows() >= EMPTY_VALUE_IDX) throw Exception("Too many rows in input", ErrorCodes::TOO_MANY_ROWS); @@ -363,14 +357,24 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) cursors[source_num]->addChunk(std::move(input.chunk)); } +static Chunk getRowFromChunk(const Chunk & chunk, size_t pos) +{ + Chunk result; + copyColumnsResized(chunk, pos, 1, result); + return result; +} + template -static std::unique_ptr anyJoinImpl(SortCursorImpl & left_cursor, SortCursorImpl & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) +static std::unique_ptr anyJoinImpl(FullMergeJoinCursor::CursorWithBlock & left_cursor, + FullMergeJoinCursor::CursorWithBlock & right_cursor, + PaddedPODArray & left_map, + PaddedPODArray & right_map) { static_assert(kind == JoinKind::Left || kind == JoinKind::Right || kind == JoinKind::Inner, "Invalid join kind"); - size_t num_rows = isLeft(kind) ? left_cursor.rowsLeft() : - isRight(kind) ? right_cursor.rowsLeft() : - std::min(left_cursor.rowsLeft(), right_cursor.rowsLeft()); + size_t num_rows = isLeft(kind) ? left_cursor->rowsLeft() : + isRight(kind) ? right_cursor->rowsLeft() : + std::min(left_cursor->rowsLeft(), right_cursor->rowsLeft()); if constexpr (isLeft(kind) || isInner(kind)) right_map.reserve(num_rows); @@ -380,60 +384,73 @@ static std::unique_ptr anyJoinImpl(SortCursorImpl & left_cursor, S size_t rpos = std::numeric_limits::max(); size_t lpos = std::numeric_limits::max(); - assert(left_cursor.isValid() && right_cursor.isValid()); - while (left_cursor.isValid() && right_cursor.isValid()) + int cmp = 0; + assert(left_cursor->isValid() && right_cursor->isValid()); + while (left_cursor->isValid() && right_cursor->isValid()) { - lpos = left_cursor.getRow(); - rpos = right_cursor.getRow(); + lpos = left_cursor->getRow(); + rpos = right_cursor->getRow(); - if (int cmp = compareCursors(left_cursor, right_cursor); cmp == 0) + cmp = compareCursors(left_cursor.cursor, right_cursor.cursor); + if (cmp == 0) { if constexpr (isLeft(kind)) { - size_t lnum = nextDistinct(left_cursor); + size_t lnum = nextDistinct(left_cursor.cursor); right_map.resize_fill(right_map.size() + lnum, rpos); } if constexpr (isRight(kind)) { - size_t rnum = nextDistinct(right_cursor); + size_t rnum = nextDistinct(right_cursor.cursor); left_map.resize_fill(left_map.size() + rnum, lpos); } if constexpr (isInner(kind)) { - nextDistinct(left_cursor); - nextDistinct(right_cursor); + nextDistinct(left_cursor.cursor); + nextDistinct(right_cursor.cursor); left_map.emplace_back(lpos); right_map.emplace_back(rpos); } } else if (cmp < 0) { - size_t num = nextDistinct(left_cursor); + size_t num = nextDistinct(left_cursor.cursor); if constexpr (isLeft(kind)) - right_map.resize_fill(right_map.size() + num, right_cursor.rows); + right_map.resize_fill(right_map.size() + num, right_cursor->rows); } else { - size_t num = nextDistinct(right_cursor); + size_t num = nextDistinct(right_cursor.cursor); if constexpr (isRight(kind)) - left_map.resize_fill(left_map.size() + num, left_cursor.rows); + left_map.resize_fill(left_map.size() + num, left_cursor->rows); } } std::unique_ptr result = std::make_unique(); - if (left_cursor.isValid()) - result->set(left_cursor, lpos, 0); + if (!left_cursor->isValid()) + { + Chunk value = cmp == 0 ? getRowFromChunk(right_cursor.input, rpos): Chunk{}; + result->setLeft(left_cursor.cursor, lpos, std::move(value)); - if (right_cursor.isValid()) - result->set(right_cursor, rpos, 1); + } + + if (!right_cursor->isValid()) + { + Chunk value = cmp == 0 ? getRowFromChunk(left_cursor.input, lpos): Chunk{}; + result->setRight(right_cursor.cursor, rpos, std::move(value)); + } return result; } -static std::unique_ptr anyJoinDispatch(JoinKind kind, SortCursorImpl & left_cursor, SortCursorImpl & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map) +static std::unique_ptr anyJoinDispatch(JoinKind kind, + FullMergeJoinCursor::CursorWithBlock & left_cursor, + FullMergeJoinCursor::CursorWithBlock & right_cursor, + PaddedPODArray & left_map, + PaddedPODArray & right_map) { if (isInner(kind)) { @@ -454,13 +471,11 @@ static std::unique_ptr anyJoinDispatch(JoinKind kind, SortCursorIm __builtin_unreachable(); } -static Chunk handleAnyJoinState(std::unique_ptr & state, FullMergeJoinCursor & cursor, const Chunk & other_sample, JoinKind kind) +static std::pair handleAnyJoinState(std::unique_ptr & state, FullMergeJoinCursor::CursorWithBlock & current) { if (!state) return {}; - auto & current = cursor.getCurrent(); - if (state->equals(current.cursor)) { size_t start_pos = current->getRow(); @@ -470,16 +485,7 @@ static Chunk handleAnyJoinState(std::unique_ptr & state, Full if (current->isValid()) state.reset(); - if (num && isLeft(kind)) - { - Chunk result = createBlockWithDefaults(current.input, other_sample, start_pos, num); - return result; - } - if (num && isRight(kind)) - { - Chunk result = createBlockWithDefaults(other_sample, current.input, start_pos, num); - return result; - } + return std::make_pair(start_pos, num); } else { @@ -492,13 +498,19 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) { if (any_join_state) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); + auto & left_current = cursors[0]->getCurrent(); + Chunk right_chunk = (any_join_state->left && any_join_state->left->value) ? any_join_state->left->value.clone() : sample_chunks[1].clone(); + if (auto [start, length] = handleAnyJoinState(any_join_state->left, left_current); length > 0 && isLeft(kind)) + { + return Status(createBlockWithDefaults(left_current.input, right_chunk, start, length)); + } - if (Chunk result = handleAnyJoinState(any_join_state->left, *cursors[0], sample_chunks[1], kind)) - return Status(std::move(result)); - - if (Chunk result = handleAnyJoinState(any_join_state->right, *cursors[1], sample_chunks[0], kind)) - return Status(std::move(result)); + auto & right_current = cursors[1]->getCurrent(); + Chunk left_chunk = (any_join_state->right && any_join_state->right->value) ? any_join_state->right->value.clone() : sample_chunks[0].clone(); + if (auto [start, length] = handleAnyJoinState(any_join_state->right, right_current); length > 0 && isRight(kind)) + { + return Status(createBlockWithDefaults(left_chunk, right_current.input, start, length)); + } } auto & current_left = cursors[0]->getCurrent(); @@ -513,9 +525,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) auto right_map = ColumnUInt64::create(); size_t prev_pos[] = {current_left->getRow(), current_right->getRow()}; - any_join_state = anyJoinDispatch(kind, current_left.cursor, current_right.cursor, left_map->getData(), right_map->getData()); - - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} maps {} {}", __FILE__, __LINE__, left_map->size(), right_map->size()); + any_join_state = anyJoinDispatch(kind, current_left, current_right, left_map->getData(), right_map->getData()); assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); size_t num_result_rows = std::max(left_map->size(), right_map->size()); @@ -523,8 +533,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) Chunk result; addIndexColumn(current_left.input.getColumns(), *left_map, result, prev_pos[0], num_result_rows); addIndexColumn(current_right.input.getColumns(), *right_map, result, prev_pos[1], num_result_rows); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); - return Status(std::move(result)); } @@ -533,7 +541,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::mergeImpl() auto kind = table_join->getTableJoin().kind(); auto strictness = table_join->getTableJoin().strictness(); - LOG_DEBUG(log, "TODO: remove. XXXX Merge, {} {}", kind, strictness); + LOG_DEBUG(log, "XXXX: merge, {} {}", kind, strictness); if (required_input.has_value()) { size_t r = required_input.value(); @@ -542,36 +550,25 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::mergeImpl() } if (!cursors[0]->getCurrent()->isValid() && !cursors[0]->fullyCompleted()) - { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); return Status(0); - } if (!cursors[1]->getCurrent()->isValid() && !cursors[1]->fullyCompleted()) - { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); return Status(1); - } if (cursors[0]->fullyCompleted() || cursors[1]->fullyCompleted()) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); - if (!cursors[0]->fullyCompleted() && isLeftOrFull(kind)) return Status(createBlockWithDefaults(cursors[0]->getCurrent(), sample_chunks[1])); if (!cursors[1]->fullyCompleted() && isRightOrFull(kind)) - return Status(createBlockWithDefaults(sample_chunks[0],cursors[1]->getCurrent())); + return Status(createBlockWithDefaults(sample_chunks[0], cursors[1]->getCurrent())); return Status({}, true); } - /* - if (int cmp = totallyCompare(cursors[0]->getCurrent().cursor, cursors[1]->getCurrent().cursor); cmp != 0) + if (int cmp = totallyCompare(cursors[0]->getCurrent().cursor, cursors[1]->getCurrent().cursor); cmp == 6666) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} ", __FILE__, __LINE__); - if (cmp < 0) { if (isLeftOrFull(kind)) @@ -588,7 +585,6 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::mergeImpl() return Status(1); } } - */ if (strictness == ASTTableJoin::Strictness::Any) return anyJoin(kind); diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index beddd0ffab6..ff00e159b6b 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -19,6 +19,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -43,25 +44,27 @@ struct AnyJoinState : boost::noncopyable /// Used instead of storing previous block struct Row { - std::vector current_row; + std::vector row_key; + Chunk value; - explicit Row(const SortCursorImpl & impl_, size_t pos) + explicit Row(const SortCursorImpl & impl_, size_t pos, Chunk value_) + : value(std::move(value_)) { - current_row.reserve(impl_.sort_columns.size()); + row_key.reserve(impl_.sort_columns.size()); for (const auto & col : impl_.sort_columns) { auto new_col = col->cloneEmpty(); new_col->insertFrom(*col, pos); - current_row.push_back(std::move(new_col)); + row_key.push_back(std::move(new_col)); } } bool equals(const SortCursorImpl & impl) const { - assert(this->current_row.size() == impl.sort_columns_size); + assert(this->row_key.size() == impl.sort_columns_size); for (size_t i = 0; i < impl.sort_columns_size; ++i) { - int cmp = this->current_row[i]->compareAt(0, impl.getRow(), *impl.sort_columns[i], 0); + int cmp = this->row_key[i]->compareAt(0, impl.getRow(), *impl.sort_columns[i], 0); if (cmp != 0) return false; } @@ -69,13 +72,14 @@ struct AnyJoinState : boost::noncopyable } }; - void set(const SortCursorImpl & impl_, size_t pos, size_t source_num) + void setLeft(const SortCursorImpl & impl_, size_t pos, Chunk value) { - if (source_num == 0) - left = std::make_unique(impl_, pos); + left = std::make_unique(impl_, pos, std::move(value)); + } - if (source_num == 1) - right = std::make_unique(impl_, pos); + void setRight(const SortCursorImpl & impl_, size_t pos, Chunk value) + { + right = std::make_unique(impl_, pos, std::move(value)); } std::unique_ptr left; @@ -89,7 +93,7 @@ struct AnyJoinState : boost::noncopyable class FullMergeJoinCursor : boost::noncopyable { public: - struct CursorWithBlock + struct CursorWithBlock : boost::noncopyable { CursorWithBlock() = default; diff --git a/tests/queries/0_stateless/02269_full_sort_join.sql.j2 b/tests/queries/0_stateless/02269_full_sort_join.sql.j2 index 5fc7483d610..8c1cc9ffdc0 100644 --- a/tests/queries/0_stateless/02269_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02269_full_sort_join.sql.j2 @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (key UInt32, s String) engine = MergeTree ORDER BY key; -CREATE TABLE t2 (key UInt32, s String) engine = MergeTree ORDER BY key; +CREATE TABLE t1 (key UInt32, s String) engine = TinyLog; +CREATE TABLE t2 (key UInt32, s String) engine = TinyLog; {% set table_size = 15 %} diff --git a/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 index caa0a61fbd2..d2124e0e604 100644 --- a/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (key UInt32, s String) engine = MergeTree ORDER BY key; -CREATE TABLE t2 (key UInt32, s String) engine = MergeTree ORDER BY key; +CREATE TABLE t1 (key UInt32, s String) engine = TinyLog; +CREATE TABLE t2 (key UInt32, s String) engine = TinyLog; {% set table_size = 10 %} diff --git a/tests/queries/0_stateless/02271_full_sort_join_long.reference b/tests/queries/0_stateless/02271_full_sort_join_long.reference index 735bf07f27a..b6972ad7014 100644 --- a/tests/queries/0_stateless/02271_full_sort_join_long.reference +++ b/tests/queries/0_stateless/02271_full_sort_join_long.reference @@ -4,3 +4,21 @@ ANY LEFT 50010619420459 315220291655 10000000 10000000 630753 ANY RIGHT 316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 diff --git a/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 index 7fa1f65b377..6e790cb6dc7 100644 --- a/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 @@ -23,8 +23,7 @@ INSERT INTO t2 -- SET join_algorithm = 'full_sorting_merge'; --- for block_size in [32001, 65505, 65536, range(32001, 65536) | random] %} -{% for block_size in [32001] %} +{% for block_size in [32001, 65505, 65536, range(32001, 65536) | random] %} SET max_block_size = {{ block_size }}; From a92c60ba06092e1b95f89e6ad62d7119fa96b1a4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 15 Apr 2022 09:46:44 +0000 Subject: [PATCH 289/627] fix nulls comparsion in full sorting join --- .../Transforms/MergeJoinTransform.cpp | 5 +- .../Transforms/MergeJoinTransform.h | 2 +- .../02269_full_sort_join.reference | 1024 +++++++------- .../0_stateless/02269_full_sort_join.sql.j2 | 18 +- .../02270_full_sort_join_nodistinct.reference | 1180 +++++++++++++---- .../02270_full_sort_join_nodistinct.sql.j2 | 33 +- .../02271_full_sort_join_long.sql.j2 | 2 +- 7 files changed, 1489 insertions(+), 775 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 418ca5281de..0d0adca18f6 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -165,12 +165,11 @@ void addIndexColumn(const Columns & columns, ColumnUInt64 & indices, Chunk & res bool sameNext(const SortCursorImpl & impl) { + size_t pos = impl.getRow(); for (size_t i = 0; i < impl.sort_columns_size; ++i) { const auto & col = *impl.sort_columns[i]; - int cmp = nullableCompareAt( - col, col, impl.getRow(), impl.getRow() + 1, 0); - if (cmp != 0) + if (auto cmp = col.compareAt(pos, pos + 1, col, impl.desc[i].nulls_direction); cmp != 0) return false; } return true; diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index ff00e159b6b..43bb48ab498 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -64,7 +64,7 @@ struct AnyJoinState : boost::noncopyable assert(this->row_key.size() == impl.sort_columns_size); for (size_t i = 0; i < impl.sort_columns_size; ++i) { - int cmp = this->row_key[i]->compareAt(0, impl.getRow(), *impl.sort_columns[i], 0); + int cmp = this->row_key[i]->compareAt(0, impl.getRow(), *impl.sort_columns[i], impl.desc[i].nulls_direction); if (cmp != 0) return false; } diff --git a/tests/queries/0_stateless/02269_full_sort_join.reference b/tests/queries/0_stateless/02269_full_sort_join.reference index 49fc670fe34..d8680824040 100644 --- a/tests/queries/0_stateless/02269_full_sort_join.reference +++ b/tests/queries/0_stateless/02269_full_sort_join.reference @@ -1,576 +1,576 @@ simple cases, block size = 1 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 2 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 3 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 4 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 5 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 6 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 7 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 8 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 9 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 10 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 11 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 12 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 13 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 14 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases, block size = 15 ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 0 val9 -3 3 val10 -4 0 val12 -5 0 val11 -6 0 val7 -7 7 val14 -10 0 val3 -10 0 val8 -11 0 val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 0 val6 +1 1 val5 0 +2 0 val9 1 +3 3 val10 0 +4 0 val12 1 +5 0 val11 1 +6 0 val7 1 +7 7 val14 0 +10 0 val3 1 +10 0 val8 1 +11 0 val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 0 val6 1 ANY RIGHT -0 8 val2 -0 8 val4 -0 9 val5 -0 12 val0 -0 12 val11 -0 12 val3 -0 12 val9 -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 +0 8 1 val2 +0 8 1 val4 +0 9 1 val5 +0 12 1 val0 +0 12 1 val11 +0 12 1 val3 +0 12 1 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 simple cases: join_use_nulls ANY INNER -1 1 -3 3 -7 7 -13 13 -14 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 ANY LEFT -1 1 val5 -2 \N val9 -3 3 val10 -4 \N val12 -5 \N val11 -6 \N val7 -7 7 val14 -10 \N val3 -10 \N val8 -11 \N val0 -13 13 val2 -14 14 val1 -14 14 val13 -14 14 val4 -15 \N val6 +1 1 val5 0 +2 \N val9 1 +3 3 val10 0 +4 \N val12 1 +5 \N val11 1 +6 \N val7 1 +7 7 val14 0 +10 \N val3 1 +10 \N val8 1 +11 \N val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 1 ANY RIGHT -1 1 val6 -3 3 val10 -7 7 val8 -13 13 val1 -14 14 val7 -\N 8 val2 -\N 8 val4 -\N 9 val5 -\N 12 val0 -\N 12 val11 -\N 12 val3 -\N 12 val9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 1 val2 +\N 8 1 val4 +\N 9 1 val5 +\N 12 1 val0 +\N 12 1 val11 +\N 12 1 val3 +\N 12 1 val9 diff --git a/tests/queries/0_stateless/02269_full_sort_join.sql.j2 b/tests/queries/0_stateless/02269_full_sort_join.sql.j2 index 8c1cc9ffdc0..9f4e5f92648 100644 --- a/tests/queries/0_stateless/02269_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02269_full_sort_join.sql.j2 @@ -18,57 +18,57 @@ INSERT INTO t2 'val' || toString(number) as s FROM numbers_mt({{ table_size - 3 }}); --- SET join_algorithm = 'full_sorting_merge'; +SET join_algorithm = 'full_sorting_merge'; -{% for block_size in range (1, table_size + 1) %} +{% for block_size in range (1, table_size + 1) -%} SET max_block_size = {{ block_size }}; SELECT 'simple cases, block size = {{ block_size }}'; SELECT 'ANY INNER'; -SELECT t1.key, t2.key FROM t1 +SELECT t1.key, t2.key, empty(t1.s), empty(t2.s) FROM t1 ANY INNER JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key ; SELECT 'ANY LEFT'; -SELECT t1.key, t2.key, t1.s FROM t1 +SELECT t1.key, t2.key, t1.s, empty(t2.s) FROM t1 ANY LEFT JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t1.s ; SELECT 'ANY RIGHT'; -SELECT t1.key, t2.key, t2.s FROM t1 +SELECT t1.key, t2.key, empty(t1.s), t2.s FROM t1 ANY RIGHT JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t2.s ; -{% endfor %} +{% endfor -%} SET join_use_nulls = 1; SELECT 'simple cases: join_use_nulls'; SELECT 'ANY INNER'; -SELECT t1.key, t2.key FROM t1 +SELECT t1.key, t2.key, isNull(t1.s), isNull(t2.s) FROM t1 ANY INNER JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key ; SELECT 'ANY LEFT'; -SELECT t1.key, t2.key, t1.s FROM t1 +SELECT t1.key, t2.key, t1.s, isNull(t2.s) FROM t1 ANY LEFT JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t1.s ; SELECT 'ANY RIGHT'; -SELECT t1.key, t2.key, t2.s FROM t1 +SELECT t1.key, t2.key, isNull(t1.s), t2.s FROM t1 ANY RIGHT JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t2.s diff --git a/tests/queries/0_stateless/02270_full_sort_join_nodistinct.reference b/tests/queries/0_stateless/02270_full_sort_join_nodistinct.reference index 4efc5041625..f140d841375 100644 --- a/tests/queries/0_stateless/02270_full_sort_join_nodistinct.reference +++ b/tests/queries/0_stateless/02270_full_sort_join_nodistinct.reference @@ -1,260 +1,980 @@ -simple cases, block size = 1 +block size = 1 ANY INNER -1 1 -2 2 -3 3 +1 1 4 5 +2 2 5 5 +3 3 4 4 ANY LEFT -1 1 val1 -2 2 val21 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -3 3 val3 +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 ANY RIGHT -1 1 val11 -1 1 val12 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -2 2 val28 -3 3 val3 -simple cases, block size = 2 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 ANY INNER -1 1 -2 2 -3 3 +1 1 4 5 +3 3 4 4 ANY LEFT -1 1 val1 -2 2 val21 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -3 3 val3 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 ANY RIGHT -1 1 val11 -1 1 val12 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -2 2 val28 -3 3 val3 -simple cases, block size = 3 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 ANY INNER -1 1 -2 2 -3 3 +1 1 4 5 +3 3 4 4 ANY LEFT -1 1 val1 -2 2 val21 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -3 3 val3 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 ANY RIGHT -1 1 val11 -1 1 val12 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -2 2 val28 -3 3 val3 -simple cases, block size = 4 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 ANY INNER -1 1 -2 2 -3 3 +1 1 4 5 +3 3 4 4 ANY LEFT -1 1 val1 -2 2 val21 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -3 3 val3 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 ANY RIGHT -1 1 val11 -1 1 val12 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -2 2 val28 -3 3 val3 -simple cases, block size = 5 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +block size = 2 ANY INNER -1 1 -2 2 -3 3 +1 1 4 5 +2 2 5 5 +3 3 4 4 ANY LEFT -1 1 val1 -2 2 val21 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -3 3 val3 +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 ANY RIGHT -1 1 val11 -1 1 val12 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -2 2 val28 -3 3 val3 -simple cases, block size = 6 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 ANY INNER -1 1 -2 2 -3 3 +1 1 4 5 +3 3 4 4 ANY LEFT -1 1 val1 -2 2 val21 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -3 3 val3 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 ANY RIGHT -1 1 val11 -1 1 val12 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -2 2 val28 -3 3 val3 -simple cases, block size = 7 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 ANY INNER -1 1 -2 2 -3 3 +1 1 4 5 +3 3 4 4 ANY LEFT -1 1 val1 -2 2 val21 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -3 3 val3 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 ANY RIGHT -1 1 val11 -1 1 val12 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -2 2 val28 -3 3 val3 -simple cases, block size = 8 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 ANY INNER -1 1 -2 2 -3 3 +1 1 4 5 +3 3 4 4 ANY LEFT -1 1 val1 -2 2 val21 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -3 3 val3 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 ANY RIGHT -1 1 val11 -1 1 val12 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -2 2 val28 -3 3 val3 -simple cases, block size = 9 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +block size = 3 ANY INNER -1 1 -2 2 -3 3 +1 1 4 5 +2 2 5 5 +3 3 4 4 ANY LEFT -1 1 val1 -2 2 val21 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -3 3 val3 +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 ANY RIGHT -1 1 val11 -1 1 val12 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -2 2 val28 -3 3 val3 -simple cases, block size = 10 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 ANY INNER -1 1 -2 2 -3 3 +1 1 4 5 +3 3 4 4 ANY LEFT -1 1 val1 -2 2 val21 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -3 3 val3 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 ANY RIGHT -1 1 val11 -1 1 val12 -2 2 val22 -2 2 val23 -2 2 val24 -2 2 val25 -2 2 val26 -2 2 val27 -2 2 val28 -3 3 val3 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +block size = 4 +ANY INNER +1 1 4 5 +2 2 5 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +ANY RIGHT +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +block size = 5 +ANY INNER +1 1 4 5 +2 2 5 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +ANY RIGHT +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +block size = 6 +ANY INNER +1 1 4 5 +2 2 5 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +ANY RIGHT +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +block size = 7 +ANY INNER +1 1 4 5 +2 2 5 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +ANY RIGHT +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +block size = 8 +ANY INNER +1 1 4 5 +2 2 5 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +ANY RIGHT +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +block size = 9 +ANY INNER +1 1 4 5 +2 2 5 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +ANY RIGHT +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +block size = 10 +ANY INNER +1 1 4 5 +2 2 5 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +ANY RIGHT +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +ANY INNER +1 1 4 5 +3 3 4 4 +ANY LEFT +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +ANY RIGHT +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 diff --git a/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 index d2124e0e604..78661978d3d 100644 --- a/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 @@ -2,40 +2,35 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; CREATE TABLE t1 (key UInt32, s String) engine = TinyLog; +CREATE TABLE tn1 (key Nullable(UInt32), s String) engine = TinyLog; CREATE TABLE t2 (key UInt32, s String) engine = TinyLog; +CREATE TABLE tn2 (key Nullable(UInt32), s String) engine = TinyLog; {% set table_size = 10 %} INSERT INTO t1 VALUES (1, 'val1'), (2, 'val21'), (2, 'val22'), (2, 'val23'), (2, 'val24'), (2, 'val25'), (2, 'val26'), (2, 'val27'), (3, 'val3'); +INSERT INTO tn1 VALUES (1, 'val1'), (NULL, 'val21'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (3, 'val3'); INSERT INTO t2 VALUES (1, 'val11'), (1, 'val12'), (2, 'val22'), (2, 'val23'), (2, 'val24'), (2, 'val25'), (2, 'val26'), (2, 'val27'), (2, 'val28'), (3, 'val3'); +INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (NULL, 'val28'), (3, 'val3'); --- SET join_algorithm = 'full_sorting_merge'; +SET join_algorithm = 'full_sorting_merge'; -{% for block_size in range (1, table_size + 1) %} +{% for block_size in range (1, table_size + 1) -%} SET max_block_size = {{ block_size }}; -SELECT 'simple cases, block size = {{ block_size }}'; +SELECT 'block size = {{ block_size }}'; + +{% for t1, t2 in [('t1', 't2'), ('t1', 'tn2'), ('tn1', 't2'), ('tn1', 'tn2')] -%} SELECT 'ANY INNER'; -SELECT t1.key, t2.key FROM t1 -ANY INNER JOIN t2 -ON t1.key == t2.key -ORDER BY t1.key, t2.key -; +SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 ANY INNER JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key; SELECT 'ANY LEFT'; -SELECT t1.key, t2.key, t1.s FROM t1 -ANY LEFT JOIN t2 -ON t1.key == t2.key -ORDER BY t1.key, t2.key, t1.s -; +SELECT t1.key, t2.key, t1.s, length(t2.s) FROM {{ t1 }} AS t1 ANY LEFT JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t1.s; SELECT 'ANY RIGHT'; -SELECT t1.key, t2.key, t2.s FROM t1 -ANY RIGHT JOIN t2 -ON t1.key == t2.key -ORDER BY t1.key, t2.key, t2.s -; +SELECT t1.key, t2.key, length(t1.s), t2.s FROM {{ t1 }} AS t1 ANY RIGHT JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t2.s; -{% endfor %} +{% endfor -%} +{% endfor -%} diff --git a/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 index 6e790cb6dc7..2a986ea64d8 100644 --- a/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 @@ -21,7 +21,7 @@ INSERT INTO t2 FROM numbers_mt({{ rtable_size }}) ; --- SET join_algorithm = 'full_sorting_merge'; +SET join_algorithm = 'full_sorting_merge'; {% for block_size in [32001, 65505, 65536, range(32001, 65536) | random] %} From 94192a23fced4ab79bb76dc5781a0cc40ccfb84a Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 15 Apr 2022 10:01:08 +0000 Subject: [PATCH 290/627] enable total compare in MergeJoinAlgorithm --- src/Processors/Transforms/MergeJoinTransform.cpp | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 0d0adca18f6..51e511c27a3 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -121,14 +121,10 @@ int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImp return compareCursors(lhs, lhs.getRow(), rhs, rhs.getRow()); } - bool ALWAYS_INLINE totallyLess(SortCursorImpl & lhs, SortCursorImpl & rhs) { - if (!lhs.isValid() || !rhs.isValid()) - return false; - /// The last row of this cursor is no larger than the first row of the another cursor. - int cmp = compareCursors(lhs, lhs.rows - 1, rhs, 0); + int cmp = compareCursors(lhs, lhs.rows - 1, rhs, rhs.getRow()); return cmp < 0; } @@ -565,8 +561,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::mergeImpl() return Status({}, true); } - - if (int cmp = totallyCompare(cursors[0]->getCurrent().cursor, cursors[1]->getCurrent().cursor); cmp == 6666) + if (int cmp = totallyCompare(cursors[0]->getCurrent().cursor, cursors[1]->getCurrent().cursor); cmp != 0) { if (cmp < 0) { From 074a9706fcce7cb93b731176e1e2e66b62a31472 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 25 Apr 2022 12:44:38 +0000 Subject: [PATCH 291/627] rename tests full_sort_join --- ...69_full_sort_join.reference => 02273_full_sort_join.reference} | 0 .../{02269_full_sort_join.sql.j2 => 02273_full_sort_join.sql.j2} | 0 ...stinct.reference => 02274_full_sort_join_nodistinct.reference} | 0 ...n_nodistinct.sql.j2 => 02274_full_sort_join_nodistinct.sql.j2} | 0 ...rt_join_long.reference => 02275_full_sort_join_long.reference} | 0 ...ull_sort_join_long.sql.j2 => 02275_full_sort_join_long.sql.j2} | 0 6 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02269_full_sort_join.reference => 02273_full_sort_join.reference} (100%) rename tests/queries/0_stateless/{02269_full_sort_join.sql.j2 => 02273_full_sort_join.sql.j2} (100%) rename tests/queries/0_stateless/{02270_full_sort_join_nodistinct.reference => 02274_full_sort_join_nodistinct.reference} (100%) rename tests/queries/0_stateless/{02270_full_sort_join_nodistinct.sql.j2 => 02274_full_sort_join_nodistinct.sql.j2} (100%) rename tests/queries/0_stateless/{02271_full_sort_join_long.reference => 02275_full_sort_join_long.reference} (100%) rename tests/queries/0_stateless/{02271_full_sort_join_long.sql.j2 => 02275_full_sort_join_long.sql.j2} (100%) diff --git a/tests/queries/0_stateless/02269_full_sort_join.reference b/tests/queries/0_stateless/02273_full_sort_join.reference similarity index 100% rename from tests/queries/0_stateless/02269_full_sort_join.reference rename to tests/queries/0_stateless/02273_full_sort_join.reference diff --git a/tests/queries/0_stateless/02269_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 similarity index 100% rename from tests/queries/0_stateless/02269_full_sort_join.sql.j2 rename to tests/queries/0_stateless/02273_full_sort_join.sql.j2 diff --git a/tests/queries/0_stateless/02270_full_sort_join_nodistinct.reference b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference similarity index 100% rename from tests/queries/0_stateless/02270_full_sort_join_nodistinct.reference rename to tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference diff --git a/tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 similarity index 100% rename from tests/queries/0_stateless/02270_full_sort_join_nodistinct.sql.j2 rename to tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 diff --git a/tests/queries/0_stateless/02271_full_sort_join_long.reference b/tests/queries/0_stateless/02275_full_sort_join_long.reference similarity index 100% rename from tests/queries/0_stateless/02271_full_sort_join_long.reference rename to tests/queries/0_stateless/02275_full_sort_join_long.reference diff --git a/tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 similarity index 100% rename from tests/queries/0_stateless/02271_full_sort_join_long.sql.j2 rename to tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 From 4e88e8f5ec1cbb793b9d57d337b23c0494c15606 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 25 Apr 2022 21:29:23 +0000 Subject: [PATCH 292/627] full sort join: move block list to all join state --- src/Interpreters/TreeRewriter.cpp | 5 +- .../Transforms/MergeJoinTransform.cpp | 398 +++++++++++++----- .../Transforms/MergeJoinTransform.h | 234 ++++++---- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 +- .../02274_full_sort_join_nodistinct.sql.j2 | 2 +- 5 files changed, 455 insertions(+), 188 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 73ec7fb1d7a..15d7128e2d3 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -636,9 +636,8 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul } else { - if (table_join.strictness == ASTTableJoin::Strictness::Any) - if (table_join.kind == ASTTableJoin::Kind::Full) - throw Exception("ANY FULL JOINs are not implemented.", ErrorCodes::NOT_IMPLEMENTED); + if (table_join.strictness == ASTTableJoin::Strictness::Any && table_join.kind == ASTTableJoin::Kind::Full) + throw Exception("ANY FULL JOINs are not implemented", ErrorCodes::NOT_IMPLEMENTED); } out_table_join = table_join; diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 51e511c27a3..c2921663e9a 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -123,7 +123,7 @@ int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImp bool ALWAYS_INLINE totallyLess(SortCursorImpl & lhs, SortCursorImpl & rhs) { - /// The last row of this cursor is no larger than the first row of the another cursor. + /// The last row of left cursor is less than the current row of the right cursor. int cmp = compareCursors(lhs, lhs.rows - 1, rhs, rhs.getRow()); return cmp < 0; } @@ -159,9 +159,9 @@ void addIndexColumn(const Columns & columns, ColumnUInt64 & indices, Chunk & res } } -bool sameNext(const SortCursorImpl & impl) +bool sameNext(const SortCursorImpl & impl, std::optional pos_opt = {}) { - size_t pos = impl.getRow(); + size_t pos = pos_opt.value_or(impl.getRow()); for (size_t i = 0; i < impl.sort_columns_size; ++i) { const auto & col = *impl.sort_columns[i]; @@ -188,57 +188,50 @@ size_t nextDistinct(SortCursorImpl & impl) } -FullMergeJoinCursor::CursorWithBlock & FullMergeJoinCursor::getCurrent() +const Chunk & FullMergeJoinCursor::getCurrent() const { - while (current != inputs.end() && !current->cursor.isValid()) - current++; - - return current != inputs.end() ? *current : empty_cursor; + return current_chunk; } -void FullMergeJoinCursor::addChunk(Chunk && chunk) +Chunk FullMergeJoinCursor::detach() +{ + cursor = SortCursorImpl(); + return std::move(current_chunk); +} + +void FullMergeJoinCursor::setChunk(Chunk && chunk) { assert(!recieved_all_blocks); if (!chunk) { recieved_all_blocks = true; + detach(); return; } - dropBlocksUntilCurrent(); - inputs.emplace_back(sample_block, desc, std::move(chunk)); - - if (current == inputs.end()) - { - current = std::prev(inputs.end()); - } + current_chunk = std::move(chunk); + cursor = SortCursorImpl(sample_block, current_chunk.getColumns(), desc); } -void FullMergeJoinCursor::dropBlocksUntilCurrent() +bool FullMergeJoinCursor::fullyCompleted() const { - while (current != inputs.end() && !current->cursor.isValid()) - current++; - - inputs.erase(inputs.begin(), current); -} - -bool FullMergeJoinCursor::fullyCompleted() -{ - return !getCurrent()->isValid() && recieved_all_blocks; + return !cursor.isValid() && recieved_all_blocks; } MergeJoinAlgorithm::MergeJoinAlgorithm( JoinPtr table_join_, - const Blocks & input_headers) + const Blocks & input_headers, + size_t max_block_size_) : table_join(table_join_) + , max_block_size(max_block_size_) , log(&Poco::Logger::get("MergeJoinAlgorithm")) { if (input_headers.size() != 2) throw Exception("MergeJoinAlgorithm requires exactly two inputs", ErrorCodes::LOGICAL_ERROR); auto strictness = table_join->getTableJoin().strictness(); - if (strictness != ASTTableJoin::Strictness::Any) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for strictness != ANY"); + if (strictness != ASTTableJoin::Strictness::Any && strictness != ASTTableJoin::Strictness::All) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for strictness {}", strictness); auto kind = table_join->getTableJoin().kind(); if (!isInner(kind) && !isLeft(kind) && !isRight(kind) && !isFull(kind)) @@ -250,13 +243,13 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( cursors.push_back(createCursor(input_headers[1], join_on.key_names_right)); } - -static ColumnPtr replicateRow(const IColumn & column, size_t num) +static ColumnPtr replicateRow(const IColumn & column, size_t pos, size_t num) { - assert(column.size() == 1); MutableColumnPtr res = column.cloneEmpty(); - res->insertManyFrom(column, 0, num); - return std::move(res); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} replicateRow {} | {} | {}", __FILE__, __LINE__, column.dumpStructure(), pos, num); + res->insertManyFrom(column, pos, num); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} replicateRow >>> {} | {} | {}", __FILE__, __LINE__, res->dumpStructure(), pos, num); + return res; } static void copyColumnsResized(const Chunk & chunk, size_t start, size_t size, Chunk & result_chunk) @@ -266,14 +259,17 @@ static void copyColumnsResized(const Chunk & chunk, size_t start, size_t size, C { if (col->empty()) { + /// add defaults result_chunk.addColumn(col->cloneResized(size)); } else if (col->size() == 1) { - result_chunk.addColumn(replicateRow(*col, size)); + /// copy same row n times + result_chunk.addColumn(replicateRow(*col, 0, size)); } else { + /// cut column assert(start + size <= col->size()); result_chunk.addColumn(col->cut(start, size)); } @@ -288,14 +284,14 @@ static Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs, size_ return result; } -static Chunk createBlockWithDefaults(const Chunk & lhs, FullMergeJoinCursor::CursorWithBlock & rhs) +static Chunk createBlockWithDefaults(const Chunk & lhs, FullMergeJoinCursor & rhs) { size_t start = rhs->getRow(); size_t num_rows = rhs->rowsLeft(); return createBlockWithDefaults(lhs, rhs.detach(), start, num_rows); } -static Chunk createBlockWithDefaults(FullMergeJoinCursor::CursorWithBlock & lhs, const Chunk & rhs) +static Chunk createBlockWithDefaults(FullMergeJoinCursor & lhs, const Chunk & rhs) { size_t start = lhs->getRow(); size_t num_rows = lhs->rowsLeft(); @@ -349,7 +345,9 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) } prepareChunk(input.chunk); - cursors[source_num]->addChunk(std::move(input.chunk)); + cursors[source_num]->setChunk(std::move(input.chunk)); + + // stat.max_blocks_loaded = std::max(stat.max_blocks_loaded, std::max(cursors[0]->blocksCount(), cursors[1]->blocksCount())); } static Chunk getRowFromChunk(const Chunk & chunk, size_t pos) @@ -359,11 +357,192 @@ static Chunk getRowFromChunk(const Chunk & chunk, size_t pos) return result; } +static void ALWAYS_INLINE addRange(PaddedPODArray & left_map, size_t start, size_t end) +{ + assert(end > start); + for (size_t i = start; i < end; ++i) + left_map.push_back(i); +} + +static void ALWAYS_INLINE addMany(PaddedPODArray & left_map, size_t idx, size_t num) +{ + for (size_t i = 0; i < num; ++i) + left_map.push_back(idx); +} + template -static std::unique_ptr anyJoinImpl(FullMergeJoinCursor::CursorWithBlock & left_cursor, - FullMergeJoinCursor::CursorWithBlock & right_cursor, - PaddedPODArray & left_map, - PaddedPODArray & right_map) +struct AllJoinImpl +{ + constexpr static bool enabled = isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind); + + static void join(FullMergeJoinCursor & left_cursor, + FullMergeJoinCursor & right_cursor, + size_t max_block_size, + PaddedPODArray & left_map, + PaddedPODArray & right_map, + std::unique_ptr & state) + { + size_t num_rows = isLeft(kind) ? left_cursor->rowsLeft() : + isRight(kind) ? right_cursor->rowsLeft() : + std::min(left_cursor->rowsLeft(), right_cursor->rowsLeft()); + + if constexpr (isLeft(kind) || isInner(kind)) + right_map.reserve(num_rows); + + if constexpr (isRight(kind) || isInner(kind)) + left_map.reserve(num_rows); + + size_t rpos = std::numeric_limits::max(); + size_t lpos = std::numeric_limits::max(); + int cmp = 0; + assert(left_cursor->isValid() && right_cursor->isValid()); + while (left_cursor->isValid() && right_cursor->isValid()) + { + lpos = left_cursor->getRow(); + rpos = right_cursor->getRow(); + + cmp = compareCursors(left_cursor.cursor, right_cursor.cursor); + if (cmp == 0) + { + size_t lnum = nextDistinct(left_cursor.cursor); + size_t rnum = nextDistinct(right_cursor.cursor); + + bool all_fit_in_block = std::max(left_map.size(), right_map.size()) + lnum * rnum <= max_block_size; + bool have_all_ranges = left_cursor.cursor.isValid() && right_cursor.cursor.isValid(); + if (all_fit_in_block && have_all_ranges) + { + /// fast path if all joined rows fit in one block + for (size_t i = 0; i < rnum; ++i) + { + addRange(left_map, lpos, left_cursor.cursor.getRow()); + addMany(right_map, rpos + i, lnum); + } + } + else + { + state = std::make_unique(left_cursor.cursor, lpos, right_cursor.cursor, rpos); + state->addRange(0, left_cursor.getCurrent().clone(), lpos, lnum); + state->addRange(1, right_cursor.getCurrent().clone(), rpos, rnum); + return; + } + } + else if (cmp < 0) + { + size_t num = nextDistinct(left_cursor.cursor); + right_map.resize_fill(right_map.size() + num, right_cursor->rows); + for (size_t i = lpos; i < left_cursor->getRow(); ++i) + left_map.push_back(i); + } + else + { + size_t num = nextDistinct(right_cursor.cursor); + left_map.resize_fill(left_map.size() + num, left_cursor->rows); + for (size_t i = rpos; i < right_cursor->getRow(); ++i) + right_map.push_back(i); + } + } + } +}; + +template class Impl, typename ... Args> +void dispatchKind(JoinKind kind, Args && ... args) +{ + if (Impl::enabled && kind == JoinKind::Inner) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Left) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Right) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Full) + return Impl::join(std::forward(args)...); + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); + + __builtin_unreachable(); +} + +MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) +{ + if (all_join_state && all_join_state->finished()) + { + all_join_state.reset(); + } + + if (all_join_state) + { + /// Accumulate blocks with same key in all_join_state + for (size_t i = 0; i < cursors.size(); ++i) + { + if (cursors[i]->cursor.isValid() && all_join_state->left_key.equals(cursors[i]->cursor)) + { + size_t pos = cursors[i]->cursor.getRow(); + size_t num = nextDistinct(cursors[i]->cursor); + all_join_state->addRange(i, cursors[i]->getCurrent().clone(), pos, num); + } + } + + for (size_t i = 0; i < cursors.size(); ++i) + { + if (!cursors[i]->cursor.isValid() && !cursors[i]->fullyCompleted()) + { + return Status(i); + } + } + /// If current position is valid, then we've found new key, can join accumulated data + + /// join all rows with current key + MutableColumns result_cols; + for (size_t i = 0; i < 2; ++i) + { + for (const auto & col : sample_chunks[i].getColumns()) + result_cols.push_back(col->cloneEmpty()); + } + + size_t total_rows = 0; + while (total_rows < max_block_size) + { + const auto & left_range = all_join_state->getLeft(); + const auto & right_range = all_join_state->getRight(); + + total_rows += left_range.length; + + size_t i = 0; + /// Copy left block + for (const auto & col : left_range.chunk.getColumns()) + result_cols[i++]->insertRangeFrom(*col, left_range.begin, left_range.length); + /// And replicate current right column + for (const auto & col : right_range.chunk.getColumns()) + result_cols[i++]->insertManyFrom(*col, right_range.current, left_range.length); + bool valid = all_join_state->next(); + if (!valid) + break; + } + + if (total_rows) + return Status(Chunk(std::move(result_cols), total_rows)); + } + + auto left_map = ColumnUInt64::create(); + auto right_map = ColumnUInt64::create(); + size_t prev_pos[] = {cursors[0]->cursor.getRow(), cursors[1]->cursor.getRow()}; + + dispatchKind(kind, *cursors[0], *cursors[1], max_block_size, left_map->getData(), right_map->getData(), all_join_state); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} left_map->size({}) == right_map->size({})", + __FILE__, __LINE__, left_map->size(), right_map->size()); + assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); + size_t num_result_rows = std::max(left_map->size(), right_map->size()); + + Chunk result; + addIndexColumn(cursors[0]->getCurrent().getColumns(), *left_map, result, prev_pos[0], num_result_rows); + addIndexColumn(cursors[1]->getCurrent().getColumns(), *right_map, result, prev_pos[1], num_result_rows); + return Status(std::move(result)); +} + + +template +static void anyJoinImpl(FullMergeJoinCursor & left_cursor, FullMergeJoinCursor & right_cursor, + PaddedPODArray & left_map, PaddedPODArray & right_map, + AnyJoinState & state) { static_assert(kind == JoinKind::Left || kind == JoinKind::Right || kind == JoinKind::Inner, "Invalid join kind"); @@ -379,8 +558,8 @@ static std::unique_ptr anyJoinImpl(FullMergeJoinCursor::CursorWith size_t rpos = std::numeric_limits::max(); size_t lpos = std::numeric_limits::max(); - int cmp = 0; assert(left_cursor->isValid() && right_cursor->isValid()); + int cmp = 0; while (left_cursor->isValid() && right_cursor->isValid()) { lpos = left_cursor->getRow(); @@ -423,41 +602,42 @@ static std::unique_ptr anyJoinImpl(FullMergeJoinCursor::CursorWith } } - std::unique_ptr result = std::make_unique(); + /// Remember index of last joined row to propagate it to next block + state.setValue({}); if (!left_cursor->isValid()) { - Chunk value = cmp == 0 ? getRowFromChunk(right_cursor.input, rpos): Chunk{}; - result->setLeft(left_cursor.cursor, lpos, std::move(value)); - + state.set(0, left_cursor.cursor); + if (cmp == 0 && isLeft(kind)) + state.setValue(getRowFromChunk(right_cursor.getCurrent(), rpos)); } if (!right_cursor->isValid()) { - Chunk value = cmp == 0 ? getRowFromChunk(left_cursor.input, lpos): Chunk{}; - result->setRight(right_cursor.cursor, rpos, std::move(value)); + state.set(1, right_cursor.cursor); + if (cmp == 0 && isRight(kind)) + state.setValue(getRowFromChunk(left_cursor.getCurrent(), lpos)); } - - return result; } -static std::unique_ptr anyJoinDispatch(JoinKind kind, - FullMergeJoinCursor::CursorWithBlock & left_cursor, - FullMergeJoinCursor::CursorWithBlock & right_cursor, +static void anyJoinDispatch(JoinKind kind, + FullMergeJoinCursor & left_cursor, + FullMergeJoinCursor & right_cursor, PaddedPODArray & left_map, - PaddedPODArray & right_map) + PaddedPODArray & right_map, + AnyJoinState & state) { if (isInner(kind)) { - return anyJoinImpl(left_cursor, right_cursor, left_map, right_map); + return anyJoinImpl(left_cursor, right_cursor, left_map, right_map, state); } else if (isLeft(kind)) { - return anyJoinImpl(left_cursor, right_cursor, left_map, right_map); + return anyJoinImpl(left_cursor, right_cursor, left_map, right_map, state); } else if (isRight(kind)) { - return anyJoinImpl(left_cursor, right_cursor, left_map, right_map); + return anyJoinImpl(left_cursor, right_cursor, left_map, right_map, state); } else { @@ -466,68 +646,68 @@ static std::unique_ptr anyJoinDispatch(JoinKind kind, __builtin_unreachable(); } -static std::pair handleAnyJoinState(std::unique_ptr & state, FullMergeJoinCursor::CursorWithBlock & current) +static bool handleAnyJoinState(JoinKind kind, + size_t source_num, + AnyJoinState & state, + FullMergeJoinCursor & current, + const Chunk & sample_chunk, + Chunk & result) { - if (!state) - return {}; - - if (state->equals(current.cursor)) + bool has_more = state.keys[source_num].equals(current.cursor); + if (has_more) { size_t start_pos = current->getRow(); - size_t num = nextDistinct(current.cursor); + size_t length = nextDistinct(current.cursor); + if (length && isLeft(kind) && source_num == 0) + result = createBlockWithDefaults(current.getCurrent(), state.value ? state.value : sample_chunk, start_pos, length); + if (length && isRight(kind) && source_num == 1) + result = createBlockWithDefaults(state.value ? state.value : sample_chunk, current.getCurrent(), start_pos, length); - /// We've found row with other key, no need to skip any rows with current key. + /// We've found row with other key, no need to skip more rows with current key if (current->isValid()) - state.reset(); - - return std::make_pair(start_pos, num); + has_more = false; } - else - { - state.reset(); - } - return {}; + return !has_more; } MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) { - if (any_join_state) + if (!any_join_state.empty()) { - auto & left_current = cursors[0]->getCurrent(); - Chunk right_chunk = (any_join_state->left && any_join_state->left->value) ? any_join_state->left->value.clone() : sample_chunks[1].clone(); - if (auto [start, length] = handleAnyJoinState(any_join_state->left, left_current); length > 0 && isLeft(kind)) - { - return Status(createBlockWithDefaults(left_current.input, right_chunk, start, length)); - } + Chunk result; - auto & right_current = cursors[1]->getCurrent(); - Chunk left_chunk = (any_join_state->right && any_join_state->right->value) ? any_join_state->right->value.clone() : sample_chunks[0].clone(); - if (auto [start, length] = handleAnyJoinState(any_join_state->right, right_current); length > 0 && isRight(kind)) - { - return Status(createBlockWithDefaults(left_chunk, right_current.input, start, length)); - } + bool left_finished = handleAnyJoinState(kind, 0, any_join_state, *cursors[0], sample_chunks[1], result); + if (left_finished) + any_join_state.keys[0].reset(); + + bool right_finished = handleAnyJoinState(kind, 1, any_join_state, *cursors[1], sample_chunks[0], result); + if (right_finished) + any_join_state.keys[1].reset(); + + if (result) + return Status(std::move(result)); } - auto & current_left = cursors[0]->getCurrent(); - if (!current_left->isValid()) + auto & current_left = cursors[0]->cursor; + if (!current_left.isValid()) return Status(0); - auto & current_right = cursors[1]->getCurrent(); - if (!current_right->isValid()) + auto & current_right = cursors[1]->cursor; + if (!current_right.isValid()) return Status(1); auto left_map = ColumnUInt64::create(); auto right_map = ColumnUInt64::create(); - size_t prev_pos[] = {current_left->getRow(), current_right->getRow()}; + size_t prev_pos[] = {current_left.getRow(), current_right.getRow()}; - any_join_state = anyJoinDispatch(kind, current_left, current_right, left_map->getData(), right_map->getData()); + anyJoinDispatch(kind, *cursors[0], *cursors[1], left_map->getData(), right_map->getData(), any_join_state); assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); size_t num_result_rows = std::max(left_map->size(), right_map->size()); Chunk result; - addIndexColumn(current_left.input.getColumns(), *left_map, result, prev_pos[0], num_result_rows); - addIndexColumn(current_right.input.getColumns(), *right_map, result, prev_pos[1], num_result_rows); + addIndexColumn(cursors[0]->getCurrent().getColumns(), *left_map, result, prev_pos[0], num_result_rows); + addIndexColumn(cursors[1]->getCurrent().getColumns(), *right_map, result, prev_pos[1], num_result_rows); return Status(std::move(result)); } @@ -537,45 +717,39 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::mergeImpl() auto strictness = table_join->getTableJoin().strictness(); LOG_DEBUG(log, "XXXX: merge, {} {}", kind, strictness); - if (required_input.has_value()) - { - size_t r = required_input.value(); - required_input = {}; - return Status(r); - } - if (!cursors[0]->getCurrent()->isValid() && !cursors[0]->fullyCompleted()) + if (!cursors[0]->cursor.isValid() && !cursors[0]->fullyCompleted()) return Status(0); - if (!cursors[1]->getCurrent()->isValid() && !cursors[1]->fullyCompleted()) + if (!cursors[1]->cursor.isValid() && !cursors[1]->fullyCompleted()) return Status(1); if (cursors[0]->fullyCompleted() || cursors[1]->fullyCompleted()) { if (!cursors[0]->fullyCompleted() && isLeftOrFull(kind)) - return Status(createBlockWithDefaults(cursors[0]->getCurrent(), sample_chunks[1])); + return Status(createBlockWithDefaults(*cursors[0], sample_chunks[1])); if (!cursors[1]->fullyCompleted() && isRightOrFull(kind)) - return Status(createBlockWithDefaults(sample_chunks[0], cursors[1]->getCurrent())); + return Status(createBlockWithDefaults(sample_chunks[0], *cursors[1])); return Status({}, true); } - if (int cmp = totallyCompare(cursors[0]->getCurrent().cursor, cursors[1]->getCurrent().cursor); cmp != 0) + if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor); cmp != 0) { if (cmp < 0) { if (isLeftOrFull(kind)) - return Status(createBlockWithDefaults(cursors[0]->getCurrent(), sample_chunks[1])); - cursors[0]->getCurrent().detach(); + return Status(createBlockWithDefaults(*cursors[0], sample_chunks[1])); + cursors[0]->detach(); return Status(0); } if (cmp > 0) { if (isRightOrFull(kind)) - return Status(createBlockWithDefaults(sample_chunks[0], cursors[1]->getCurrent())); - cursors[1]->getCurrent().detach(); + return Status(createBlockWithDefaults(sample_chunks[0], *cursors[1])); + cursors[1]->detach(); return Status(1); } } @@ -583,6 +757,9 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::mergeImpl() if (strictness == ASTTableJoin::Strictness::Any) return anyJoin(kind); + if (strictness == ASTTableJoin::Strictness::All) + return allJoin(kind); + throw Exception("Unsupported strictness: " + toString(strictness), ErrorCodes::NOT_IMPLEMENTED); } @@ -590,8 +767,9 @@ MergeJoinTransform::MergeJoinTransform( JoinPtr table_join, const Blocks & input_headers, const Block & output_header, + size_t max_block_size, UInt64 limit_hint) - : IMergingTransform(input_headers, output_header, true, limit_hint, table_join, input_headers) + : IMergingTransform(input_headers, output_header, true, limit_hint, table_join, input_headers, max_block_size) , log(&Poco::Logger::get("MergeJoinTransform")) { LOG_TRACE(log, "Will use MergeJoinTransform"); diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 43bb48ab498..5283f09009b 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -39,51 +39,159 @@ class FullMergeJoinCursor; using FullMergeJoinCursorPtr = std::unique_ptr; + +/// Used instead of storing previous block +struct JoinKeyRow +{ + std::vector row; + + JoinKeyRow() = default; + + explicit JoinKeyRow(const SortCursorImpl & impl_, size_t pos) + { + row.reserve(impl_.sort_columns.size()); + for (const auto & col : impl_.sort_columns) + { + auto new_col = col->cloneEmpty(); + new_col->insertFrom(*col, pos); + row.push_back(std::move(new_col)); + } + } + + void reset() + { + row.clear(); + } + + bool equals(const SortCursorImpl & impl) const + { + if (row.empty()) + return false; + + assert(this->row.size() == impl.sort_columns_size); + for (size_t i = 0; i < impl.sort_columns_size; ++i) + { + int cmp = this->row[i]->compareAt(0, impl.getRow(), *impl.sort_columns[i], impl.desc[i].nulls_direction); + if (cmp != 0) + return false; + } + return true; + } +}; + struct AnyJoinState : boost::noncopyable { - /// Used instead of storing previous block - struct Row + AnyJoinState() = default; + + void set(size_t source_num, const SortCursorImpl & cursor) { - std::vector row_key; - Chunk value; + assert(cursor.rows); + keys[source_num] = JoinKeyRow(cursor, cursor.rows - 1); + } - explicit Row(const SortCursorImpl & impl_, size_t pos, Chunk value_) - : value(std::move(value_)) + void setValue(Chunk value_) + { + value = std::move(value_); + } + + bool empty() const + { + return keys[0].row.empty() && keys[1].row.empty(); + } + + JoinKeyRow keys[2]; + Chunk value; +}; + +struct AllJoinState : boost::noncopyable +{ + struct Range + { + Range() = default; + + explicit Range(Chunk chunk_, size_t begin_, size_t length_) + : begin(begin_) + , length(length_) + , current(begin_) + , chunk(std::move(chunk_)) { - row_key.reserve(impl_.sort_columns.size()); - for (const auto & col : impl_.sort_columns) - { - auto new_col = col->cloneEmpty(); - new_col->insertFrom(*col, pos); - row_key.push_back(std::move(new_col)); - } + assert(length > 0 && begin + length <= chunk.getNumRows()); } - bool equals(const SortCursorImpl & impl) const - { - assert(this->row_key.size() == impl.sort_columns_size); - for (size_t i = 0; i < impl.sort_columns_size; ++i) - { - int cmp = this->row_key[i]->compareAt(0, impl.getRow(), *impl.sort_columns[i], impl.desc[i].nulls_direction); - if (cmp != 0) - return false; - } - return true; - } + size_t begin; + size_t length; + + size_t current; + Chunk chunk; }; - void setLeft(const SortCursorImpl & impl_, size_t pos, Chunk value) + AllJoinState(const SortCursorImpl & lcursor, size_t lpos, + const SortCursorImpl & rcursor, size_t rpos) + : left_key(lcursor, lpos) + , right_key(rcursor, rpos) { - left = std::make_unique(impl_, pos, std::move(value)); } - void setRight(const SortCursorImpl & impl_, size_t pos, Chunk value) + void addRange(size_t source_num, Chunk chunk, size_t begin, size_t length) { - right = std::make_unique(impl_, pos, std::move(value)); + if (source_num == 0) + left.emplace_back(std::move(chunk), begin, length); + else + right.emplace_back(std::move(chunk), begin, length); } - std::unique_ptr left; - std::unique_ptr right; + bool next() + { + assert(!left.empty() && !right.empty()); + + if (finished()) + return false; + + bool has_next_right = nextRight(); + if (has_next_right) + return true; + + return nextLeft(); + } + + bool finished() const { return lidx >= left.size(); } + + bool nextLeft() + { + lidx += 1; + return lidx < left.size(); + } + + bool nextRight() + { + /// cycle through right rows + right[ridx].current += 1; + if (right[ridx].current >= right[ridx].begin + right[ridx].length) + { + /// reset current row index to the beginning, because range will be accessed again + right[ridx].current = right[ridx].begin; + ridx += 1; + if (ridx >= right.size()) + { + ridx = 0; + return false; + } + } + return true; + } + + const Range & getLeft() const { return left[lidx]; } + const Range & getRight() const { return right[ridx]; } + + std::vector left; + std::vector right; + + /// Left and right types can be different because of nullable + JoinKeyRow left_key; + JoinKeyRow right_key; + + size_t lidx = 0; + size_t ridx = 0; }; /* @@ -93,53 +201,27 @@ struct AnyJoinState : boost::noncopyable class FullMergeJoinCursor : boost::noncopyable { public: - struct CursorWithBlock : boost::noncopyable - { - CursorWithBlock() = default; - - CursorWithBlock(const Block & header, const SortDescription & desc_, Chunk && chunk) - : input(std::move(chunk)) - , cursor(header, input.getColumns(), desc_) - { - } - - Chunk detach() - { - cursor = SortCursorImpl(); - return std::move(input); - } - - SortCursorImpl * operator-> () { return &cursor; } - const SortCursorImpl * operator-> () const { return &cursor; } - - Chunk input; - SortCursorImpl cursor; - }; - - using CursorList = std::list; - using CursorListIt = CursorList::iterator; - explicit FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_) : sample_block(sample_block_.cloneEmpty()) , desc(description_) - , current(inputs.end()) { } - bool fullyCompleted(); - void addChunk(Chunk && chunk); - CursorWithBlock & getCurrent(); + bool fullyCompleted() const; + void setChunk(Chunk && chunk); + const Chunk & getCurrent() const; + Chunk detach(); + + SortCursorImpl * operator-> () { return &cursor; } + const SortCursorImpl * operator-> () const { return &cursor; } + + SortCursorImpl cursor; private: - void dropBlocksUntilCurrent(); - Block sample_block; SortDescription desc; - CursorList inputs; - CursorListIt current; - CursorWithBlock empty_cursor; - + Chunk current_chunk; bool recieved_all_blocks = false; }; @@ -150,7 +232,7 @@ private: class MergeJoinAlgorithm final : public IMergingAlgorithm { public: - explicit MergeJoinAlgorithm(JoinPtr table_join, const Blocks & input_headers); + explicit MergeJoinAlgorithm(JoinPtr table_join, const Blocks & input_headers, size_t max_block_size_); virtual void initialize(Inputs inputs) override; virtual void consume(Input & input, size_t source_num) override; @@ -164,27 +246,36 @@ public: void onFinish(double seconds) { - LOG_TRACE(log, "Finished pocessing in {} seconds - left: {} blocks, {} rows; right: {} blocks, {} rows", - seconds, stat.num_blocks[0], stat.num_rows[0], stat.num_blocks[1], stat.num_rows[1]); + LOG_TRACE(log, + "Finished pocessing in {} seconds" + ", left: {} blocks, {} rows; right: {} blocks, {} rows" + ", max blocks loaded to memory: {}", + seconds, stat.num_blocks[0], stat.num_rows[0], stat.num_blocks[1], stat.num_rows[1], + stat.max_blocks_loaded); } private: Status mergeImpl(); Status anyJoin(ASTTableJoin::Kind kind); + Status allJoin(ASTTableJoin::Kind kind); std::vector cursors; std::vector sample_chunks; - std::optional required_input = std::nullopt; - std::unique_ptr any_join_state; + AnyJoinState any_join_state; + std::unique_ptr all_join_state; JoinPtr table_join; + size_t max_block_size; + struct Statistic { size_t num_blocks[2] = {0, 0}; size_t num_rows[2] = {0, 0}; + + size_t max_blocks_loaded = 0; }; Statistic stat; @@ -198,6 +289,7 @@ public: JoinPtr table_join, const Blocks & input_headers, const Block & output_header, + size_t max_block_size, UInt64 limit_hint = 0); String getName() const override { return "MergeJoinTransform"; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index e127c2823ab..45f36264559 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -336,8 +336,6 @@ std::unique_ptr QueryPipelineBuilder::joinPipelines2( size_t max_block_size, Processors * collected_processors) { - UNUSED(max_block_size); - left->checkInitializedAndNotCompleted(); right->checkInitializedAndNotCompleted(); @@ -349,7 +347,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelines2( Blocks inputs = {left->getHeader(), right->getHeader()}; - auto joining = std::make_shared(join, inputs, out_header); + auto joining = std::make_shared(join, inputs, out_header, max_block_size); auto result = mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); return result; diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index 78661978d3d..bb54f49b754 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -15,7 +15,7 @@ INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val2 SET join_algorithm = 'full_sorting_merge'; -{% for block_size in range (1, table_size + 1) -%} +{% for block_size in range (1, table_size + 1) -%} SET max_block_size = {{ block_size }}; From a0144e115d80712927c0be33d9e075ccafe3575b Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 26 Apr 2022 13:51:33 +0000 Subject: [PATCH 293/627] full sorting all join --- .../Transforms/MergeJoinTransform.cpp | 318 +- .../Transforms/MergeJoinTransform.h | 38 +- .../02273_full_sort_join.reference | 736 ++++- .../0_stateless/02273_full_sort_join.sql.j2 | 35 +- .../02274_full_sort_join_nodistinct.reference | 2610 ++++++++++++++++- .../02274_full_sort_join_nodistinct.sql.j2 | 22 +- .../02275_full_sort_join_long.reference | 24 + .../02275_full_sort_join_long.sql.j2 | 19 +- 8 files changed, 3387 insertions(+), 415 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index c2921663e9a..806372cd90a 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -5,22 +5,21 @@ #include #include #include -#include + +#include #include -#include -#include -#include +#include #include + +#include #include #include -#include #include +#include +#include +#include #include -#include -#include -#include -#include "Common/Exception.h" -#include "Core/SettingsEnums.h" +#include namespace DB @@ -29,14 +28,11 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; - extern const int TOO_MANY_ROWS; extern const int LOGICAL_ERROR; } -constexpr size_t EMPTY_VALUE_IDX = std::numeric_limits::max(); using JoinKind = ASTTableJoin::Kind; - namespace { @@ -49,7 +45,6 @@ FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns) return std::make_unique(block, desc); } - template int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos, int null_direction_hint = 1) { @@ -100,16 +95,8 @@ int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, size_t lpos, { for (size_t i = 0; i < lhs.sort_columns_size; ++i) { - const auto & desc = lhs.desc[i]; - int direction = desc.direction; - int nulls_direction = desc.nulls_direction; - - int cmp = direction * nullableCompareAt( - *lhs.sort_columns[i], - *rhs.sort_columns[i], - lpos, - rpos, - nulls_direction); + /// TODO(@vdimir): use nullableCompareAt only if there's nullable columns + int cmp = nullableCompareAt(*lhs.sort_columns[i], *rhs.sort_columns[i], lpos, rpos); if (cmp != 0) return cmp; } @@ -246,9 +233,7 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( static ColumnPtr replicateRow(const IColumn & column, size_t pos, size_t num) { MutableColumnPtr res = column.cloneEmpty(); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} replicateRow {} | {} | {}", __FILE__, __LINE__, column.dumpStructure(), pos, num); res->insertManyFrom(column, pos, num); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} replicateRow >>> {} | {} | {}", __FILE__, __LINE__, res->dumpStructure(), pos, num); return res; } @@ -333,11 +318,6 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) if (input.permutation) throw DB::Exception("permutation is not supported", ErrorCodes::NOT_IMPLEMENTED); - LOG_DEBUG(log, "XXXX: consume from {} chunk: {}", source_num, input.chunk.getNumRows()); - - if (input.chunk.getNumRows() >= EMPTY_VALUE_IDX) - throw Exception("Too many rows in input", ErrorCodes::TOO_MANY_ROWS); - if (input.chunk) { stat.num_blocks[source_num] += 1; @@ -346,8 +326,6 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) prepareChunk(input.chunk); cursors[source_num]->setChunk(std::move(input.chunk)); - - // stat.max_blocks_loaded = std::max(stat.max_blocks_loaded, std::max(cursors[0]->blocksCount(), cursors[1]->blocksCount())); } static Chunk getRowFromChunk(const Chunk & chunk, size_t pos) @@ -429,16 +407,22 @@ struct AllJoinImpl else if (cmp < 0) { size_t num = nextDistinct(left_cursor.cursor); - right_map.resize_fill(right_map.size() + num, right_cursor->rows); - for (size_t i = lpos; i < left_cursor->getRow(); ++i) - left_map.push_back(i); + if constexpr (isLeft(kind)) + { + right_map.resize_fill(right_map.size() + num, right_cursor->rows); + for (size_t i = lpos; i < left_cursor->getRow(); ++i) + left_map.push_back(i); + } } else { size_t num = nextDistinct(right_cursor.cursor); - left_map.resize_fill(left_map.size() + num, left_cursor->rows); - for (size_t i = rpos; i < right_cursor->getRow(); ++i) - right_map.push_back(i); + if constexpr (isRight(kind)) + { + left_map.resize_fill(left_map.size() + num, left_cursor->rows); + for (size_t i = rpos; i < right_cursor->getRow(); ++i) + right_map.push_back(i); + } } } } @@ -461,7 +445,7 @@ void dispatchKind(JoinKind kind, Args && ... args) __builtin_unreachable(); } -MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) +std::optional MergeJoinAlgorithm::handleAllJoinState() { if (all_join_state && all_join_state->finished()) { @@ -473,7 +457,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) /// Accumulate blocks with same key in all_join_state for (size_t i = 0; i < cursors.size(); ++i) { - if (cursors[i]->cursor.isValid() && all_join_state->left_key.equals(cursors[i]->cursor)) + if (cursors[i]->cursor.isValid() && all_join_state->keys[i].equals(cursors[i]->cursor)) { size_t pos = cursors[i]->cursor.getRow(); size_t num = nextDistinct(cursors[i]->cursor); @@ -490,6 +474,8 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) } /// If current position is valid, then we've found new key, can join accumulated data + stat.max_blocks_loaded = std::max(stat.max_blocks_loaded, all_join_state->blocksStored()); + /// join all rows with current key MutableColumns result_cols; for (size_t i = 0; i < 2; ++i) @@ -521,14 +507,17 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) if (total_rows) return Status(Chunk(std::move(result_cols), total_rows)); } + return {}; +} +MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) +{ auto left_map = ColumnUInt64::create(); auto right_map = ColumnUInt64::create(); size_t prev_pos[] = {cursors[0]->cursor.getRow(), cursors[1]->cursor.getRow()}; dispatchKind(kind, *cursors[0], *cursors[1], max_block_size, left_map->getData(), right_map->getData(), all_join_state); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} left_map->size({}) == right_map->size({})", - __FILE__, __LINE__, left_map->size(), right_map->size()); + assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); size_t num_result_rows = std::max(left_map->size(), right_map->size()); @@ -540,153 +529,134 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) template -static void anyJoinImpl(FullMergeJoinCursor & left_cursor, FullMergeJoinCursor & right_cursor, - PaddedPODArray & left_map, PaddedPODArray & right_map, - AnyJoinState & state) +struct AnyJoinImpl { - static_assert(kind == JoinKind::Left || kind == JoinKind::Right || kind == JoinKind::Inner, "Invalid join kind"); + constexpr static bool enabled = isInner(kind) || isLeft(kind) || isRight(kind); - size_t num_rows = isLeft(kind) ? left_cursor->rowsLeft() : - isRight(kind) ? right_cursor->rowsLeft() : - std::min(left_cursor->rowsLeft(), right_cursor->rowsLeft()); - - if constexpr (isLeft(kind) || isInner(kind)) - right_map.reserve(num_rows); - - if constexpr (isRight(kind) || isInner(kind)) - left_map.reserve(num_rows); - - size_t rpos = std::numeric_limits::max(); - size_t lpos = std::numeric_limits::max(); - assert(left_cursor->isValid() && right_cursor->isValid()); - int cmp = 0; - while (left_cursor->isValid() && right_cursor->isValid()) + static void join(FullMergeJoinCursor & left_cursor, + FullMergeJoinCursor & right_cursor, + PaddedPODArray & left_map, + PaddedPODArray & right_map, + AnyJoinState & state) { - lpos = left_cursor->getRow(); - rpos = right_cursor->getRow(); + size_t num_rows = isLeft(kind) ? left_cursor->rowsLeft() : + isRight(kind) ? right_cursor->rowsLeft() : + std::min(left_cursor->rowsLeft(), right_cursor->rowsLeft()); - cmp = compareCursors(left_cursor.cursor, right_cursor.cursor); - if (cmp == 0) + if constexpr (isLeft(kind) || isInner(kind)) + right_map.reserve(num_rows); + + if constexpr (isRight(kind) || isInner(kind)) + left_map.reserve(num_rows); + + size_t rpos = std::numeric_limits::max(); + size_t lpos = std::numeric_limits::max(); + assert(left_cursor->isValid() && right_cursor->isValid()); + int cmp = 0; + while (left_cursor->isValid() && right_cursor->isValid()) { - if constexpr (isLeft(kind)) - { - size_t lnum = nextDistinct(left_cursor.cursor); - right_map.resize_fill(right_map.size() + lnum, rpos); - } + lpos = left_cursor->getRow(); + rpos = right_cursor->getRow(); - if constexpr (isRight(kind)) + cmp = compareCursors(left_cursor.cursor, right_cursor.cursor); + if (cmp == 0) { - size_t rnum = nextDistinct(right_cursor.cursor); - left_map.resize_fill(left_map.size() + rnum, lpos); - } + if constexpr (isLeft(kind)) + { + size_t lnum = nextDistinct(left_cursor.cursor); + right_map.resize_fill(right_map.size() + lnum, rpos); + } - if constexpr (isInner(kind)) + if constexpr (isRight(kind)) + { + size_t rnum = nextDistinct(right_cursor.cursor); + left_map.resize_fill(left_map.size() + rnum, lpos); + } + + if constexpr (isInner(kind)) + { + nextDistinct(left_cursor.cursor); + nextDistinct(right_cursor.cursor); + left_map.emplace_back(lpos); + right_map.emplace_back(rpos); + } + } + else if (cmp < 0) { - nextDistinct(left_cursor.cursor); - nextDistinct(right_cursor.cursor); - left_map.emplace_back(lpos); - right_map.emplace_back(rpos); + size_t num = nextDistinct(left_cursor.cursor); + if constexpr (isLeft(kind)) + right_map.resize_fill(right_map.size() + num, right_cursor->rows); + } + else + { + size_t num = nextDistinct(right_cursor.cursor); + if constexpr (isRight(kind)) + left_map.resize_fill(left_map.size() + num, left_cursor->rows); } } - else if (cmp < 0) + + /// Remember index of last joined row to propagate it to next block + + state.setValue({}); + if (!left_cursor->isValid()) { - size_t num = nextDistinct(left_cursor.cursor); - if constexpr (isLeft(kind)) - right_map.resize_fill(right_map.size() + num, right_cursor->rows); + state.set(0, left_cursor.cursor); + if (cmp == 0 && isLeft(kind)) + state.setValue(getRowFromChunk(right_cursor.getCurrent(), rpos)); + } + + if (!right_cursor->isValid()) + { + state.set(1, right_cursor.cursor); + if (cmp == 0 && isRight(kind)) + state.setValue(getRowFromChunk(left_cursor.getCurrent(), lpos)); + } + } +}; + +std::optional MergeJoinAlgorithm::handleAnyJoinState() +{ + if (any_join_state.empty()) + return {}; + + auto kind = table_join->getTableJoin().kind(); + + Chunk result; + + for (size_t source_num = 0; source_num < 2; ++source_num) + { + auto & current = *cursors[source_num]; + auto & state = any_join_state; + if (any_join_state.keys[source_num].equals(current.cursor)) + { + size_t start_pos = current->getRow(); + size_t length = nextDistinct(current.cursor); + if (length && isLeft(kind) && source_num == 0) + result = createBlockWithDefaults(current.getCurrent(), state.value ? state.value : sample_chunks[1 - source_num], start_pos, length); + if (length && isRight(kind) && source_num == 1) + result = createBlockWithDefaults(state.value ? state.value : sample_chunks[1 - source_num], current.getCurrent(), start_pos, length); + + /// We've found row with other key, no need to skip more rows with current key + if (current->isValid()) + { + state.keys[source_num].reset(); + } } else { - size_t num = nextDistinct(right_cursor.cursor); - if constexpr (isRight(kind)) - left_map.resize_fill(left_map.size() + num, left_cursor->rows); + any_join_state.keys[source_num].reset(); } } - /// Remember index of last joined row to propagate it to next block - - state.setValue({}); - if (!left_cursor->isValid()) - { - state.set(0, left_cursor.cursor); - if (cmp == 0 && isLeft(kind)) - state.setValue(getRowFromChunk(right_cursor.getCurrent(), rpos)); - } - - if (!right_cursor->isValid()) - { - state.set(1, right_cursor.cursor); - if (cmp == 0 && isRight(kind)) - state.setValue(getRowFromChunk(left_cursor.getCurrent(), lpos)); - } -} - -static void anyJoinDispatch(JoinKind kind, - FullMergeJoinCursor & left_cursor, - FullMergeJoinCursor & right_cursor, - PaddedPODArray & left_map, - PaddedPODArray & right_map, - AnyJoinState & state) -{ - if (isInner(kind)) - { - return anyJoinImpl(left_cursor, right_cursor, left_map, right_map, state); - } - else if (isLeft(kind)) - { - return anyJoinImpl(left_cursor, right_cursor, left_map, right_map, state); - } - else if (isRight(kind)) - { - return anyJoinImpl(left_cursor, right_cursor, left_map, right_map, state); - } - else - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); - } - __builtin_unreachable(); -} - -static bool handleAnyJoinState(JoinKind kind, - size_t source_num, - AnyJoinState & state, - FullMergeJoinCursor & current, - const Chunk & sample_chunk, - Chunk & result) -{ - bool has_more = state.keys[source_num].equals(current.cursor); - if (has_more) - { - size_t start_pos = current->getRow(); - size_t length = nextDistinct(current.cursor); - if (length && isLeft(kind) && source_num == 0) - result = createBlockWithDefaults(current.getCurrent(), state.value ? state.value : sample_chunk, start_pos, length); - if (length && isRight(kind) && source_num == 1) - result = createBlockWithDefaults(state.value ? state.value : sample_chunk, current.getCurrent(), start_pos, length); - - /// We've found row with other key, no need to skip more rows with current key - if (current->isValid()) - has_more = false; - } - return !has_more; + if (result) + return Status(std::move(result)); + return {}; } MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) { - if (!any_join_state.empty()) - { - Chunk result; - - bool left_finished = handleAnyJoinState(kind, 0, any_join_state, *cursors[0], sample_chunks[1], result); - if (left_finished) - any_join_state.keys[0].reset(); - - bool right_finished = handleAnyJoinState(kind, 1, any_join_state, *cursors[1], sample_chunks[0], result); - if (right_finished) - any_join_state.keys[1].reset(); - - if (result) - return Status(std::move(result)); - } + if (auto result = handleAnyJoinState()) + return std::move(*result); auto & current_left = cursors[0]->cursor; if (!current_left.isValid()) @@ -700,7 +670,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) auto right_map = ColumnUInt64::create(); size_t prev_pos[] = {current_left.getRow(), current_right.getRow()}; - anyJoinDispatch(kind, *cursors[0], *cursors[1], left_map->getData(), right_map->getData(), any_join_state); + dispatchKind(kind, *cursors[0], *cursors[1], left_map->getData(), right_map->getData(), any_join_state); assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); size_t num_result_rows = std::max(left_map->size(), right_map->size()); @@ -711,19 +681,20 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) return Status(std::move(result)); } -IMergingAlgorithm::Status MergeJoinAlgorithm::mergeImpl() +IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { auto kind = table_join->getTableJoin().kind(); auto strictness = table_join->getTableJoin().strictness(); - LOG_DEBUG(log, "XXXX: merge, {} {}", kind, strictness); - if (!cursors[0]->cursor.isValid() && !cursors[0]->fullyCompleted()) return Status(0); if (!cursors[1]->cursor.isValid() && !cursors[1]->fullyCompleted()) return Status(1); + if (auto result = handleAllJoinState()) + return std::move(*result); + if (cursors[0]->fullyCompleted() || cursors[1]->fullyCompleted()) { if (!cursors[0]->fullyCompleted() && isLeftOrFull(kind)) @@ -772,7 +743,7 @@ MergeJoinTransform::MergeJoinTransform( : IMergingTransform(input_headers, output_header, true, limit_hint, table_join, input_headers, max_block_size) , log(&Poco::Logger::get("MergeJoinTransform")) { - LOG_TRACE(log, "Will use MergeJoinTransform"); + LOG_TRACE(log, "Use MergeJoinTransform"); } void MergeJoinTransform::onFinish() @@ -780,5 +751,4 @@ void MergeJoinTransform::onFinish() algorithm.onFinish(total_stopwatch.elapsedSeconds()); } - } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 5283f09009b..eba4291f065 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -79,8 +79,9 @@ struct JoinKeyRow } }; -struct AnyJoinState : boost::noncopyable +class AnyJoinState : boost::noncopyable { +public: AnyJoinState() = default; void set(size_t source_num, const SortCursorImpl & cursor) @@ -103,8 +104,9 @@ struct AnyJoinState : boost::noncopyable Chunk value; }; -struct AllJoinState : boost::noncopyable +class AllJoinState : boost::noncopyable { +public: struct Range { Range() = default; @@ -127,8 +129,7 @@ struct AllJoinState : boost::noncopyable AllJoinState(const SortCursorImpl & lcursor, size_t lpos, const SortCursorImpl & rcursor, size_t rpos) - : left_key(lcursor, lpos) - , right_key(rcursor, rpos) + : keys{JoinKeyRow(lcursor, lpos), JoinKeyRow(rcursor, rpos)} { } @@ -156,6 +157,14 @@ struct AllJoinState : boost::noncopyable bool finished() const { return lidx >= left.size(); } + size_t blocksStored() const { return left.size() + right.size(); } + const Range & getLeft() const { return left[lidx]; } + const Range & getRight() const { return right[ridx]; } + + /// Left and right types can be different because of nullable + JoinKeyRow keys[2]; + +private: bool nextLeft() { lidx += 1; @@ -179,17 +188,9 @@ struct AllJoinState : boost::noncopyable } return true; } - - const Range & getLeft() const { return left[lidx]; } - const Range & getRight() const { return right[ridx]; } - std::vector left; std::vector right; - /// Left and right types can be different because of nullable - JoinKeyRow left_key; - JoinKeyRow right_key; - size_t lidx = 0; size_t ridx = 0; }; @@ -236,13 +237,7 @@ public: virtual void initialize(Inputs inputs) override; virtual void consume(Input & input, size_t source_num) override; - virtual Status merge() override - { - Status result = mergeImpl(); - LOG_TRACE(log, "XXXX: merge result: chunk: {}, required: {}, finished: {}", - result.chunk.getNumRows(), result.required_source, result.is_finished); - return result ; - } + virtual Status merge() override; void onFinish(double seconds) { @@ -255,9 +250,10 @@ public: } private: - Status mergeImpl(); - + std::optional handleAnyJoinState(); Status anyJoin(ASTTableJoin::Kind kind); + + std::optional handleAllJoinState(); Status allJoin(ASTTableJoin::Kind kind); std::vector cursors; diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference b/tests/queries/0_stateless/02273_full_sort_join.reference index d8680824040..360a1540b8d 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.reference +++ b/tests/queries/0_stateless/02273_full_sort_join.reference @@ -1,11 +1,12 @@ -simple cases, block size = 1 -ANY INNER +ALL INNER | bs = 1 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 1 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -21,7 +22,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 1 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -34,14 +35,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 2 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 2 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 2 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -57,7 +61,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 2 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -70,14 +74,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 3 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 3 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 3 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -93,7 +100,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 3 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -106,14 +113,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 4 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 4 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 4 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -129,7 +139,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 4 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -142,14 +152,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 5 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 5 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 5 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -165,7 +178,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 5 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -178,14 +191,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 6 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 6 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 6 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -201,7 +217,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 6 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -214,14 +230,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 7 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 7 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 7 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -237,7 +256,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 7 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -250,14 +269,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 8 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 8 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 8 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -273,7 +295,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 8 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -286,14 +308,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 9 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 9 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 9 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -309,7 +334,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 9 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -322,14 +347,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 10 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 10 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 10 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -345,7 +373,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 10 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -358,14 +386,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 11 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 11 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 11 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -381,7 +412,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 11 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -394,14 +425,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 12 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 12 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 12 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -417,7 +451,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 12 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -430,14 +464,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 13 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 13 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 13 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -453,7 +490,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 13 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -466,14 +503,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 14 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 14 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 14 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -489,7 +529,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 14 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -502,14 +542,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases, block size = 15 -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | bs = 15 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = 15 1 1 val5 0 2 0 val9 1 3 3 val10 0 @@ -525,7 +568,7 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 0 val6 1 -ANY RIGHT +ALL RIGHT | bs = 15 0 8 1 val2 0 8 1 val4 0 9 1 val5 @@ -538,14 +581,17 @@ ANY RIGHT 7 7 0 val8 13 13 0 val1 14 14 0 val7 -simple cases: join_use_nulls -ANY INNER +14 14 0 val7 +14 14 0 val7 +ALL INNER | join_use_nulls = 1 1 1 0 0 3 3 0 0 7 7 0 0 13 13 0 0 14 14 0 0 -ANY LEFT +14 14 0 0 +14 14 0 0 +ALL LEFT | join_use_nulls = 1 1 1 val5 0 2 \N val9 1 3 3 val10 0 @@ -561,7 +607,569 @@ ANY LEFT 14 14 val13 0 14 14 val4 0 15 \N val6 1 -ANY RIGHT +ALL RIGHT | join_use_nulls = 1 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +14 14 0 val7 +14 14 0 val7 +\N 8 1 val2 +\N 8 1 val4 +\N 9 1 val5 +\N 12 1 val0 +\N 12 1 val11 +\N 12 1 val3 +\N 12 1 val9 +ANY INNER | bs = 1 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 1 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 1 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 2 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 2 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 2 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 3 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 3 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 3 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 4 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 4 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 4 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 5 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 5 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 5 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 6 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 6 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 6 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 7 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 7 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 7 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 8 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 8 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 8 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 9 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 9 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 9 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 10 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 10 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 10 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 11 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 11 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 11 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 12 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 12 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 12 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 13 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 13 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 13 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 14 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 14 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 14 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | bs = 15 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = 15 +1 1 val5 0 +2 \N val9 \N +3 3 val10 0 +4 \N val12 \N +5 \N val11 \N +6 \N val7 \N +7 7 val14 0 +10 \N val3 \N +10 \N val8 \N +11 \N val0 \N +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 \N +ANY RIGHT | bs = 15 +1 1 0 val6 +3 3 0 val10 +7 7 0 val8 +13 13 0 val1 +14 14 0 val7 +\N 8 \N val2 +\N 8 \N val4 +\N 9 \N val5 +\N 12 \N val0 +\N 12 \N val11 +\N 12 \N val3 +\N 12 \N val9 +ANY INNER | join_use_nulls = 1 +1 1 0 0 +3 3 0 0 +7 7 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | join_use_nulls = 1 +1 1 val5 0 +2 \N val9 1 +3 3 val10 0 +4 \N val12 1 +5 \N val11 1 +6 \N val7 1 +7 7 val14 0 +10 \N val3 1 +10 \N val8 1 +11 \N val0 1 +13 13 val2 0 +14 14 val1 0 +14 14 val13 0 +14 14 val4 0 +15 \N val6 1 +ANY RIGHT | join_use_nulls = 1 1 1 0 val6 3 3 0 val10 7 7 0 val8 diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 9f4e5f92648..6737cede6cd 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -18,31 +18,30 @@ INSERT INTO t2 'val' || toString(number) as s FROM numbers_mt({{ table_size - 3 }}); -SET join_algorithm = 'full_sorting_merge'; +-- SET join_algorithm = 'full_sorting_merge'; -{% for block_size in range (1, table_size + 1) -%} +{% for kind in ['ALL', 'ANY'] -%} +{% for block_size in range(1, table_size + 1) -%} SET max_block_size = {{ block_size }}; -SELECT 'simple cases, block size = {{ block_size }}'; - -SELECT 'ANY INNER'; +SELECT '{{ kind }} INNER | bs = {{ block_size }}'; SELECT t1.key, t2.key, empty(t1.s), empty(t2.s) FROM t1 -ANY INNER JOIN t2 +{{ kind }} INNER JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key ; -SELECT 'ANY LEFT'; +SELECT '{{ kind }} LEFT | bs = {{ block_size }}'; SELECT t1.key, t2.key, t1.s, empty(t2.s) FROM t1 -ANY LEFT JOIN t2 +{{ kind }} LEFT JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t1.s ; -SELECT 'ANY RIGHT'; +SELECT '{{ kind }} RIGHT | bs = {{ block_size }}'; SELECT t1.key, t2.key, empty(t1.s), t2.s FROM t1 -ANY RIGHT JOIN t2 +{{ kind }} RIGHT JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t2.s ; @@ -51,25 +50,25 @@ ORDER BY t1.key, t2.key, t2.s SET join_use_nulls = 1; -SELECT 'simple cases: join_use_nulls'; - -SELECT 'ANY INNER'; +SELECT '{{ kind }} INNER | join_use_nulls = 1'; SELECT t1.key, t2.key, isNull(t1.s), isNull(t2.s) FROM t1 -ANY INNER JOIN t2 +{{ kind }} INNER JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key ; -SELECT 'ANY LEFT'; +SELECT '{{ kind }} LEFT | join_use_nulls = 1'; SELECT t1.key, t2.key, t1.s, isNull(t2.s) FROM t1 -ANY LEFT JOIN t2 +{{ kind }} LEFT JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t1.s ; -SELECT 'ANY RIGHT'; +SELECT '{{ kind }} RIGHT | join_use_nulls = 1'; SELECT t1.key, t2.key, isNull(t1.s), t2.s FROM t1 -ANY RIGHT JOIN t2 +{{ kind }} RIGHT JOIN t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t2.s ; + +{% endfor -%} diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference index f140d841375..99336d442b2 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference @@ -1,9 +1,168 @@ block size = 1 -ANY INNER +t1 ALL INNER JOIN t2 | bs = 1 +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = 1 +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = 1 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = 1 1 1 4 5 2 2 5 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN t2 | bs = 1 1 1 val1 5 2 2 val21 5 2 2 val22 5 @@ -13,7 +172,7 @@ ANY LEFT 2 2 val26 5 2 2 val27 5 3 3 val3 4 -ANY RIGHT +t1 ANY RIGHT JOIN t2 | bs = 1 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -24,10 +183,12 @@ ANY RIGHT 2 2 5 val27 2 2 5 val28 3 3 4 val3 -ANY INNER +t1 ALL INNER JOIN tn2 | bs = 1 +1 1 4 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ALL LEFT JOIN tn2 | bs = 1 +1 1 val1 5 1 1 val1 5 2 \N val21 0 2 \N val22 0 @@ -37,7 +198,7 @@ ANY LEFT 2 \N val26 0 2 \N val27 0 3 3 val3 4 -ANY RIGHT +t1 ALL RIGHT JOIN tn2 | bs = 1 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -48,10 +209,36 @@ ANY RIGHT 1 1 4 val11 1 1 4 val12 3 3 4 val3 -ANY INNER +t1 ANY INNER JOIN tn2 | bs = 1 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN tn2 | bs = 1 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = 1 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = 1 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = 1 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N 0 val21 0 @@ -61,7 +248,7 @@ ANY LEFT \N 0 val25 0 \N 0 val26 0 \N 0 val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN t2 | bs = 1 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -72,10 +259,36 @@ ANY RIGHT \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -ANY INNER +tn1 ANY INNER JOIN t2 | bs = 1 1 1 4 5 3 3 4 4 -ANY LEFT +tn1 ANY LEFT JOIN t2 | bs = 1 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = 1 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = 1 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = 1 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N \N val21 0 @@ -85,7 +298,31 @@ ANY LEFT \N \N val25 0 \N \N val26 0 \N \N val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN tn2 | bs = 1 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = 1 +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = 1 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = 1 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -97,11 +334,170 @@ ANY RIGHT \N \N 0 val27 \N \N 0 val28 block size = 2 -ANY INNER +t1 ALL INNER JOIN t2 | bs = 2 +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = 2 +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = 2 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = 2 1 1 4 5 2 2 5 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN t2 | bs = 2 1 1 val1 5 2 2 val21 5 2 2 val22 5 @@ -111,7 +507,7 @@ ANY LEFT 2 2 val26 5 2 2 val27 5 3 3 val3 4 -ANY RIGHT +t1 ANY RIGHT JOIN t2 | bs = 2 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -122,10 +518,12 @@ ANY RIGHT 2 2 5 val27 2 2 5 val28 3 3 4 val3 -ANY INNER +t1 ALL INNER JOIN tn2 | bs = 2 +1 1 4 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ALL LEFT JOIN tn2 | bs = 2 +1 1 val1 5 1 1 val1 5 2 \N val21 0 2 \N val22 0 @@ -135,7 +533,7 @@ ANY LEFT 2 \N val26 0 2 \N val27 0 3 3 val3 4 -ANY RIGHT +t1 ALL RIGHT JOIN tn2 | bs = 2 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -146,10 +544,36 @@ ANY RIGHT 1 1 4 val11 1 1 4 val12 3 3 4 val3 -ANY INNER +t1 ANY INNER JOIN tn2 | bs = 2 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN tn2 | bs = 2 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = 2 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = 2 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = 2 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N 0 val21 0 @@ -159,7 +583,7 @@ ANY LEFT \N 0 val25 0 \N 0 val26 0 \N 0 val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN t2 | bs = 2 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -170,10 +594,36 @@ ANY RIGHT \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -ANY INNER +tn1 ANY INNER JOIN t2 | bs = 2 1 1 4 5 3 3 4 4 -ANY LEFT +tn1 ANY LEFT JOIN t2 | bs = 2 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = 2 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = 2 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = 2 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N \N val21 0 @@ -183,7 +633,31 @@ ANY LEFT \N \N val25 0 \N \N val26 0 \N \N val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN tn2 | bs = 2 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = 2 +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = 2 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = 2 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -195,11 +669,170 @@ ANY RIGHT \N \N 0 val27 \N \N 0 val28 block size = 3 -ANY INNER +t1 ALL INNER JOIN t2 | bs = 3 +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = 3 +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = 3 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = 3 1 1 4 5 2 2 5 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN t2 | bs = 3 1 1 val1 5 2 2 val21 5 2 2 val22 5 @@ -209,7 +842,7 @@ ANY LEFT 2 2 val26 5 2 2 val27 5 3 3 val3 4 -ANY RIGHT +t1 ANY RIGHT JOIN t2 | bs = 3 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -220,10 +853,12 @@ ANY RIGHT 2 2 5 val27 2 2 5 val28 3 3 4 val3 -ANY INNER +t1 ALL INNER JOIN tn2 | bs = 3 +1 1 4 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ALL LEFT JOIN tn2 | bs = 3 +1 1 val1 5 1 1 val1 5 2 \N val21 0 2 \N val22 0 @@ -233,7 +868,7 @@ ANY LEFT 2 \N val26 0 2 \N val27 0 3 3 val3 4 -ANY RIGHT +t1 ALL RIGHT JOIN tn2 | bs = 3 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -244,10 +879,36 @@ ANY RIGHT 1 1 4 val11 1 1 4 val12 3 3 4 val3 -ANY INNER +t1 ANY INNER JOIN tn2 | bs = 3 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN tn2 | bs = 3 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = 3 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = 3 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = 3 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N 0 val21 0 @@ -257,7 +918,7 @@ ANY LEFT \N 0 val25 0 \N 0 val26 0 \N 0 val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN t2 | bs = 3 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -268,10 +929,36 @@ ANY RIGHT \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -ANY INNER +tn1 ANY INNER JOIN t2 | bs = 3 1 1 4 5 3 3 4 4 -ANY LEFT +tn1 ANY LEFT JOIN t2 | bs = 3 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = 3 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = 3 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = 3 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N \N val21 0 @@ -281,7 +968,31 @@ ANY LEFT \N \N val25 0 \N \N val26 0 \N \N val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN tn2 | bs = 3 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = 3 +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = 3 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = 3 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -293,11 +1004,170 @@ ANY RIGHT \N \N 0 val27 \N \N 0 val28 block size = 4 -ANY INNER +t1 ALL INNER JOIN t2 | bs = 4 +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = 4 +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = 4 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = 4 1 1 4 5 2 2 5 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN t2 | bs = 4 1 1 val1 5 2 2 val21 5 2 2 val22 5 @@ -307,7 +1177,7 @@ ANY LEFT 2 2 val26 5 2 2 val27 5 3 3 val3 4 -ANY RIGHT +t1 ANY RIGHT JOIN t2 | bs = 4 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -318,10 +1188,12 @@ ANY RIGHT 2 2 5 val27 2 2 5 val28 3 3 4 val3 -ANY INNER +t1 ALL INNER JOIN tn2 | bs = 4 +1 1 4 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ALL LEFT JOIN tn2 | bs = 4 +1 1 val1 5 1 1 val1 5 2 \N val21 0 2 \N val22 0 @@ -331,7 +1203,7 @@ ANY LEFT 2 \N val26 0 2 \N val27 0 3 3 val3 4 -ANY RIGHT +t1 ALL RIGHT JOIN tn2 | bs = 4 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -342,10 +1214,36 @@ ANY RIGHT 1 1 4 val11 1 1 4 val12 3 3 4 val3 -ANY INNER +t1 ANY INNER JOIN tn2 | bs = 4 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN tn2 | bs = 4 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = 4 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = 4 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = 4 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N 0 val21 0 @@ -355,7 +1253,7 @@ ANY LEFT \N 0 val25 0 \N 0 val26 0 \N 0 val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN t2 | bs = 4 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -366,10 +1264,36 @@ ANY RIGHT \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -ANY INNER +tn1 ANY INNER JOIN t2 | bs = 4 1 1 4 5 3 3 4 4 -ANY LEFT +tn1 ANY LEFT JOIN t2 | bs = 4 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = 4 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = 4 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = 4 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N \N val21 0 @@ -379,7 +1303,31 @@ ANY LEFT \N \N val25 0 \N \N val26 0 \N \N val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN tn2 | bs = 4 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = 4 +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = 4 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = 4 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -391,11 +1339,170 @@ ANY RIGHT \N \N 0 val27 \N \N 0 val28 block size = 5 -ANY INNER +t1 ALL INNER JOIN t2 | bs = 5 +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = 5 +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = 5 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = 5 1 1 4 5 2 2 5 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN t2 | bs = 5 1 1 val1 5 2 2 val21 5 2 2 val22 5 @@ -405,7 +1512,7 @@ ANY LEFT 2 2 val26 5 2 2 val27 5 3 3 val3 4 -ANY RIGHT +t1 ANY RIGHT JOIN t2 | bs = 5 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -416,10 +1523,12 @@ ANY RIGHT 2 2 5 val27 2 2 5 val28 3 3 4 val3 -ANY INNER +t1 ALL INNER JOIN tn2 | bs = 5 +1 1 4 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ALL LEFT JOIN tn2 | bs = 5 +1 1 val1 5 1 1 val1 5 2 \N val21 0 2 \N val22 0 @@ -429,7 +1538,7 @@ ANY LEFT 2 \N val26 0 2 \N val27 0 3 3 val3 4 -ANY RIGHT +t1 ALL RIGHT JOIN tn2 | bs = 5 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -440,10 +1549,36 @@ ANY RIGHT 1 1 4 val11 1 1 4 val12 3 3 4 val3 -ANY INNER +t1 ANY INNER JOIN tn2 | bs = 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN tn2 | bs = 5 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = 5 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = 5 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = 5 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N 0 val21 0 @@ -453,7 +1588,7 @@ ANY LEFT \N 0 val25 0 \N 0 val26 0 \N 0 val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN t2 | bs = 5 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -464,10 +1599,36 @@ ANY RIGHT \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -ANY INNER +tn1 ANY INNER JOIN t2 | bs = 5 1 1 4 5 3 3 4 4 -ANY LEFT +tn1 ANY LEFT JOIN t2 | bs = 5 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = 5 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = 5 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = 5 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N \N val21 0 @@ -477,7 +1638,31 @@ ANY LEFT \N \N val25 0 \N \N val26 0 \N \N val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN tn2 | bs = 5 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = 5 +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = 5 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = 5 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -489,11 +1674,170 @@ ANY RIGHT \N \N 0 val27 \N \N 0 val28 block size = 6 -ANY INNER +t1 ALL INNER JOIN t2 | bs = 6 +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = 6 +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = 6 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = 6 1 1 4 5 2 2 5 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN t2 | bs = 6 1 1 val1 5 2 2 val21 5 2 2 val22 5 @@ -503,7 +1847,7 @@ ANY LEFT 2 2 val26 5 2 2 val27 5 3 3 val3 4 -ANY RIGHT +t1 ANY RIGHT JOIN t2 | bs = 6 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -514,10 +1858,12 @@ ANY RIGHT 2 2 5 val27 2 2 5 val28 3 3 4 val3 -ANY INNER +t1 ALL INNER JOIN tn2 | bs = 6 +1 1 4 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ALL LEFT JOIN tn2 | bs = 6 +1 1 val1 5 1 1 val1 5 2 \N val21 0 2 \N val22 0 @@ -527,7 +1873,7 @@ ANY LEFT 2 \N val26 0 2 \N val27 0 3 3 val3 4 -ANY RIGHT +t1 ALL RIGHT JOIN tn2 | bs = 6 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -538,10 +1884,36 @@ ANY RIGHT 1 1 4 val11 1 1 4 val12 3 3 4 val3 -ANY INNER +t1 ANY INNER JOIN tn2 | bs = 6 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN tn2 | bs = 6 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = 6 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = 6 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = 6 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N 0 val21 0 @@ -551,7 +1923,7 @@ ANY LEFT \N 0 val25 0 \N 0 val26 0 \N 0 val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN t2 | bs = 6 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -562,10 +1934,36 @@ ANY RIGHT \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -ANY INNER +tn1 ANY INNER JOIN t2 | bs = 6 1 1 4 5 3 3 4 4 -ANY LEFT +tn1 ANY LEFT JOIN t2 | bs = 6 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = 6 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = 6 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = 6 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N \N val21 0 @@ -575,7 +1973,31 @@ ANY LEFT \N \N val25 0 \N \N val26 0 \N \N val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN tn2 | bs = 6 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = 6 +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = 6 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = 6 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -587,11 +2009,170 @@ ANY RIGHT \N \N 0 val27 \N \N 0 val28 block size = 7 -ANY INNER +t1 ALL INNER JOIN t2 | bs = 7 +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = 7 +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = 7 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = 7 1 1 4 5 2 2 5 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN t2 | bs = 7 1 1 val1 5 2 2 val21 5 2 2 val22 5 @@ -601,7 +2182,7 @@ ANY LEFT 2 2 val26 5 2 2 val27 5 3 3 val3 4 -ANY RIGHT +t1 ANY RIGHT JOIN t2 | bs = 7 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -612,10 +2193,12 @@ ANY RIGHT 2 2 5 val27 2 2 5 val28 3 3 4 val3 -ANY INNER +t1 ALL INNER JOIN tn2 | bs = 7 +1 1 4 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ALL LEFT JOIN tn2 | bs = 7 +1 1 val1 5 1 1 val1 5 2 \N val21 0 2 \N val22 0 @@ -625,7 +2208,7 @@ ANY LEFT 2 \N val26 0 2 \N val27 0 3 3 val3 4 -ANY RIGHT +t1 ALL RIGHT JOIN tn2 | bs = 7 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -636,10 +2219,36 @@ ANY RIGHT 1 1 4 val11 1 1 4 val12 3 3 4 val3 -ANY INNER +t1 ANY INNER JOIN tn2 | bs = 7 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN tn2 | bs = 7 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = 7 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = 7 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = 7 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N 0 val21 0 @@ -649,7 +2258,7 @@ ANY LEFT \N 0 val25 0 \N 0 val26 0 \N 0 val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN t2 | bs = 7 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -660,10 +2269,36 @@ ANY RIGHT \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -ANY INNER +tn1 ANY INNER JOIN t2 | bs = 7 1 1 4 5 3 3 4 4 -ANY LEFT +tn1 ANY LEFT JOIN t2 | bs = 7 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = 7 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = 7 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = 7 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N \N val21 0 @@ -673,7 +2308,31 @@ ANY LEFT \N \N val25 0 \N \N val26 0 \N \N val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN tn2 | bs = 7 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = 7 +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = 7 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = 7 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -685,11 +2344,170 @@ ANY RIGHT \N \N 0 val27 \N \N 0 val28 block size = 8 -ANY INNER +t1 ALL INNER JOIN t2 | bs = 8 +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = 8 +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = 8 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = 8 1 1 4 5 2 2 5 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN t2 | bs = 8 1 1 val1 5 2 2 val21 5 2 2 val22 5 @@ -699,7 +2517,7 @@ ANY LEFT 2 2 val26 5 2 2 val27 5 3 3 val3 4 -ANY RIGHT +t1 ANY RIGHT JOIN t2 | bs = 8 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -710,10 +2528,12 @@ ANY RIGHT 2 2 5 val27 2 2 5 val28 3 3 4 val3 -ANY INNER +t1 ALL INNER JOIN tn2 | bs = 8 +1 1 4 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ALL LEFT JOIN tn2 | bs = 8 +1 1 val1 5 1 1 val1 5 2 \N val21 0 2 \N val22 0 @@ -723,7 +2543,7 @@ ANY LEFT 2 \N val26 0 2 \N val27 0 3 3 val3 4 -ANY RIGHT +t1 ALL RIGHT JOIN tn2 | bs = 8 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -734,10 +2554,36 @@ ANY RIGHT 1 1 4 val11 1 1 4 val12 3 3 4 val3 -ANY INNER +t1 ANY INNER JOIN tn2 | bs = 8 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN tn2 | bs = 8 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = 8 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = 8 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = 8 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N 0 val21 0 @@ -747,7 +2593,7 @@ ANY LEFT \N 0 val25 0 \N 0 val26 0 \N 0 val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN t2 | bs = 8 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -758,10 +2604,36 @@ ANY RIGHT \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -ANY INNER +tn1 ANY INNER JOIN t2 | bs = 8 1 1 4 5 3 3 4 4 -ANY LEFT +tn1 ANY LEFT JOIN t2 | bs = 8 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = 8 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = 8 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = 8 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N \N val21 0 @@ -771,7 +2643,31 @@ ANY LEFT \N \N val25 0 \N \N val26 0 \N \N val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN tn2 | bs = 8 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = 8 +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = 8 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = 8 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -783,11 +2679,170 @@ ANY RIGHT \N \N 0 val27 \N \N 0 val28 block size = 9 -ANY INNER +t1 ALL INNER JOIN t2 | bs = 9 +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = 9 +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = 9 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = 9 1 1 4 5 2 2 5 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN t2 | bs = 9 1 1 val1 5 2 2 val21 5 2 2 val22 5 @@ -797,7 +2852,7 @@ ANY LEFT 2 2 val26 5 2 2 val27 5 3 3 val3 4 -ANY RIGHT +t1 ANY RIGHT JOIN t2 | bs = 9 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -808,10 +2863,12 @@ ANY RIGHT 2 2 5 val27 2 2 5 val28 3 3 4 val3 -ANY INNER +t1 ALL INNER JOIN tn2 | bs = 9 +1 1 4 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ALL LEFT JOIN tn2 | bs = 9 +1 1 val1 5 1 1 val1 5 2 \N val21 0 2 \N val22 0 @@ -821,7 +2878,7 @@ ANY LEFT 2 \N val26 0 2 \N val27 0 3 3 val3 4 -ANY RIGHT +t1 ALL RIGHT JOIN tn2 | bs = 9 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -832,10 +2889,36 @@ ANY RIGHT 1 1 4 val11 1 1 4 val12 3 3 4 val3 -ANY INNER +t1 ANY INNER JOIN tn2 | bs = 9 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN tn2 | bs = 9 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = 9 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = 9 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = 9 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N 0 val21 0 @@ -845,7 +2928,7 @@ ANY LEFT \N 0 val25 0 \N 0 val26 0 \N 0 val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN t2 | bs = 9 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -856,10 +2939,36 @@ ANY RIGHT \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -ANY INNER +tn1 ANY INNER JOIN t2 | bs = 9 1 1 4 5 3 3 4 4 -ANY LEFT +tn1 ANY LEFT JOIN t2 | bs = 9 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = 9 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = 9 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = 9 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N \N val21 0 @@ -869,7 +2978,31 @@ ANY LEFT \N \N val25 0 \N \N val26 0 \N \N val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN tn2 | bs = 9 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = 9 +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = 9 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = 9 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -881,11 +3014,170 @@ ANY RIGHT \N \N 0 val27 \N \N 0 val28 block size = 10 -ANY INNER +t1 ALL INNER JOIN t2 | bs = 10 +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = 10 +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = 10 +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = 10 1 1 4 5 2 2 5 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN t2 | bs = 10 1 1 val1 5 2 2 val21 5 2 2 val22 5 @@ -895,7 +3187,7 @@ ANY LEFT 2 2 val26 5 2 2 val27 5 3 3 val3 4 -ANY RIGHT +t1 ANY RIGHT JOIN t2 | bs = 10 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -906,10 +3198,12 @@ ANY RIGHT 2 2 5 val27 2 2 5 val28 3 3 4 val3 -ANY INNER +t1 ALL INNER JOIN tn2 | bs = 10 +1 1 4 5 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ALL LEFT JOIN tn2 | bs = 10 +1 1 val1 5 1 1 val1 5 2 \N val21 0 2 \N val22 0 @@ -919,7 +3213,7 @@ ANY LEFT 2 \N val26 0 2 \N val27 0 3 3 val3 4 -ANY RIGHT +t1 ALL RIGHT JOIN tn2 | bs = 10 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -930,10 +3224,36 @@ ANY RIGHT 1 1 4 val11 1 1 4 val12 3 3 4 val3 -ANY INNER +t1 ANY INNER JOIN tn2 | bs = 10 1 1 4 5 3 3 4 4 -ANY LEFT +t1 ANY LEFT JOIN tn2 | bs = 10 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = 10 +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = 10 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = 10 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N 0 val21 0 @@ -943,7 +3263,7 @@ ANY LEFT \N 0 val25 0 \N 0 val26 0 \N 0 val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN t2 | bs = 10 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -954,10 +3274,36 @@ ANY RIGHT \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -ANY INNER +tn1 ANY INNER JOIN t2 | bs = 10 1 1 4 5 3 3 4 4 -ANY LEFT +tn1 ANY LEFT JOIN t2 | bs = 10 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = 10 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = 10 +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = 10 +1 1 val1 5 1 1 val1 5 3 3 val3 4 \N \N val21 0 @@ -967,7 +3313,31 @@ ANY LEFT \N \N val25 0 \N \N val26 0 \N \N val27 0 -ANY RIGHT +tn1 ALL RIGHT JOIN tn2 | bs = 10 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = 10 +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = 10 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = 10 1 1 4 val11 1 1 4 val12 3 3 4 val3 diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index bb54f49b754..082464aa081 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -1,36 +1,38 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS tn1; +DROP TABLE IF EXISTS tn2; CREATE TABLE t1 (key UInt32, s String) engine = TinyLog; CREATE TABLE tn1 (key Nullable(UInt32), s String) engine = TinyLog; CREATE TABLE t2 (key UInt32, s String) engine = TinyLog; CREATE TABLE tn2 (key Nullable(UInt32), s String) engine = TinyLog; -{% set table_size = 10 %} - INSERT INTO t1 VALUES (1, 'val1'), (2, 'val21'), (2, 'val22'), (2, 'val23'), (2, 'val24'), (2, 'val25'), (2, 'val26'), (2, 'val27'), (3, 'val3'); INSERT INTO tn1 VALUES (1, 'val1'), (NULL, 'val21'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (3, 'val3'); INSERT INTO t2 VALUES (1, 'val11'), (1, 'val12'), (2, 'val22'), (2, 'val23'), (2, 'val24'), (2, 'val25'), (2, 'val26'), (2, 'val27'), (2, 'val28'), (3, 'val3'); INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (NULL, 'val28'), (3, 'val3'); -SET join_algorithm = 'full_sorting_merge'; +-- SET join_algorithm = 'full_sorting_merge'; -{% for block_size in range (1, table_size + 1) -%} +{% for block_size in range(1, 11) -%} SET max_block_size = {{ block_size }}; SELECT 'block size = {{ block_size }}'; {% for t1, t2 in [('t1', 't2'), ('t1', 'tn2'), ('tn1', 't2'), ('tn1', 'tn2')] -%} +{% for kind in ['ALL', 'ANY'] -%} -SELECT 'ANY INNER'; -SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 ANY INNER JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key; +SELECT '{{ t1 }} {{ kind }} INNER JOIN {{ t2 }} | bs = {{ block_size }}'; +SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} INNER JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key; -SELECT 'ANY LEFT'; -SELECT t1.key, t2.key, t1.s, length(t2.s) FROM {{ t1 }} AS t1 ANY LEFT JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t1.s; +SELECT '{{ t1 }} {{ kind }} LEFT JOIN {{ t2 }} | bs = {{ block_size }}'; +SELECT t1.key, t2.key, t1.s, length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} LEFT JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t1.s; -SELECT 'ANY RIGHT'; -SELECT t1.key, t2.key, length(t1.s), t2.s FROM {{ t1 }} AS t1 ANY RIGHT JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t2.s; +SELECT '{{ t1 }} {{ kind }} RIGHT JOIN {{ t2 }} | bs = {{ block_size }}'; +SELECT t1.key, t2.key, length(t1.s), t2.s FROM {{ t1 }} AS t1 {{ kind }} RIGHT JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t2.s; {% endfor -%} {% endfor -%} +{% endfor -%} diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.reference b/tests/queries/0_stateless/02275_full_sort_join_long.reference index b6972ad7014..91b81d5ab3a 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.reference +++ b/tests/queries/0_stateless/02275_full_sort_join_long.reference @@ -1,3 +1,27 @@ +ALL INNER +500353531835 500353531835 1000342 1000342 1000342 +ALL LEFT +50195752660639 500353531835 10369589 10369589 1000342 +ALL RIGHT +500353531835 684008812186 1367170 1000342 1367170 +ALL INNER +500353531835 500353531835 1000342 1000342 1000342 +ALL LEFT +50195752660639 500353531835 10369589 10369589 1000342 +ALL RIGHT +500353531835 684008812186 1367170 1000342 1367170 +ALL INNER +500353531835 500353531835 1000342 1000342 1000342 +ALL LEFT +50195752660639 500353531835 10369589 10369589 1000342 +ALL RIGHT +500353531835 684008812186 1367170 1000342 1367170 +ALL INNER +500353531835 500353531835 1000342 1000342 1000342 +ALL LEFT +50195752660639 500353531835 10369589 10369589 1000342 +ALL RIGHT +500353531835 684008812186 1367170 1000342 1367170 ANY INNER 199622811843 199622811843 399458 399458 399458 ANY LEFT diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 2a986ea64d8..a326d2635f4 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -1,3 +1,4 @@ +-- Tags: long DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; @@ -21,28 +22,30 @@ INSERT INTO t2 FROM numbers_mt({{ rtable_size }}) ; -SET join_algorithm = 'full_sorting_merge'; +-- SET join_algorithm = 'full_sorting_merge'; +{% for kind in ['ALL', 'ANY'] -%} {% for block_size in [32001, 65505, 65536, range(32001, 65536) | random] %} SET max_block_size = {{ block_size }}; -SELECT 'ANY INNER'; +SELECT '{{ kind }} INNER'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 -ANY INNER JOIN t2 +{{ kind }} INNER JOIN t2 ON t1.key == t2.key ; -SELECT 'ANY LEFT'; +SELECT '{{ kind }} LEFT'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 -ANY LEFT JOIN t2 +{{ kind }} LEFT JOIN t2 ON t1.key == t2.key ; -SELECT 'ANY RIGHT'; +SELECT '{{ kind }} RIGHT'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 -ANY RIGHT JOIN t2 +{{ kind }} RIGHT JOIN t2 ON t1.key == t2.key ; -{% endfor %} +{% endfor -%} +{% endfor -%} From 92ff43eb7ce79ab0048a35973bd1546a666a4a29 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 26 Apr 2022 14:32:48 +0000 Subject: [PATCH 294/627] tests full sort join --- .../Transforms/MergeJoinTransform.cpp | 11 +- .../Transforms/MergeJoinTransform.h | 19 +- .../02273_full_sort_join.reference | 1184 ------ .../02273_full_sort_join.reference.j2 | 303 ++ .../0_stateless/02273_full_sort_join.sql.j2 | 64 +- .../02274_full_sort_join_nodistinct.reference | 3350 ----------------- ...274_full_sort_join_nodistinct.reference.j2 | 336 ++ .../02274_full_sort_join_nodistinct.sql.j2 | 4 +- .../02275_full_sort_join_long.sql.j2 | 2 +- .../02276_full_sort_join_composite_key.sql.j2 | 0 10 files changed, 715 insertions(+), 4558 deletions(-) delete mode 100644 tests/queries/0_stateless/02273_full_sort_join.reference create mode 100644 tests/queries/0_stateless/02273_full_sort_join.reference.j2 delete mode 100644 tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference create mode 100644 tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 create mode 100644 tests/queries/0_stateless/02276_full_sort_join_composite_key.sql.j2 diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 806372cd90a..3b4c2959d32 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -138,7 +138,8 @@ void addIndexColumn(const Columns & columns, ColumnUInt64 & indices, Chunk & res limit = indices.size(); assert(limit == indices.size()); - + /// rows where default value shold be inserted have index == size + /// add row with defaults to handle it auto tmp_col = col->cloneResized(col->size() + 1); ColumnPtr new_col = tmp_col->index(indices, limit); result.addColumn(std::move(new_col)); @@ -666,6 +667,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) if (!current_right.isValid()) return Status(1); + /// join doen't build result block, but returns indices where result rows should be placed auto left_map = ColumnUInt64::create(); auto right_map = ColumnUInt64::create(); size_t prev_pos[] = {current_left.getRow(), current_right.getRow()}; @@ -675,6 +677,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); size_t num_result_rows = std::max(left_map->size(), right_map->size()); + /// build result block from indices Chunk result; addIndexColumn(cursors[0]->getCurrent().getColumns(), *left_map, result, prev_pos[0], num_result_rows); addIndexColumn(cursors[1]->getCurrent().getColumns(), *right_map, result, prev_pos[1], num_result_rows); @@ -684,7 +687,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { auto kind = table_join->getTableJoin().kind(); - auto strictness = table_join->getTableJoin().strictness(); if (!cursors[0]->cursor.isValid() && !cursors[0]->fullyCompleted()) return Status(0); @@ -706,6 +708,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() return Status({}, true); } + /// check if blocks are not intersecting at all if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor); cmp != 0) { if (cmp < 0) @@ -725,13 +728,15 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() } } + auto strictness = table_join->getTableJoin().strictness(); + if (strictness == ASTTableJoin::Strictness::Any) return anyJoin(kind); if (strictness == ASTTableJoin::Strictness::All) return allJoin(kind); - throw Exception("Unsupported strictness: " + toString(strictness), ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported strictness '{}'", strictness); } MergeJoinTransform::MergeJoinTransform( diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index eba4291f065..166f0d220c9 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -38,8 +38,6 @@ class FullMergeJoinCursor; using FullMergeJoinCursorPtr = std::unique_ptr; - - /// Used instead of storing previous block struct JoinKeyRow { @@ -79,6 +77,7 @@ struct JoinKeyRow } }; +/// Remembers previous key if it was joined in previous block class AnyJoinState : boost::noncopyable { public: @@ -90,20 +89,18 @@ public: keys[source_num] = JoinKeyRow(cursor, cursor.rows - 1); } - void setValue(Chunk value_) - { - value = std::move(value_); - } + void setValue(Chunk value_) { value = std::move(value_); } - bool empty() const - { - return keys[0].row.empty() && keys[1].row.empty(); - } + bool empty() const { return keys[0].row.empty() && keys[1].row.empty(); } + /// current keys JoinKeyRow keys[2]; + + /// for LEFT/RIGHT join use previously joined row from other table. Chunk value; }; +/// Accumulate blocks with same key and cross-join them class AllJoinState : boost::noncopyable { public: @@ -143,6 +140,7 @@ public: bool next() { + /// advance right to one row, when right finished, advance left to next block assert(!left.empty() && !right.empty()); if (finished()) @@ -197,7 +195,6 @@ private: /* * Wrapper for SortCursorImpl - * It is used to keep cursor for list of blocks. */ class FullMergeJoinCursor : boost::noncopyable { diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference b/tests/queries/0_stateless/02273_full_sort_join.reference deleted file mode 100644 index 360a1540b8d..00000000000 --- a/tests/queries/0_stateless/02273_full_sort_join.reference +++ /dev/null @@ -1,1184 +0,0 @@ -ALL INNER | bs = 1 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 1 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 1 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 2 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 2 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 2 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 3 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 3 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 3 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 4 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 4 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 4 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 5 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 5 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 5 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 6 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 6 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 6 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 7 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 7 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 7 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 8 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 8 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 8 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 9 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 9 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 9 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 10 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 10 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 10 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 11 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 11 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 11 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 12 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 12 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 12 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 13 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 13 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 13 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 14 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 14 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 14 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | bs = 15 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | bs = 15 -1 1 val5 0 -2 0 val9 1 -3 3 val10 0 -4 0 val12 1 -5 0 val11 1 -6 0 val7 1 -7 7 val14 0 -10 0 val3 1 -10 0 val8 1 -11 0 val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 0 val6 1 -ALL RIGHT | bs = 15 -0 8 1 val2 -0 8 1 val4 -0 9 1 val5 -0 12 1 val0 -0 12 1 val11 -0 12 1 val3 -0 12 1 val9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -ALL INNER | join_use_nulls = 1 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -14 14 0 0 -14 14 0 0 -ALL LEFT | join_use_nulls = 1 -1 1 val5 0 -2 \N val9 1 -3 3 val10 0 -4 \N val12 1 -5 \N val11 1 -6 \N val7 1 -7 7 val14 0 -10 \N val3 1 -10 \N val8 1 -11 \N val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 1 -ALL RIGHT | join_use_nulls = 1 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -14 14 0 val7 -14 14 0 val7 -\N 8 1 val2 -\N 8 1 val4 -\N 9 1 val5 -\N 12 1 val0 -\N 12 1 val11 -\N 12 1 val3 -\N 12 1 val9 -ANY INNER | bs = 1 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 1 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 1 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 2 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 2 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 2 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 3 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 3 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 3 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 4 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 4 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 4 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 5 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 5 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 5 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 6 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 6 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 6 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 7 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 7 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 7 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 8 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 8 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 8 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 9 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 9 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 9 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 10 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 10 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 10 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 11 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 11 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 11 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 12 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 12 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 12 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 13 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 13 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 13 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 14 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 14 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 14 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | bs = 15 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | bs = 15 -1 1 val5 0 -2 \N val9 \N -3 3 val10 0 -4 \N val12 \N -5 \N val11 \N -6 \N val7 \N -7 7 val14 0 -10 \N val3 \N -10 \N val8 \N -11 \N val0 \N -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 \N -ANY RIGHT | bs = 15 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 \N val2 -\N 8 \N val4 -\N 9 \N val5 -\N 12 \N val0 -\N 12 \N val11 -\N 12 \N val3 -\N 12 \N val9 -ANY INNER | join_use_nulls = 1 -1 1 0 0 -3 3 0 0 -7 7 0 0 -13 13 0 0 -14 14 0 0 -ANY LEFT | join_use_nulls = 1 -1 1 val5 0 -2 \N val9 1 -3 3 val10 0 -4 \N val12 1 -5 \N val11 1 -6 \N val7 1 -7 7 val14 0 -10 \N val3 1 -10 \N val8 1 -11 \N val0 1 -13 13 val2 0 -14 14 val1 0 -14 14 val13 0 -14 14 val4 0 -15 \N val6 1 -ANY RIGHT | join_use_nulls = 1 -1 1 0 val6 -3 3 0 val10 -7 7 0 val8 -13 13 0 val1 -14 14 0 val7 -\N 8 1 val2 -\N 8 1 val4 -\N 9 1 val5 -\N 12 1 val0 -\N 12 1 val11 -\N 12 1 val3 -\N 12 1 val9 diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 new file mode 100644 index 00000000000..50b4e5c41e5 --- /dev/null +++ b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 @@ -0,0 +1,303 @@ +{% set table_size = 15 -%} +{% for block_size in range(1, table_size + 1) -%} +ALL INNER | bs = {{ block_size }} +4 4 0 0 +5 5 0 0 +6 6 0 0 +8 8 0 0 +9 9 0 0 +11 11 0 0 +11 11 0 0 +12 12 0 0 +13 13 0 0 +13 13 0 0 +13 13 0 0 +14 14 0 0 +14 14 0 0 +ALL LEFT | bs = {{ block_size }} +1 0 val7 1 +2 0 val3 1 +2 0 val5 1 +4 4 val0 0 +5 5 val12 0 +6 6 val11 0 +8 8 val4 0 +9 9 val10 0 +10 0 val1 1 +10 0 val14 1 +11 11 val6 0 +11 11 val8 0 +12 12 val2 0 +13 13 val13 0 +13 13 val13 0 +13 13 val13 0 +14 14 val9 0 +14 14 val9 0 +ALL RIGHT | bs = {{ block_size }} +4 4 0 val10 +5 5 0 val6 +6 6 0 val8 +8 8 0 val1 +9 9 0 val5 +11 11 0 val11 +11 11 0 val11 +12 12 0 val0 +13 13 0 val2 +13 13 0 val4 +13 13 0 val9 +14 14 0 val3 +14 14 0 val7 +ALL INNER | bs = {{ block_size }} | copmosite key +2 2 2 2 2 2 0 0 +2 2 2 2 2 2 0 0 +ALL LEFT | bs = {{ block_size }} | copmosite key +1 1 2 0 0 \N val14 1 +1 1 2 0 0 \N val3 1 +1 2 1 0 0 \N val7 1 +1 2 2 0 0 \N val1 1 +1 2 2 0 0 \N val13 1 +1 2 2 0 0 \N val4 1 +1 2 2 0 0 \N val8 1 +1 \N 1 0 0 \N val0 1 +1 \N 1 0 0 \N val5 1 +1 \N 2 0 0 \N val10 1 +2 2 1 0 0 \N val11 1 +2 2 1 0 0 \N val2 1 +2 2 1 0 0 \N val6 1 +2 2 2 2 2 2 val12 0 +2 2 2 2 2 2 val9 0 +ALL RIGHT | bs = {{ block_size }} | copmosite key +0 \N 0 1 1 1 1 val2 +0 \N 0 1 1 1 1 val7 +0 \N 0 1 1 2 1 val5 +0 \N 0 1 1 \N 1 val0 +0 \N 0 1 1 \N 1 val6 +0 \N 0 2 1 1 1 val1 +0 \N 0 2 1 1 1 val11 +0 \N 0 2 1 2 1 val10 +0 \N 0 2 1 2 1 val8 +0 \N 0 2 1 \N 1 val3 +0 \N 0 2 2 \N 1 val9 +2 2 2 2 2 2 0 val4 +2 2 2 2 2 2 0 val4 +ANY INNER | bs = {{ block_size }} +4 4 0 0 +5 5 0 0 +6 6 0 0 +8 8 0 0 +9 9 0 0 +11 11 0 0 +12 12 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | bs = {{ block_size }} +1 0 val7 1 +2 0 val3 1 +2 0 val5 1 +4 4 val0 0 +5 5 val12 0 +6 6 val11 0 +8 8 val4 0 +9 9 val10 0 +10 0 val1 1 +10 0 val14 1 +11 11 val6 0 +11 11 val8 0 +12 12 val2 0 +13 13 val13 0 +14 14 val9 0 +ANY RIGHT | bs = {{ block_size }} +4 4 0 val10 +5 5 0 val6 +6 6 0 val8 +8 8 0 val1 +9 9 0 val5 +11 11 0 val11 +12 12 0 val0 +13 13 0 val2 +13 13 0 val4 +13 13 0 val9 +14 14 0 val3 +14 14 0 val7 +ANY INNER | bs = {{ block_size }} | copmosite key +2 2 2 2 2 2 0 0 +ANY LEFT | bs = {{ block_size }} | copmosite key +1 1 2 0 0 \N val14 1 +1 1 2 0 0 \N val3 1 +1 2 1 0 0 \N val7 1 +1 2 2 0 0 \N val1 1 +1 2 2 0 0 \N val13 1 +1 2 2 0 0 \N val4 1 +1 2 2 0 0 \N val8 1 +1 \N 1 0 0 \N val0 1 +1 \N 1 0 0 \N val5 1 +1 \N 2 0 0 \N val10 1 +2 2 1 0 0 \N val11 1 +2 2 1 0 0 \N val2 1 +2 2 1 0 0 \N val6 1 +2 2 2 2 2 2 val12 0 +2 2 2 2 2 2 val9 0 +ANY RIGHT | bs = {{ block_size }} | copmosite key +0 \N 0 1 1 1 1 val2 +0 \N 0 1 1 1 1 val7 +0 \N 0 1 1 2 1 val5 +0 \N 0 1 1 \N 1 val0 +0 \N 0 1 1 \N 1 val6 +0 \N 0 2 1 1 1 val1 +0 \N 0 2 1 1 1 val11 +0 \N 0 2 1 2 1 val10 +0 \N 0 2 1 2 1 val8 +0 \N 0 2 1 \N 1 val3 +0 \N 0 2 2 \N 1 val9 +2 2 2 2 2 2 0 val4 +{% endfor -%} +ALL INNER | join_use_nulls = 1 +4 4 0 0 +5 5 0 0 +6 6 0 0 +8 8 0 0 +9 9 0 0 +11 11 0 0 +11 11 0 0 +12 12 0 0 +13 13 0 0 +13 13 0 0 +13 13 0 0 +14 14 0 0 +14 14 0 0 +ALL LEFT | join_use_nulls = 1 +1 \N val7 1 +2 \N val3 1 +2 \N val5 1 +4 4 val0 0 +5 5 val12 0 +6 6 val11 0 +8 8 val4 0 +9 9 val10 0 +10 \N val1 1 +10 \N val14 1 +11 11 val6 0 +11 11 val8 0 +12 12 val2 0 +13 13 val13 0 +13 13 val13 0 +13 13 val13 0 +14 14 val9 0 +14 14 val9 0 +ALL RIGHT | join_use_nulls = 1 +4 4 0 val10 +5 5 0 val6 +6 6 0 val8 +8 8 0 val1 +9 9 0 val5 +11 11 0 val11 +11 11 0 val11 +12 12 0 val0 +13 13 0 val2 +13 13 0 val4 +13 13 0 val9 +14 14 0 val3 +14 14 0 val7 +ALL INNER | join_use_nulls = 1 | copmosite key +2 2 2 2 2 2 0 0 +2 2 2 2 2 2 0 0 +ALL LEFT | join_use_nulls = 1 | copmosite key +1 1 2 \N \N \N val14 \N +1 1 2 \N \N \N val3 \N +1 2 1 \N \N \N val7 \N +1 2 2 \N \N \N val1 \N +1 2 2 \N \N \N val13 \N +1 2 2 \N \N \N val4 \N +1 2 2 \N \N \N val8 \N +1 \N 1 \N \N \N val0 \N +1 \N 1 \N \N \N val5 \N +1 \N 2 \N \N \N val10 \N +2 2 1 \N \N \N val11 \N +2 2 1 \N \N \N val2 \N +2 2 1 \N \N \N val6 \N +2 2 2 2 2 2 val12 0 +2 2 2 2 2 2 val9 0 +ALL RIGHT | join_use_nulls = 1 | copmosite key +2 2 2 2 2 2 0 val4 +2 2 2 2 2 2 0 val4 +\N \N \N 1 1 1 \N val2 +\N \N \N 1 1 1 \N val7 +\N \N \N 1 1 2 \N val5 +\N \N \N 1 1 \N \N val0 +\N \N \N 1 1 \N \N val6 +\N \N \N 2 1 1 \N val1 +\N \N \N 2 1 1 \N val11 +\N \N \N 2 1 2 \N val10 +\N \N \N 2 1 2 \N val8 +\N \N \N 2 1 \N \N val3 +\N \N \N 2 2 \N \N val9 +ANY INNER | join_use_nulls = 1 +4 4 0 0 +5 5 0 0 +6 6 0 0 +8 8 0 0 +9 9 0 0 +11 11 0 0 +12 12 0 0 +13 13 0 0 +14 14 0 0 +ANY LEFT | join_use_nulls = 1 +1 \N val7 1 +2 \N val3 1 +2 \N val5 1 +4 4 val0 0 +5 5 val12 0 +6 6 val11 0 +8 8 val4 0 +9 9 val10 0 +10 \N val1 1 +10 \N val14 1 +11 11 val6 0 +11 11 val8 0 +12 12 val2 0 +13 13 val13 0 +14 14 val9 0 +ANY RIGHT | join_use_nulls = 1 +4 4 0 val10 +5 5 0 val6 +6 6 0 val8 +8 8 0 val1 +9 9 0 val5 +11 11 0 val11 +12 12 0 val0 +13 13 0 val2 +13 13 0 val4 +13 13 0 val9 +14 14 0 val3 +14 14 0 val7 +ANY INNER | join_use_nulls = 1 | copmosite key +2 2 2 2 2 2 0 0 +ANY LEFT | join_use_nulls = 1 | copmosite key +1 1 2 \N \N \N val14 \N +1 1 2 \N \N \N val3 \N +1 2 1 \N \N \N val7 \N +1 2 2 \N \N \N val1 \N +1 2 2 \N \N \N val13 \N +1 2 2 \N \N \N val4 \N +1 2 2 \N \N \N val8 \N +1 \N 1 \N \N \N val0 \N +1 \N 1 \N \N \N val5 \N +1 \N 2 \N \N \N val10 \N +2 2 1 \N \N \N val11 \N +2 2 1 \N \N \N val2 \N +2 2 1 \N \N \N val6 \N +2 2 2 2 2 2 val12 0 +2 2 2 2 2 2 val9 0 +ANY RIGHT | join_use_nulls = 1 | copmosite key +2 2 2 2 2 2 0 val4 +\N \N \N 1 1 1 \N val2 +\N \N \N 1 1 1 \N val7 +\N \N \N 1 1 2 \N val5 +\N \N \N 1 1 \N \N val0 +\N \N \N 1 1 \N \N val6 +\N \N \N 2 1 1 \N val1 +\N \N \N 2 1 1 \N val11 +\N \N \N 2 1 2 \N val10 +\N \N \N 2 1 2 \N val8 +\N \N \N 2 1 \N \N val3 +\N \N \N 2 2 \N \N val9 diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 6737cede6cd..ec93cee2df1 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -1,27 +1,33 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (key UInt32, s String) engine = TinyLog; -CREATE TABLE t2 (key UInt32, s String) engine = TinyLog; +CREATE TABLE t1 (key UInt32, key1 UInt32, key2 Nullable(UInt32), key3 UInt32, s String) engine = TinyLog; +CREATE TABLE t2 (key UInt32, key1 UInt32, key2 UInt32, key3 Nullable(UInt32), s String) engine = TinyLog; {% set table_size = 15 %} INSERT INTO t1 SELECT - sipHash64(number, 'x') % {{ table_size }} + 1 as key, + sipHash64(number, 't1') % {{ table_size }} + 1 as key, + sipHash64(number, 'x') % 2 + 1 as key1, + if(number % 5 == 0, NULL, sipHash64(number, 'y') % 2 + 1) as key2, + sipHash64(number, 'z') % 2 + 1 as key3, 'val' || toString(number) as s FROM numbers_mt({{ table_size }}); INSERT INTO t2 SELECT - sipHash64(number, 'y') % {{ table_size }} + 1 as key, + sipHash64(number, 't2') % {{ table_size }} + 1 as key, + sipHash64(number, 'a') % 2 + 1 as key1, + sipHash64(number, 'b') % 2 + 1 as key2, + if(number % 3 == 0, NULL, sipHash64(number, 'c') % 2 + 1) as key3, 'val' || toString(number) as s FROM numbers_mt({{ table_size - 3 }}); --- SET join_algorithm = 'full_sorting_merge'; +SET join_algorithm = 'full_sorting_merge'; -{% for kind in ['ALL', 'ANY'] -%} {% for block_size in range(1, table_size + 1) -%} +{% for kind in ['ALL', 'ANY'] -%} SET max_block_size = {{ block_size }}; @@ -46,7 +52,32 @@ ON t1.key == t2.key ORDER BY t1.key, t2.key, t2.s ; +SELECT '{{ kind }} INNER | bs = {{ block_size }} | copmosite key'; +SELECT t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, empty(t1.s), empty(t2.s) FROM t1 +{{ kind }} INNER JOIN t2 +ON t1.key1 == t2.key1 AND t1.key2 == t2.key2 AND t1.key3 == t2.key3 AND t1.key1 == t2.key3 +ORDER BY t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3 +; + +SELECT '{{ kind }} LEFT | bs = {{ block_size }} | copmosite key'; +SELECT t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, t1.s, empty(t2.s) FROM t1 +{{ kind }} LEFT JOIN t2 +ON t1.key1 == t2.key1 AND t1.key2 == t2.key2 AND t1.key3 == t2.key3 AND t1.key1 == t2.key3 +ORDER BY t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, t1.s +; + +SELECT '{{ kind }} RIGHT | bs = {{ block_size }} | copmosite key'; +SELECT t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, empty(t1.s), t2.s FROM t1 +{{ kind }} RIGHT JOIN t2 +ON t1.key1 == t2.key1 AND t1.key2 == t2.key2 AND t1.key3 == t2.key3 AND t1.key1 == t2.key3 +ORDER BY t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, t2.s +; + {% endfor -%} +{% endfor -%} + + +{% for kind in ['ALL', 'ANY'] -%} SET join_use_nulls = 1; @@ -71,4 +102,25 @@ ON t1.key == t2.key ORDER BY t1.key, t2.key, t2.s ; +SELECT '{{ kind }} INNER | join_use_nulls = 1 | copmosite key'; +SELECT t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, empty(t1.s), empty(t2.s) FROM t1 +{{ kind }} INNER JOIN t2 +ON t1.key1 == t2.key1 AND t1.key2 == t2.key2 AND t1.key3 == t2.key3 AND t1.key1 == t2.key3 +ORDER BY t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3 +; + +SELECT '{{ kind }} LEFT | join_use_nulls = 1 | copmosite key'; +SELECT t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, t1.s, empty(t2.s) FROM t1 +{{ kind }} LEFT JOIN t2 +ON t1.key1 == t2.key1 AND t1.key2 == t2.key2 AND t1.key3 == t2.key3 AND t1.key1 == t2.key3 +ORDER BY t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, t1.s +; + +SELECT '{{ kind }} RIGHT | join_use_nulls = 1 | copmosite key'; +SELECT t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, empty(t1.s), t2.s FROM t1 +{{ kind }} RIGHT JOIN t2 +ON t1.key1 == t2.key1 AND t1.key2 == t2.key2 AND t1.key3 == t2.key3 AND t1.key1 == t2.key3 +ORDER BY t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, t2.s +; + {% endfor -%} diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference deleted file mode 100644 index 99336d442b2..00000000000 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference +++ /dev/null @@ -1,3350 +0,0 @@ -block size = 1 -t1 ALL INNER JOIN t2 | bs = 1 -1 1 4 5 -1 1 4 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -3 3 4 4 -t1 ALL LEFT JOIN t2 | bs = 1 -1 1 val1 5 -1 1 val1 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -3 3 val3 4 -t1 ALL RIGHT JOIN t2 | bs = 1 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -3 3 4 val3 -t1 ANY INNER JOIN t2 | bs = 1 -1 1 4 5 -2 2 5 5 -3 3 4 4 -t1 ANY LEFT JOIN t2 | bs = 1 -1 1 val1 5 -2 2 val21 5 -2 2 val22 5 -2 2 val23 5 -2 2 val24 5 -2 2 val25 5 -2 2 val26 5 -2 2 val27 5 -3 3 val3 4 -t1 ANY RIGHT JOIN t2 | bs = 1 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val23 -2 2 5 val24 -2 2 5 val25 -2 2 5 val26 -2 2 5 val27 -2 2 5 val28 -3 3 4 val3 -t1 ALL INNER JOIN tn2 | bs = 1 -1 1 4 5 -1 1 4 5 -3 3 4 4 -t1 ALL LEFT JOIN tn2 | bs = 1 -1 1 val1 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ALL RIGHT JOIN tn2 | bs = 1 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -t1 ANY INNER JOIN tn2 | bs = 1 -1 1 4 5 -3 3 4 4 -t1 ANY LEFT JOIN tn2 | bs = 1 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ANY RIGHT JOIN tn2 | bs = 1 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -tn1 ALL INNER JOIN t2 | bs = 1 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN t2 | bs = 1 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ALL RIGHT JOIN t2 | bs = 1 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ANY INNER JOIN t2 | bs = 1 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN t2 | bs = 1 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ANY RIGHT JOIN t2 | bs = 1 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ALL INNER JOIN tn2 | bs = 1 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN tn2 | bs = 1 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ALL RIGHT JOIN tn2 | bs = 1 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -tn1 ANY INNER JOIN tn2 | bs = 1 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN tn2 | bs = 1 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ANY RIGHT JOIN tn2 | bs = 1 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -block size = 2 -t1 ALL INNER JOIN t2 | bs = 2 -1 1 4 5 -1 1 4 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -3 3 4 4 -t1 ALL LEFT JOIN t2 | bs = 2 -1 1 val1 5 -1 1 val1 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -3 3 val3 4 -t1 ALL RIGHT JOIN t2 | bs = 2 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -3 3 4 val3 -t1 ANY INNER JOIN t2 | bs = 2 -1 1 4 5 -2 2 5 5 -3 3 4 4 -t1 ANY LEFT JOIN t2 | bs = 2 -1 1 val1 5 -2 2 val21 5 -2 2 val22 5 -2 2 val23 5 -2 2 val24 5 -2 2 val25 5 -2 2 val26 5 -2 2 val27 5 -3 3 val3 4 -t1 ANY RIGHT JOIN t2 | bs = 2 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val23 -2 2 5 val24 -2 2 5 val25 -2 2 5 val26 -2 2 5 val27 -2 2 5 val28 -3 3 4 val3 -t1 ALL INNER JOIN tn2 | bs = 2 -1 1 4 5 -1 1 4 5 -3 3 4 4 -t1 ALL LEFT JOIN tn2 | bs = 2 -1 1 val1 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ALL RIGHT JOIN tn2 | bs = 2 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -t1 ANY INNER JOIN tn2 | bs = 2 -1 1 4 5 -3 3 4 4 -t1 ANY LEFT JOIN tn2 | bs = 2 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ANY RIGHT JOIN tn2 | bs = 2 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -tn1 ALL INNER JOIN t2 | bs = 2 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN t2 | bs = 2 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ALL RIGHT JOIN t2 | bs = 2 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ANY INNER JOIN t2 | bs = 2 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN t2 | bs = 2 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ANY RIGHT JOIN t2 | bs = 2 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ALL INNER JOIN tn2 | bs = 2 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN tn2 | bs = 2 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ALL RIGHT JOIN tn2 | bs = 2 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -tn1 ANY INNER JOIN tn2 | bs = 2 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN tn2 | bs = 2 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ANY RIGHT JOIN tn2 | bs = 2 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -block size = 3 -t1 ALL INNER JOIN t2 | bs = 3 -1 1 4 5 -1 1 4 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -3 3 4 4 -t1 ALL LEFT JOIN t2 | bs = 3 -1 1 val1 5 -1 1 val1 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -3 3 val3 4 -t1 ALL RIGHT JOIN t2 | bs = 3 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -3 3 4 val3 -t1 ANY INNER JOIN t2 | bs = 3 -1 1 4 5 -2 2 5 5 -3 3 4 4 -t1 ANY LEFT JOIN t2 | bs = 3 -1 1 val1 5 -2 2 val21 5 -2 2 val22 5 -2 2 val23 5 -2 2 val24 5 -2 2 val25 5 -2 2 val26 5 -2 2 val27 5 -3 3 val3 4 -t1 ANY RIGHT JOIN t2 | bs = 3 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val23 -2 2 5 val24 -2 2 5 val25 -2 2 5 val26 -2 2 5 val27 -2 2 5 val28 -3 3 4 val3 -t1 ALL INNER JOIN tn2 | bs = 3 -1 1 4 5 -1 1 4 5 -3 3 4 4 -t1 ALL LEFT JOIN tn2 | bs = 3 -1 1 val1 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ALL RIGHT JOIN tn2 | bs = 3 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -t1 ANY INNER JOIN tn2 | bs = 3 -1 1 4 5 -3 3 4 4 -t1 ANY LEFT JOIN tn2 | bs = 3 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ANY RIGHT JOIN tn2 | bs = 3 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -tn1 ALL INNER JOIN t2 | bs = 3 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN t2 | bs = 3 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ALL RIGHT JOIN t2 | bs = 3 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ANY INNER JOIN t2 | bs = 3 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN t2 | bs = 3 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ANY RIGHT JOIN t2 | bs = 3 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ALL INNER JOIN tn2 | bs = 3 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN tn2 | bs = 3 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ALL RIGHT JOIN tn2 | bs = 3 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -tn1 ANY INNER JOIN tn2 | bs = 3 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN tn2 | bs = 3 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ANY RIGHT JOIN tn2 | bs = 3 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -block size = 4 -t1 ALL INNER JOIN t2 | bs = 4 -1 1 4 5 -1 1 4 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -3 3 4 4 -t1 ALL LEFT JOIN t2 | bs = 4 -1 1 val1 5 -1 1 val1 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -3 3 val3 4 -t1 ALL RIGHT JOIN t2 | bs = 4 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -3 3 4 val3 -t1 ANY INNER JOIN t2 | bs = 4 -1 1 4 5 -2 2 5 5 -3 3 4 4 -t1 ANY LEFT JOIN t2 | bs = 4 -1 1 val1 5 -2 2 val21 5 -2 2 val22 5 -2 2 val23 5 -2 2 val24 5 -2 2 val25 5 -2 2 val26 5 -2 2 val27 5 -3 3 val3 4 -t1 ANY RIGHT JOIN t2 | bs = 4 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val23 -2 2 5 val24 -2 2 5 val25 -2 2 5 val26 -2 2 5 val27 -2 2 5 val28 -3 3 4 val3 -t1 ALL INNER JOIN tn2 | bs = 4 -1 1 4 5 -1 1 4 5 -3 3 4 4 -t1 ALL LEFT JOIN tn2 | bs = 4 -1 1 val1 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ALL RIGHT JOIN tn2 | bs = 4 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -t1 ANY INNER JOIN tn2 | bs = 4 -1 1 4 5 -3 3 4 4 -t1 ANY LEFT JOIN tn2 | bs = 4 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ANY RIGHT JOIN tn2 | bs = 4 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -tn1 ALL INNER JOIN t2 | bs = 4 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN t2 | bs = 4 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ALL RIGHT JOIN t2 | bs = 4 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ANY INNER JOIN t2 | bs = 4 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN t2 | bs = 4 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ANY RIGHT JOIN t2 | bs = 4 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ALL INNER JOIN tn2 | bs = 4 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN tn2 | bs = 4 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ALL RIGHT JOIN tn2 | bs = 4 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -tn1 ANY INNER JOIN tn2 | bs = 4 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN tn2 | bs = 4 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ANY RIGHT JOIN tn2 | bs = 4 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -block size = 5 -t1 ALL INNER JOIN t2 | bs = 5 -1 1 4 5 -1 1 4 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -3 3 4 4 -t1 ALL LEFT JOIN t2 | bs = 5 -1 1 val1 5 -1 1 val1 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -3 3 val3 4 -t1 ALL RIGHT JOIN t2 | bs = 5 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -3 3 4 val3 -t1 ANY INNER JOIN t2 | bs = 5 -1 1 4 5 -2 2 5 5 -3 3 4 4 -t1 ANY LEFT JOIN t2 | bs = 5 -1 1 val1 5 -2 2 val21 5 -2 2 val22 5 -2 2 val23 5 -2 2 val24 5 -2 2 val25 5 -2 2 val26 5 -2 2 val27 5 -3 3 val3 4 -t1 ANY RIGHT JOIN t2 | bs = 5 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val23 -2 2 5 val24 -2 2 5 val25 -2 2 5 val26 -2 2 5 val27 -2 2 5 val28 -3 3 4 val3 -t1 ALL INNER JOIN tn2 | bs = 5 -1 1 4 5 -1 1 4 5 -3 3 4 4 -t1 ALL LEFT JOIN tn2 | bs = 5 -1 1 val1 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ALL RIGHT JOIN tn2 | bs = 5 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -t1 ANY INNER JOIN tn2 | bs = 5 -1 1 4 5 -3 3 4 4 -t1 ANY LEFT JOIN tn2 | bs = 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ANY RIGHT JOIN tn2 | bs = 5 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -tn1 ALL INNER JOIN t2 | bs = 5 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN t2 | bs = 5 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ALL RIGHT JOIN t2 | bs = 5 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ANY INNER JOIN t2 | bs = 5 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN t2 | bs = 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ANY RIGHT JOIN t2 | bs = 5 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ALL INNER JOIN tn2 | bs = 5 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN tn2 | bs = 5 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ALL RIGHT JOIN tn2 | bs = 5 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -tn1 ANY INNER JOIN tn2 | bs = 5 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN tn2 | bs = 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ANY RIGHT JOIN tn2 | bs = 5 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -block size = 6 -t1 ALL INNER JOIN t2 | bs = 6 -1 1 4 5 -1 1 4 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -3 3 4 4 -t1 ALL LEFT JOIN t2 | bs = 6 -1 1 val1 5 -1 1 val1 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -3 3 val3 4 -t1 ALL RIGHT JOIN t2 | bs = 6 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -3 3 4 val3 -t1 ANY INNER JOIN t2 | bs = 6 -1 1 4 5 -2 2 5 5 -3 3 4 4 -t1 ANY LEFT JOIN t2 | bs = 6 -1 1 val1 5 -2 2 val21 5 -2 2 val22 5 -2 2 val23 5 -2 2 val24 5 -2 2 val25 5 -2 2 val26 5 -2 2 val27 5 -3 3 val3 4 -t1 ANY RIGHT JOIN t2 | bs = 6 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val23 -2 2 5 val24 -2 2 5 val25 -2 2 5 val26 -2 2 5 val27 -2 2 5 val28 -3 3 4 val3 -t1 ALL INNER JOIN tn2 | bs = 6 -1 1 4 5 -1 1 4 5 -3 3 4 4 -t1 ALL LEFT JOIN tn2 | bs = 6 -1 1 val1 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ALL RIGHT JOIN tn2 | bs = 6 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -t1 ANY INNER JOIN tn2 | bs = 6 -1 1 4 5 -3 3 4 4 -t1 ANY LEFT JOIN tn2 | bs = 6 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ANY RIGHT JOIN tn2 | bs = 6 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -tn1 ALL INNER JOIN t2 | bs = 6 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN t2 | bs = 6 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ALL RIGHT JOIN t2 | bs = 6 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ANY INNER JOIN t2 | bs = 6 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN t2 | bs = 6 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ANY RIGHT JOIN t2 | bs = 6 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ALL INNER JOIN tn2 | bs = 6 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN tn2 | bs = 6 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ALL RIGHT JOIN tn2 | bs = 6 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -tn1 ANY INNER JOIN tn2 | bs = 6 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN tn2 | bs = 6 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ANY RIGHT JOIN tn2 | bs = 6 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -block size = 7 -t1 ALL INNER JOIN t2 | bs = 7 -1 1 4 5 -1 1 4 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -3 3 4 4 -t1 ALL LEFT JOIN t2 | bs = 7 -1 1 val1 5 -1 1 val1 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -3 3 val3 4 -t1 ALL RIGHT JOIN t2 | bs = 7 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -3 3 4 val3 -t1 ANY INNER JOIN t2 | bs = 7 -1 1 4 5 -2 2 5 5 -3 3 4 4 -t1 ANY LEFT JOIN t2 | bs = 7 -1 1 val1 5 -2 2 val21 5 -2 2 val22 5 -2 2 val23 5 -2 2 val24 5 -2 2 val25 5 -2 2 val26 5 -2 2 val27 5 -3 3 val3 4 -t1 ANY RIGHT JOIN t2 | bs = 7 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val23 -2 2 5 val24 -2 2 5 val25 -2 2 5 val26 -2 2 5 val27 -2 2 5 val28 -3 3 4 val3 -t1 ALL INNER JOIN tn2 | bs = 7 -1 1 4 5 -1 1 4 5 -3 3 4 4 -t1 ALL LEFT JOIN tn2 | bs = 7 -1 1 val1 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ALL RIGHT JOIN tn2 | bs = 7 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -t1 ANY INNER JOIN tn2 | bs = 7 -1 1 4 5 -3 3 4 4 -t1 ANY LEFT JOIN tn2 | bs = 7 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ANY RIGHT JOIN tn2 | bs = 7 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -tn1 ALL INNER JOIN t2 | bs = 7 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN t2 | bs = 7 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ALL RIGHT JOIN t2 | bs = 7 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ANY INNER JOIN t2 | bs = 7 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN t2 | bs = 7 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ANY RIGHT JOIN t2 | bs = 7 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ALL INNER JOIN tn2 | bs = 7 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN tn2 | bs = 7 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ALL RIGHT JOIN tn2 | bs = 7 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -tn1 ANY INNER JOIN tn2 | bs = 7 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN tn2 | bs = 7 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ANY RIGHT JOIN tn2 | bs = 7 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -block size = 8 -t1 ALL INNER JOIN t2 | bs = 8 -1 1 4 5 -1 1 4 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -3 3 4 4 -t1 ALL LEFT JOIN t2 | bs = 8 -1 1 val1 5 -1 1 val1 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -3 3 val3 4 -t1 ALL RIGHT JOIN t2 | bs = 8 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -3 3 4 val3 -t1 ANY INNER JOIN t2 | bs = 8 -1 1 4 5 -2 2 5 5 -3 3 4 4 -t1 ANY LEFT JOIN t2 | bs = 8 -1 1 val1 5 -2 2 val21 5 -2 2 val22 5 -2 2 val23 5 -2 2 val24 5 -2 2 val25 5 -2 2 val26 5 -2 2 val27 5 -3 3 val3 4 -t1 ANY RIGHT JOIN t2 | bs = 8 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val23 -2 2 5 val24 -2 2 5 val25 -2 2 5 val26 -2 2 5 val27 -2 2 5 val28 -3 3 4 val3 -t1 ALL INNER JOIN tn2 | bs = 8 -1 1 4 5 -1 1 4 5 -3 3 4 4 -t1 ALL LEFT JOIN tn2 | bs = 8 -1 1 val1 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ALL RIGHT JOIN tn2 | bs = 8 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -t1 ANY INNER JOIN tn2 | bs = 8 -1 1 4 5 -3 3 4 4 -t1 ANY LEFT JOIN tn2 | bs = 8 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ANY RIGHT JOIN tn2 | bs = 8 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -tn1 ALL INNER JOIN t2 | bs = 8 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN t2 | bs = 8 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ALL RIGHT JOIN t2 | bs = 8 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ANY INNER JOIN t2 | bs = 8 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN t2 | bs = 8 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ANY RIGHT JOIN t2 | bs = 8 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ALL INNER JOIN tn2 | bs = 8 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN tn2 | bs = 8 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ALL RIGHT JOIN tn2 | bs = 8 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -tn1 ANY INNER JOIN tn2 | bs = 8 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN tn2 | bs = 8 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ANY RIGHT JOIN tn2 | bs = 8 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -block size = 9 -t1 ALL INNER JOIN t2 | bs = 9 -1 1 4 5 -1 1 4 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -3 3 4 4 -t1 ALL LEFT JOIN t2 | bs = 9 -1 1 val1 5 -1 1 val1 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -3 3 val3 4 -t1 ALL RIGHT JOIN t2 | bs = 9 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -3 3 4 val3 -t1 ANY INNER JOIN t2 | bs = 9 -1 1 4 5 -2 2 5 5 -3 3 4 4 -t1 ANY LEFT JOIN t2 | bs = 9 -1 1 val1 5 -2 2 val21 5 -2 2 val22 5 -2 2 val23 5 -2 2 val24 5 -2 2 val25 5 -2 2 val26 5 -2 2 val27 5 -3 3 val3 4 -t1 ANY RIGHT JOIN t2 | bs = 9 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val23 -2 2 5 val24 -2 2 5 val25 -2 2 5 val26 -2 2 5 val27 -2 2 5 val28 -3 3 4 val3 -t1 ALL INNER JOIN tn2 | bs = 9 -1 1 4 5 -1 1 4 5 -3 3 4 4 -t1 ALL LEFT JOIN tn2 | bs = 9 -1 1 val1 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ALL RIGHT JOIN tn2 | bs = 9 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -t1 ANY INNER JOIN tn2 | bs = 9 -1 1 4 5 -3 3 4 4 -t1 ANY LEFT JOIN tn2 | bs = 9 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ANY RIGHT JOIN tn2 | bs = 9 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -tn1 ALL INNER JOIN t2 | bs = 9 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN t2 | bs = 9 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ALL RIGHT JOIN t2 | bs = 9 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ANY INNER JOIN t2 | bs = 9 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN t2 | bs = 9 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ANY RIGHT JOIN t2 | bs = 9 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ALL INNER JOIN tn2 | bs = 9 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN tn2 | bs = 9 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ALL RIGHT JOIN tn2 | bs = 9 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -tn1 ANY INNER JOIN tn2 | bs = 9 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN tn2 | bs = 9 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ANY RIGHT JOIN tn2 | bs = 9 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -block size = 10 -t1 ALL INNER JOIN t2 | bs = 10 -1 1 4 5 -1 1 4 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -2 2 5 5 -3 3 4 4 -t1 ALL LEFT JOIN t2 | bs = 10 -1 1 val1 5 -1 1 val1 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val21 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val22 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val23 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val24 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val25 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val26 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -2 2 val27 5 -3 3 val3 4 -t1 ALL RIGHT JOIN t2 | bs = 10 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val22 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val23 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val24 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val25 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val26 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val27 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -2 2 5 val28 -3 3 4 val3 -t1 ANY INNER JOIN t2 | bs = 10 -1 1 4 5 -2 2 5 5 -3 3 4 4 -t1 ANY LEFT JOIN t2 | bs = 10 -1 1 val1 5 -2 2 val21 5 -2 2 val22 5 -2 2 val23 5 -2 2 val24 5 -2 2 val25 5 -2 2 val26 5 -2 2 val27 5 -3 3 val3 4 -t1 ANY RIGHT JOIN t2 | bs = 10 -1 1 4 val11 -1 1 4 val12 -2 2 5 val22 -2 2 5 val23 -2 2 5 val24 -2 2 5 val25 -2 2 5 val26 -2 2 5 val27 -2 2 5 val28 -3 3 4 val3 -t1 ALL INNER JOIN tn2 | bs = 10 -1 1 4 5 -1 1 4 5 -3 3 4 4 -t1 ALL LEFT JOIN tn2 | bs = 10 -1 1 val1 5 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ALL RIGHT JOIN tn2 | bs = 10 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -t1 ANY INNER JOIN tn2 | bs = 10 -1 1 4 5 -3 3 4 4 -t1 ANY LEFT JOIN tn2 | bs = 10 -1 1 val1 5 -2 \N val21 0 -2 \N val22 0 -2 \N val23 0 -2 \N val24 0 -2 \N val25 0 -2 \N val26 0 -2 \N val27 0 -3 3 val3 4 -t1 ANY RIGHT JOIN tn2 | bs = 10 -0 \N 0 val22 -0 \N 0 val23 -0 \N 0 val24 -0 \N 0 val25 -0 \N 0 val26 -0 \N 0 val27 -0 \N 0 val28 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -tn1 ALL INNER JOIN t2 | bs = 10 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN t2 | bs = 10 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ALL RIGHT JOIN t2 | bs = 10 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ANY INNER JOIN t2 | bs = 10 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN t2 | bs = 10 -1 1 val1 5 -3 3 val3 4 -\N 0 val21 0 -\N 0 val22 0 -\N 0 val23 0 -\N 0 val24 0 -\N 0 val25 0 -\N 0 val26 0 -\N 0 val27 0 -tn1 ANY RIGHT JOIN t2 | bs = 10 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N 2 0 val22 -\N 2 0 val23 -\N 2 0 val24 -\N 2 0 val25 -\N 2 0 val26 -\N 2 0 val27 -\N 2 0 val28 -tn1 ALL INNER JOIN tn2 | bs = 10 -1 1 4 5 -1 1 4 5 -3 3 4 4 -tn1 ALL LEFT JOIN tn2 | bs = 10 -1 1 val1 5 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ALL RIGHT JOIN tn2 | bs = 10 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 -tn1 ANY INNER JOIN tn2 | bs = 10 -1 1 4 5 -3 3 4 4 -tn1 ANY LEFT JOIN tn2 | bs = 10 -1 1 val1 5 -3 3 val3 4 -\N \N val21 0 -\N \N val22 0 -\N \N val23 0 -\N \N val24 0 -\N \N val25 0 -\N \N val26 0 -\N \N val27 0 -tn1 ANY RIGHT JOIN tn2 | bs = 10 -1 1 4 val11 -1 1 4 val12 -3 3 4 val3 -\N \N 0 val22 -\N \N 0 val23 -\N \N 0 val24 -\N \N 0 val25 -\N \N 0 val26 -\N \N 0 val27 -\N \N 0 val28 diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 new file mode 100644 index 00000000000..9fe48d539d6 --- /dev/null +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -0,0 +1,336 @@ +{% for block_size in range(1, 11) -%} +t1 ALL INNER JOIN t2 | bs = {{ block_size }} +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL LEFT JOIN t2 | bs = {{ block_size }} +1 1 val1 5 +1 1 val1 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val21 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val22 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val23 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val24 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val25 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val26 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +2 2 val27 5 +3 3 val3 4 +t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val22 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val23 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val24 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val25 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val26 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val27 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +2 2 5 val28 +3 3 4 val3 +t1 ANY INNER JOIN t2 | bs = {{ block_size }} +1 1 4 5 +2 2 5 5 +3 3 4 4 +t1 ANY LEFT JOIN t2 | bs = {{ block_size }} +1 1 val1 5 +2 2 val21 5 +2 2 val22 5 +2 2 val23 5 +2 2 val24 5 +2 2 val25 5 +2 2 val26 5 +2 2 val27 5 +3 3 val3 4 +t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} +1 1 4 val11 +1 1 4 val12 +2 2 5 val22 +2 2 5 val23 +2 2 5 val24 +2 2 5 val25 +2 2 5 val26 +2 2 5 val27 +2 2 5 val28 +3 3 4 val3 +t1 ALL INNER JOIN tn2 | bs = {{ block_size }} +1 1 4 5 +1 1 4 5 +3 3 4 4 +t1 ALL LEFT JOIN tn2 | bs = {{ block_size }} +1 1 val1 5 +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +t1 ANY INNER JOIN tn2 | bs = {{ block_size }} +1 1 4 5 +3 3 4 4 +t1 ANY LEFT JOIN tn2 | bs = {{ block_size }} +1 1 val1 5 +2 \N val21 0 +2 \N val22 0 +2 \N val23 0 +2 \N val24 0 +2 \N val25 0 +2 \N val26 0 +2 \N val27 0 +3 3 val3 4 +t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} +0 \N 0 val22 +0 \N 0 val23 +0 \N 0 val24 +0 \N 0 val25 +0 \N 0 val26 +0 \N 0 val27 +0 \N 0 val28 +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +tn1 ALL INNER JOIN t2 | bs = {{ block_size }} +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN t2 | bs = {{ block_size }} +1 1 val1 5 +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ANY INNER JOIN t2 | bs = {{ block_size }} +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN t2 | bs = {{ block_size }} +1 1 val1 5 +3 3 val3 4 +\N 0 val21 0 +\N 0 val22 0 +\N 0 val23 0 +\N 0 val24 0 +\N 0 val25 0 +\N 0 val26 0 +\N 0 val27 0 +tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N 2 0 val22 +\N 2 0 val23 +\N 2 0 val24 +\N 2 0 val25 +\N 2 0 val26 +\N 2 0 val27 +\N 2 0 val28 +tn1 ALL INNER JOIN tn2 | bs = {{ block_size }} +1 1 4 5 +1 1 4 5 +3 3 4 4 +tn1 ALL LEFT JOIN tn2 | bs = {{ block_size }} +1 1 val1 5 +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +tn1 ANY INNER JOIN tn2 | bs = {{ block_size }} +1 1 4 5 +3 3 4 4 +tn1 ANY LEFT JOIN tn2 | bs = {{ block_size }} +1 1 val1 5 +3 3 val3 4 +\N \N val21 0 +\N \N val22 0 +\N \N val23 0 +\N \N val24 0 +\N \N val25 0 +\N \N val26 0 +\N \N val27 0 +tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} +1 1 4 val11 +1 1 4 val12 +3 3 4 val3 +\N \N 0 val22 +\N \N 0 val23 +\N \N 0 val24 +\N \N 0 val25 +\N \N 0 val26 +\N \N 0 val27 +\N \N 0 val28 +{% endfor -%} diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index 082464aa081..4b69162cbc4 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -13,14 +13,12 @@ INSERT INTO tn1 VALUES (1, 'val1'), (NULL, 'val21'), (NULL, 'val22'), (NULL, 'va INSERT INTO t2 VALUES (1, 'val11'), (1, 'val12'), (2, 'val22'), (2, 'val23'), (2, 'val24'), (2, 'val25'), (2, 'val26'), (2, 'val27'), (2, 'val28'), (3, 'val3'); INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (NULL, 'val28'), (3, 'val3'); --- SET join_algorithm = 'full_sorting_merge'; +SET join_algorithm = 'full_sorting_merge'; {% for block_size in range(1, 11) -%} SET max_block_size = {{ block_size }}; -SELECT 'block size = {{ block_size }}'; - {% for t1, t2 in [('t1', 't2'), ('t1', 'tn2'), ('tn1', 't2'), ('tn1', 'tn2')] -%} {% for kind in ['ALL', 'ANY'] -%} diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index a326d2635f4..29f1d46e2c8 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -22,7 +22,7 @@ INSERT INTO t2 FROM numbers_mt({{ rtable_size }}) ; --- SET join_algorithm = 'full_sorting_merge'; +SET join_algorithm = 'full_sorting_merge'; {% for kind in ['ALL', 'ANY'] -%} {% for block_size in [32001, 65505, 65536, range(32001, 65536) | random] %} diff --git a/tests/queries/0_stateless/02276_full_sort_join_composite_key.sql.j2 b/tests/queries/0_stateless/02276_full_sort_join_composite_key.sql.j2 new file mode 100644 index 00000000000..e69de29bb2d From a2a7abc2e9ae04ced83157fa28e15794fa6fd99f Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 26 Apr 2022 14:55:20 +0000 Subject: [PATCH 295/627] add not implemented checks, add using testcase to full sort join --- .../Transforms/MergeJoinTransform.cpp | 23 ++++++++++++++++-- .../Transforms/MergeJoinTransform.h | 16 ++++++------- .../02273_full_sort_join.reference.j2 | 24 +++++++++++++++++++ .../0_stateless/02273_full_sort_join.sql.j2 | 7 ++++++ .../02276_full_sort_join_composite_key.sql.j2 | 0 5 files changed, 60 insertions(+), 10 deletions(-) delete mode 100644 tests/queries/0_stateless/02276_full_sort_join_composite_key.sql.j2 diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 3b4c2959d32..a27aea21e3e 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -138,7 +138,7 @@ void addIndexColumn(const Columns & columns, ColumnUInt64 & indices, Chunk & res limit = indices.size(); assert(limit == indices.size()); - /// rows where default value shold be inserted have index == size + /// rows where default value should be inserted have index == size /// add row with defaults to handle it auto tmp_col = col->cloneResized(col->size() + 1); ColumnPtr new_col = tmp_col->index(indices, limit); @@ -227,6 +227,25 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( const auto & join_on = table_join->getTableJoin().getOnlyClause(); + for (const auto & key : join_on.key_names_left) + { + if (input_headers[0].getByName(key).type->lowCardinality() || + input_headers[0].getByName(key).type->isLowCardinalityNullable()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support low cardinality columns"); + } + } + for (const auto & key : join_on.key_names_right) + { + if (input_headers[1].getByName(key).type->lowCardinality() || + input_headers[1].getByName(key).type->isLowCardinalityNullable()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support low cardinality columns"); + } + } + if (join_on.on_filter_condition_left || join_on.on_filter_condition_right) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support ON filter conditions"); + cursors.push_back(createCursor(input_headers[0], join_on.key_names_left)); cursors.push_back(createCursor(input_headers[1], join_on.key_names_right)); } @@ -667,7 +686,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) if (!current_right.isValid()) return Status(1); - /// join doen't build result block, but returns indices where result rows should be placed + /// join doesn't build result block, but returns indices where result rows should be placed auto left_map = ColumnUInt64::create(); auto right_map = ColumnUInt64::create(); size_t prev_pos[] = {current_left.getRow(), current_right.getRow()}; diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 166f0d220c9..d5a0f6be52b 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -7,19 +7,19 @@ #include #include #include -#include + +#include + +#include + #include -#include "Columns/IColumn.h" -#include "Core/Field.h" -#include "Interpreters/IJoin.h" +#include #include +#include +#include #include #include #include -#include -#include -#include -#include namespace Poco { class Logger; } diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 index 50b4e5c41e5..1059108a03b 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 @@ -1,5 +1,19 @@ {% set table_size = 15 -%} {% for block_size in range(1, table_size + 1) -%} +ALL INNER USING | bs = {{ block_size }} +4 0 0 +5 0 0 +6 0 0 +8 0 0 +9 0 0 +11 0 0 +11 0 0 +12 0 0 +13 0 0 +13 0 0 +13 0 0 +14 0 0 +14 0 0 ALL INNER | bs = {{ block_size }} 4 4 0 0 5 5 0 0 @@ -80,6 +94,16 @@ ALL RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 +ANY INNER USING | bs = {{ block_size }} +4 0 0 +5 0 0 +6 0 0 +8 0 0 +9 0 0 +11 0 0 +12 0 0 +13 0 0 +14 0 0 ANY INNER | bs = {{ block_size }} 4 4 0 0 5 5 0 0 diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index ec93cee2df1..5b1eaf99f20 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -31,6 +31,13 @@ SET join_algorithm = 'full_sorting_merge'; SET max_block_size = {{ block_size }}; +SELECT '{{ kind }} INNER USING | bs = {{ block_size }}'; +SELECT key, empty(t1.s), empty(t2.s) FROM t1 +{{ kind }} INNER JOIN t2 +USING (key) +ORDER BY t1.key, t2.key +; + SELECT '{{ kind }} INNER | bs = {{ block_size }}'; SELECT t1.key, t2.key, empty(t1.s), empty(t2.s) FROM t1 {{ kind }} INNER JOIN t2 diff --git a/tests/queries/0_stateless/02276_full_sort_join_composite_key.sql.j2 b/tests/queries/0_stateless/02276_full_sort_join_composite_key.sql.j2 deleted file mode 100644 index e69de29bb2d..00000000000 From d184e184b44d83a99b007207e0221223fdec86b3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 26 Apr 2022 20:22:57 +0000 Subject: [PATCH 296/627] full sort join: check key types, more tests --- src/Interpreters/FullSortingMergeJoin.h | 33 ++++++++++++- src/Interpreters/TableJoin.cpp | 1 - src/Interpreters/join_common.cpp | 2 +- .../Transforms/MergeJoinTransform.cpp | 48 +++++++------------ .../Transforms/MergeJoinTransform.h | 7 +-- src/QueryPipeline/QueryPipelineBuilder.cpp | 3 ++ ...02276_full_sort_join_unsupported.reference | 0 .../02276_full_sort_join_unsupported.sql | 43 +++++++++++++++++ .../02277_full_sort_join_misc.reference | 6 +++ .../0_stateless/02277_full_sort_join_misc.sql | 11 +++++ 10 files changed, 115 insertions(+), 39 deletions(-) create mode 100644 tests/queries/0_stateless/02276_full_sort_join_unsupported.reference create mode 100644 tests/queries/0_stateless/02276_full_sort_join_unsupported.sql create mode 100644 tests/queries/0_stateless/02277_full_sort_join_misc.reference create mode 100644 tests/queries/0_stateless/02277_full_sort_join_misc.sql diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index df9bdfa5fc8..336295c4d63 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -2,11 +2,19 @@ #include #include +#include +#include #include namespace DB { +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; + extern const int NOT_IMPLEMENTED; +} + /// Dummy class, actual joining is done by MergeTransform class FullSortingMergeJoin : public IJoin { @@ -22,8 +30,31 @@ public: bool addJoinedBlock(const Block & /* block */, bool /* check_limits */) override { __builtin_unreachable(); } - void checkTypesOfKeys(const Block & /* block */) const override + void checkTypesOfKeys(const Block & left_block) const override { + if (table_join->getClauses().size() != 1) + throw Exception("FullSortingMergeJoin supports only one join key", ErrorCodes::NOT_IMPLEMENTED); + + const auto & onexpr = table_join->getOnlyClause(); + + for (size_t i = 0; i < onexpr.key_names_left.size(); ++i) + { + DataTypePtr left_type = left_block.getByName(onexpr.key_names_left[i]).type; + DataTypePtr right_type = right_sample_block.getByName(onexpr.key_names_right[i]).type; + + if (!removeNullable(left_type)->equals(*removeNullable(right_type))) + { + DataTypePtr left_type_no_lc = removeNullable(recursiveRemoveLowCardinality(left_type)); + DataTypePtr right_type_no_lc = removeNullable(recursiveRemoveLowCardinality(right_type)); + /// if types equal after removing low cardinality, then it is ok and can be supported + bool equals_up_to_lc = left_type_no_lc->equals(*right_type_no_lc); + throw DB::Exception( + equals_up_to_lc ? ErrorCodes::NOT_IMPLEMENTED : ErrorCodes::TYPE_MISMATCH, + "Type mismatch of columns to JOIN by: {} :: {} at left, {} :: {} at right", + onexpr.key_names_left[i], left_type->getName(), + onexpr.key_names_right[i], right_type->getName()); + } + } } /// Used just to get result header diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index db75a86fff6..cdb7f42ce11 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -818,7 +818,6 @@ void TableJoin::assertHasOneOnExpr() const text.push_back(onexpr.formatDebug()); throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have only one join clause, got {}: [{}], query: '{}'", clauses.size(), fmt::join(text, " | "), queryToString(table_join)); - } } diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 37f46d6a65e..c81f4a193c3 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -440,7 +440,7 @@ void checkTypesOfKeys(const Block & block_left, const Names & key_names_left, { throw DB::Exception( ErrorCodes::TYPE_MISMATCH, - "Type mismatch of columns to JOIN by: {} {} at left, {} {} at right", + "Type mismatch of columns to JOIN by: {} :: {} at left, {} :: {} at right", key_names_left[i], left_type->getName(), key_names_right[i], right_type->getName()); } diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index a27aea21e3e..981f59d8849 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -42,7 +42,7 @@ FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns) desc.reserve(columns.size()); for (const auto & name : columns) desc.emplace_back(name); - return std::make_unique(block, desc); + return std::make_unique(materializeBlock(block), desc); } template @@ -227,22 +227,6 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( const auto & join_on = table_join->getTableJoin().getOnlyClause(); - for (const auto & key : join_on.key_names_left) - { - if (input_headers[0].getByName(key).type->lowCardinality() || - input_headers[0].getByName(key).type->isLowCardinalityNullable()) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support low cardinality columns"); - } - } - for (const auto & key : join_on.key_names_right) - { - if (input_headers[1].getByName(key).type->lowCardinality() || - input_headers[1].getByName(key).type->isLowCardinalityNullable()) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support low cardinality columns"); - } - } if (join_on.on_filter_condition_left || join_on.on_filter_condition_right) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support ON filter conditions"); @@ -303,20 +287,6 @@ static Chunk createBlockWithDefaults(FullMergeJoinCursor & lhs, const Chunk & rh return createBlockWithDefaults(lhs.detach(), rhs, start, num_rows); } -void MergeJoinAlgorithm::initialize(Inputs inputs) -{ - if (inputs.size() != 2) - throw Exception("MergeJoinAlgorithm requires exactly two inputs", ErrorCodes::LOGICAL_ERROR); - - LOG_DEBUG(log, "MergeJoinAlgorithm initialize, number of inputs: {}", inputs.size()); - - for (size_t i = 0; i < inputs.size(); ++i) - { - copyColumnsResized(inputs[i].chunk, 0, 0, sample_chunks.emplace_back()); - consume(inputs[i], i); - } -} - static void prepareChunk(Chunk & chunk) { if (!chunk) @@ -330,6 +300,21 @@ static void prepareChunk(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } +void MergeJoinAlgorithm::initialize(Inputs inputs) +{ + if (inputs.size() != 2) + throw Exception("MergeJoinAlgorithm requires exactly two inputs", ErrorCodes::LOGICAL_ERROR); + + LOG_DEBUG(log, "MergeJoinAlgorithm initialize, number of inputs: {}", inputs.size()); + for (size_t i = 0; i < inputs.size(); ++i) + { + assert(inputs[i].chunk.getNumColumns() == cursors[i]->sampleBlock().columns()); + prepareChunk(inputs[i].chunk); + copyColumnsResized(inputs[i].chunk, 0, 0, sample_chunks.emplace_back()); + consume(inputs[i], i); + } +} + void MergeJoinAlgorithm::consume(Input & input, size_t source_num) { if (input.skip_last_row) @@ -418,6 +403,7 @@ struct AllJoinImpl } else { + assert(state == nullptr); state = std::make_unique(left_cursor.cursor, lpos, right_cursor.cursor, rpos); state->addRange(0, left_cursor.getCurrent().clone(), lpos, lnum); state->addRange(1, right_cursor.getCurrent().clone(), rpos, rnum); diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index d5a0f6be52b..1824b201724 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -26,11 +26,6 @@ namespace Poco { class Logger; } namespace DB { -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - class IJoin; using JoinPtr = std::shared_ptr; @@ -215,6 +210,8 @@ public: SortCursorImpl cursor; + const Block & sampleBlock() const { return sample_block; } + private: Block sample_block; SortDescription desc; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 45f36264559..fc2e8624f2a 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -345,6 +345,9 @@ std::unique_ptr QueryPipelineBuilder::joinPipelines2( if (left->pipe.output_ports.size() != 1 || right->pipe.output_ports.size() != 1) throw Exception("Join is supported only for pipelines with one output port", ErrorCodes::LOGICAL_ERROR); + if (left->hasTotals() || right->hasTotals()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Current join algorithm is supported only for pipelines without totals"); + Blocks inputs = {left->getHeader(), right->getHeader()}; auto joining = std::make_shared(join, inputs, out_header, max_block_size); diff --git a/tests/queries/0_stateless/02276_full_sort_join_unsupported.reference b/tests/queries/0_stateless/02276_full_sort_join_unsupported.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql new file mode 100644 index 00000000000..a10025e7b3f --- /dev/null +++ b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql @@ -0,0 +1,43 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (key UInt32, val UInt32) ENGINE = Memory; +INSERT INTO t1 VALUES (1, 1); + +CREATE TABLE t2 (key UInt32, val UInt32) ENGINE = Memory; +INSERT INTO t2 VALUES (1, 2); + +SET join_algorithm = 'full_sorting_merge'; + +SELECT * FROM t1 JOIN t2 ON t1.key = t2.key AND t2.key > 0; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM t1 JOIN t2 ON t1.key = t2.key AND t1.key > 0; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM t1 JOIN t2 ON t1.key = t2.key OR t1.val = t2.key; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM t1 ANTI JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM t1 SEMI JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM t1 ASOF JOIN t2 ON t1.key = t2.key AND t1.val > t2.val; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM t1 ANY JOIN t2 ON t1.key = t2.key SETTINGS any_join_distinct_right_table_keys = 1; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM ( SELECT key, sum(val) AS val FROM t1 GROUP BY key WITH TOTALS ) as t1 +JOIN ( SELECT key, sum(val) AS val FROM t2 GROUP BY key WITH TOTALS ) as t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM t1 JOIN ( SELECT key, sum(val) AS val FROM t2 GROUP BY key WITH TOTALS ) as t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM ( SELECT key, sum(val) AS val FROM t1 GROUP BY key WITH TOTALS ) as t1 JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM ( SELECT 'a' :: LowCardinality(String) AS key ) AS t1 +JOIN ( SELECT 'a' :: String AS key ) AS t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM ( SELECT 'a' :: LowCardinality(Nullable(String)) AS key ) AS t1 +JOIN ( SELECT 'a' :: String AS key ) AS t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } + +SELECT * FROM ( SELECT 'a' :: LowCardinality(Nullable(String)) AS key ) AS t1 +JOIN ( SELECT 'a' :: Nullable(String) AS key ) AS t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/02277_full_sort_join_misc.reference b/tests/queries/0_stateless/02277_full_sort_join_misc.reference new file mode 100644 index 00000000000..f648ee7e9c3 --- /dev/null +++ b/tests/queries/0_stateless/02277_full_sort_join_misc.reference @@ -0,0 +1,6 @@ +1 1 +1 +1 1 +-1 0 +\N 4294967295 +a a diff --git a/tests/queries/0_stateless/02277_full_sort_join_misc.sql b/tests/queries/0_stateless/02277_full_sort_join_misc.sql new file mode 100644 index 00000000000..f36e6b54cbd --- /dev/null +++ b/tests/queries/0_stateless/02277_full_sort_join_misc.sql @@ -0,0 +1,11 @@ +SET join_algorithm = 'full_sorting_merge'; + +SELECT * FROM (SELECT 1 as key ) AS t1 JOIN (SELECT 1 as key) t2 ON t1.key = t2.key; + +SELECT * FROM (SELECT 1 as key ) AS t1 JOIN (SELECT 1 as key) t2 USING key; + +SELECT * FROM (SELECT 1 :: Int32 as key ) AS t1 JOIN (SELECT 1 :: UInt32 as key) t2 ON t1.key = t2.key; + +SELECT * FROM (SELECT -1 :: Nullable(Int32) as key ) AS t1 FULL JOIN (SELECT 4294967295 :: UInt32 as key) t2 ON t1.key = t2.key; + +SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: LowCardinality(String) AS key) AS t2 ON t1.key = t2.key; From a90ac59ee5065b9ff6b111a687eb6c32eda96c58 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 29 Apr 2022 14:02:00 +0000 Subject: [PATCH 297/627] MergeJoinAlgorithm::createBlockWithDefaults --- .../Transforms/MergeJoinTransform.cpp | 177 +++++++++++++----- .../Transforms/MergeJoinTransform.h | 2 + 2 files changed, 133 insertions(+), 46 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 981f59d8849..fb27823ce2e 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -190,6 +190,8 @@ Chunk FullMergeJoinCursor::detach() void FullMergeJoinCursor::setChunk(Chunk && chunk) { assert(!recieved_all_blocks); + assert(!cursor.isValid()); + if (!chunk) { recieved_all_blocks = true; @@ -234,16 +236,16 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( cursors.push_back(createCursor(input_headers[1], join_on.key_names_right)); } -static ColumnPtr replicateRow(const IColumn & column, size_t pos, size_t num) +static ColumnPtr replicateRow(const IColumn & column, size_t num) { MutableColumnPtr res = column.cloneEmpty(); - res->insertManyFrom(column, pos, num); + res->insertManyFrom(column, 0, num); return res; } -static void copyColumnsResized(const Chunk & chunk, size_t start, size_t size, Chunk & result_chunk) +template +static void copyColumnsResized(const TColumns & cols, size_t start, size_t size, Chunk & result_chunk) { - const auto & cols = chunk.getColumns(); for (const auto & col : cols) { if (col->empty()) @@ -254,7 +256,7 @@ static void copyColumnsResized(const Chunk & chunk, size_t start, size_t size, C else if (col->size() == 1) { /// copy same row n times - result_chunk.addColumn(replicateRow(*col, 0, size)); + result_chunk.addColumn(replicateRow(*col, size)); } else { @@ -265,28 +267,14 @@ static void copyColumnsResized(const Chunk & chunk, size_t start, size_t size, C } } -static Chunk createBlockWithDefaults(const Chunk & lhs, const Chunk & rhs, size_t start, size_t num_rows) +static Chunk copyChunkResized(const Chunk & lhs, const Chunk & rhs, size_t start, size_t num_rows) { Chunk result; - copyColumnsResized(lhs, start, num_rows, result); - copyColumnsResized(rhs, start, num_rows, result); + copyColumnsResized(lhs.getColumns(), start, num_rows, result); + copyColumnsResized(rhs.getColumns(), start, num_rows, result); return result; } -static Chunk createBlockWithDefaults(const Chunk & lhs, FullMergeJoinCursor & rhs) -{ - size_t start = rhs->getRow(); - size_t num_rows = rhs->rowsLeft(); - return createBlockWithDefaults(lhs, rhs.detach(), start, num_rows); -} - -static Chunk createBlockWithDefaults(FullMergeJoinCursor & lhs, const Chunk & rhs) -{ - size_t start = lhs->getRow(); - size_t num_rows = lhs->rowsLeft(); - return createBlockWithDefaults(lhs.detach(), rhs, start, num_rows); -} - static void prepareChunk(Chunk & chunk) { if (!chunk) @@ -310,7 +298,7 @@ void MergeJoinAlgorithm::initialize(Inputs inputs) { assert(inputs[i].chunk.getNumColumns() == cursors[i]->sampleBlock().columns()); prepareChunk(inputs[i].chunk); - copyColumnsResized(inputs[i].chunk, 0, 0, sample_chunks.emplace_back()); + copyColumnsResized(inputs[i].chunk.getColumns(), 0, 0, sample_chunks.emplace_back()); consume(inputs[i], i); } } @@ -336,7 +324,7 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) static Chunk getRowFromChunk(const Chunk & chunk, size_t pos) { Chunk result; - copyColumnsResized(chunk, pos, 1, result); + copyColumnsResized(chunk.getColumns(), pos, 1, result); return result; } @@ -365,15 +353,10 @@ struct AllJoinImpl PaddedPODArray & right_map, std::unique_ptr & state) { - size_t num_rows = isLeft(kind) ? left_cursor->rowsLeft() : - isRight(kind) ? right_cursor->rowsLeft() : - std::min(left_cursor->rowsLeft(), right_cursor->rowsLeft()); - - if constexpr (isLeft(kind) || isInner(kind)) - right_map.reserve(num_rows); - - if constexpr (isRight(kind) || isInner(kind)) - left_map.reserve(num_rows); + right_map.clear(); + right_map.reserve(max_block_size); + left_map.clear(); + left_map.reserve(max_block_size); size_t rpos = std::numeric_limits::max(); size_t lpos = std::numeric_limits::max(); @@ -413,7 +396,7 @@ struct AllJoinImpl else if (cmp < 0) { size_t num = nextDistinct(left_cursor.cursor); - if constexpr (isLeft(kind)) + if constexpr (isLeftOrFull(kind)) { right_map.resize_fill(right_map.size() + num, right_cursor->rows); for (size_t i = lpos; i < left_cursor->getRow(); ++i) @@ -423,7 +406,7 @@ struct AllJoinImpl else { size_t num = nextDistinct(right_cursor.cursor); - if constexpr (isRight(kind)) + if constexpr (isRightOrFull(kind)) { left_map.resize_fill(left_map.size() + num, left_cursor->rows); for (size_t i = rpos; i < right_cursor->getRow(); ++i) @@ -545,6 +528,8 @@ struct AnyJoinImpl PaddedPODArray & right_map, AnyJoinState & state) { + assert(enabled); + size_t num_rows = isLeft(kind) ? left_cursor->rowsLeft() : isRight(kind) ? right_cursor->rowsLeft() : std::min(left_cursor->rowsLeft(), right_cursor->rowsLeft()); @@ -567,13 +552,13 @@ struct AnyJoinImpl cmp = compareCursors(left_cursor.cursor, right_cursor.cursor); if (cmp == 0) { - if constexpr (isLeft(kind)) + if constexpr (isLeftOrFull(kind)) { size_t lnum = nextDistinct(left_cursor.cursor); right_map.resize_fill(right_map.size() + lnum, rpos); } - if constexpr (isRight(kind)) + if constexpr (isRightOrFull(kind)) { size_t rnum = nextDistinct(right_cursor.cursor); left_map.resize_fill(left_map.size() + rnum, lpos); @@ -590,13 +575,13 @@ struct AnyJoinImpl else if (cmp < 0) { size_t num = nextDistinct(left_cursor.cursor); - if constexpr (isLeft(kind)) + if constexpr (isLeftOrFull(kind)) right_map.resize_fill(right_map.size() + num, right_cursor->rows); } else { size_t num = nextDistinct(right_cursor.cursor); - if constexpr (isRight(kind)) + if constexpr (isRightOrFull(kind)) left_map.resize_fill(left_map.size() + num, left_cursor->rows); } } @@ -638,9 +623,9 @@ std::optional MergeJoinAlgorithm::handleAnyJoinState size_t start_pos = current->getRow(); size_t length = nextDistinct(current.cursor); if (length && isLeft(kind) && source_num == 0) - result = createBlockWithDefaults(current.getCurrent(), state.value ? state.value : sample_chunks[1 - source_num], start_pos, length); + result = copyChunkResized(current.getCurrent(), state.value ? state.value : sample_chunks[1 - source_num], start_pos, length); if (length && isRight(kind) && source_num == 1) - result = createBlockWithDefaults(state.value ? state.value : sample_chunks[1 - source_num], current.getCurrent(), start_pos, length); + result = copyChunkResized(state.value ? state.value : sample_chunks[1 - source_num], current.getCurrent(), start_pos, length); /// We've found row with other key, no need to skip more rows with current key if (current->isValid()) @@ -689,6 +674,106 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) return Status(std::move(result)); } + +String logRow(const SortCursorImpl & lhs, std::optional pos = {}) +{ + std::vector text; + size_t lhs_pos = pos.value_or(lhs.getRow()); + + if (lhs_pos >= lhs.rows) + return fmt::format("[{}/{}] : END", lhs_pos, lhs.rows); + + for (size_t i = 0; i < lhs.sort_columns_size; ++i) + { + if (const auto * left_nullable = checkAndGetColumn(lhs.sort_columns[i])) + { + if (left_nullable->isNullAt(lhs_pos)) + { + text.push_back(""); + } + else + { + Field f; + left_nullable->getNestedColumn().get(lhs_pos, f); + text.push_back(fmt::format("<{}>", f.dump())); + } + } + else + { + Field f; + lhs.sort_columns[i]->get(lhs_pos, f); + text.push_back(fmt::format("{}", f.dump())); + } + } + return fmt::format("[{}/{}] : ({})", lhs_pos, lhs.rows, fmt::join(text, ", ")); +} + + +String dumpBlock(Block block, const Chunk & chunk = {}) +{ + if (chunk) + { + assert(block.columns() == chunk.getNumColumns()); + block.setColumns(chunk.getColumns()); + } + + WriteBufferFromOwnString out; + for (size_t i = 0; i < block.columns(); ++i) + { + const auto & col = block.safeGetByPosition(i); + writeString(col.name, out); + writeString(" :: ", out); + writeString(col.type->getName(), out); + writeString(" [", out); + + size_t limit = std::min(col.column->size(), 10); + for (size_t j = 0; j < limit; ++j) + { + if (j != 0) + writeString(", ", out); + { + Field f; + col.column->get(j, f); + writeString(f.dump(), out); + } + + // { + // FormatSettings fs; + // col.type->getDefaultSerialization()->serializeText(*col.column, j, out, fs); + // } + } + if (limit < col.column->size()) + writeString("...", out); + writeString("] ", out); + } + out.finalize(); + return out.str(); +} + +/// if `source_num == 0` get data from left cursor and fill defaults at right +/// otherwise vise versa +Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num) +{ + size_t start = cursors[source_num]->cursor.getRow(); + size_t num_rows = cursors[source_num]->cursor.rowsLeft(); + + ColumnRawPtrs cols; + { + const Chunk & chunk_left = source_num == 0 ? cursors[0]->getCurrent() : sample_chunks[0]; + for (const auto & col : chunk_left.getColumns()) + cols.push_back(col.get()); + + const Chunk & chunk_right = source_num == 1 ? cursors[1]->getCurrent() : sample_chunks[1]; + for (const auto & col : chunk_right.getColumns()) + cols.push_back(col.get()); + } + + Chunk result_chunk; + copyColumnsResized(cols, start, num_rows, result_chunk); + cursors[source_num]->detach(); + return result_chunk; +} + IMergingAlgorithm::Status MergeJoinAlgorithm::merge() { auto kind = table_join->getTableJoin().kind(); @@ -705,21 +790,21 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() if (cursors[0]->fullyCompleted() || cursors[1]->fullyCompleted()) { if (!cursors[0]->fullyCompleted() && isLeftOrFull(kind)) - return Status(createBlockWithDefaults(*cursors[0], sample_chunks[1])); + return Status(createBlockWithDefaults(0)); if (!cursors[1]->fullyCompleted() && isRightOrFull(kind)) - return Status(createBlockWithDefaults(sample_chunks[0], *cursors[1])); + return Status(createBlockWithDefaults(1)); return Status({}, true); } /// check if blocks are not intersecting at all - if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor); cmp != 0) + if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor); cmp == 111) { if (cmp < 0) { if (isLeftOrFull(kind)) - return Status(createBlockWithDefaults(*cursors[0], sample_chunks[1])); + return Status(createBlockWithDefaults(0)); cursors[0]->detach(); return Status(0); } @@ -727,7 +812,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() if (cmp > 0) { if (isRightOrFull(kind)) - return Status(createBlockWithDefaults(sample_chunks[0], *cursors[1])); + return Status(createBlockWithDefaults(1)); cursors[1]->detach(); return Status(1); } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 1824b201724..a40308c8cfe 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -250,6 +250,8 @@ private: std::optional handleAllJoinState(); Status allJoin(ASTTableJoin::Kind kind); + Chunk createBlockWithDefaults(size_t source_num); + std::vector cursors; std::vector sample_chunks; From 753a567da8962681af27d577f01028b03150302b Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 2 May 2022 14:00:24 +0000 Subject: [PATCH 298/627] full sorting join with using --- src/Interpreters/TableJoin.cpp | 33 +-- src/Interpreters/TableJoin.h | 2 +- .../Transforms/MergeJoinTransform.cpp | 237 +++++++++--------- .../Transforms/MergeJoinTransform.h | 7 +- .../01720_join_implicit_cast.reference.j2 | 18 +- .../01720_join_implicit_cast.sql.j2 | 18 +- .../0_stateless/02273_full_sort_join.sql.j2 | 6 +- ...274_full_sort_join_nodistinct.reference.j2 | 214 ++++++++++++++++ .../02274_full_sort_join_nodistinct.sql.j2 | 17 +- .../02277_full_sort_join_misc.reference | 3 + .../0_stateless/02277_full_sort_join_misc.sql | 4 + 11 files changed, 408 insertions(+), 151 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index cdb7f42ce11..b5a97204a00 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -363,7 +363,7 @@ void TableJoin::addJoinedColumnsAndCorrectTypesImpl(TColumns & left_columns, boo * For `JOIN ON expr1 == expr2` we will infer common type later in makeTableJoin, * when part of plan built and types of expression will be known. */ - inferJoinKeyCommonType(left_columns, columns_from_joined_table, !isSpecialStorage()); + inferJoinKeyCommonType(left_columns, columns_from_joined_table, !isSpecialStorage(), forceFullSortingMergeJoin()); if (auto it = left_type_map.find(col.name); it != left_type_map.end()) { @@ -511,7 +511,7 @@ TableJoin::createConvertingActions( const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns) { - inferJoinKeyCommonType(left_sample_columns, right_sample_columns, !isSpecialStorage()); + inferJoinKeyCommonType(left_sample_columns, right_sample_columns, !isSpecialStorage(), forceFullSortingMergeJoin()); NameToNameMap left_key_column_rename; NameToNameMap right_key_column_rename; @@ -568,7 +568,7 @@ TableJoin::createConvertingActions( } template -void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right) +void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right, bool strict) { if (!left_type_map.empty() || !right_type_map.empty()) return; @@ -593,39 +593,42 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) { - auto ltype = left_types.find(left_key_name); - auto rtype = right_types.find(right_key_name); - if (ltype == left_types.end() || rtype == right_types.end()) + auto ltypeit = left_types.find(left_key_name); + auto rtypeit = right_types.find(right_key_name); + if (ltypeit == left_types.end() || rtypeit == right_types.end()) { /// Name mismatch, give up left_type_map.clear(); right_type_map.clear(); return false; } + const auto & ltype = ltypeit->second; + const auto & rtype = rtypeit->second; - if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second)) + bool type_equals = strict ? ltype->equals(*rtype) : JoinCommon::typesEqualUpToNullability(ltype, rtype); + if (type_equals) return true; DataTypePtr common_type; try { /// TODO(vdimir): use getMostSubtype if possible - common_type = DB::getLeastSupertype(DataTypes{ltype->second, rtype->second}); + common_type = DB::getLeastSupertype(DataTypes{ltype, rtype}); } catch (DB::Exception & ex) { throw DB::Exception(ErrorCodes::TYPE_MISMATCH, - "Can't infer common type for joined columns: {}: {} at left, {}: {} at right ({})", - left_key_name, ltype->second->getName(), - right_key_name, rtype->second->getName(), + "Can't infer common type for joined columns: {}: {} at left, {}: {} at right. {}", + left_key_name, ltype->getName(), + right_key_name, rtype->getName(), ex.message()); } - bool right_side_changed = !common_type->equals(*rtype->second); - if (right_side_changed && !allow_right) + + if (!allow_right && !common_type->equals(*rtype)) { throw DB::Exception(ErrorCodes::TYPE_MISMATCH, - "Can't change type for right table: {}: {} -> {}", - right_key_name, rtype->second->getName(), common_type->getName()); + "Can't change type for right table: {}: {} -> {}.", + right_key_name, rtype->getName(), common_type->getName()); } left_type_map[left_key_name] = right_type_map[right_key_name] = common_type; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 45a4106d040..38540f695d9 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -166,7 +166,7 @@ private: /// Calculates common supertypes for corresponding join key columns. template - void inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right); + void inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right, bool strict); NamesAndTypesList correctedColumnsAddedByJoin() const; diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index fb27823ce2e..aa0a89bce7d 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include @@ -124,27 +123,30 @@ int ALWAYS_INLINE totallyCompare(SortCursorImpl & lhs, SortCursorImpl & rhs) return 0; } -void addIndexColumn(const Columns & columns, ColumnUInt64 & indices, Chunk & result, size_t start, size_t limit) +ColumnPtr indexColumn(const ColumnPtr & column, const PaddedPODArray & indices) { - for (const auto & col : columns) + auto new_col = column->cloneEmpty(); + new_col->reserve(indices.size()); + for (size_t i = 0; i < indices.size(); ++i) { - if (indices.empty()) - { - result.addColumn(col->cut(start, limit)); - } + /// rows where default value should be inserted have index == size + if (indices[i] < column->size()) + new_col->insertFrom(*column, indices[i]); else - { - if (limit == 0) - limit = indices.size(); - - assert(limit == indices.size()); - /// rows where default value should be inserted have index == size - /// add row with defaults to handle it - auto tmp_col = col->cloneResized(col->size() + 1); - ColumnPtr new_col = tmp_col->index(indices, limit); - result.addColumn(std::move(new_col)); - } + new_col->insertDefault(); } + return new_col; +} + +Columns indexColumns(const Columns & columns, const PaddedPODArray & indices) +{ + Columns new_columns; + new_columns.reserve(columns.size()); + for (const auto & column : columns) + { + new_columns.emplace_back(indexColumn(column, indices)); + } + return new_columns; } bool sameNext(const SortCursorImpl & impl, std::optional pos_opt = {}) @@ -234,6 +236,13 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( cursors.push_back(createCursor(input_headers[0], join_on.key_names_left)); cursors.push_back(createCursor(input_headers[1], join_on.key_names_right)); + + for (const auto & [left_key, right_key] : table_join->getTableJoin().leftToRightKeyRemap()) + { + size_t left_idx = input_headers[0].getPositionByName(left_key); + size_t right_idx = input_headers[1].getPositionByName(right_key); + left_to_right_key_remap[left_idx] = right_idx; + } } static ColumnPtr replicateRow(const IColumn & column, size_t num) @@ -501,18 +510,55 @@ std::optional MergeJoinAlgorithm::handleAllJoinState MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) { - auto left_map = ColumnUInt64::create(); - auto right_map = ColumnUInt64::create(); - size_t prev_pos[] = {cursors[0]->cursor.getRow(), cursors[1]->cursor.getRow()}; + PaddedPODArray idx_map[2]; - dispatchKind(kind, *cursors[0], *cursors[1], max_block_size, left_map->getData(), right_map->getData(), all_join_state); - - assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); - size_t num_result_rows = std::max(left_map->size(), right_map->size()); + dispatchKind(kind, *cursors[0], *cursors[1], max_block_size, idx_map[0], idx_map[1], all_join_state); + assert(idx_map[0].size() == idx_map[1].size()); Chunk result; - addIndexColumn(cursors[0]->getCurrent().getColumns(), *left_map, result, prev_pos[0], num_result_rows); - addIndexColumn(cursors[1]->getCurrent().getColumns(), *right_map, result, prev_pos[1], num_result_rows); + + Columns rcols = indexColumns(cursors[1]->getCurrent().getColumns(), idx_map[1]); + Columns lcols; + if (!left_to_right_key_remap.empty()) + { + /// If we have remapped columns, then we need to get values from right columns insead of defaults + const auto & indices = idx_map[0]; + const auto & right_indices = idx_map[1]; + + const auto & left_src = cursors[0]->getCurrent().getColumns(); + for (size_t col_idx = 0; col_idx < left_src.size(); ++col_idx) + { + const auto & col = left_src[col_idx]; + auto new_col = col->cloneEmpty(); + new_col->reserve(indices.size()); + for (size_t i = 0; i < indices.size(); ++i) + { + if (indices[i] < col->size()) + { + new_col->insertFrom(*col, indices[i]); + } + else + { + if (auto it = left_to_right_key_remap.find(col_idx); it != left_to_right_key_remap.end()) + new_col->insertFrom(*rcols[it->second], right_indices[i]); + else + new_col->insertDefault(); + } + } + lcols.push_back(std::move(new_col)); + } + } + else + { + lcols = indexColumns(cursors[0]->getCurrent().getColumns(), idx_map[0]); + } + + for (auto & col : lcols) + result.addColumn(std::move(col)); + + for (auto & col : rcols) + result.addColumn(std::move(col)); + return Status(std::move(result)); } @@ -622,10 +668,22 @@ std::optional MergeJoinAlgorithm::handleAnyJoinState { size_t start_pos = current->getRow(); size_t length = nextDistinct(current.cursor); + if (length && isLeft(kind) && source_num == 0) - result = copyChunkResized(current.getCurrent(), state.value ? state.value : sample_chunks[1 - source_num], start_pos, length); + { + if (state.value) + result = copyChunkResized(current.getCurrent(), state.value, start_pos, length); + else + result = createBlockWithDefaults(source_num, start_pos, length); + } + if (length && isRight(kind) && source_num == 1) - result = copyChunkResized(state.value ? state.value : sample_chunks[1 - source_num], current.getCurrent(), start_pos, length); + { + if (state.value) + result = copyChunkResized(state.value, current.getCurrent(), start_pos, length); + else + result = createBlockWithDefaults(source_num, start_pos, length); + } /// We've found row with other key, no need to skip more rows with current key if (current->isValid()) @@ -658,118 +716,73 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) return Status(1); /// join doesn't build result block, but returns indices where result rows should be placed - auto left_map = ColumnUInt64::create(); - auto right_map = ColumnUInt64::create(); + PaddedPODArray idx_map[2]; size_t prev_pos[] = {current_left.getRow(), current_right.getRow()}; - dispatchKind(kind, *cursors[0], *cursors[1], left_map->getData(), right_map->getData(), any_join_state); + dispatchKind(kind, *cursors[0], *cursors[1], idx_map[0], idx_map[1], any_join_state); - assert(left_map->empty() || right_map->empty() || left_map->size() == right_map->size()); - size_t num_result_rows = std::max(left_map->size(), right_map->size()); + assert(idx_map[0].empty() || idx_map[1].empty() || idx_map[0].size() == idx_map[1].size()); + size_t num_result_rows = std::max(idx_map[0].size(), idx_map[1].size()); /// build result block from indices Chunk result; - addIndexColumn(cursors[0]->getCurrent().getColumns(), *left_map, result, prev_pos[0], num_result_rows); - addIndexColumn(cursors[1]->getCurrent().getColumns(), *right_map, result, prev_pos[1], num_result_rows); - return Status(std::move(result)); -} - - -String logRow(const SortCursorImpl & lhs, std::optional pos = {}) -{ - std::vector text; - size_t lhs_pos = pos.value_or(lhs.getRow()); - - if (lhs_pos >= lhs.rows) - return fmt::format("[{}/{}] : END", lhs_pos, lhs.rows); - - for (size_t i = 0; i < lhs.sort_columns_size; ++i) + for (size_t source_num = 0; source_num < 2; ++source_num) { - if (const auto * left_nullable = checkAndGetColumn(lhs.sort_columns[i])) + /// empty map means identity mapping + if (!idx_map[source_num].empty()) { - if (left_nullable->isNullAt(lhs_pos)) + for (const auto & col : cursors[source_num]->getCurrent().getColumns()) { - text.push_back(""); - } - else - { - Field f; - left_nullable->getNestedColumn().get(lhs_pos, f); - text.push_back(fmt::format("<{}>", f.dump())); + result.addColumn(indexColumn(col, idx_map[source_num])); } } else { - Field f; - lhs.sort_columns[i]->get(lhs_pos, f); - text.push_back(fmt::format("{}", f.dump())); - } - } - return fmt::format("[{}/{}] : ({})", lhs_pos, lhs.rows, fmt::join(text, ", ")); -} - - -String dumpBlock(Block block, const Chunk & chunk = {}) -{ - if (chunk) - { - assert(block.columns() == chunk.getNumColumns()); - block.setColumns(chunk.getColumns()); - } - - WriteBufferFromOwnString out; - for (size_t i = 0; i < block.columns(); ++i) - { - const auto & col = block.safeGetByPosition(i); - writeString(col.name, out); - writeString(" :: ", out); - writeString(col.type->getName(), out); - writeString(" [", out); - - size_t limit = std::min(col.column->size(), 10); - for (size_t j = 0; j < limit; ++j) - { - if (j != 0) - writeString(", ", out); + for (const auto & col : cursors[source_num]->getCurrent().getColumns()) { - Field f; - col.column->get(j, f); - writeString(f.dump(), out); + result.addColumn(col->cut(prev_pos[source_num], num_result_rows)); } - - // { - // FormatSettings fs; - // col.type->getDefaultSerialization()->serializeText(*col.column, j, out, fs); - // } } - if (limit < col.column->size()) - writeString("...", out); - writeString("] ", out); } - out.finalize(); - return out.str(); + return Status(std::move(result)); } /// if `source_num == 0` get data from left cursor and fill defaults at right -/// otherwise vise versa -Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num) +/// otherwise - vice versa +Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) { - size_t start = cursors[source_num]->cursor.getRow(); - size_t num_rows = cursors[source_num]->cursor.rowsLeft(); - ColumnRawPtrs cols; { - const Chunk & chunk_left = source_num == 0 ? cursors[0]->getCurrent() : sample_chunks[0]; - for (const auto & col : chunk_left.getColumns()) - cols.push_back(col.get()); + const auto & columns_left = source_num == 0 ? cursors[0]->getCurrent().getColumns() : sample_chunks[0].getColumns(); + const auto & columns_right = source_num == 1 ? cursors[1]->getCurrent().getColumns() : sample_chunks[1].getColumns(); - const Chunk & chunk_right = source_num == 1 ? cursors[1]->getCurrent() : sample_chunks[1]; - for (const auto & col : chunk_right.getColumns()) + for (size_t i = 0; i < columns_left.size(); ++i) + { + if (auto it = left_to_right_key_remap.find(i); source_num == 0 || it == left_to_right_key_remap.end()) + { + cols.push_back(columns_left[i].get()); + } + else + { + cols.push_back(columns_right[it->second].get()); + } + } + + for (const auto & col : columns_right) + { cols.push_back(col.get()); + } } Chunk result_chunk; copyColumnsResized(cols, start, num_rows, result_chunk); + return result_chunk; +} + +/// This overload also flushes cursort +Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num) +{ + Chunk result_chunk = createBlockWithDefaults(source_num, cursors[source_num]->cursor.getRow(), cursors[source_num]->cursor.rowsLeft()); cursors[source_num]->detach(); return result_chunk; } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index a40308c8cfe..6e78f7d4e2e 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -5,13 +5,12 @@ #include #include #include +#include #include #include #include -#include - #include #include #include @@ -251,7 +250,11 @@ private: Status allJoin(ASTTableJoin::Kind kind); Chunk createBlockWithDefaults(size_t source_num); + Chunk createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows); + + /// For `USING` join key columns should have values from right side instead of defaults + std::unordered_map left_to_right_key_remap; std::vector cursors; std::vector sample_chunks; diff --git a/tests/queries/0_stateless/01720_join_implicit_cast.reference.j2 b/tests/queries/0_stateless/01720_join_implicit_cast.reference.j2 index 807088d2d5d..516030811ca 100644 --- a/tests/queries/0_stateless/01720_join_implicit_cast.reference.j2 +++ b/tests/queries/0_stateless/01720_join_implicit_cast.reference.j2 @@ -1,40 +1,40 @@ -{% for join_type in ['hash', 'partial_merge'] -%} +{% for join_type in ['hash', 'full_sorting_merge', 'partial_merge'] -%} === {{ join_type }} === -= full = += full using = 1 1 2 2 -1 1 1 \N 1 257 1 -1 -= left = += left using = 1 1 2 2 -= right = += right using = 1 1 -1 1 1 \N 1 257 1 -1 -= inner = += inner using = 1 1 -= full = += full on = 1 1 1 1 2 2 0 \N 0 0 -1 1 0 0 1 \N 0 0 1 257 0 0 1 -1 -= left = += left on = 1 1 1 1 2 2 0 \N -= right = += right on = 1 1 1 1 0 0 -1 1 0 0 1 \N 0 0 1 257 0 0 1 -1 -= inner = += inner on = 1 1 1 1 = agg = 5 260 diff --git a/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 b/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 index d1de6d06593..f7439c8c56c 100644 --- a/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 +++ b/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 @@ -6,28 +6,28 @@ CREATE TABLE t_ab2 (id Nullable(Int32), a Int16, b Nullable(Int64)) ENGINE = Tin INSERT INTO t_ab1 VALUES (0, 1, 1), (1, 2, 2); INSERT INTO t_ab2 VALUES (2, -1, 1), (3, 1, NULL), (4, 1, 257), (5, 1, -1), (6, 1, 1); -{% for join_type in ['hash', 'partial_merge'] -%} +{% for join_type in ['hash', 'full_sorting_merge', 'partial_merge'] -%} SELECT '=== {{ join_type }} ==='; SET join_algorithm = '{{ join_type }}'; -SELECT '= full ='; +SELECT '= full using ='; SELECT a, b FROM t_ab1 FULL JOIN t_ab2 USING (a, b) ORDER BY ifNull(t_ab1.id, t_ab2.id); -SELECT '= left ='; +SELECT '= left using ='; SELECT a, b FROM t_ab1 LEFT JOIN t_ab2 USING (a, b) ORDER BY ifNull(t_ab1.id, t_ab2.id); -SELECT '= right ='; +SELECT '= right using ='; SELECT a, b FROM t_ab1 RIGHT JOIN t_ab2 USING (a, b) ORDER BY ifNull(t_ab1.id, t_ab2.id); -SELECT '= inner ='; +SELECT '= inner using ='; SELECT a, b FROM t_ab1 INNER JOIN t_ab2 USING (a, b) ORDER BY ifNull(t_ab1.id, t_ab2.id); -SELECT '= full ='; +SELECT '= full on ='; SELECT a, b, t_ab2.a, t_ab2.b FROM t_ab1 FULL JOIN t_ab2 ON (t_ab1.a == t_ab2.a AND t_ab1.b == t_ab2.b) ORDER BY ifNull(t_ab1.id, t_ab2.id); -SELECT '= left ='; +SELECT '= left on ='; SELECT a, b, t_ab2.a, t_ab2.b FROM t_ab1 LEFT JOIN t_ab2 ON (t_ab1.a == t_ab2.a AND t_ab1.b == t_ab2.b) ORDER BY ifNull(t_ab1.id, t_ab2.id); -SELECT '= right ='; +SELECT '= right on ='; SELECT a, b, t_ab2.a, t_ab2.b FROM t_ab1 RIGHT JOIN t_ab2 ON (t_ab1.a == t_ab2.a AND t_ab1.b == t_ab2.b) ORDER BY ifNull(t_ab1.id, t_ab2.id); -SELECT '= inner ='; +SELECT '= inner on ='; SELECT a, b, t_ab2.a, t_ab2.b FROM t_ab1 INNER JOIN t_ab2 ON (t_ab1.a == t_ab2.a AND t_ab1.b == t_ab2.b) ORDER BY ifNull(t_ab1.id, t_ab2.id); SELECT '= agg ='; diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 5b1eaf99f20..b70d1e5f55f 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -1,3 +1,5 @@ +-- Tags: long + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; @@ -83,7 +85,6 @@ ORDER BY t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, t2.s {% endfor -%} {% endfor -%} - {% for kind in ['ALL', 'ANY'] -%} SET join_use_nulls = 1; @@ -131,3 +132,6 @@ ORDER BY t1.key1, t1.key2, t1.key3, t2.key1, t2.key2, t2.key3, t2.s ; {% endfor -%} + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 9fe48d539d6..ca2e47d7208 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -183,6 +183,112 @@ t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val27 2 2 5 val28 3 3 4 val3 +t1 ALL FULL JOIN t2 | bs = {{ block_size }} +1 1 4 5 +1 1 4 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +2 2 5 5 +3 3 4 4 +t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} +1 4 5 +1 4 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +2 5 5 +3 4 4 t1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -233,6 +339,42 @@ t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 +t1 ALL FULL JOIN tn2 | bs = {{ block_size }} +0 \N 0 5 +0 \N 0 5 +0 \N 0 5 +0 \N 0 5 +0 \N 0 5 +0 \N 0 5 +0 \N 0 5 +1 1 4 5 +1 1 4 5 +2 \N 5 0 +2 \N 5 0 +2 \N 5 0 +2 \N 5 0 +2 \N 5 0 +2 \N 5 0 +2 \N 5 0 +3 3 4 4 +t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} +1 4 5 +1 4 5 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +3 4 4 +\N 0 5 +\N 0 5 +\N 0 5 +\N 0 5 +\N 0 5 +\N 0 5 +\N 0 5 tn1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -283,6 +425,42 @@ tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 +tn1 ALL FULL JOIN t2 | bs = {{ block_size }} +1 1 4 5 +1 1 4 5 +3 3 4 4 +\N 0 5 0 +\N 0 5 0 +\N 0 5 0 +\N 0 5 0 +\N 0 5 0 +\N 0 5 0 +\N 0 5 0 +\N 2 0 5 +\N 2 0 5 +\N 2 0 5 +\N 2 0 5 +\N 2 0 5 +\N 2 0 5 +\N 2 0 5 +tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} +1 4 5 +1 4 5 +2 0 5 +2 0 5 +2 0 5 +2 0 5 +2 0 5 +2 0 5 +2 0 5 +3 4 4 +\N 5 0 +\N 5 0 +\N 5 0 +\N 5 0 +\N 5 0 +\N 5 0 +\N 5 0 tn1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -333,4 +511,40 @@ tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 +tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} +1 1 4 5 +1 1 4 5 +3 3 4 4 +\N \N 0 5 +\N \N 0 5 +\N \N 0 5 +\N \N 0 5 +\N \N 0 5 +\N \N 0 5 +\N \N 0 5 +\N \N 5 0 +\N \N 5 0 +\N \N 5 0 +\N \N 5 0 +\N \N 5 0 +\N \N 5 0 +\N \N 5 0 +tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} +1 4 5 +1 4 5 +3 4 4 +\N 0 5 +\N 0 5 +\N 0 5 +\N 0 5 +\N 0 5 +\N 0 5 +\N 0 5 +\N 5 0 +\N 5 0 +\N 5 0 +\N 5 0 +\N 5 0 +\N 5 0 +\N 5 0 {% endfor -%} diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index 4b69162cbc4..95d3a564016 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -1,3 +1,5 @@ +-- Tags: long + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS tn1; @@ -16,14 +18,13 @@ INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val2 SET join_algorithm = 'full_sorting_merge'; {% for block_size in range(1, 11) -%} - SET max_block_size = {{ block_size }}; {% for t1, t2 in [('t1', 't2'), ('t1', 'tn2'), ('tn1', 't2'), ('tn1', 'tn2')] -%} {% for kind in ['ALL', 'ANY'] -%} SELECT '{{ t1 }} {{ kind }} INNER JOIN {{ t2 }} | bs = {{ block_size }}'; -SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} INNER JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key; +SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} INNER JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key; SELECT '{{ t1 }} {{ kind }} LEFT JOIN {{ t2 }} | bs = {{ block_size }}'; SELECT t1.key, t2.key, t1.s, length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} LEFT JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t1.s; @@ -32,5 +33,17 @@ SELECT '{{ t1 }} {{ kind }} RIGHT JOIN {{ t2 }} | bs = {{ block_size }}'; SELECT t1.key, t2.key, length(t1.s), t2.s FROM {{ t1 }} AS t1 {{ kind }} RIGHT JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, t2.s; {% endfor -%} + +SELECT '{{ t1 }} ALL FULL JOIN {{ t2 }} | bs = {{ block_size }}'; +SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} FULL JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, length(t1.s), length(t2.s); + +SELECT '{{ t1 }} ALL FULL JOIN USING {{ t2 }} | bs = {{ block_size }}'; +SELECT key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 ALL FULL JOIN {{ t2 }} AS t2 USING (key) ORDER BY key, length(t1.s), length(t2.s); + {% endfor -%} {% endfor -%} + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS tn1; +DROP TABLE IF EXISTS tn2; diff --git a/tests/queries/0_stateless/02277_full_sort_join_misc.reference b/tests/queries/0_stateless/02277_full_sort_join_misc.reference index f648ee7e9c3..8bdbcad9115 100644 --- a/tests/queries/0_stateless/02277_full_sort_join_misc.reference +++ b/tests/queries/0_stateless/02277_full_sort_join_misc.reference @@ -1,5 +1,8 @@ 1 1 1 +1 +1 +\N 1 1 -1 0 \N 4294967295 diff --git a/tests/queries/0_stateless/02277_full_sort_join_misc.sql b/tests/queries/0_stateless/02277_full_sort_join_misc.sql index f36e6b54cbd..7a51bb779cd 100644 --- a/tests/queries/0_stateless/02277_full_sort_join_misc.sql +++ b/tests/queries/0_stateless/02277_full_sort_join_misc.sql @@ -4,6 +4,10 @@ SELECT * FROM (SELECT 1 as key ) AS t1 JOIN (SELECT 1 as key) t2 ON t1.key = t2. SELECT * FROM (SELECT 1 as key ) AS t1 JOIN (SELECT 1 as key) t2 USING key; +SELECT * FROM (SELECT 1 :: UInt32 as key ) AS t1 FULL JOIN (SELECT 1 :: Nullable(UInt32) as key) t2 USING (key); + +SELECT * FROM (SELECT 1 :: UInt32 as key ) AS t1 FULL JOIN (SELECT NULL :: Nullable(UInt32) as key) t2 USING (key); + SELECT * FROM (SELECT 1 :: Int32 as key ) AS t1 JOIN (SELECT 1 :: UInt32 as key) t2 ON t1.key = t2.key; SELECT * FROM (SELECT -1 :: Nullable(Int32) as key ) AS t1 FULL JOIN (SELECT 4294967295 :: UInt32 as key) t2 ON t1.key = t2.key; From c200ce490b34f99d21bbd61a80c552ede4f16e4e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 2 May 2022 15:54:42 +0000 Subject: [PATCH 299/627] full sorting join using and nulls --- src/Interpreters/FullSortingMergeJoin.h | 11 +- src/Interpreters/TableJoin.cpp | 2 +- src/Interpreters/TableJoin.h | 1 + .../01721_join_implicit_cast_long.reference | 201 ++++++++++++++++++ .../01721_join_implicit_cast_long.sql.j2 | 3 +- .../02276_full_sort_join_unsupported.sql | 11 +- .../02277_full_sort_join_misc.reference | 3 + .../0_stateless/02277_full_sort_join_misc.sql | 18 +- 8 files changed, 226 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 336295c4d63..a7429f6877e 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -36,7 +36,6 @@ public: throw Exception("FullSortingMergeJoin supports only one join key", ErrorCodes::NOT_IMPLEMENTED); const auto & onexpr = table_join->getOnlyClause(); - for (size_t i = 0; i < onexpr.key_names_left.size(); ++i) { DataTypePtr left_type = left_block.getByName(onexpr.key_names_left[i]).type; @@ -44,17 +43,17 @@ public: if (!removeNullable(left_type)->equals(*removeNullable(right_type))) { - DataTypePtr left_type_no_lc = removeNullable(recursiveRemoveLowCardinality(left_type)); - DataTypePtr right_type_no_lc = removeNullable(recursiveRemoveLowCardinality(right_type)); - /// if types equal after removing low cardinality, then it is ok and can be supported - bool equals_up_to_lc = left_type_no_lc->equals(*right_type_no_lc); throw DB::Exception( - equals_up_to_lc ? ErrorCodes::NOT_IMPLEMENTED : ErrorCodes::TYPE_MISMATCH, + ErrorCodes::TYPE_MISMATCH, "Type mismatch of columns to JOIN by: {} :: {} at left, {} :: {} at right", onexpr.key_names_left[i], left_type->getName(), onexpr.key_names_right[i], right_type->getName()); } } + + /// Key column can change nullability and it's not handled on type conversion stage, so algorithm shold be aware of it + if (table_join->hasUsing() && table_join->joinUseNulls()) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "FullSortingMergeJoin doesn't support USING with join_use_nulls"); } /// Used just to get result header diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index b5a97204a00..1352e3777f1 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -712,7 +712,7 @@ ActionsDAGPtr TableJoin::applyKeyConvertToTable( /// Create DAG to make columns nullable if needed if (make_nullable) { - /// Do not need to make nullable temporary columns that would be used only as join keys, but not shown to user + /// Do not need to make nullable temporary columns that would be used only as join keys, but is not visible to user NameSet cols_not_nullable; for (const auto & t : key_column_rename) cols_not_nullable.insert(t.second); diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 38540f695d9..f953ebb111e 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -212,6 +212,7 @@ public: || join_algorithm == MultiEnum(JoinAlgorithm::PARALLEL_HASH); } + bool joinUseNulls() const { return join_use_nulls; } bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(table_join.kind); } bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(table_join.kind); } size_t defaultMaxBytes() const { return default_max_bytes; } diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference index 51a20d9f524..07c240fa784 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference @@ -400,6 +400,207 @@ 1 1 1 +=== full_sorting_merge === += full = +-4 0 196 +-3 0 197 +-2 0 198 +-1 0 199 +0 0 200 +1 101 201 +2 102 202 +3 103 203 +4 104 204 +5 105 205 +6 106 \N +7 107 \N +8 108 \N +9 109 \N +10 110 \N += left = +1 101 201 +2 102 202 +3 103 203 +4 104 204 +5 105 205 +6 106 \N +7 107 \N +8 108 \N +9 109 \N +10 110 \N += right = +-4 0 196 +-3 0 197 +-2 0 198 +-1 0 199 +0 0 200 +1 101 201 +2 102 202 +3 103 203 +4 104 204 +5 105 205 += inner = +1 101 201 +2 102 202 +3 103 203 +4 104 204 +5 105 205 += full = +0 0 -4 +0 0 -3 +0 0 -2 +0 0 -1 +0 0 0 +1 1 1 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +6 6 0 +7 7 0 +8 8 0 +9 9 0 +10 10 0 += left = +1 1 1 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +6 6 0 +7 7 0 +8 8 0 +9 9 0 +10 10 0 += right = +0 0 -4 +0 0 -3 +0 0 -2 +0 0 -1 +0 0 0 +1 1 1 +2 2 2 +3 3 3 +4 4 4 +5 5 5 += inner = +1 1 1 +2 2 2 +3 3 3 +4 4 4 +5 5 5 += join on = += full = +0 0 -4 196 +0 0 -3 197 +0 0 -2 198 +0 0 -1 199 +0 0 0 200 +1 101 1 201 +2 102 2 202 +3 103 3 203 +4 104 4 204 +5 105 5 205 +6 106 0 \N +7 107 0 \N +8 108 0 \N +9 109 0 \N +10 110 0 \N += left = +1 101 1 201 +2 102 2 202 +3 103 3 203 +4 104 4 204 +5 105 5 205 +6 106 0 \N +7 107 0 \N +8 108 0 \N +9 109 0 \N +10 110 0 \N += right = +0 0 -4 196 +0 0 -3 197 +0 0 -2 198 +0 0 -1 199 +0 0 0 200 +1 101 1 201 +2 102 2 202 +3 103 3 203 +4 104 4 204 +5 105 5 205 += inner = +1 101 1 201 +2 102 2 202 +3 103 3 203 +4 104 4 204 +5 105 5 205 += full = +0 0 -4 196 +0 0 -3 197 +0 0 -2 198 +0 0 -1 199 +0 0 0 200 +1 101 1 201 +2 102 2 202 +3 103 3 203 +4 104 4 204 +5 105 5 205 +6 106 0 \N +7 107 0 \N +8 108 0 \N +9 109 0 \N +10 110 0 \N += left = +1 101 1 201 +2 102 2 202 +3 103 3 203 +4 104 4 204 +5 105 5 205 +6 106 0 \N +7 107 0 \N +8 108 0 \N +9 109 0 \N +10 110 0 \N += right = +0 0 -4 196 +0 0 -3 197 +0 0 -2 198 +0 0 -1 199 +0 0 0 200 +1 101 1 201 +2 102 2 202 +3 103 3 203 +4 104 4 204 +5 105 5 205 += inner = +1 101 1 201 +2 102 2 202 +3 103 3 203 +4 104 4 204 +5 105 5 205 += agg = +1 +1 +1 +1 +1 +1 +0 -10 0 +1 55 1055 +0 0 -10 0 990 +1 55 15 1055 1015 += types = +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 === auto === = full = -4 0 196 diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 index 8c3ed03fce7..3846f527bba 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 @@ -9,7 +9,7 @@ CREATE TABLE t2 (a Int16, b Nullable(Int64)) ENGINE = TinyLog; INSERT INTO t1 SELECT number as a, 100 + number as b FROM system.numbers LIMIT 1, 10; INSERT INTO t2 SELECT number - 5 as a, 200 + number - 5 as b FROM system.numbers LIMIT 1, 10; -{% for join_type in ['hash', 'partial_merge', 'auto'] -%} +{% for join_type in ['hash', 'partial_merge', 'full_sorting_merge', 'auto'] -%} SELECT '=== {{ join_type }} ==='; SET join_algorithm = '{{ join_type }}'; @@ -99,7 +99,6 @@ SELECT '=== join use nulls ==='; SET join_use_nulls = 1; - SELECT '= full ='; SELECT a, b, t2.b FROM t1 FULL JOIN t2 USING (a) ORDER BY (a); SELECT '= left ='; diff --git a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql index a10025e7b3f..a4e60ff54dd 100644 --- a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql +++ b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql @@ -23,6 +23,8 @@ SELECT * FROM t1 ASOF JOIN t2 ON t1.key = t2.key AND t1.val > t2.val; -- { serve SELECT * FROM t1 ANY JOIN t2 ON t1.key = t2.key SETTINGS any_join_distinct_right_table_keys = 1; -- { serverError NOT_IMPLEMENTED } +SELECT * FROM t1 JOIN t2 USING (key) SETTINGS join_use_nulls = 1; -- { serverError NOT_IMPLEMENTED } + SELECT * FROM ( SELECT key, sum(val) AS val FROM t1 GROUP BY key WITH TOTALS ) as t1 JOIN ( SELECT key, sum(val) AS val FROM t2 GROUP BY key WITH TOTALS ) as t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } @@ -30,14 +32,5 @@ SELECT * FROM t1 JOIN ( SELECT key, sum(val) AS val FROM t2 GROUP BY key WITH TO SELECT * FROM ( SELECT key, sum(val) AS val FROM t1 GROUP BY key WITH TOTALS ) as t1 JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } -SELECT * FROM ( SELECT 'a' :: LowCardinality(String) AS key ) AS t1 -JOIN ( SELECT 'a' :: String AS key ) AS t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } - -SELECT * FROM ( SELECT 'a' :: LowCardinality(Nullable(String)) AS key ) AS t1 -JOIN ( SELECT 'a' :: String AS key ) AS t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } - -SELECT * FROM ( SELECT 'a' :: LowCardinality(Nullable(String)) AS key ) AS t1 -JOIN ( SELECT 'a' :: Nullable(String) AS key ) AS t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } - DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/02277_full_sort_join_misc.reference b/tests/queries/0_stateless/02277_full_sort_join_misc.reference index 8bdbcad9115..7a22f9ef5ea 100644 --- a/tests/queries/0_stateless/02277_full_sort_join_misc.reference +++ b/tests/queries/0_stateless/02277_full_sort_join_misc.reference @@ -7,3 +7,6 @@ -1 0 \N 4294967295 a a +a a +a a +a a diff --git a/tests/queries/0_stateless/02277_full_sort_join_misc.sql b/tests/queries/0_stateless/02277_full_sort_join_misc.sql index 7a51bb779cd..6621881f304 100644 --- a/tests/queries/0_stateless/02277_full_sort_join_misc.sql +++ b/tests/queries/0_stateless/02277_full_sort_join_misc.sql @@ -1,15 +1,21 @@ SET join_algorithm = 'full_sorting_merge'; -SELECT * FROM (SELECT 1 as key ) AS t1 JOIN (SELECT 1 as key) t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 1 as key) AS t1 JOIN (SELECT 1 as key) t2 ON t1.key = t2.key; -SELECT * FROM (SELECT 1 as key ) AS t1 JOIN (SELECT 1 as key) t2 USING key; +SELECT * FROM (SELECT 1 as key) AS t1 JOIN (SELECT 1 as key) t2 USING key; -SELECT * FROM (SELECT 1 :: UInt32 as key ) AS t1 FULL JOIN (SELECT 1 :: Nullable(UInt32) as key) t2 USING (key); +SELECT * FROM (SELECT 1 :: UInt32 as key) AS t1 FULL JOIN (SELECT 1 :: Nullable(UInt32) as key) t2 USING (key); -SELECT * FROM (SELECT 1 :: UInt32 as key ) AS t1 FULL JOIN (SELECT NULL :: Nullable(UInt32) as key) t2 USING (key); +SELECT * FROM (SELECT 1 :: UInt32 as key) AS t1 FULL JOIN (SELECT NULL :: Nullable(UInt32) as key) t2 USING (key); -SELECT * FROM (SELECT 1 :: Int32 as key ) AS t1 JOIN (SELECT 1 :: UInt32 as key) t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 1 :: Int32 as key) AS t1 JOIN (SELECT 1 :: UInt32 as key) t2 ON t1.key = t2.key; -SELECT * FROM (SELECT -1 :: Nullable(Int32) as key ) AS t1 FULL JOIN (SELECT 4294967295 :: UInt32 as key) t2 ON t1.key = t2.key; +SELECT * FROM (SELECT -1 :: Nullable(Int32) as key) AS t1 FULL JOIN (SELECT 4294967295 :: UInt32 as key) t2 ON t1.key = t2.key; + +SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: String AS key) AS t2 ON t1.key = t2.key; + +SELECT * FROM (SELECT 'a' :: LowCardinality(Nullable(String)) AS key) AS t1 JOIN (SELECT 'a' :: String AS key) AS t2 ON t1.key = t2.key; + +SELECT * FROM (SELECT 'a' :: LowCardinality(Nullable(String)) AS key) AS t1 JOIN (SELECT 'a' :: Nullable(String) AS key) AS t2 ON t1.key = t2.key; SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: LowCardinality(String) AS key) AS t2 ON t1.key = t2.key; From c8cd962bacb93a78caa517b4021a7ec2b0e345de Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 2 May 2022 16:27:43 +0000 Subject: [PATCH 300/627] couple of tests for full_sorting_merge --- src/Interpreters/FullSortingMergeJoin.h | 13 ++++++++----- ...> 00800_low_cardinality_merge_join.reference.j2} | 2 ++ ....sql => 00800_low_cardinality_merge_join.sql.j2} | 6 +++++- .../01010_partial_merge_join_const_and_lc.reference | 5 +++++ .../01010_partial_merge_join_const_and_lc.sql | 8 ++++++++ tests/queries/0_stateless/02000_join_on_const.sql | 2 ++ 6 files changed, 30 insertions(+), 6 deletions(-) rename tests/queries/0_stateless/{00800_low_cardinality_merge_join.reference => 00800_low_cardinality_merge_join.reference.j2} (63%) rename tests/queries/0_stateless/{00800_low_cardinality_merge_join.sql => 00800_low_cardinality_merge_join.sql.j2} (97%) diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index a7429f6877e..8577b5267b7 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -35,13 +35,20 @@ public: if (table_join->getClauses().size() != 1) throw Exception("FullSortingMergeJoin supports only one join key", ErrorCodes::NOT_IMPLEMENTED); + /// Key column can change nullability and it's not handled on type conversion stage, so algorithm shold be aware of it + if (table_join->hasUsing() && table_join->joinUseNulls()) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "FullSortingMergeJoin doesn't support USING with join_use_nulls"); + const auto & onexpr = table_join->getOnlyClause(); for (size_t i = 0; i < onexpr.key_names_left.size(); ++i) { DataTypePtr left_type = left_block.getByName(onexpr.key_names_left[i]).type; DataTypePtr right_type = right_sample_block.getByName(onexpr.key_names_right[i]).type; - if (!removeNullable(left_type)->equals(*removeNullable(right_type))) + bool type_equals + = table_join->hasUsing() ? left_type->equals(*right_type) : removeNullable(left_type)->equals(*removeNullable(right_type)); + + if (!type_equals) { throw DB::Exception( ErrorCodes::TYPE_MISMATCH, @@ -50,10 +57,6 @@ public: onexpr.key_names_right[i], right_type->getName()); } } - - /// Key column can change nullability and it's not handled on type conversion stage, so algorithm shold be aware of it - if (table_join->hasUsing() && table_join->joinUseNulls()) - throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "FullSortingMergeJoin doesn't support USING with join_use_nulls"); } /// Used just to get result header diff --git a/tests/queries/0_stateless/00800_low_cardinality_merge_join.reference b/tests/queries/0_stateless/00800_low_cardinality_merge_join.reference.j2 similarity index 63% rename from tests/queries/0_stateless/00800_low_cardinality_merge_join.reference rename to tests/queries/0_stateless/00800_low_cardinality_merge_join.reference.j2 index 8e032c0a542..06001261088 100644 --- a/tests/queries/0_stateless/00800_low_cardinality_merge_join.reference +++ b/tests/queries/0_stateless/00800_low_cardinality_merge_join.reference.j2 @@ -1,3 +1,4 @@ +{% for join_algorithm in ['partial_merge', 'full_sorting_merge'] -%} 0 0 0 @@ -39,3 +40,4 @@ 0 1 1 2 2 \N +{% endfor -%} diff --git a/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql b/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 similarity index 97% rename from tests/queries/0_stateless/00800_low_cardinality_merge_join.sql rename to tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 index 1181ee453a6..d0dd908ae67 100644 --- a/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql +++ b/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 @@ -1,4 +1,6 @@ -set join_algorithm = 'partial_merge'; +{% for join_algorithm in ['partial_merge', 'full_sorting_merge'] -%} + +set join_algorithm = '{{ join_algorithm }}'; select * from (select dummy as val from system.one) s1 any left join (select dummy as val from system.one) s2 using val; select * from (select toLowCardinality(dummy) as val from system.one) s1 any left join (select dummy as val from system.one) s2 using val; @@ -28,3 +30,5 @@ select * from (select toLowCardinality(number) as l from system.numbers limit 3) select * from (select toLowCardinality(toNullable(number)) as l from system.numbers limit 3) s1 any left join (select toLowCardinality(number) as r from system.numbers limit 3) s2 on l + 1 = r * 1; select * from (select toLowCardinality(number) as l from system.numbers limit 3) s1 any left join (select toLowCardinality(toNullable(number)) as r from system.numbers limit 3) s2 on l + 1 = r * 1; select * from (select toLowCardinality(toNullable(number)) as l from system.numbers limit 3) s1 any left join (select toLowCardinality(toNullable(number)) as r from system.numbers limit 3) s2 on l + 1 = r * 1; + +{% endfor -%} diff --git a/tests/queries/0_stateless/01010_partial_merge_join_const_and_lc.reference b/tests/queries/0_stateless/01010_partial_merge_join_const_and_lc.reference index 43974985672..95859e3e0a4 100644 --- a/tests/queries/0_stateless/01010_partial_merge_join_const_and_lc.reference +++ b/tests/queries/0_stateless/01010_partial_merge_join_const_and_lc.reference @@ -3,3 +3,8 @@ 3 4 5 +1 1 +2 +3 +4 +5 diff --git a/tests/queries/0_stateless/01010_partial_merge_join_const_and_lc.sql b/tests/queries/0_stateless/01010_partial_merge_join_const_and_lc.sql index 9d266a0ece7..9abfc425d83 100644 --- a/tests/queries/0_stateless/01010_partial_merge_join_const_and_lc.sql +++ b/tests/queries/0_stateless/01010_partial_merge_join_const_and_lc.sql @@ -5,3 +5,11 @@ select * from (select materialize(2) as x) s1 left join (select 2 as x) s2 using select * from (select 3 as x) s1 left join (select materialize(3) as x) s2 using x; select * from (select toLowCardinality(4) as x) s1 left join (select 4 as x) s2 using x; select * from (select 5 as x) s1 left join (select toLowCardinality(5) as x) s2 using x; + +SET join_algorithm = 'full_sorting_merge'; + +select s1.x, s2.x from (select 1 as x) s1 left join (select 1 as x) s2 using x; +select * from (select materialize(2) as x) s1 left join (select 2 as x) s2 using x; +select * from (select 3 as x) s1 left join (select materialize(3) as x) s2 using x; +select * from (select toLowCardinality(4) as x) s1 left join (select 4 as x) s2 using x; +select * from (select 5 as x) s1 left join (select toLowCardinality(5) as x) s2 using x; diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index 92ded98b5f4..7496e754a0a 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -43,8 +43,10 @@ SELECT * FROM t1 RIGHT JOIN t2 ON NULL ORDER BY t1.id NULLS FIRST, t2.id SETTING SELECT '- full -'; SELECT * FROM t1 FULL JOIN t2 ON NULL ORDER BY t1.id NULLS FIRST, t2.id SETTINGS join_use_nulls = 1; +SELECT * FROM t1 JOIN t2 ON 1 = 1 SETTINGS join_algorithm = 'full_sorting_merge'; -- { serverError 48 } SELECT * FROM t1 JOIN t2 ON 1 = 1 SETTINGS join_algorithm = 'partial_merge'; -- { serverError 48 } SELECT * FROM t1 JOIN t2 ON 1 = 1 SETTINGS join_algorithm = 'auto'; -- { serverError 48 } +SELECT * FROM t1 JOIN t2 ON NULL SETTINGS join_algorithm = 'full_sorting_merge'; -- { serverError 48 } SELECT * FROM t1 JOIN t2 ON NULL SETTINGS join_algorithm = 'partial_merge'; -- { serverError 48 } SELECT * FROM t1 LEFT JOIN t2 ON NULL SETTINGS join_algorithm = 'partial_merge'; -- { serverError 48 } SELECT * FROM t1 RIGHT JOIN t2 ON NULL SETTINGS join_algorithm = 'auto'; -- { serverError 48 } From 0b994bb258aad6cc578be380fe7bee12e4c43547 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 3 May 2022 12:05:22 +0000 Subject: [PATCH 301/627] fix build --- src/Interpreters/FullSortingMergeJoin.h | 2 +- src/Processors/Merges/IMergingTransform.h | 1 - .../Transforms/MergeJoinTransform.cpp | 130 +++++++++--------- src/QueryPipeline/QueryPipelineBuilder.cpp | 1 + 4 files changed, 67 insertions(+), 67 deletions(-) diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 8577b5267b7..5895937be73 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -35,7 +35,7 @@ public: if (table_join->getClauses().size() != 1) throw Exception("FullSortingMergeJoin supports only one join key", ErrorCodes::NOT_IMPLEMENTED); - /// Key column can change nullability and it's not handled on type conversion stage, so algorithm shold be aware of it + /// Key column can change nullability and it's not handled on type conversion stage, so algorithm should be aware of it if (table_join->hasUsing() && table_join->joinUseNulls()) throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "FullSortingMergeJoin doesn't support USING with join_use_nulls"); diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index f7178d7b1ae..061750d91e4 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index aa0a89bce7d..a41fcea325e 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -7,9 +7,9 @@ #include #include -#include #include +#include #include #include #include @@ -127,11 +127,11 @@ ColumnPtr indexColumn(const ColumnPtr & column, const PaddedPODArray & i { auto new_col = column->cloneEmpty(); new_col->reserve(indices.size()); - for (size_t i = 0; i < indices.size(); ++i) + for (size_t idx : indices) { /// rows where default value should be inserted have index == size - if (indices[i] < column->size()) - new_col->insertFrom(*column, indices[i]); + if (idx < column->size()) + new_col->insertFrom(*column, idx); else new_col->insertDefault(); } @@ -176,6 +176,65 @@ size_t nextDistinct(SortCursorImpl & impl) return impl.rows - start_pos; } +ColumnPtr replicateRow(const IColumn & column, size_t num) +{ + MutableColumnPtr res = column.cloneEmpty(); + res->insertManyFrom(column, 0, num); + return res; +} + +template +void copyColumnsResized(const TColumns & cols, size_t start, size_t size, Chunk & result_chunk) +{ + for (const auto & col : cols) + { + if (col->empty()) + { + /// add defaults + result_chunk.addColumn(col->cloneResized(size)); + } + else if (col->size() == 1) + { + /// copy same row n times + result_chunk.addColumn(replicateRow(*col, size)); + } + else + { + /// cut column + assert(start + size <= col->size()); + result_chunk.addColumn(col->cut(start, size)); + } + } +} + +Chunk copyChunkResized(const Chunk & lhs, const Chunk & rhs, size_t start, size_t num_rows) +{ + Chunk result; + copyColumnsResized(lhs.getColumns(), start, num_rows, result); + copyColumnsResized(rhs.getColumns(), start, num_rows, result); + return result; +} + +Chunk getRowFromChunk(const Chunk & chunk, size_t pos) +{ + Chunk result; + copyColumnsResized(chunk.getColumns(), pos, 1, result); + return result; +} + +void inline addRange(PaddedPODArray & left_map, size_t start, size_t end) +{ + assert(end > start); + for (size_t i = start; i < end; ++i) + left_map.push_back(i); +} + +void inline addMany(PaddedPODArray & left_map, size_t idx, size_t num) +{ + for (size_t i = 0; i < num; ++i) + left_map.push_back(idx); +} + } const Chunk & FullMergeJoinCursor::getCurrent() const @@ -245,45 +304,6 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( } } -static ColumnPtr replicateRow(const IColumn & column, size_t num) -{ - MutableColumnPtr res = column.cloneEmpty(); - res->insertManyFrom(column, 0, num); - return res; -} - -template -static void copyColumnsResized(const TColumns & cols, size_t start, size_t size, Chunk & result_chunk) -{ - for (const auto & col : cols) - { - if (col->empty()) - { - /// add defaults - result_chunk.addColumn(col->cloneResized(size)); - } - else if (col->size() == 1) - { - /// copy same row n times - result_chunk.addColumn(replicateRow(*col, size)); - } - else - { - /// cut column - assert(start + size <= col->size()); - result_chunk.addColumn(col->cut(start, size)); - } - } -} - -static Chunk copyChunkResized(const Chunk & lhs, const Chunk & rhs, size_t start, size_t num_rows) -{ - Chunk result; - copyColumnsResized(lhs.getColumns(), start, num_rows, result); - copyColumnsResized(rhs.getColumns(), start, num_rows, result); - return result; -} - static void prepareChunk(Chunk & chunk) { if (!chunk) @@ -330,26 +350,6 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) cursors[source_num]->setChunk(std::move(input.chunk)); } -static Chunk getRowFromChunk(const Chunk & chunk, size_t pos) -{ - Chunk result; - copyColumnsResized(chunk.getColumns(), pos, 1, result); - return result; -} - -static void ALWAYS_INLINE addRange(PaddedPODArray & left_map, size_t start, size_t end) -{ - assert(end > start); - for (size_t i = start; i < end; ++i) - left_map.push_back(i); -} - -static void ALWAYS_INLINE addMany(PaddedPODArray & left_map, size_t idx, size_t num) -{ - for (size_t i = 0; i < num; ++i) - left_map.push_back(idx); -} - template struct AllJoinImpl { @@ -847,8 +847,8 @@ MergeJoinTransform::MergeJoinTransform( const Blocks & input_headers, const Block & output_header, size_t max_block_size, - UInt64 limit_hint) - : IMergingTransform(input_headers, output_header, true, limit_hint, table_join, input_headers, max_block_size) + UInt64 limit_hint_) + : IMergingTransform(input_headers, output_header, true, limit_hint_, table_join, input_headers, max_block_size) , log(&Poco::Logger::get("MergeJoinTransform")) { LOG_TRACE(log, "Use MergeJoinTransform"); diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index fc2e8624f2a..353f37836de 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -33,6 +33,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; } void QueryPipelineBuilder::checkInitialized() From f8e66601a7fa55561a5ecc49ff6bef9f99cbb335 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 13 May 2022 15:46:38 +0000 Subject: [PATCH 302/627] Fix column remap in MergeJoinTransform --- src/Processors/Transforms/MergeJoinTransform.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index a41fcea325e..539ac67a57d 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -523,7 +523,6 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) { /// If we have remapped columns, then we need to get values from right columns insead of defaults const auto & indices = idx_map[0]; - const auto & right_indices = idx_map[1]; const auto & left_src = cursors[0]->getCurrent().getColumns(); for (size_t col_idx = 0; col_idx < left_src.size(); ++col_idx) @@ -540,7 +539,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) else { if (auto it = left_to_right_key_remap.find(col_idx); it != left_to_right_key_remap.end()) - new_col->insertFrom(*rcols[it->second], right_indices[i]); + new_col->insertFrom(*rcols[it->second], i); else new_col->insertDefault(); } From aff6654d528d83c14b33dc1d796e4d3b719a8ee0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 18 May 2022 13:57:41 +0000 Subject: [PATCH 303/627] minor changes in full sort join --- src/Processors/Merges/IMergingTransform.h | 7 +++++-- src/Processors/Transforms/MergeJoinTransform.cpp | 14 ++++++++++---- src/Processors/Transforms/MergeJoinTransform.h | 5 +++-- 3 files changed, 18 insertions(+), 8 deletions(-) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 061750d91e4..1d223179f8a 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -95,11 +95,12 @@ public: const Block & output_header, bool have_all_inputs_, UInt64 limit_hint_, + bool empty_chunk_on_finish_, Args && ... args) : IMergingTransformBase(input_headers, output_header, have_all_inputs_, limit_hint_) + , empty_chunk_on_finish(empty_chunk_on_finish_) , algorithm(std::forward(args) ...) { - empty_chunk_on_finish = true; } void work() override @@ -144,12 +145,14 @@ public: } protected: + /// Call `consume` with empty chunk when there is no more data. + bool empty_chunk_on_finish = false; + Algorithm algorithm; /// Profile info. Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; - bool empty_chunk_on_finish = false; private: using IMergingTransformBase::state; }; diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 539ac67a57d..dc7462ce24f 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -748,7 +748,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) /// if `source_num == 0` get data from left cursor and fill defaults at right /// otherwise - vice versa -Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) +Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const { ColumnRawPtrs cols; { @@ -778,7 +778,7 @@ Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num, size_t star return result_chunk; } -/// This overload also flushes cursort +/// This function also flushes cursor Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num) { Chunk result_chunk = createBlockWithDefaults(source_num, cursors[source_num]->cursor.getRow(), cursors[source_num]->cursor.rowsLeft()); @@ -811,7 +811,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() } /// check if blocks are not intersecting at all - if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor); cmp == 111) + if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor); cmp != 0) { if (cmp < 0) { @@ -847,7 +847,13 @@ MergeJoinTransform::MergeJoinTransform( const Block & output_header, size_t max_block_size, UInt64 limit_hint_) - : IMergingTransform(input_headers, output_header, true, limit_hint_, table_join, input_headers, max_block_size) + : IMergingTransform( + input_headers, + output_header, + /* have_all_inputs_= */ true, + limit_hint_, + /* empty_chunk_on_finish_= */ true, + table_join, input_headers, max_block_size) , log(&Poco::Logger::get("MergeJoinTransform")) { LOG_TRACE(log, "Use MergeJoinTransform"); diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 6e78f7d4e2e..d776900e205 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -250,14 +250,15 @@ private: Status allJoin(ASTTableJoin::Kind kind); Chunk createBlockWithDefaults(size_t source_num); - Chunk createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows); - + Chunk createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const; /// For `USING` join key columns should have values from right side instead of defaults std::unordered_map left_to_right_key_remap; + std::vector cursors; std::vector sample_chunks; + /// Keep some state to make connection between data in different blocks AnyJoinState any_join_state; std::unique_ptr all_join_state; From a4b93c74549e3d2c5bc3b97d2aa37ee0803550ad Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 20 May 2022 12:00:37 +0000 Subject: [PATCH 304/627] Add algorithms to 00172_parallel_join --- ...n.reference => 00172_parallel_join.reference.j2} | 5 +++++ ...parallel_join.sql => 00172_parallel_join.sql.j2} | 13 +++++++++++-- 2 files changed, 16 insertions(+), 2 deletions(-) rename tests/queries/1_stateful/{00172_parallel_join.reference => 00172_parallel_join.reference.j2} (98%) rename tests/queries/1_stateful/{00172_parallel_join.sql => 00172_parallel_join.sql.j2} (94%) diff --git a/tests/queries/1_stateful/00172_parallel_join.reference b/tests/queries/1_stateful/00172_parallel_join.reference.j2 similarity index 98% rename from tests/queries/1_stateful/00172_parallel_join.reference rename to tests/queries/1_stateful/00172_parallel_join.reference.j2 index 039f6ca9967..30088c91500 100644 --- a/tests/queries/1_stateful/00172_parallel_join.reference +++ b/tests/queries/1_stateful/00172_parallel_join.reference.j2 @@ -1,3 +1,5 @@ +{% for join_algorithm in ['hash', 'parallel_hash', 'full_sorting_merge'] -%} +--- {{ join_algorithm }} --- 2014-03-17 1406958 265108 2014-03-19 1405797 261624 2014-03-18 1383658 258723 @@ -22,6 +24,7 @@ mail.ru 87949 22225 best.ru 58537 55 korablitz.ru 51844 0 hurpass.com 49671 1251 +{% if join_algorithm != 'full_sorting_merge' -%} 37292 0 35642 92887 252214 0 7842 196036 0 @@ -287,6 +290,7 @@ hurpass.com 49671 1251 4 4 2 6 6 3 8 8 4 +{% endif -%} 0 0 0 0 0 1 1 1 2 @@ -297,3 +301,4 @@ hurpass.com 49671 1251 3 3 7 4 4 8 4 4 9 +{% endfor -%} diff --git a/tests/queries/1_stateful/00172_parallel_join.sql b/tests/queries/1_stateful/00172_parallel_join.sql.j2 similarity index 94% rename from tests/queries/1_stateful/00172_parallel_join.sql rename to tests/queries/1_stateful/00172_parallel_join.sql.j2 index 36b12a43b88..39c981e0d31 100644 --- a/tests/queries/1_stateful/00172_parallel_join.sql +++ b/tests/queries/1_stateful/00172_parallel_join.sql.j2 @@ -1,4 +1,8 @@ -SET join_algorithm='parallel_hash'; +{% for join_algorithm in ['hash', 'parallel_hash', 'full_sorting_merge'] -%} + +SELECT '--- {{ join_algorithm }} ---'; + +SET join_algorithm = '{{ join_algorithm }}'; SELECT EventDate, @@ -65,8 +69,9 @@ ORDER BY hits DESC LIMIT 10 SETTINGS joined_subquery_requires_alias = 0; -SELECT CounterID FROM test.visits ARRAY JOIN Goals.ID WHERE CounterID = 942285 ORDER BY CounterID; +{% if join_algorithm != 'full_sorting_merge' -%} +SELECT CounterID FROM test.visits ARRAY JOIN Goals.ID WHERE CounterID = 942285 ORDER BY CounterID; SELECT CounterID, @@ -194,6 +199,8 @@ ANY INNER JOIN USING k ORDER BY joined SETTINGS any_join_distinct_right_table_keys = 1; +{% endif -%} + SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 @@ -203,3 +210,5 @@ ALL INNER JOIN SELECT intDiv(number, 2) AS k, number AS joined FROM system.numbers LIMIT 10 ) AS b USING k ORDER BY joined; + +{% endfor %} From b0c0ec370d27cf04a82f32cb3b82022689080ab5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 25 May 2022 16:14:18 +0000 Subject: [PATCH 305/627] Add join_algorithm='full_sorting_merge' to stress tests --- docker/test/stress/stress | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index e195f81b551..ab25d13695b 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -28,18 +28,20 @@ def get_options(i, backward_compatibility_check): if i % 2 == 1: options.append(" --database=test_{}".format(i)) - if i % 5 == 1: + if i % 3 == 1: client_options.append("join_use_nulls=1") - if i % 15 == 1: - client_options.append("join_algorithm='parallel_hash'") - - if i % 15 == 6: - client_options.append("join_algorithm='partial_merge'") - - if i % 15 == 11: - client_options.append("join_algorithm='auto'") - client_options.append("max_rows_in_join=1000") + if i % 2 == 1: + join_alg_num = i // 2 + if join_alg_num % 4 == 0: + client_options.append("join_algorithm='parallel_hash'") + if join_alg_num % 4 == 1: + client_options.append("join_algorithm='partial_merge'") + if join_alg_num % 4 == 2: + client_options.append("join_algorithm='full_sorting_merge'") + if join_alg_num % 4 == 3: + client_options.append("join_algorithm='auto'") + client_options.append('max_rows_in_join=1000') if i == 13: client_options.append("memory_tracker_fault_probability=0.001") From 881d352e056f5acc9f08642b9f50a75d9c125b60 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 5 Jul 2022 14:06:10 +0000 Subject: [PATCH 306/627] upd full sorting join --- src/Interpreters/ExpressionAnalyzer.cpp | 2 ++ src/Interpreters/TableJoin.h | 2 +- .../Transforms/MergeJoinTransform.cpp | 6 +++--- src/Processors/Transforms/MergeJoinTransform.h | 18 +++++++++++++++++- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 ++-- 5 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 9cbf6258199..203797a0731 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1098,6 +1098,8 @@ static std::shared_ptr chooseJoinAlgorithm(std::shared_ptr ana { if (analyzed_join->getClauses().size() != 1) throw Exception("Full sorting merge join is supported only for single-condition joins", ErrorCodes::NOT_IMPLEMENTED); + if (analyzed_join->isSpecialStorage()) + throw Exception("Full sorting merge join is not supported for special storage", ErrorCodes::NOT_IMPLEMENTED); return std::make_shared(analyzed_join, sample_block); } return std::make_shared(analyzed_join, sample_block); diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index f953ebb111e..4210da6ae76 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -202,7 +202,7 @@ public: bool forceMergeJoin() const { return join_algorithm == MultiEnum(JoinAlgorithm::PARTIAL_MERGE); } bool allowParallelHashJoin() const; - bool forceFullSortingMergeJoin() const { return join_algorithm.isSet(JoinAlgorithm::FULL_SORTING_MERGE); } + bool forceFullSortingMergeJoin() const { return !isSpecialStorage() && join_algorithm.isSet(JoinAlgorithm::FULL_SORTING_MERGE); } bool forceHashJoin() const { diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index dc7462ce24f..5ad2ea77c13 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -320,9 +320,9 @@ static void prepareChunk(Chunk & chunk) void MergeJoinAlgorithm::initialize(Inputs inputs) { if (inputs.size() != 2) - throw Exception("MergeJoinAlgorithm requires exactly two inputs", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Two inputs arerequired, got {}", inputs.size()); - LOG_DEBUG(log, "MergeJoinAlgorithm initialize, number of inputs: {}", inputs.size()); + LOG_DEBUG(log, "Initialize, number of inputs: {}", inputs.size()); for (size_t i = 0; i < inputs.size(); ++i) { assert(inputs[i].chunk.getNumColumns() == cursors[i]->sampleBlock().columns()); @@ -861,7 +861,7 @@ MergeJoinTransform::MergeJoinTransform( void MergeJoinTransform::onFinish() { - algorithm.onFinish(total_stopwatch.elapsedSeconds()); + algorithm.logElapsed(total_stopwatch.elapsedSeconds(), true); } } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index d776900e205..5c588d5db94 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -232,14 +232,19 @@ public: virtual void consume(Input & input, size_t source_num) override; virtual Status merge() override; - void onFinish(double seconds) + void logElapsed(double seconds, bool force) { + /// Do not log more frequently than once per ten seconds + if (seconds - stat.last_log_seconds < 10 && !force) + return; + LOG_TRACE(log, "Finished pocessing in {} seconds" ", left: {} blocks, {} rows; right: {} blocks, {} rows" ", max blocks loaded to memory: {}", seconds, stat.num_blocks[0], stat.num_rows[0], stat.num_blocks[1], stat.num_rows[1], stat.max_blocks_loaded); + stat.last_log_seconds = seconds; } private: @@ -272,7 +277,10 @@ private: size_t num_rows[2] = {0, 0}; size_t max_blocks_loaded = 0; + + double last_log_seconds = 0; }; + Statistic stat; Poco::Logger * log; @@ -280,6 +288,8 @@ private: class MergeJoinTransform final : public IMergingTransform { + using Base = IMergingTransform; + public: MergeJoinTransform( JoinPtr table_join, @@ -293,6 +303,12 @@ public: protected: void onFinish() override; + void work() override + { + algorithm.logElapsed(total_stopwatch.elapsedSeconds(), true); + Base::work(); + } + Poco::Logger * log; }; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 353f37836de..ec186971b50 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -311,7 +311,7 @@ QueryPipelineBuilderPtr QueryPipelineBuilder::mergePipelines( Processors * collected_processors) { if (transform->getOutputs().size() != 1) - throw Exception("Merge transform must have exactly 1 output", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Merge transform must have exactly 1 output, got {}", transform->getOutputs().size()); connect(*left->pipe.output_ports.front(), transform->getInputs().front()); connect(*right->pipe.output_ports.front(), transform->getInputs().back()); @@ -323,7 +323,7 @@ QueryPipelineBuilderPtr QueryPipelineBuilder::mergePipelines( left->pipe.processors.emplace_back(transform); left->pipe.processors.insert(left->pipe.processors.end(), right->pipe.processors.begin(), right->pipe.processors.end()); - left->pipe.holder = std::move(right->pipe.holder); + // left->pipe.holder = std::move(right->pipe.holder); left->pipe.header = left->pipe.output_ports.front()->getHeader(); left->pipe.max_parallel_streams = std::max(left->pipe.max_parallel_streams, right->pipe.max_parallel_streams); return left; From 7c586a9e7c0a1eecfb9ecd3ca059fe964ae3610b Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 5 Jul 2022 16:21:35 +0000 Subject: [PATCH 307/627] Minor updates for full soring merge join --- src/Interpreters/FullSortingMergeJoin.h | 26 ++++++++++++++----- src/Interpreters/HashJoin.h | 3 +++ src/Interpreters/IJoin.h | 7 ++--- src/Processors/QueryPlan/JoinStep.cpp | 4 +-- .../Transforms/MergeJoinTransform.cpp | 2 -- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 +-- src/QueryPipeline/QueryPipelineBuilder.h | 6 +++-- 7 files changed, 35 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 5895937be73..06dc50e6036 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -28,12 +28,15 @@ public: const TableJoin & getTableJoin() const override { return *table_join; } - bool addJoinedBlock(const Block & /* block */, bool /* check_limits */) override { __builtin_unreachable(); } + bool addJoinedBlock(const Block & /* block */, bool /* check_limits */) override + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "FullSortingMergeJoin::addJoinedBlock should not be called"); + } void checkTypesOfKeys(const Block & left_block) const override { if (table_join->getClauses().size() != 1) - throw Exception("FullSortingMergeJoin supports only one join key", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::LOGICAL_ERROR, "FullSortingMergeJoin supports only one join key"); /// Key column can change nullability and it's not handled on type conversion stage, so algorithm should be aware of it if (table_join->hasUsing() && table_join->joinUseNulls()) @@ -48,6 +51,8 @@ public: bool type_equals = table_join->hasUsing() ? left_type->equals(*right_type) : removeNullable(left_type)->equals(*removeNullable(right_type)); + /// Even slightly different types should be converted on previous pipeline steps. + /// If we still have some differences, we can't join, because the algorithm expects strict type equality. if (!type_equals) { throw DB::Exception( @@ -70,16 +75,25 @@ public: void setTotals(const Block & block) override { totals = block; } const Block & getTotals() const override { return totals; } - size_t getTotalRowCount() const override { __builtin_unreachable(); } - size_t getTotalByteCount() const override { __builtin_unreachable(); } - bool alwaysReturnsEmptySet() const override { __builtin_unreachable(); } + size_t getTotalRowCount() const override + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "FullSortingMergeJoin::getTotalRowCount should not be called"); + } + + size_t getTotalByteCount() const override + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "FullSortingMergeJoin::getTotalByteCount should not be called"); + } + + bool alwaysReturnsEmptySet() const override { return false; } std::shared_ptr getNonJoinedBlocks(const Block & /* left_sample_block */, const Block & /* result_sample_block */, UInt64 /* max_block_size */) const override { - __builtin_unreachable(); + throw Exception(ErrorCodes::LOGICAL_ERROR, "FullSortingMergeJoin::getNonJoinedBlocks should not be called"); } + /// Left and right streams have the same priority and are processed simultaneously virtual JoinPipelineType pipelineType() const override { return JoinPipelineType::YShaped; } private: diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 8402a220f21..7fd1b7771d0 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -180,9 +180,12 @@ public: virtual JoinPipelineType pipelineType() const override { + /// No need to process anything in the right stream if it's a dictionary will just join the left stream with it. bool is_filled = from_storage_join || data->type == Type::DICT; if (is_filled) return JoinPipelineType::FilledRight; + + /// Default pipeline processes right stream at first and then left. return JoinPipelineType::FillRightFirst; } diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 83c1eb31277..b699988e926 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -22,19 +22,20 @@ class NotJoinedBlocks; enum class JoinPipelineType { /* - * Right stream processed first. + * Right stream processed first, then when join data structures are ready, the left stream is processed using it. * The pipeline is not sorted. */ FillRightFirst, /* - * Only left stream is processed, right is already filled. + * Only the left stream is processed. Right is already filled. */ FilledRight, /* * The pipeline is created from the left and right streams processed with merging transform. - * The pipeline is sorted. + * Left and right streams have the same priority and are processed simultaneously. + * The pipelines are sorted. */ YShaped, }; diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index ff3063f01a3..d481c745798 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -34,12 +34,12 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); if (join->pipelineType() == JoinPipelineType::YShaped) - return QueryPipelineBuilder::joinPipelines2( + return QueryPipelineBuilder::joinPipelinesYShaped( std::move(pipelines[0]), std::move(pipelines[1]), join, output_stream->header, max_block_size, &processors); - return QueryPipelineBuilder::joinPipelines(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, max_streams, keep_left_read_in_order, &processors); + return QueryPipelineBuilder::joinPipelinesRightLeft(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, max_streams, keep_left_read_in_order, &processors); } void JoinStep::describePipeline(FormatSettings & settings) const diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 5ad2ea77c13..fdd48b75bee 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -439,8 +439,6 @@ void dispatchKind(JoinKind kind, Args && ... args) return Impl::join(std::forward(args)...); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); - - __builtin_unreachable(); } std::optional MergeJoinAlgorithm::handleAllJoinState() diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index ec186971b50..5f83037e8b0 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -329,7 +329,7 @@ QueryPipelineBuilderPtr QueryPipelineBuilder::mergePipelines( return left; } -std::unique_ptr QueryPipelineBuilder::joinPipelines2( +std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped( std::unique_ptr left, std::unique_ptr right, JoinPtr join, @@ -357,7 +357,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelines2( return result; } -std::unique_ptr QueryPipelineBuilder::joinPipelines( +std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLeft( std::unique_ptr left, std::unique_ptr right, JoinPtr join, diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index d12de77230a..18e9d718445 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -109,7 +109,8 @@ public: /// Join two pipelines together using JoinPtr. /// If collector is used, it will collect only newly-added processors, but not processors from pipelines. - static std::unique_ptr joinPipelines( + /// Process right stream to fill JoinPtr and then process left pipeline using it + static std::unique_ptr joinPipelinesRightLeft( std::unique_ptr left, std::unique_ptr right, JoinPtr join, @@ -118,7 +119,8 @@ public: bool keep_left_read_in_order, Processors * collected_processors = nullptr); - static std::unique_ptr joinPipelines2( + /// Join two independent pipelines, processing them simultaneously. + static std::unique_ptr joinPipelinesYShaped( std::unique_ptr left, std::unique_ptr right, JoinPtr table_join, From 2fd229448c41a06e1996ab00fa1039c34bb91495 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Jul 2022 12:25:12 +0000 Subject: [PATCH 308/627] Fix style FullSortingMergeJoin.h --- src/Interpreters/FullSortingMergeJoin.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 06dc50e6036..90ec67f54b1 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -11,8 +11,9 @@ namespace DB namespace ErrorCodes { - extern const int TYPE_MISMATCH; + extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; + extern const int TYPE_MISMATCH; } /// Dummy class, actual joining is done by MergeTransform From 0c4da85e750f146d3094581e816a192a61697bab Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Jul 2022 14:29:42 +0000 Subject: [PATCH 309/627] More uniform naming --- .../FunctionsMultiStringFuzzySearch.h | 20 +++++++++---------- src/Functions/FunctionsMultiStringSearch.h | 12 +++++------ 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 1e13b300136..7e2d004070b 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -65,26 +65,26 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - const ColumnPtr & column_haystack = arguments[0].column; - const ColumnPtr & num_ptr = arguments[1].column; - const ColumnPtr & arr_ptr = arguments[2].column; + const ColumnPtr & haystack_ptr = arguments[0].column; + const ColumnPtr & edit_distance_ptr = arguments[1].column; + const ColumnPtr & needles_ptr = arguments[2].column; - const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); - const ColumnConst * col_haystack_const = typeid_cast(&*column_haystack); + const ColumnString * col_haystack_vector = checkAndGetColumn(&*haystack_ptr); + const ColumnConst * col_haystack_const = typeid_cast(&*haystack_ptr); assert(static_cast(col_haystack_vector) ^ static_cast(col_haystack_const)); UInt32 edit_distance = 0; - if (const auto * col_const_uint8 = checkAndGetColumnConst(num_ptr.get())) + if (const auto * col_const_uint8 = checkAndGetColumnConst(edit_distance_ptr.get())) edit_distance = col_const_uint8->getValue(); - else if (const auto * col_const_uint16 = checkAndGetColumnConst(num_ptr.get())) + else if (const auto * col_const_uint16 = checkAndGetColumnConst(edit_distance_ptr.get())) edit_distance = col_const_uint16->getValue(); - else if (const auto * col_const_uint32 = checkAndGetColumnConst(num_ptr.get())) + else if (const auto * col_const_uint32 = checkAndGetColumnConst(edit_distance_ptr.get())) edit_distance = col_const_uint32->getValue(); else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}. The number is not const or does not fit in UInt32", arguments[1].column->getName()); - const ColumnArray * col_needles = checkAndGetColumn(arr_ptr.get()); - const ColumnConst * col_needles_const = checkAndGetColumnConst(arr_ptr.get()); + const ColumnArray * col_needles = checkAndGetColumn(needles_ptr.get()); + const ColumnConst * col_needles_const = checkAndGetColumnConst(needles_ptr.get()); assert(static_cast(col_needles) ^ static_cast(col_needles_const)); if (col_haystack_const && col_needles) diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 1a47a54c04c..560fe1313e2 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -77,15 +77,15 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - const ColumnPtr & column_haystack = arguments[0].column; - const ColumnPtr & arr_ptr = arguments[1].column; + const ColumnPtr & haystack_ptr = arguments[0].column; + const ColumnPtr & needles_ptr = arguments[1].column; - const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); - const ColumnConst * col_haystack_const = typeid_cast(&*column_haystack); + const ColumnString * col_haystack_vector = checkAndGetColumn(&*haystack_ptr); + const ColumnConst * col_haystack_const = typeid_cast(&*haystack_ptr); assert(static_cast(col_haystack_vector) ^ static_cast(col_haystack_const)); - const ColumnArray * col_needles = checkAndGetColumn(arr_ptr.get()); - const ColumnConst * col_needles_const = checkAndGetColumnConst(arr_ptr.get()); + const ColumnArray * col_needles = checkAndGetColumn(needles_ptr.get()); + const ColumnConst * col_needles_const = checkAndGetColumnConst(needles_ptr.get()); assert(static_cast(col_needles) ^ static_cast(col_needles_const)); if (col_haystack_const && col_needles) From 12bffd57c92e65c8333a5c2c6708d613a6495c45 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 6 Jul 2022 14:30:30 +0000 Subject: [PATCH 310/627] Update test --- .../0_stateless/02323_null_modifier_in_table_function.reference | 2 ++ .../0_stateless/02323_null_modifier_in_table_function.sql | 1 + 2 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02323_null_modifier_in_table_function.reference b/tests/queries/0_stateless/02323_null_modifier_in_table_function.reference index 316141783da..32b08cc2059 100644 --- a/tests/queries/0_stateless/02323_null_modifier_in_table_function.reference +++ b/tests/queries/0_stateless/02323_null_modifier_in_table_function.reference @@ -6,3 +6,5 @@ \N 1 \N +\N +660095071900147438 diff --git a/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql b/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql index 810480ed868..d3c24b2a1d7 100644 --- a/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql +++ b/tests/queries/0_stateless/02323_null_modifier_in_table_function.sql @@ -4,3 +4,4 @@ select * from values('x UInt8 NULL', NULL); insert into function file(data_02323.tsv) select number % 2 ? number : NULL from numbers(3) settings engine_file_truncate_on_insert=1; select * from file(data_02323.tsv, auto, 'x UInt32 NOT NULL'); select * from file(data_02323.tsv, auto, 'x UInt32 NULL'); +select * from generateRandom('x UInt64 NULL', 7, 3) limit 2; From 7f84cf396866b0637e4644b7530f59a280f524c3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 6 Jul 2022 12:03:10 +0200 Subject: [PATCH 311/627] Fix style. --- src/Backups/BackupCoordinationRemote.cpp | 6 +++--- src/Backups/BackupCoordinationRemote.h | 2 +- src/Backups/BackupCoordinationReplicatedAccess.cpp | 2 +- src/Backups/BackupCoordinationReplicatedTables.cpp | 14 +++++++------- src/Backups/BackupCoordinationReplicatedTables.h | 6 +++--- src/Backups/BackupEntriesCollector.cpp | 6 +++--- src/Backups/BackupEntriesCollector.h | 2 +- src/Backups/IBackupCoordination.h | 6 +++--- src/Backups/RestorerFromBackup.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/MergeTree/MergeTreeMutationEntry.cpp | 2 +- src/Storages/StorageMemory.cpp | 6 +++--- src/Storages/StorageMergeTree.cpp | 2 +- 14 files changed, 34 insertions(+), 34 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index b9ce43291c5..c65cc7390be 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -273,7 +273,7 @@ void BackupCoordinationRemote::addReplicatedDataPath( if (replicated_tables) throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedDataPath() must not be called after preparing"); } - + auto zookeeper = get_zookeeper(); String path = zookeeper_path + "/repl_data_paths/" + escapeForFileName(table_shared_id); zookeeper->createIfNotExists(path, ""); @@ -350,7 +350,7 @@ void BackupCoordinationRemote::addReplicatedAccessFilePath(const String & access if (replicated_access) throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedAccessFilePath() must not be called after preparing"); } - + auto zookeeper = get_zookeeper(); String path = zookeeper_path + "/repl_access/" + escapeForFileName(access_zk_path); zookeeper->createIfNotExists(path, ""); @@ -374,7 +374,7 @@ void BackupCoordinationRemote::prepareReplicatedAccess() const replicated_access.emplace(); auto zookeeper = get_zookeeper(); - + String path = zookeeper_path + "/repl_access"; for (const String & escaped_access_zk_path : zookeeper->getChildren(path)) { diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index e5ecf87d7bf..9a3575ba222 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -33,7 +33,7 @@ public: const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) override; - + std::vector getReplicatedMutations(const String & table_shared_id, const String & replica_name) const override; void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; diff --git a/src/Backups/BackupCoordinationReplicatedAccess.cpp b/src/Backups/BackupCoordinationReplicatedAccess.cpp index 3211e4258fa..23d35d0ecc4 100644 --- a/src/Backups/BackupCoordinationReplicatedAccess.cpp +++ b/src/Backups/BackupCoordinationReplicatedAccess.cpp @@ -22,7 +22,7 @@ Strings BackupCoordinationReplicatedAccess::getFilePaths(const String & access_z if (it == file_paths_by_zk_path.end()) return {}; - auto & file_paths = it->second; + const auto & file_paths = it->second; if (file_paths.host_to_store_access != host_id) return {}; diff --git a/src/Backups/BackupCoordinationReplicatedTables.cpp b/src/Backups/BackupCoordinationReplicatedTables.cpp index 3c12e5b062f..27137edb008 100644 --- a/src/Backups/BackupCoordinationReplicatedTables.cpp +++ b/src/Backups/BackupCoordinationReplicatedTables.cpp @@ -203,16 +203,16 @@ void BackupCoordinationReplicatedTables::addPartNames( Strings BackupCoordinationReplicatedTables::getPartNames(const String & table_shared_id, const String & replica_name) const { prepare(); - + auto it = table_infos.find(table_shared_id); if (it == table_infos.end()) return {}; - + const auto & part_names_by_replica_name = it->second.part_names_by_replica_name; auto it2 = part_names_by_replica_name.find(replica_name); if (it2 == part_names_by_replica_name.end()) return {}; - + return it2->second; } @@ -238,11 +238,11 @@ std::vector BackupCoordinationReplicatedTables::getMutations(const String & table_shared_id, const String & replica_name) const { prepare(); - + auto it = table_infos.find(table_shared_id); if (it == table_infos.end()) return {}; - + const auto & table_info = it->second; if (table_info.replica_name_to_store_mutations != replica_name) return {}; @@ -264,7 +264,7 @@ Strings BackupCoordinationReplicatedTables::getDataPaths(const String & table_sh auto it = table_infos.find(table_shared_id); if (it == table_infos.end()) return {}; - + const auto & table_info = it->second; return Strings{table_info.data_paths.begin(), table_info.data_paths.end()}; } @@ -284,7 +284,7 @@ void BackupCoordinationReplicatedTables::prepare() const for (const auto & [part_name, part_replicas] : table_info.replicas_by_part_name) { auto part_info = MergeTreePartInfo::fromPartName(part_name, MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING); - + auto & min_data_versions_by_partition = table_info.min_data_versions_by_partition; auto it2 = min_data_versions_by_partition.find(part_info.partition_id); if (it2 == min_data_versions_by_partition.end()) diff --git a/src/Backups/BackupCoordinationReplicatedTables.h b/src/Backups/BackupCoordinationReplicatedTables.h index eb7b2e15e5a..7b9f608d28b 100644 --- a/src/Backups/BackupCoordinationReplicatedTables.h +++ b/src/Backups/BackupCoordinationReplicatedTables.h @@ -23,7 +23,7 @@ namespace DB /// /shards/1/replicas/1/data/mydb/mytable/all_2_2_0 /// /shards/1/replicas/2/data/mydb/mytable/all_1_1_0 /// /shards/1/replicas/2/data/mydb/mytable/all_2_2_0 -/// +/// /// Example 2: Let's consider two replicas again, and let the first replica contain parts all_1_1_0 and all_2_2_0 and /// the second replica contain part all_1_2_1 (i.e. the second replica have those parts merged). /// In this case the files in the backup will look like this: @@ -61,8 +61,8 @@ public: const String & table_name_for_logs, const String & replica_name, const std::vector & mutations); - - /// Returns all mutations of a replicated table which are not finished for some data parts added by addReplicatedPartNames(). + + /// Returns all mutations of a replicated table which are not finished for some data parts added by addReplicatedPartNames(). std::vector getMutations(const String & table_shared_id, const String & replica_name) const; /// Adds a data path in backup for a replicated table. diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 8c96f24311c..b660ed295a8 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -710,10 +710,10 @@ void BackupEntriesCollector::makeBackupEntriesForTableData(const QualifiedTableN if (backup_settings.structure_only) return; - const auto & table_info = table_infos.at(table_name); + const auto & table_info = table_infos.at(table_name); const auto & storage = table_info.storage; const auto & data_path_in_backup = table_info.data_path_in_backup; - + if (!storage) { /// If storage == null that means this storage exists on other replicas but it has not been created on this replica yet. @@ -726,7 +726,7 @@ void BackupEntriesCollector::makeBackupEntriesForTableData(const QualifiedTableN } LOG_TRACE(log, "Collecting data of {} for backup", tableNameWithTypeToString(table_name.database, table_name.table, false)); - + try { storage->backupData(*this, data_path_in_backup, table_info.partitions); diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index b83846047c0..03710605654 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -83,7 +83,7 @@ private: void makeBackupEntriesForTablesDefs(); void makeBackupEntriesForTablesData(); void makeBackupEntriesForTableData(const QualifiedTableName & table_name); - + void runPostTasks(); Strings setStatus(const String & new_status, const String & message = ""); diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 0658aac346d..2eeafc085ab 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -51,8 +51,8 @@ public: /// Adds information about mutations of a replicated table. virtual void addReplicatedMutations(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) = 0; - /// Returns all mutations of a replicated table which are not finished for some data parts added by addReplicatedPartNames(). - virtual std::vector getReplicatedMutations(const String & table_shared_id, const String & replica_name) const = 0; + /// Returns all mutations of a replicated table which are not finished for some data parts added by addReplicatedPartNames(). + virtual std::vector getReplicatedMutations(const String & table_shared_id, const String & replica_name) const = 0; /// Adds a data path in backup for a replicated table. /// Multiple replicas of the replicated table call this function and then all the added paths can be returned by call of the function @@ -65,7 +65,7 @@ public: /// Adds a path to access.txt file keeping access entities of a ReplicatedAccessStorage. virtual void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id, const String & file_path) = 0; virtual Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const = 0; - + struct FileInfo { String file_name; diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index ea00be5f514..bcec692583d 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -385,7 +385,7 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name { if (!access_restorer) access_restorer = std::make_unique(backup, restore_settings); - + try { /// addDataPath() will parse access*.txt files and extract access entities from them. @@ -601,9 +601,9 @@ void RestorerFromBackup::createDatabase(const String & database_name) const create_database_query = create_database_query->clone(); create_database_query->as().if_not_exists = true; } - + LOG_TRACE(log, "Creating database {}: {}", backQuoteIfNeed(database_name), serializeAST(*create_database_query)); - + try { /// Execute CREATE DATABASE query. @@ -625,7 +625,7 @@ void RestorerFromBackup::checkDatabase(const String & database_name) { DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name); database_info.database = database; - + if (!restore_settings.allow_different_database_def && !database_info.is_predefined_database) { /// Check that the database's definition is the same as expected. @@ -756,7 +756,7 @@ void RestorerFromBackup::insertDataToTable(const QualifiedTableName & table_name { if (restore_settings.structure_only) return; - + auto & table_info = table_infos.at(table_name); auto storage = table_info.storage; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 29146ef9833..bb38f884366 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4131,7 +4131,7 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const restored_parts_holder->setNumParts(num_parts); } -void MergeTreeData::restorePartFromBackup(std::shared_ptr restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup) +void MergeTreeData::restorePartFromBackup(std::shared_ptr restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup) const { String part_name = part_info.getPartName(); auto backup = restored_parts_holder->getBackup(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 91d8b87151e..5132f28e88e 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1232,7 +1232,7 @@ protected: /// Restores the parts of this table from backup. void restorePartsFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions); - void restorePartFromBackup(std::shared_ptr restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup); + void restorePartFromBackup(std::shared_ptr restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup) const; /// Attaches restored parts to the storage. virtual void attachRestoredParts(MutableDataPartsVector && parts) = 0; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index cad749ced21..a222f2a8ad8 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -172,7 +172,7 @@ std::shared_ptr MergeTreeMutationEntry::backup() const { WriteBufferFromOwnString out; out << "block number: " << block_number << "\n"; - + out << "commands: "; commands.writeText(out); out << "\n"; diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index bf71d539cc2..f3f1162287f 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -465,7 +465,7 @@ namespace num_rows += block.rows(); backup_entries[count_txt_pos] = {file_paths[count_txt_pos], std::make_shared(toString(num_rows))}; } - + /// Writing sizes.json { auto sizes_json_path = temp_dir / fs::path{file_paths[sizes_json_pos]}.filename(); @@ -522,7 +522,7 @@ void StorageMemory::restoreDataFromBackup(RestorerFromBackup & restorer, const S { storage->restoreDataImpl(backup, data_path_in_backup, temp_disk); }); } -void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, const DiskPtr & temp_disk) +void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, const DiskPtr & temporary_disk) { /// Our data are in the StripeLog format. @@ -555,7 +555,7 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat std::optional temp_data_file; if (!dynamic_cast(in.get())) { - temp_data_file.emplace(temp_disk); + temp_data_file.emplace(temporary_disk); auto out = std::make_unique(temp_data_file->getPath()); copyData(*in, *out); out.reset(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2fe9beca9cf..b081555cf2a 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1817,7 +1817,7 @@ void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collec BackupEntries StorageMergeTree::backupMutations(UInt64 version, const String & data_path_in_backup) const { - fs::path mutations_path_in_backup = fs::path{data_path_in_backup} / "mutations"; + fs::path mutations_path_in_backup = fs::path{data_path_in_backup} / "mutations"; BackupEntries backup_entries; for (auto it = current_mutations_by_version.lower_bound(version); it != current_mutations_by_version.end(); ++it) backup_entries.emplace_back(mutations_path_in_backup / fmt::format("{:010}.txt", it->first), it->second.backup()); From 6a907b23fb47fa018f60465eef338bdd01a3259a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Jul 2022 14:30:21 +0000 Subject: [PATCH 312/627] Replace typeid_cast() with checkAndGetColumnConst() ... syntactic sugar --- src/Functions/FunctionsMultiStringFuzzySearch.h | 2 +- src/Functions/FunctionsMultiStringSearch.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 7e2d004070b..31c8b64ec10 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -70,7 +70,7 @@ public: const ColumnPtr & needles_ptr = arguments[2].column; const ColumnString * col_haystack_vector = checkAndGetColumn(&*haystack_ptr); - const ColumnConst * col_haystack_const = typeid_cast(&*haystack_ptr); + const ColumnConst * col_haystack_const = checkAndGetColumnConst(&*haystack_ptr); assert(static_cast(col_haystack_vector) ^ static_cast(col_haystack_const)); UInt32 edit_distance = 0; diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 560fe1313e2..c7fa2b1a416 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -81,7 +81,7 @@ public: const ColumnPtr & needles_ptr = arguments[1].column; const ColumnString * col_haystack_vector = checkAndGetColumn(&*haystack_ptr); - const ColumnConst * col_haystack_const = typeid_cast(&*haystack_ptr); + const ColumnConst * col_haystack_const = checkAndGetColumnConst(&*haystack_ptr); assert(static_cast(col_haystack_vector) ^ static_cast(col_haystack_const)); const ColumnArray * col_needles = checkAndGetColumn(needles_ptr.get()); From 144c1edb035157439fa40fb6e1a5da2789aa50aa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Jul 2022 14:42:19 +0000 Subject: [PATCH 313/627] Inherit default implementation of getArgumentsThatAreAlwaysConstant() --- src/Functions/FunctionsMultiStringSearch.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index c7fa2b1a416..1f6471640a3 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -61,7 +61,6 @@ public: size_t getNumberOfArguments() const override { return 2; } bool useDefaultImplementationForConstants() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { From 0b0d64c5caa152a1837781384c8d54ec8400f641 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Jul 2022 14:45:22 +0000 Subject: [PATCH 314/627] Don't resize output vector in each loop iteration --- src/Functions/MultiMatchAllIndicesImpl.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index adba8817b33..56ef0a828a9 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -162,6 +162,7 @@ struct MultiMatchAllIndicesImpl if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); #if USE_VECTORSCAN + offsets.resize(haystack_offsets.size()); size_t prev_haystack_offset = 0; for (size_t i = 0; i < haystack_offsets.size(); ++i) { @@ -175,8 +176,6 @@ struct MultiMatchAllIndicesImpl checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); - offsets.resize(haystack_offsets.size()); - const auto & hyperscan_regex = MultiRegexps::get(needles, edit_distance); hs_scratch_t * scratch = nullptr; hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch); From 8a18705729203abff31b26f483f1d9586b61c1dd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Jul 2022 14:55:02 +0000 Subject: [PATCH 315/627] More variable renamings for more uniformity --- src/Functions/FunctionsMultiStringFuzzySearch.h | 8 ++++---- src/Functions/FunctionsMultiStringSearch.h | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 31c8b64ec10..8f4228e8cd4 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -83,11 +83,11 @@ public: else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}. The number is not const or does not fit in UInt32", arguments[1].column->getName()); - const ColumnArray * col_needles = checkAndGetColumn(needles_ptr.get()); + const ColumnArray * col_needles_vector = checkAndGetColumn(needles_ptr.get()); const ColumnConst * col_needles_const = checkAndGetColumnConst(needles_ptr.get()); - assert(static_cast(col_needles) ^ static_cast(col_needles_const)); + assert(static_cast(col_needles_vector) ^ static_cast(col_needles_const)); - if (col_haystack_const && col_needles) + if (col_haystack_const && col_needles_vector) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support search with non-constant needles in constant haystack", name); using ResultType = typename Impl::ResultType; @@ -108,7 +108,7 @@ public: else { Impl::vectorVector( - col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), *col_needles, vec_res, offsets_res, edit_distance, + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), *col_needles_vector, vec_res, offsets_res, edit_distance, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); } diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 1f6471640a3..ddd6c72d3c0 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -83,11 +83,11 @@ public: const ColumnConst * col_haystack_const = checkAndGetColumnConst(&*haystack_ptr); assert(static_cast(col_haystack_vector) ^ static_cast(col_haystack_const)); - const ColumnArray * col_needles = checkAndGetColumn(needles_ptr.get()); + const ColumnArray * col_needles_vector = checkAndGetColumn(needles_ptr.get()); const ColumnConst * col_needles_const = checkAndGetColumnConst(needles_ptr.get()); - assert(static_cast(col_needles) ^ static_cast(col_needles_const)); + assert(static_cast(col_needles_vector) ^ static_cast(col_needles_const)); - if (col_haystack_const && col_needles) + if (col_haystack_const && col_needles_vector) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support search with non-constant needles in constant haystack", name); using ResultType = typename Impl::ResultType; @@ -108,7 +108,7 @@ public: else { Impl::vectorVector( - col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), *col_needles, vec_res, offsets_res, + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), *col_needles_vector, vec_res, offsets_res, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); } From b27d3ec5ceb592751562a3497107a6b0412df9ca Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Jul 2022 15:11:23 +0000 Subject: [PATCH 316/627] Fix build --- src/Interpreters/ExpressionAnalyzer.cpp | 4 ++-- src/Interpreters/HashJoin.h | 7 +------ 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 203797a0731..b79dc9c80e9 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1100,9 +1100,9 @@ static std::shared_ptr chooseJoinAlgorithm(std::shared_ptr ana throw Exception("Full sorting merge join is supported only for single-condition joins", ErrorCodes::NOT_IMPLEMENTED); if (analyzed_join->isSpecialStorage()) throw Exception("Full sorting merge join is not supported for special storage", ErrorCodes::NOT_IMPLEMENTED); - return std::make_shared(analyzed_join, sample_block); + return std::make_shared(analyzed_join, right_sample_block); } - return std::make_shared(analyzed_join, sample_block); + return std::make_shared(analyzed_join, right_sample_block); } static std::unique_ptr buildJoinedPlan( diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 7fd1b7771d0..00bf92059bd 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -173,12 +173,7 @@ public: bool isFilled() const override { return from_storage_join || data->type == Type::DICT; } - /** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. - */ - void setTotals(const Block & block) override { totals = block; } - const Block & getTotals() const override { return totals; } - - virtual JoinPipelineType pipelineType() const override + JoinPipelineType pipelineType() const override { /// No need to process anything in the right stream if it's a dictionary will just join the left stream with it. bool is_filled = from_storage_join || data->type == Type::DICT; From beac1820dae0924c5799311de59e4262b24d6c68 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Jul 2022 15:33:11 +0000 Subject: [PATCH 317/627] Test for EXPLAIN AST after_rewrite option --- .../02353_explain_ast_after_rewrite.reference | 25 +++++++++++++++++++ .../02353_explain_ast_after_rewrite.sql | 4 +++ 2 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/02353_explain_ast_after_rewrite.reference create mode 100644 tests/queries/0_stateless/02353_explain_ast_after_rewrite.sql diff --git a/tests/queries/0_stateless/02353_explain_ast_after_rewrite.reference b/tests/queries/0_stateless/02353_explain_ast_after_rewrite.reference new file mode 100644 index 00000000000..05df82f3a4c --- /dev/null +++ b/tests/queries/0_stateless/02353_explain_ast_after_rewrite.reference @@ -0,0 +1,25 @@ +-- { echoOn } +EXPLAIN AST after_rewrite=0 SELECT * FROM numbers(0); +SelectWithUnionQuery (children 1) + ExpressionList (children 1) + SelectQuery (children 2) + ExpressionList (children 1) + Asterisk + TablesInSelectQuery (children 1) + TablesInSelectQueryElement (children 1) + TableExpression (children 1) + Function numbers (children 1) + ExpressionList (children 1) + Literal UInt64_0 +EXPLAIN AST after_rewrite=1 SELECT * FROM numbers(0); +SelectWithUnionQuery (children 1) + ExpressionList (children 1) + SelectQuery (children 2) + ExpressionList (children 1) + Identifier number + TablesInSelectQuery (children 1) + TablesInSelectQueryElement (children 1) + TableExpression (children 1) + Function numbers (children 1) + ExpressionList (children 1) + Literal UInt64_0 diff --git a/tests/queries/0_stateless/02353_explain_ast_after_rewrite.sql b/tests/queries/0_stateless/02353_explain_ast_after_rewrite.sql new file mode 100644 index 00000000000..9bede77780e --- /dev/null +++ b/tests/queries/0_stateless/02353_explain_ast_after_rewrite.sql @@ -0,0 +1,4 @@ +-- { echoOn } +EXPLAIN AST after_rewrite=0 SELECT * FROM numbers(0); +EXPLAIN AST after_rewrite=1 SELECT * FROM numbers(0); +-- { echoOff } From 810926c7fc5a3a503c50ae643a6e89e349f96ff3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Jul 2022 17:29:30 +0000 Subject: [PATCH 318/627] Rename setting `after_rewrite` -> `rewrite` --- src/Interpreters/InterpreterExplainQuery.cpp | 6 +++--- .../queries/0_stateless/02353_explain_ast_after_rewrite.sql | 4 ---- ...ewrite.reference => 02353_explain_ast_rewrite.reference} | 4 ++-- tests/queries/0_stateless/02353_explain_ast_rewrite.sql | 4 ++++ 4 files changed, 9 insertions(+), 9 deletions(-) delete mode 100644 tests/queries/0_stateless/02353_explain_ast_after_rewrite.sql rename tests/queries/0_stateless/{02353_explain_ast_after_rewrite.reference => 02353_explain_ast_rewrite.reference} (86%) create mode 100644 tests/queries/0_stateless/02353_explain_ast_rewrite.sql diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index e0de29fe415..3fad4374abe 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -146,14 +146,14 @@ namespace struct QueryASTSettings { bool graph = false; - bool after_rewrite = false; + bool rewrite = false; constexpr static char name[] = "AST"; std::unordered_map> boolean_settings = { {"graph", graph}, - {"after_rewrite", after_rewrite} + {"rewrite", rewrite} }; }; @@ -280,7 +280,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() case ASTExplainQuery::ParsedAST: { auto settings = checkAndGetSettings(ast.getSettings()); - if (settings.after_rewrite) + if (settings.rewrite) { ExplainAnalyzedSyntaxVisitor::Data data(getContext()); ExplainAnalyzedSyntaxVisitor(data).visit(query); diff --git a/tests/queries/0_stateless/02353_explain_ast_after_rewrite.sql b/tests/queries/0_stateless/02353_explain_ast_after_rewrite.sql deleted file mode 100644 index 9bede77780e..00000000000 --- a/tests/queries/0_stateless/02353_explain_ast_after_rewrite.sql +++ /dev/null @@ -1,4 +0,0 @@ --- { echoOn } -EXPLAIN AST after_rewrite=0 SELECT * FROM numbers(0); -EXPLAIN AST after_rewrite=1 SELECT * FROM numbers(0); --- { echoOff } diff --git a/tests/queries/0_stateless/02353_explain_ast_after_rewrite.reference b/tests/queries/0_stateless/02353_explain_ast_rewrite.reference similarity index 86% rename from tests/queries/0_stateless/02353_explain_ast_after_rewrite.reference rename to tests/queries/0_stateless/02353_explain_ast_rewrite.reference index 05df82f3a4c..5ee3e0d126c 100644 --- a/tests/queries/0_stateless/02353_explain_ast_after_rewrite.reference +++ b/tests/queries/0_stateless/02353_explain_ast_rewrite.reference @@ -1,5 +1,5 @@ -- { echoOn } -EXPLAIN AST after_rewrite=0 SELECT * FROM numbers(0); +EXPLAIN AST rewrite=0 SELECT * FROM numbers(0); SelectWithUnionQuery (children 1) ExpressionList (children 1) SelectQuery (children 2) @@ -11,7 +11,7 @@ SelectWithUnionQuery (children 1) Function numbers (children 1) ExpressionList (children 1) Literal UInt64_0 -EXPLAIN AST after_rewrite=1 SELECT * FROM numbers(0); +EXPLAIN AST rewrite=1 SELECT * FROM numbers(0); SelectWithUnionQuery (children 1) ExpressionList (children 1) SelectQuery (children 2) diff --git a/tests/queries/0_stateless/02353_explain_ast_rewrite.sql b/tests/queries/0_stateless/02353_explain_ast_rewrite.sql new file mode 100644 index 00000000000..7310aa62704 --- /dev/null +++ b/tests/queries/0_stateless/02353_explain_ast_rewrite.sql @@ -0,0 +1,4 @@ +-- { echoOn } +EXPLAIN AST rewrite=0 SELECT * FROM numbers(0); +EXPLAIN AST rewrite=1 SELECT * FROM numbers(0); +-- { echoOff } From dec184f61ddb8cba1b4d6e4b01994a31af810493 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Jul 2022 15:27:01 +0000 Subject: [PATCH 319/627] Add comment about const needle + const haystack --- src/Functions/FunctionsMultiStringFuzzySearch.h | 4 ++++ src/Functions/FunctionsMultiStringSearch.h | 3 +++ 2 files changed, 7 insertions(+) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 8f4228e8cd4..34295b7e914 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -112,6 +112,10 @@ public: allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); } + // the combination of const haystack + const needle is not implemented because + // useDefaultImplementationForConstants() == true makes upper layers convert both to + // non-const columns + if constexpr (Impl::is_column_array) return ColumnArray::create(std::move(col_res), std::move(col_offsets)); else diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index ddd6c72d3c0..f18a7d0904c 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -112,6 +112,9 @@ public: allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); } + // the combination of const haystack + const needle is not implemented because + // useDefaultImplementationForConstants() == true makes upper layers convert both to + // non-const columns if constexpr (Impl::is_column_array) return ColumnArray::create(std::move(col_res), std::move(col_offsets)); From 88bc5a84e5806b5a6c3285a4b25337a4132a53f9 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 6 Jul 2022 14:06:53 -0400 Subject: [PATCH 320/627] update README --- docker/server/README.md | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/docker/server/README.md b/docker/server/README.md index 2ff08620658..352ebc9b29d 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -2,9 +2,9 @@ ## What is ClickHouse? -ClickHouse is an open-source column-oriented database management system that allows the generation of analytical data reports in real-time. +We are the creators of the popular open-source column-oriented DBMS (columnar database management system) for online analytical processing (OLAP) which allows users to generate analytical reports using SQL queries in real-time. -ClickHouse manages extremely large volumes of data. It currently powers [Yandex.Metrica](https://metrica.yandex.com/), the world’s [second-largest](http://w3techs.com/technologies/overview/traffic_analysis/all) web analytics platform, with over 13 trillion database records and over 20 billion events a day, generating customized reports on-the-fly, directly from non-aggregated data. This system was successfully implemented at [CERN’s LHCb experiment](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) to store and process metadata on 10bn events with over 1000 attributes per event registered in 2011. +ClickHouse works 100-1000x faster than traditional database management systems, and processes hundreds of millions to over a billion rows and tens of gigabytes of data per server per second. With a widespread user base around the globe, the technology has received praise for its reliability, ease of use, and fault tolerance. For more information and documentation see https://clickhouse.com/. @@ -52,7 +52,10 @@ You can expose your ClickHouse running in docker by [mapping a particular port]( ```bash docker run -d -p 18123:8123 -p19000:9000 --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server echo 'SELECT version()' | curl 'http://localhost:18123/' --data-binary @- -20.12.3.3 +``` + +```response +22.6.3.35 ``` or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows archiving better network performance): @@ -60,7 +63,10 @@ or by allowing the container to use [host ports directly](https://docs.docker.co ```bash docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server echo 'SELECT version()' | curl 'http://localhost:8123/' --data-binary @- -20.12.3.3 +``` + +```response +22.6.3.35 ``` ### Volumes From af76e17e2e41cc1fc0ac46f36949251970b2bb4c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Jul 2022 22:55:09 +0200 Subject: [PATCH 321/627] Add Infobright --- benchmark/infobright/benchmark.sh | 6 +++ benchmark/infobright/results/c6a.4xlarge.txt | 46 +++++++++++++++++++- 2 files changed, 51 insertions(+), 1 deletion(-) diff --git a/benchmark/infobright/benchmark.sh b/benchmark/infobright/benchmark.sh index 97ae6284d82..8075894248c 100755 --- a/benchmark/infobright/benchmark.sh +++ b/benchmark/infobright/benchmark.sh @@ -30,3 +30,9 @@ sudo docker exec mysql_ib du -bcs /mnt/mysql_data/ /usr/local/infobright-4.0.7-x # 13 760 341 294 ./run.sh 2>&1 | log + +cat log.txt | + grep -P 'rows? in set|Empty set|^ERROR' | + sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) days? )?(([0-9.]+) hours? )?(([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \4 \6 \7/' | + awk '{ if ($4) { print $1 * 86400 + $2 * 3600 + $3 * 60 + $4 } else if ($3) { print $1 * 3600 + $2 * 60 + $3 } else if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/infobright/results/c6a.4xlarge.txt b/benchmark/infobright/results/c6a.4xlarge.txt index 7dce1056bd6..03c8950cdec 100644 --- a/benchmark/infobright/results/c6a.4xlarge.txt +++ b/benchmark/infobright/results/c6a.4xlarge.txt @@ -1,2 +1,46 @@ Load time: 2317 sec. -Data size: +Data size: 13 760 341 294 bytes + +[0.01,0.00,0.00], +[2.15,2.16,2.20], +[0.00,0.00,0.00], +[6.49,5.44,6.22], +[14.48,15.17,14.12], +[43.92,38.13,43.77], +[0.00,0.00,0.00], +[3.13,2.18,2.18], +[21.20,22.30,19.99], +[29.58,28.54,31.03], +[13.32,13.35,12.70], +[15.03,14.88,15.63], +[1577.62,1799.89,1765.26], +[1074.09,1050.3,1098.42], +[1966.33,2084.51,2001.73], +[29.32,27.62,28.42], +[270.15,14599.2,14551.6], +[110.16,108.44,112.2], +[71034.7,71325.4,70654.4], +[3.04,1.10,1.09], +[260.76,272.07,257.25], +[374.24,350.31,362.95], +[516.44,531.73,517.55], +[270.12,264.56,257.08], +[37.28,33.73,35.68], +[67.68,67.83,64.86], +[35.30,37.37,36.10], +[404.6,400.53,403.81], +[null,null,null], +[405.78,439.47,408.45], +[52.82,53.36,52.26], +[76.02,71.03,71.44], +[466.17,468.26,454.46], +[164222,164222,164222], +[164222,164222,164222], +[61.59,60.24,60.91], +[7.47,3.26,3.25], +[0.94,0.56,0.56], +[0.20,0.16,0.16], +[511,509.87,506.72], +[0.26,0.13,0.10], +[0.15,0.07,0.07], +[1.23,1.21,1.19] From a20a15ff304e924a046be7730ad113e56b163990 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Jul 2022 22:27:53 +0000 Subject: [PATCH 322/627] Tests + check that EXPLAIN SYNTAX return the same result for ordinary ORDER BY and ORDER BY tuple + performance --- tests/performance/order_by_read_in_order.xml | 4 +-- tests/performance/order_by_tuple.xml | 2 ++ .../02353_order_by_tuple.reference | 1 + .../0_stateless/02353_order_by_tuple.sh | 25 +++++++++++++++++++ 4 files changed, 30 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02353_order_by_tuple.reference create mode 100755 tests/queries/0_stateless/02353_order_by_tuple.sh diff --git a/tests/performance/order_by_read_in_order.xml b/tests/performance/order_by_read_in_order.xml index 9ba116bb955..358ef0da65d 100644 --- a/tests/performance/order_by_read_in_order.xml +++ b/tests/performance/order_by_read_in_order.xml @@ -1,9 +1,9 @@ -SELECT * FROM hits_100m_single ORDER BY CounterID, EventDate LIMIT 100 +SELECT * FROM hits_100m_single ORDER BY (CounterID, EventDate) LIMIT 100 SELECT * FROM hits_100m_single ORDER BY CounterID DESC, toStartOfWeek(EventDate) DESC LIMIT 100 -SELECT * FROM hits_100m_single ORDER BY CounterID, EventDate, URL LIMIT 100 +SELECT * FROM hits_100m_single ORDER BY (CounterID, EventDate, URL) LIMIT 100 SELECT * FROM hits_100m_single WHERE CounterID IN (152220, 168777, 149234, 149234) ORDER BY CounterID DESC, EventDate DESC LIMIT 100 SELECT * FROM hits_100m_single WHERE UserID=1988954671305023629 ORDER BY CounterID, EventDate LIMIT 100 diff --git a/tests/performance/order_by_tuple.xml b/tests/performance/order_by_tuple.xml index 74588e96869..bb6c4b205e3 100644 --- a/tests/performance/order_by_tuple.xml +++ b/tests/performance/order_by_tuple.xml @@ -1,3 +1,5 @@ select * from numbers(300000000) order by (1 - number , number + 1 , number) limit 10; + SELECT * FROM hits_100m_single ORDER BY (CounterID, EventDate) format Null + SELECT * FROM hits_100m_single ORDER BY (CounterID, EventDate, URL) format Null diff --git a/tests/queries/0_stateless/02353_order_by_tuple.reference b/tests/queries/0_stateless/02353_order_by_tuple.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02353_order_by_tuple.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02353_order_by_tuple.sh b/tests/queries/0_stateless/02353_order_by_tuple.sh new file mode 100755 index 00000000000..f7daee20cd1 --- /dev/null +++ b/tests/queries/0_stateless/02353_order_by_tuple.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +[ ! -z "$CLICKHOUSE_CLIENT_REDEFINED" ] && CLICKHOUSE_CLIENT=$CLICKHOUSE_CLIENT_REDEFINED + +################## +# checking that both queries have the same `EXPLAIN SYNTAX` output +# todo: `EXPLAIN SYTNAX` can be replaced by `EXPLAIN AST rewrite=1` when available +################## +QUERY_ORDER_BY="SELECT number AS a, number % 2 AS b FROM numbers(10) ORDER BY a DESC NULLS FIRST WITH FILL FROM 2 TO 1 STEP -1, b DESC NULLS FIRST WITH FILL FROM 2 TO 1 STEP -1" +QUERY_ORDER_BY_TUPLE="SELECT number AS a, number % 2 AS b FROM numbers(10) ORDER BY (a, b) DESC NULLS FIRST WITH FILL FROM 2 TO 1 STEP -1" + +EXPLAIN="EXPLAIN SYNTAX" +OUTPUT_EXPLAIN_ORDER_BY=$($CLICKHOUSE_CLIENT -q "$EXPLAIN $QUERY_ORDER_BY") +OUTPUT_EXPLAIN_ORDER_BY_TUPLE=$($CLICKHOUSE_CLIENT -q "$EXPLAIN $QUERY_ORDER_BY_TUPLE") + +$CLICKHOUSE_CLIENT -q "drop table if exists order_by_syntax" +$CLICKHOUSE_CLIENT -q "create table order_by_syntax (explain String) engine=Memory" +$CLICKHOUSE_CLIENT -q "insert into order_by_syntax values('$OUTPUT_EXPLAIN_ORDER_BY')" +$CLICKHOUSE_CLIENT -q "insert into order_by_syntax values('$OUTPUT_EXPLAIN_ORDER_BY_TUPLE')" +$CLICKHOUSE_CLIENT -q "select count(distinct explain) from order_by_syntax" +$CLICKHOUSE_CLIENT -q "drop table if exists order_by_syntax" From 13889be1d446a1264c414678b19e371b5201d486 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 00:28:56 +0200 Subject: [PATCH 323/627] Change generator for a new page --- benchmark/generate-results.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/benchmark/generate-results.sh b/benchmark/generate-results.sh index c317c3691a4..3dff8cc3f1c 100755 --- a/benchmark/generate-results.sh +++ b/benchmark/generate-results.sh @@ -1,5 +1,11 @@ #!/bin/bash -ls -1 */results/*.txt | while read file; do SYSTEM=$(echo "$file" | grep -oP '^[\w-]+'); SETUP=$(echo "$file" | sed -r -e 's/^.*\/([a-zA-Z0-9_.-]+)\.txt$/\1/'); echo "[{\"system\": \"${SYSTEM} (${SETUP})\", \"version\": \"\", \"data_size\": 100000000, \"time\": \"2022-07-01 00:00:00\", \"comments\": \"\", \"result\": [ +ls -1 */results/*.txt | while read file +do + SYSTEM=$(echo "$file" | grep -oP '^[\w-]+'); + SETUP=$(echo "$file" | sed -r -e 's/^.*\/([a-zA-Z0-9_.-]+)\.txt$/\1/'); + + echo "{\"system\": \"${SYSTEM}\", \"machine\": \"${SETUP}\", \"time\": \"$(git log -1 --pretty="format:%cs" $file)\", \"result\": [ $(grep -P '^\[.+\]' $file) -]}]" > ../website/benchmark/dbms/results/${SYSTEM}.${SETUP}.json; done +]}," +done From 4b3020c9a539df08cf6571944a8f770833e4ab13 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 00:33:13 +0200 Subject: [PATCH 324/627] Add a test for simdjson --- .../0_stateless/02353_simdjson_buffer_overflow.reference | 0 tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql | 4 ++++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02353_simdjson_buffer_overflow.reference create mode 100644 tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql diff --git a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.reference b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql new file mode 100644 index 00000000000..763c99a77c3 --- /dev/null +++ b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql @@ -0,0 +1,4 @@ +SET max_execution_time = 3; +SET timeout_overflow_mode = 'break'; + +SELECT count() FROM system.numbers_mt WHERE NOT ignore(JSONExtract('{' || repeat('"a":"b",', rand() % 10) || '"c":"d"}', 'a', 'String')) FORMAT Null; From d51f9b954b87359336c2c1ad91f3c3a5da3d201d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 6 Jul 2022 20:41:01 -0300 Subject: [PATCH 325/627] Update exchange.md --- docs/en/sql-reference/statements/exchange.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/exchange.md b/docs/en/sql-reference/statements/exchange.md index cf1be28fe7c..8bfb142e8e3 100644 --- a/docs/en/sql-reference/statements/exchange.md +++ b/docs/en/sql-reference/statements/exchange.md @@ -15,7 +15,7 @@ The `EXCHANGE` query is supported by the [Atomic](../../engines/database-engines **Syntax** ```sql -EXCHANGE TABLES|DICTIONARIES [db0.]name_A AND [db1.]name_B +EXCHANGE TABLES|DICTIONARIES [db0.]name_A AND [db1.]name_B [ON CLUSTER cluster] ``` ## EXCHANGE TABLES @@ -25,7 +25,7 @@ Exchanges the names of two tables. **Syntax** ```sql -EXCHANGE TABLES [db0.]table_A AND [db1.]table_B +EXCHANGE TABLES [db0.]table_A AND [db1.]table_B [ON CLUSTER cluster] ``` ## EXCHANGE DICTIONARIES @@ -35,7 +35,7 @@ Exchanges the names of two dictionaries. **Syntax** ```sql -EXCHANGE DICTIONARIES [db0.]dict_A AND [db1.]dict_B +EXCHANGE DICTIONARIES [db0.]dict_A AND [db1.]dict_B [ON CLUSTER cluster] ``` **See Also** From d542e8fb7ba408b5f5ae2a915aa09b224e16011d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 6 Jul 2022 20:43:45 -0300 Subject: [PATCH 326/627] Update exchange.md --- docs/ru/sql-reference/statements/exchange.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/statements/exchange.md b/docs/ru/sql-reference/statements/exchange.md index 91de858fb9a..c8c97a1daa4 100644 --- a/docs/ru/sql-reference/statements/exchange.md +++ b/docs/ru/sql-reference/statements/exchange.md @@ -14,7 +14,7 @@ sidebar_label: EXCHANGE **Синтаксис** ```sql -EXCHANGE TABLES|DICTIONARIES [db0.]name_A AND [db1.]name_B +EXCHANGE TABLES|DICTIONARIES [db0.]name_A AND [db1.]name_B [ON CLUSTER cluster] ``` ## EXCHANGE TABLES {#exchange_tables} @@ -24,7 +24,7 @@ EXCHANGE TABLES|DICTIONARIES [db0.]name_A AND [db1.]name_B **Синтаксис** ```sql -EXCHANGE TABLES [db0.]table_A AND [db1.]table_B +EXCHANGE TABLES [db0.]table_A AND [db1.]table_B [ON CLUSTER cluster] ``` ## EXCHANGE DICTIONARIES {#exchange_dictionaries} @@ -34,7 +34,7 @@ EXCHANGE TABLES [db0.]table_A AND [db1.]table_B **Синтаксис** ```sql -EXCHANGE DICTIONARIES [db0.]dict_A AND [db1.]dict_B +EXCHANGE DICTIONARIES [db0.]dict_A AND [db1.]dict_B [ON CLUSTER cluster] ``` **Смотрите также** From f361ddcf495ad41f6de6e87d7171cbd1095ff0dc Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 6 Jul 2022 20:48:07 -0300 Subject: [PATCH 327/627] Update partition.md --- .../statements/alter/partition.md | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/ru/sql-reference/statements/alter/partition.md b/docs/ru/sql-reference/statements/alter/partition.md index 780d443a847..036f72fc951 100644 --- a/docs/ru/sql-reference/statements/alter/partition.md +++ b/docs/ru/sql-reference/statements/alter/partition.md @@ -25,7 +25,7 @@ sidebar_label: PARTITION ## DETACH PARTITION\|PART {#alter_detach-partition} ``` sql -ALTER TABLE table_name DETACH PARTITION|PART partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] DETACH PARTITION|PART partition_expr ``` Перемещает заданную партицию в директорию `detached`. Сервер не будет знать об этой партиции до тех пор, пока вы не выполните запрос [ATTACH](#alter_attach-partition). @@ -46,7 +46,7 @@ ALTER TABLE mt DETACH PART 'all_2_2_0'; ## DROP PARTITION\|PART {#alter_drop-partition} ``` sql -ALTER TABLE table_name DROP PARTITION|PART partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] DROP PARTITION|PART partition_expr ``` Удаляет партицию. Партиция помечается как неактивная и будет полностью удалена примерно через 10 минут. @@ -65,7 +65,7 @@ ALTER TABLE mt DROP PART 'all_4_4_0'; ## DROP DETACHED PARTITION\|PART {#alter_drop-detached} ``` sql -ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] DROP DETACHED PARTITION|PART partition_expr ``` Удаляет из `detached` кусок или все куски, принадлежащие партиции. @@ -74,7 +74,7 @@ ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr ## ATTACH PARTITION\|PART {#alter_attach-partition} ``` sql -ALTER TABLE table_name ATTACH PARTITION|PART partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] ATTACH PARTITION|PART partition_expr ``` Добавляет данные в таблицу из директории `detached`. Можно добавить данные как для целой партиции, так и для отдельного куска. Примеры: @@ -97,7 +97,7 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0; ## ATTACH PARTITION FROM {#alter_attach-partition-from} ``` sql -ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1 +ALTER TABLE table2 [ON CLUSTER cluster] ATTACH PARTITION partition_expr FROM table1 ``` Копирует партицию из таблицы `table1` в таблицу `table2`. @@ -113,7 +113,7 @@ ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1 ## REPLACE PARTITION {#alter_replace-partition} ``` sql -ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1 +ALTER TABLE table2 [ON CLUSTER cluster] REPLACE PARTITION partition_expr FROM table1 ``` Копирует партицию из таблицы `table1` в таблицу `table2` с заменой существующих данных в `table2`. Данные из `table1` не удаляются. @@ -128,7 +128,7 @@ ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1 ## MOVE PARTITION TO TABLE {#alter_move_to_table-partition} ``` sql -ALTER TABLE table_source MOVE PARTITION partition_expr TO TABLE table_dest +ALTER TABLE table_source [ON CLUSTER cluster] MOVE PARTITION partition_expr TO TABLE table_dest ``` Перемещает партицию из таблицы `table_source` в таблицу `table_dest` (добавляет к существующим данным в `table_dest`) с удалением данных из таблицы `table_source`. @@ -143,7 +143,7 @@ ALTER TABLE table_source MOVE PARTITION partition_expr TO TABLE table_dest ## CLEAR COLUMN IN PARTITION {#alter_clear-column-partition} ``` sql -ALTER TABLE table_name CLEAR COLUMN column_name IN PARTITION partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] CLEAR COLUMN column_name IN PARTITION partition_expr ``` Сбрасывает все значения в столбце для заданной партиции. Если для столбца определено значение по умолчанию (в секции `DEFAULT`), то будет выставлено это значение. @@ -157,7 +157,7 @@ ALTER TABLE visits CLEAR COLUMN hour in PARTITION 201902 ## CLEAR INDEX IN PARTITION {#alter_clear-index-partition} ``` sql -ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] CLEAR INDEX index_name IN PARTITION partition_expr ``` Работает как `CLEAR COLUMN`, но сбрасывает индексы вместо данных в столбцах. @@ -165,7 +165,7 @@ ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr ## FREEZE PARTITION {#alter_freeze-partition} ``` sql -ALTER TABLE table_name FREEZE [PARTITION partition_expr] [WITH NAME 'backup_name'] +ALTER TABLE table_name [ON CLUSTER cluster] FREEZE [PARTITION partition_expr] [WITH NAME 'backup_name'] ``` Создаёт резервную копию для заданной партиции. Если выражение `PARTITION` опущено, резервные копии будут созданы для всех партиций. @@ -205,7 +205,7 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr] [WITH NAME 'backup_name ## UNFREEZE PARTITION {#alter_unfreeze-partition} ``` sql -ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name' +ALTER TABLE table_name [ON CLUSTER cluster] UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name' ``` Удаляет с диска "замороженные" партиции с указанным именем. Если секция `PARTITION` опущена, запрос удаляет резервную копию всех партиций сразу. @@ -213,7 +213,7 @@ ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name ## FETCH PARTITION\|PART {#alter_fetch-partition} ``` sql -ALTER TABLE table_name FETCH PARTITION|PART partition_expr FROM 'path-in-zookeeper' +ALTER TABLE table_name [ON CLUSTER cluster] FETCH PARTITION|PART partition_expr FROM 'path-in-zookeeper' ``` Загружает партицию с другого сервера. Этот запрос работает только для реплицированных таблиц. @@ -250,7 +250,7 @@ ALTER TABLE users ATTACH PART 201901_2_2_0; Перемещает партицию или кусок данных на другой том или диск для таблиц с движком `MergeTree`. Смотрите [Хранение данных таблицы на нескольких блочных устройствах](../../statements/alter/index.md#table_engine-mergetree-multiple-volumes). ``` sql -ALTER TABLE table_name MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name' +ALTER TABLE table_name [ON CLUSTER cluster] MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name' ``` Запрос `ALTER TABLE t MOVE`: @@ -273,7 +273,7 @@ ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd' Синтаксис: ``` sql -ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] [IN PARTITION partition_id] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] [IN PARTITION partition_id] WHERE filter_expr ``` ### Пример @@ -293,7 +293,7 @@ ALTER TABLE mt UPDATE x = x + 1 IN PARTITION 2 WHERE p = 2; Синтаксис: ``` sql -ALTER TABLE [db.]table DELETE [IN PARTITION partition_id] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE [IN PARTITION partition_id] WHERE filter_expr ``` ### Пример From f96225cca485c1dbdc35691d2a9825b2bd6b6419 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 6 Jul 2022 20:50:13 -0300 Subject: [PATCH 328/627] Update partition.md --- .../statements/alter/partition.md | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 921ffb71066..079d462a536 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -27,7 +27,7 @@ The following operations with [partitions](../../../engines/table-engines/merget ## DETACH PARTITION\|PART ``` sql -ALTER TABLE table_name DETACH PARTITION|PART partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] DETACH PARTITION|PART partition_expr ``` Moves all data for the specified partition to the `detached` directory. The server forgets about the detached data partition as if it does not exist. The server will not know about this data until you make the [ATTACH](#alter_attach-partition) query. @@ -48,7 +48,7 @@ This query is replicated – it moves the data to the `detached` directory on al ## DROP PARTITION\|PART ``` sql -ALTER TABLE table_name DROP PARTITION|PART partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] DROP PARTITION|PART partition_expr ``` Deletes the specified partition from the table. This query tags the partition as inactive and deletes data completely, approximately in 10 minutes. @@ -67,7 +67,7 @@ ALTER TABLE mt DROP PART 'all_4_4_0'; ## DROP DETACHED PARTITION\|PART ``` sql -ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] DROP DETACHED PARTITION|PART partition_expr ``` Removes the specified part or all parts of the specified partition from `detached`. @@ -76,7 +76,7 @@ Read more about setting the partition expression in a section [How to specify th ## ATTACH PARTITION\|PART ``` sql -ALTER TABLE table_name ATTACH PARTITION|PART partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] ATTACH PARTITION|PART partition_expr ``` Adds data to the table from the `detached` directory. It is possible to add data for an entire partition or for a separate part. Examples: @@ -99,7 +99,7 @@ You can put data to the `detached` directory on one replica and use the `ALTER . ## ATTACH PARTITION FROM ``` sql -ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1 +ALTER TABLE table2 [ON CLUSTER cluster] ATTACH PARTITION partition_expr FROM table1 ``` This query copies the data partition from `table1` to `table2`. @@ -113,7 +113,7 @@ For the query to run successfully, the following conditions must be met: ## REPLACE PARTITION ``` sql -ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1 +ALTER TABLE table2 [ON CLUSTER cluster] REPLACE PARTITION partition_expr FROM table1 ``` This query copies the data partition from the `table1` to `table2` and replaces existing partition in the `table2`. Note that data won’t be deleted from `table1`. @@ -126,7 +126,7 @@ For the query to run successfully, the following conditions must be met: ## MOVE PARTITION TO TABLE ``` sql -ALTER TABLE table_source MOVE PARTITION partition_expr TO TABLE table_dest +ALTER TABLE table_source [ON CLUSTER cluster] MOVE PARTITION partition_expr TO TABLE table_dest ``` This query moves the data partition from the `table_source` to `table_dest` with deleting the data from `table_source`. @@ -141,7 +141,7 @@ For the query to run successfully, the following conditions must be met: ## CLEAR COLUMN IN PARTITION ``` sql -ALTER TABLE table_name CLEAR COLUMN column_name IN PARTITION partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] CLEAR COLUMN column_name IN PARTITION partition_expr ``` Resets all values in the specified column in a partition. If the `DEFAULT` clause was determined when creating a table, this query sets the column value to a specified default value. @@ -155,7 +155,7 @@ ALTER TABLE visits CLEAR COLUMN hour in PARTITION 201902 ## FREEZE PARTITION ``` sql -ALTER TABLE table_name FREEZE [PARTITION partition_expr] [WITH NAME 'backup_name'] +ALTER TABLE table_name [ON CLUSTER cluster] FREEZE [PARTITION partition_expr] [WITH NAME 'backup_name'] ``` This query creates a local backup of a specified partition. If the `PARTITION` clause is omitted, the query creates the backup of all partitions at once. @@ -197,7 +197,7 @@ For more information about backups and restoring data, see the [Data Backup](../ ## UNFREEZE PARTITION ``` sql -ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name' +ALTER TABLE table_name [ON CLUSTER cluster] UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name' ``` Removes `freezed` partitions with the specified name from the disk. If the `PARTITION` clause is omitted, the query removes the backup of all partitions at once. @@ -205,7 +205,7 @@ Removes `freezed` partitions with the specified name from the disk. If the `PART ## CLEAR INDEX IN PARTITION ``` sql -ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] CLEAR INDEX index_name IN PARTITION partition_expr ``` The query works similar to `CLEAR COLUMN`, but it resets an index instead of a column data. @@ -213,7 +213,7 @@ The query works similar to `CLEAR COLUMN`, but it resets an index instead of a c ## FETCH PARTITION|PART ``` sql -ALTER TABLE table_name FETCH PARTITION|PART partition_expr FROM 'path-in-zookeeper' +ALTER TABLE table_name [ON CLUSTER cluster] FETCH PARTITION|PART partition_expr FROM 'path-in-zookeeper' ``` Downloads a partition from another server. This query only works for the replicated tables. @@ -250,7 +250,7 @@ Although the query is called `ALTER TABLE`, it does not change the table structu Moves partitions or data parts to another volume or disk for `MergeTree`-engine tables. See [Using Multiple Block Devices for Data Storage](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). ``` sql -ALTER TABLE table_name MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name' +ALTER TABLE table_name [ON CLUSTER cluster] MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name' ``` The `ALTER TABLE t MOVE` query: @@ -273,7 +273,7 @@ Manipulates data in the specifies partition matching the specified filtering exp Syntax: ``` sql -ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] [IN PARTITION partition_id] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] [IN PARTITION partition_id] WHERE filter_expr ``` ### Example @@ -293,7 +293,7 @@ Deletes data in the specifies partition matching the specified filtering express Syntax: ``` sql -ALTER TABLE [db.]table DELETE [IN PARTITION partition_id] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE [IN PARTITION partition_id] WHERE filter_expr ``` ### Example From 71c4e738d020c3eb9b6f4709cccd4f36ae8bd200 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 6 Jul 2022 20:55:06 -0300 Subject: [PATCH 329/627] Update update.md --- docs/en/sql-reference/statements/alter/update.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/update.md b/docs/en/sql-reference/statements/alter/update.md index 5937f2e8f79..26a2d57f050 100644 --- a/docs/en/sql-reference/statements/alter/update.md +++ b/docs/en/sql-reference/statements/alter/update.md @@ -6,7 +6,7 @@ sidebar_label: UPDATE # ALTER TABLE … UPDATE Statements ``` sql -ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] WHERE filter_expr ``` Manipulates data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). From 0a70e596a8ba97dceefdf4956833b27f1e35fa51 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 6 Jul 2022 20:55:19 -0300 Subject: [PATCH 330/627] Update update.md --- docs/ru/sql-reference/statements/alter/update.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/update.md b/docs/ru/sql-reference/statements/alter/update.md index 0a12bb6ca0c..23c6d1b65b0 100644 --- a/docs/ru/sql-reference/statements/alter/update.md +++ b/docs/ru/sql-reference/statements/alter/update.md @@ -6,7 +6,7 @@ sidebar_label: UPDATE # ALTER TABLE … UPDATE {#alter-table-update-statements} ``` sql -ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] WHERE filter_expr ``` Манипулирует данными, соответствующими заданному выражению фильтрации. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations). From 9d520e391ff6826468792611141e7af04ca84763 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 6 Jul 2022 20:57:25 -0300 Subject: [PATCH 331/627] Update index.md --- docs/ru/sql-reference/statements/alter/index/index.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/statements/alter/index/index.md b/docs/ru/sql-reference/statements/alter/index/index.md index 9a00c90328d..f73005b1bfe 100644 --- a/docs/ru/sql-reference/statements/alter/index/index.md +++ b/docs/ru/sql-reference/statements/alter/index/index.md @@ -9,9 +9,9 @@ sidebar_label: "Манипуляции с индексами" Добавить или удалить индекс можно с помощью операций ``` sql -ALTER TABLE [db.]name ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name] -ALTER TABLE [db.]name DROP INDEX name -ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name +ALTER TABLE [db.]table_name [ON CLUSTER cluster] ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name] +ALTER TABLE [db.]table_name [ON CLUSTER cluster] DROP INDEX name +ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name IN PARTITION partition_name ``` Поддерживается только таблицами семейства `*MergeTree`. From db357f4a3c9faae7ddf0ab56be8eb91b10232c24 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 6 Jul 2022 20:57:49 -0300 Subject: [PATCH 332/627] Update index.md --- docs/en/sql-reference/statements/alter/index/index.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/index/index.md b/docs/en/sql-reference/statements/alter/index/index.md index 2c2f1c9b2f2..c9e7de22076 100644 --- a/docs/en/sql-reference/statements/alter/index/index.md +++ b/docs/en/sql-reference/statements/alter/index/index.md @@ -8,11 +8,11 @@ sidebar_label: INDEX The following operations are available: -- `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name]` - Adds index description to tables metadata. +- `ALTER TABLE [db].table_name [ON CLUSTER cluster] ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name]` - Adds index description to tables metadata. -- `ALTER TABLE [db].name DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk. +- `ALTER TABLE [db].table_name [ON CLUSTER cluster] DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk. -- `ALTER TABLE [db.]table MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. +- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. The first two commands are lightweight in a sense that they only change metadata or remove files. From 0c783800c837da9644270b566ad563ec77254d9f Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 7 Jul 2022 01:47:33 +0000 Subject: [PATCH 333/627] Better --- src/Client/ClientBase.cpp | 30 ++++++++++++------- src/IO/CompressionMethod.cpp | 13 ++++++++ src/IO/CompressionMethod.h | 3 ++ src/Parsers/ParserQueryWithOutput.cpp | 9 ++++-- .../0_stateless/02353_compression_level.sh | 4 +-- 5 files changed, 44 insertions(+), 15 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 13056c65449..b998b01681b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -524,26 +524,34 @@ try const auto & out_file_node = query_with_output->out_file->as(); out_file = out_file_node.value.safeGet(); - std::string compression_method; - UInt64 compression_level = 3; + std::string compression_method_string; + if (query_with_output->compression) { const auto & compression_method_node = query_with_output->compression->as(); - compression_method = compression_method_node.value.safeGet(); + compression_method_string = compression_method_node.value.safeGet(); + } - if (query_with_output->compression_level) - { - const auto & compression_level_node = query_with_output->compression_level->as(); - bool res = compression_level_node.value.tryGet(compression_level); + CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); + UInt64 compression_level = 3; - if (!res || compression_level < 1 || compression_level > 9) - throw Exception("Invalid compression level, must be positive integer in range 1-9", ErrorCodes::BAD_ARGUMENTS); - } + if (query_with_output->compression_level) + { + const auto & compression_level_node = query_with_output->compression_level->as(); + bool res = compression_level_node.value.tryGet(compression_level); + + auto range = getCompressionLevelRange(compression_method); + if (!res || compression_level < range.first || compression_level > range.second) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid compression level, must be positive integer in range {}-{}", + range.first, + range.second); } out_file_buf = wrapWriteBufferWithCompressionMethod( std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), - chooseCompressionMethod(out_file, compression_method), + compression_method, compression_level ); diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index 0da235c074c..449335407aa 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -98,6 +98,19 @@ CompressionMethod chooseCompressionMethod(const std::string & path, const std::s ErrorCodes::NOT_IMPLEMENTED); } +std::pair getCompressionLevelRange(const CompressionMethod & method) +{ + switch (method) + { + case CompressionMethod::Zstd: + return {1, 22}; + case CompressionMethod::Lz4: + return {1, 12}; + default: + return {1, 9}; + } +} + static std::unique_ptr createCompressedWrapper( std::unique_ptr nested, CompressionMethod method, size_t buf_size, char * existing_memory, size_t alignment, int zstd_window_log_max) { diff --git a/src/IO/CompressionMethod.h b/src/IO/CompressionMethod.h index a399a756c13..29a18e5e978 100644 --- a/src/IO/CompressionMethod.h +++ b/src/IO/CompressionMethod.h @@ -46,6 +46,9 @@ std::string toContentEncodingName(CompressionMethod method); */ CompressionMethod chooseCompressionMethod(const std::string & path, const std::string & hint); +/// Get a range of the valid compression levels for the compression method. +std::pair getCompressionLevelRange(const CompressionMethod & method); + std::unique_ptr wrapReadBufferWithCompressionMethod( std::unique_ptr nested, CompressionMethod method, diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index f3c67791309..79203c6d3d1 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -97,8 +97,13 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!compression.parse(pos, query_with_output.compression, expected)) return false; - ParserNumber compression_level; - compression_level.parse(pos, query_with_output.compression_level, expected); + ParserKeyword s_compression_level("LEVEL"); + if (s_compression_level.ignore(pos, expected)) + { + ParserNumber compression_level; + if (!compression_level.parse(pos, query_with_output.compression_level, expected)) + return false; + } } query_with_output.children.push_back(query_with_output.out_file); diff --git a/tests/queries/0_stateless/02353_compression_level.sh b/tests/queries/0_stateless/02353_compression_level.sh index 364479063a2..b08dc1e204c 100755 --- a/tests/queries/0_stateless/02353_compression_level.sh +++ b/tests/queries/0_stateless/02353_compression_level.sh @@ -14,8 +14,8 @@ mkdir "${WORKING_FOLDER_02353}" for m in gz br xz zst lz4 bz2 do - ${CLICKHOUSE_CLIENT} --query "SELECT number, 'Hello, world!' FROM numbers(6000) INTO OUTFILE '${WORKING_FOLDER_02353}/${m}_1.${m}' COMPRESSION '${m}' 1" - ${CLICKHOUSE_CLIENT} --query "SELECT number, 'Hello, world!' FROM numbers(6000) INTO OUTFILE '${WORKING_FOLDER_02353}/${m}_9.${m}' COMPRESSION '${m}' 9" + ${CLICKHOUSE_CLIENT} --query "SELECT number, 'Hello, world!' FROM numbers(6000) INTO OUTFILE '${WORKING_FOLDER_02353}/${m}_1.${m}' COMPRESSION '${m}' LEVEL 1" + ${CLICKHOUSE_CLIENT} --query "SELECT number, 'Hello, world!' FROM numbers(6000) INTO OUTFILE '${WORKING_FOLDER_02353}/${m}_9.${m}' COMPRESSION '${m}' LEVEL 9" ${CLICKHOUSE_CLIENT} --query "SELECT count(), max(x), avg(length(s)) FROM file('${WORKING_FOLDER_02353}/${m}_1.${m}', 'TabSeparated', 'x UInt32, s String')" ${CLICKHOUSE_CLIENT} --query "SELECT count(), max(x), avg(length(s)) FROM file('${WORKING_FOLDER_02353}/${m}_9.${m}', 'TabSeparated', 'x UInt32, s String')" From 54d210d714c4c5977f1648f4da2bbb2ee3ede500 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 7 Jul 2022 01:59:00 +0000 Subject: [PATCH 334/627] Add documentation --- docs/en/sql-reference/statements/select/index.md | 2 +- docs/en/sql-reference/statements/select/into-outfile.md | 6 ++++-- docs/ru/sql-reference/statements/select/index.md | 2 +- docs/ru/sql-reference/statements/select/into-outfile.md | 6 ++++-- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index b05fb2ba334..66ed8a71f94 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -25,7 +25,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] [UNION ...] -[INTO OUTFILE filename [COMPRESSION type] ] +[INTO OUTFILE filename [COMPRESSION type [LEVEL level] ] [FORMAT format] ``` diff --git a/docs/en/sql-reference/statements/select/into-outfile.md b/docs/en/sql-reference/statements/select/into-outfile.md index db1ed2551a7..2e7ea8dceb1 100644 --- a/docs/en/sql-reference/statements/select/into-outfile.md +++ b/docs/en/sql-reference/statements/select/into-outfile.md @@ -6,16 +6,18 @@ sidebar_label: INTO OUTFILE `INTO OUTFILE` clause redirects the result of a `SELECT` query to a file on the **client** side. -Compressed files are supported. Compression type is detected by the extension of the file name (mode `'auto'` is used by default). Or it can be explicitly specified in a `COMPRESSION` clause. +Compressed files are supported. Compression type is detected by the extension of the file name (mode `'auto'` is used by default). Or it can be explicitly specified in a `COMPRESSION` clause. Compression level for a certain compression type can be specified in a `LEVEL` clause. **Syntax** ```sql -SELECT INTO OUTFILE file_name [COMPRESSION type] +SELECT INTO OUTFILE file_name [COMPRESSION type [LEVEL level]] ``` `file_name` and `type` are string literals. Supported compression types are: `'none'`, `'gzip'`, `'deflate'`, `'br'`, `'xz'`, `'zstd'`, `'lz4'`, `'bz2'`. +`level` is a numeric literal. Positive integers in following ranges are supported: `1-12` for `lz4` type, `1-22` for `zstd` type and `1-9` for other compression types. + ## Implementation Details - This functionality is available in the [command-line client](../../../interfaces/cli.md) and [clickhouse-local](../../../operations/utilities/clickhouse-local.md). Thus a query sent via [HTTP interface](../../../interfaces/http.md) will fail. diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index 6222efe5dd1..1edf93faeaa 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -24,7 +24,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] [UNION ALL ...] -[INTO OUTFILE filename [COMPRESSION type] ] +[INTO OUTFILE filename [COMPRESSION type [LEVEL level]] ] [FORMAT format] ``` diff --git a/docs/ru/sql-reference/statements/select/into-outfile.md b/docs/ru/sql-reference/statements/select/into-outfile.md index 2952ef2a3a2..67344711977 100644 --- a/docs/ru/sql-reference/statements/select/into-outfile.md +++ b/docs/ru/sql-reference/statements/select/into-outfile.md @@ -6,16 +6,18 @@ sidebar_label: INTO OUTFILE Секция `INTO OUTFILE` перенаправляет результат запроса `SELECT` в файл на стороне **клиента**. -Поддерживаются сжатые файлы. Формат сжатия определяется по расширению файла (по умолчанию используется режим `'auto'`), либо он может быть задан явно в секции `COMPRESSION`. +Поддерживаются сжатые файлы. Формат сжатия определяется по расширению файла (по умолчанию используется режим `'auto'`), либо он может быть задан явно в секции `COMPRESSION`. Уровень сжатия для конкретного алгоритма может быть зада в секции `LEVEL`. **Синтаксис** ```sql -SELECT INTO OUTFILE file_name [COMPRESSION type] +SELECT INTO OUTFILE file_name [COMPRESSION type [LEVEL level]] ``` `file_name` и `type` задаются в виде строковых литералов. Поддерживаются форматы сжатия: `'none`', `'gzip'`, `'deflate'`, `'br'`, `'xz'`, `'zstd'`, `'lz4'`, `'bz2'`. +`level` задается в виде числового литерала. Поддерживаются положительные значения в следующих диапазонах: `1-12` для формата `lz4`, `1-22` для формата `zstd` и `1-9` для остальных форматов. + ## Детали реализации {#implementation-details} - Эта функция доступна только в следующих интерфейсах: [клиент командной строки](../../../interfaces/cli.md) и [clickhouse-local](../../../operations/utilities/clickhouse-local.md). Таким образом, запрос, отправленный через [HTTP интерфейс](../../../interfaces/http.md) вернет ошибку. From 5c5c7ae6dbcbdc722233bf5326c03555d5bf3c68 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 7 Jul 2022 02:09:41 +0000 Subject: [PATCH 335/627] Fix typos --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 3f4db831e3d..c51445675f5 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -131,7 +131,7 @@ Integer value in the `UInt8`, `UInt16`, `UInt32`, `UInt64` or `UInt256` data typ Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. -The behavior of functions for negative agruments and for the [NaN and Inf](../../sql-reference/data-types/float.md#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#numeric-conversion-issues), when using the functions. +The behavior of functions for negative arguments and for the [NaN and Inf](../../sql-reference/data-types/float.md#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#numeric-conversion-issues), when using the functions. **Example** From 573b81965ac8acc4bdc69f29d1d55f1a2eb25142 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 7 Jul 2022 02:20:25 +0000 Subject: [PATCH 336/627] Fix style --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b998b01681b..399975821bc 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -543,7 +543,7 @@ try auto range = getCompressionLevelRange(compression_method); if (!res || compression_level < range.first || compression_level > range.second) throw Exception( - ErrorCodes::BAD_ARGUMENTS, + ErrorCodes::BAD_ARGUMENTS, "Invalid compression level, must be positive integer in range {}-{}", range.first, range.second); From 5be9e418ed0f3c3d74375e0fd4642dac95e843fd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 05:24:26 +0200 Subject: [PATCH 337/627] Add HTML page --- benchmark/index.html | 2181 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 2181 insertions(+) create mode 100644 benchmark/index.html diff --git a/benchmark/index.html b/benchmark/index.html new file mode 100644 index 00000000000..7718d903589 --- /dev/null +++ b/benchmark/index.html @@ -0,0 +1,2181 @@ + + + + + ClickBench — a Benchmark For Analytical DBMS + + + + + + + + + + +

+

ClickBench — a Benchmark For Analytical DBMS

+ Methodology | Reproduce and Validate the Results | Add a System | Report Mistake +
+ + + + + + + + + + + + + + + + + + + + + + +
System: + All +
Type: + All +
Machine: + All +
Cluster size: + All +
Metric: + Cold RunHot RunLoad TimeStorage Size +
+ + + + + + + + + + +
+ System & Machine + + Relative time (lower is better) +
+ +
+

Detailed Comparison

+
+ + + + + + + + +
+ +
+ + + + From 367008b7fb9923bc5708c110e27328b7d78c2165 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 23 Jun 2022 10:26:38 +0300 Subject: [PATCH 338/627] Better objects removal in `S3ObjectStorage`. --- .../ObjectStorages/S3/S3ObjectStorage.cpp | 30 ++-------- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 10 +++- .../ObjectStorages/S3/registerDiskS3.cpp | 55 +++++++++++++++++-- 3 files changed, 64 insertions(+), 31 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 9236cde6e93..8a0402ef1b2 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -228,32 +228,12 @@ void S3ObjectStorage::removeObjectImpl(const std::string & path, bool if_exists) { auto client_ptr = client.get(); - // If chunk size is 0, only use single delete request - // This allows us to work with GCS, which doesn't support DeleteObjects - if (!s3_capabilities.support_batch_delete) - { - Aws::S3::Model::DeleteObjectRequest request; - request.SetBucket(bucket); - request.SetKey(path); - auto outcome = client_ptr->DeleteObject(request); + Aws::S3::Model::DeleteObjectRequest request; + request.SetBucket(bucket); + request.SetKey(path); + auto outcome = client_ptr->DeleteObject(request); - throwIfUnexpectedError(outcome, if_exists); - } - else - { - /// TODO: For AWS we prefer to use multiobject operation even for single object - /// maybe we shouldn't? - Aws::S3::Model::ObjectIdentifier obj; - obj.SetKey(path); - Aws::S3::Model::Delete delkeys; - delkeys.SetObjects({obj}); - Aws::S3::Model::DeleteObjectsRequest request; - request.SetBucket(bucket); - request.SetDelete(delkeys); - auto outcome = client_ptr->DeleteObjects(request); - - throwIfUnexpectedError(outcome, if_exists); - } + throwIfUnexpectedError(outcome, if_exists); } void S3ObjectStorage::removeObjectsImpl(const PathsWithSize & paths, bool if_exists) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 5d4300bffd3..86a8bd5d5b3 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -83,12 +83,18 @@ public: void listPrefix(const std::string & path, RelativePathsWithSize & children) const override; /// Remove file. Throws exception if file doesn't exist or it's a directory. + /// Uses `DeleteObjectRequest`. void removeObject(const std::string & path) override; + /// Uses `DeleteObjectsRequest` if it is allowed by `s3_capabilities`, otherwise `DeleteObjectRequest`. + /// `DeleteObjectsRequest` is not supported on GCS, see https://issuetracker.google.com/issues/162653700 . void removeObjects(const PathsWithSize & paths) override; + /// Uses `DeleteObjectRequest`. void removeObjectIfExists(const std::string & path) override; + /// Uses `DeleteObjectsRequest` if it is allowed by `s3_capabilities`, otherwise `DeleteObjectRequest`. + /// `DeleteObjectsRequest` does not exist on GCS, see https://issuetracker.google.com/issues/162653700 . void removeObjectsIfExist(const PathsWithSize & paths) override; ObjectMetadata getObjectMetadata(const std::string & path) const override; @@ -113,6 +119,8 @@ public: const std::string & config_prefix, ContextPtr context) override; + void setCapabilitiesSupportBatchDelete(bool value) { s3_capabilities.support_batch_delete = value; } + String getObjectsNamespace() const override { return bucket; } std::unique_ptr cloneObjectStorage( @@ -151,7 +159,7 @@ private: MultiVersion client; MultiVersion s3_settings; - const S3Capabilities s3_capabilities; + S3Capabilities s3_capabilities; const String version_id; }; diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index b078ec98551..3597cde5d76 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -11,12 +11,14 @@ #include #include + #include +#include + #include #include #include - #include #include #include @@ -26,10 +28,9 @@ #include #include -#include - #include + namespace DB { @@ -65,7 +66,35 @@ void checkReadAccess(const String & disk_name, IDisk & disk) throw Exception("No read access to S3 bucket in disk " + disk_name, ErrorCodes::PATH_ACCESS_DENIED); } -void checkRemoveAccess(IDisk & disk) { disk.removeFile("test_acl"); } +void checkRemoveAccess(IDisk & disk) +{ + disk.removeFile("test_acl"); +} + +bool checkBatchRemoveIsMissing(S3ObjectStorage & storage, const String & key) +{ + String path = key + "/test_acl"; + auto file = storage.writeObject(path, WriteMode::Rewrite); + try + { + file->write("test", 4); + } + catch (...) + { + file->finalize(); + return false; /// We don't have write access, therefore no information about batch remove. + } + try + { + /// Uses `DeleteObjects` request (batch delete). + storage.removeObjects({{ path, 0 }}); + return false; + } + catch (const Exception &) + { + return true; + } +} } @@ -91,11 +120,27 @@ void registerDiskS3(DiskFactory & factory) FileCachePtr cache = getCachePtrForDisk(name, config, config_prefix, context); S3Capabilities s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); - ObjectStoragePtr s3_storage = std::make_unique( + auto s3_storage = std::make_unique( std::move(cache), getClient(config, config_prefix, context), getSettings(config, config_prefix, context), uri.version_id, s3_capabilities, uri.bucket); + if (!config.getBool(config_prefix + ".skip_access_check", false)) + { + /// If `support_batch_delete` is turned on (default), check and possibly switch it off. + if (s3_capabilities.support_batch_delete && checkBatchRemoveIsMissing(*s3_storage, uri.key)) + { + LOG_WARNING( + &Poco::Logger::get("registerDiskS3"), + "Storage for disk {} does not support batch delete operations, " + "so `s3_capabilities.support_batch_delete` was automatically turned off during the access check. " + "To remove this message set `s3_capabilities.support_batch_delete` for the disk to `false`.", + name + ); + s3_storage->setCapabilitiesSupportBatchDelete(false); + } + } + bool send_metadata = config.getBool(config_prefix + ".send_metadata", false); uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16); From 17186730b6541a1c5c769655010fe408c8b007c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 06:45:39 +0200 Subject: [PATCH 339/627] Add HTML page --- benchmark/index.html | 155 +++++++++++++++++++++++++++++++++---------- 1 file changed, 119 insertions(+), 36 deletions(-) diff --git a/benchmark/index.html b/benchmark/index.html index 7718d903589..29669a35a11 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -22,6 +22,10 @@ padding: 1% 3% 0 3%; } + table { + border-spacing: 1px; + } + .stick-left { position: sticky; left: 0px; @@ -126,6 +130,7 @@ .shadow:hover { box-shadow: 0 0 1rem gray; + position: relative; } #legend { @@ -136,6 +141,13 @@ background: #FED; border: 1px solid #FFCB80; } + + #nothing-selected { + display: none; + font-size: 32pt; + font-weight: bold; + color: #CCC; + } From c71f637c79d71e7bdc7dfe5fdb327f2cbc0d85fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 09:29:26 +0200 Subject: [PATCH 345/627] Adjustments --- benchmark/aurora-mysql/results/16acu.json | 2 +- .../aurora-postgresql/results/16acu.json | 2 +- benchmark/citus/results/c6a.4xlarge.json | 2 +- .../results/c6a.4xlarge.partitioned.json | 2 +- benchmark/clickhouse/results/c6a.4xlarge.json | 2 +- benchmark/clickhouse/results/c6a.metal.json | 2 +- benchmark/cratedb/results/c6a.4xlarge.json | 2 +- benchmark/databend/results/c6a.4xlarge.json | 2 +- benchmark/druid/results/c6a.4xlarge.json | 2 +- benchmark/duckdb/results/c6a.4xlarge.json | 2 +- benchmark/greenplum/results/c6a.4xlarge.json | 2 +- benchmark/heavyai/results/c6a.4xlarge.json | 2 +- benchmark/index.html | 61 ++++++++++--------- benchmark/infobright/results/c6a.4xlarge.json | 2 +- .../results/c6a.4xlarge.json | 2 +- benchmark/monetdb/results/c6a.4xlarge.json | 2 +- .../mysql-myisam/results/c6a.4xlarge.json | 2 +- benchmark/mysql/results/c6a.4xlarge.json | 2 +- benchmark/postgresql/results/c6a.4xlarge.json | 2 +- benchmark/questdb/results/c6a.4xlarge.json | 2 +- benchmark/redshift/results/4x.ra3.xplus.json | 2 +- benchmark/snowflake/results/2xl.json | 2 +- benchmark/snowflake/results/3xl.json | 2 +- benchmark/snowflake/results/4xl.json | 2 +- benchmark/snowflake/results/l.json | 2 +- benchmark/snowflake/results/m.json | 2 +- benchmark/snowflake/results/s.json | 2 +- benchmark/snowflake/results/xl.json | 2 +- benchmark/snowflake/results/xs.json | 2 +- 29 files changed, 59 insertions(+), 58 deletions(-) diff --git a/benchmark/aurora-mysql/results/16acu.json b/benchmark/aurora-mysql/results/16acu.json index 783a534b674..12abda3b307 100644 --- a/benchmark/aurora-mysql/results/16acu.json +++ b/benchmark/aurora-mysql/results/16acu.json @@ -2,7 +2,7 @@ "system": "Aurora for MySQL", "date": "2022-07-01", "machine": "16acu", - "cluster_size": "1", + "cluster_size": 1, "comment": "Some queries cannot run due to ERROR 1114 (HY000) at line 1: The table '/rdsdbdata/tmp/#sqlaff_e5_0' is full", "tags": ["managed", "SQL", "C++", "MySQL compatible", "row-oriented"], diff --git a/benchmark/aurora-postgresql/results/16acu.json b/benchmark/aurora-postgresql/results/16acu.json index 37af4b7ad46..a18f4c54271 100644 --- a/benchmark/aurora-postgresql/results/16acu.json +++ b/benchmark/aurora-postgresql/results/16acu.json @@ -2,7 +2,7 @@ "system": "Aurora for PostgreSQL", "date": "2022-07-01", "machine": "16acu", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["managed", "SQL", "C", "PostgreSQL compatible", "row-oriented"], diff --git a/benchmark/citus/results/c6a.4xlarge.json b/benchmark/citus/results/c6a.4xlarge.json index 0a0c0a5009d..8364e70cbc8 100644 --- a/benchmark/citus/results/c6a.4xlarge.json +++ b/benchmark/citus/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "BigQuery", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C", "PostgreSQL compatible", "column-oriented"], diff --git a/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json b/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json index 52489a87d1b..243c20aee20 100644 --- a/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json +++ b/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json @@ -2,7 +2,7 @@ "system": "clickhouse-local (partitioned)", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless"], diff --git a/benchmark/clickhouse/results/c6a.4xlarge.json b/benchmark/clickhouse/results/c6a.4xlarge.json index 5a74407e774..cc6617717bb 100644 --- a/benchmark/clickhouse/results/c6a.4xlarge.json +++ b/benchmark/clickhouse/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "ClickHouse", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "column-oriented"], diff --git a/benchmark/clickhouse/results/c6a.metal.json b/benchmark/clickhouse/results/c6a.metal.json index 6b580ee5b69..9e01abc5b26 100644 --- a/benchmark/clickhouse/results/c6a.metal.json +++ b/benchmark/clickhouse/results/c6a.metal.json @@ -2,7 +2,7 @@ "system": "ClickHouse", "date": "2022-07-01", "machine": "c6a.metal, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "column-oriented"], diff --git a/benchmark/cratedb/results/c6a.4xlarge.json b/benchmark/cratedb/results/c6a.4xlarge.json index 0944d6cc14c..b558c24ebe7 100644 --- a/benchmark/cratedb/results/c6a.4xlarge.json +++ b/benchmark/cratedb/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "CrateDB", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "For some queries it gives \"Data too large\".", "tags": ["SQL", "Java", "column-oriented"], diff --git a/benchmark/databend/results/c6a.4xlarge.json b/benchmark/databend/results/c6a.4xlarge.json index 03549225e11..a820a90803b 100644 --- a/benchmark/databend/results/c6a.4xlarge.json +++ b/benchmark/databend/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "Databend", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Only 90% of data successfully loaded. For some queries it gives \"Data too large\".", "tags": ["SQL", "Rust", "column-oriented", "ClickHouse derivative"], diff --git a/benchmark/druid/results/c6a.4xlarge.json b/benchmark/druid/results/c6a.4xlarge.json index 679e1cef348..8adaa163b26 100644 --- a/benchmark/druid/results/c6a.4xlarge.json +++ b/benchmark/druid/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "Druid", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "Java", "column-oriented"], diff --git a/benchmark/duckdb/results/c6a.4xlarge.json b/benchmark/duckdb/results/c6a.4xlarge.json index 96e2e29640b..a59ca62863e 100644 --- a/benchmark/duckdb/results/c6a.4xlarge.json +++ b/benchmark/duckdb/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "DuckDB", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Many queries triggered OOM", "tags": ["SQL", "C", "column-oriented", "embedded"], diff --git a/benchmark/greenplum/results/c6a.4xlarge.json b/benchmark/greenplum/results/c6a.4xlarge.json index 3163acef9c1..1b6ddb22ec9 100644 --- a/benchmark/greenplum/results/c6a.4xlarge.json +++ b/benchmark/greenplum/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "Greenplum", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C", "column-oriented", "PostgreSQL compatible"], diff --git a/benchmark/heavyai/results/c6a.4xlarge.json b/benchmark/heavyai/results/c6a.4xlarge.json index 3915df803c6..5c84f71fc24 100644 --- a/benchmark/heavyai/results/c6a.4xlarge.json +++ b/benchmark/heavyai/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "HeavyAI", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Previous names: OmniSci, mapD. Many queries cannot run due to errors and limitations.", "tags": ["SQL", "C++", "column-oriented"], diff --git a/benchmark/index.html b/benchmark/index.html index 7cb8f42d17b..9966039dbbc 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -320,7 +320,7 @@ const data = [ "system": "Aurora for MySQL", "date": "2022-07-01", "machine": "16acu", - "cluster_size": "1", + "cluster_size": 1, "comment": "Some queries cannot run due to ERROR 1114 (HY000) at line 1: The table '/rdsdbdata/tmp/#sqlaff_e5_0' is full", "tags": ["managed", "SQL", "C++", "MySQL compatible", "row-oriented"], @@ -379,7 +379,7 @@ const data = [ "system": "Aurora for PostgreSQL", "date": "2022-07-01", "machine": "16acu", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["managed", "SQL", "C", "PostgreSQL compatible", "row-oriented"], @@ -497,7 +497,7 @@ const data = [ "system": "BigQuery", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C", "PostgreSQL compatible", "column-oriented"], @@ -556,7 +556,7 @@ const data = [ "system": "clickhouse-local (partitioned)", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless"], @@ -615,7 +615,7 @@ const data = [ "system": "clickhouse-local (partitioned)", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless"], @@ -674,7 +674,7 @@ const data = [ "system": "ClickHouse", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "column-oriented"], @@ -733,7 +733,7 @@ const data = [ "system": "ClickHouse", "date": "2022-07-01", "machine": "c6a.metal, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "column-oriented"], @@ -792,7 +792,7 @@ const data = [ "system": "CrateDB", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "For some queries it gives \"Data too large\".", "tags": ["SQL", "Java", "column-oriented"], @@ -851,7 +851,7 @@ const data = [ "system": "Databend", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Only 90% of data successfully loaded. For some queries it gives \"Data too large\".", "tags": ["SQL", "Rust", "column-oriented", "ClickHouse derivative"], @@ -910,7 +910,7 @@ const data = [ "system": "Druid", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "Java", "column-oriented"], @@ -969,7 +969,7 @@ const data = [ "system": "DuckDB", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Many queries triggered OOM", "tags": ["SQL", "C", "column-oriented", "embedded"], @@ -1028,7 +1028,7 @@ const data = [ "system": "Greenplum", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C", "column-oriented", "PostgreSQL compatible"], @@ -1087,7 +1087,7 @@ const data = [ "system": "HeavyAI", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Previous names: OmniSci, mapD. Many queries cannot run due to errors and limitations.", "tags": ["SQL", "C++", "column-oriented"], @@ -1146,7 +1146,7 @@ const data = [ "system": "Infobright", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Only 90% of data successfully loaded. Some queries run for days.", "tags": ["SQL", "C++", "column-oriented", "MySQL compatible"], @@ -1205,7 +1205,7 @@ const data = [ "system": "MariaDB ColumnStore", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Previous name: InfiniDB.", "tags": ["SQL", "C++", "column-oriented", "MySQL compatible"], @@ -1264,7 +1264,7 @@ const data = [ "system": "MonetDB", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C", "column-oriented"], @@ -1323,7 +1323,7 @@ const data = [ "system": "MySQL (MyISAM)", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "row-oriented", "MySQL compatible"], @@ -1382,7 +1382,7 @@ const data = [ "system": "MySQL", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "row-oriented", "MySQL compatible"], @@ -1441,7 +1441,7 @@ const data = [ "system": "PostgreSQL", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C", "row-oriented", "PostgreSQL compatible"], @@ -1500,7 +1500,7 @@ const data = [ "system": "QuestDB", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Many queries cannot run. It also crashes and hangs.", "tags": ["SQL", "Java", "time-series"], @@ -1559,7 +1559,7 @@ const data = [ "system": "Redshift", "date": "2022-07-01", "machine": "ra3.xplus", - "cluster_size": "4", + "cluster_size": 4, "comment": "One query did not run due to overflow check and another due to missing regexp function.", "tags": ["SQL", "managed", "column-oriented"], @@ -1736,7 +1736,7 @@ const data = [ "system": "Snowflake", "date": "2022-07-01", "machine": "2XL", - "cluster_size": "32", + "cluster_size": 32, "comment": "", "tags": ["SQL", "managed", "column-oriented"], @@ -1795,7 +1795,7 @@ const data = [ "system": "Snowflake", "date": "2022-07-01", "machine": "3XL", - "cluster_size": "64", + "cluster_size": 64, "comment": "", "tags": ["SQL", "managed", "column-oriented"], @@ -1854,7 +1854,7 @@ const data = [ "system": "Snowflake", "date": "2022-07-01", "machine": "4XL", - "cluster_size": "128", + "cluster_size": 128, "comment": "", "tags": ["SQL", "managed", "column-oriented"], @@ -1913,7 +1913,7 @@ const data = [ "system": "Snowflake", "date": "2022-07-01", "machine": "L", - "cluster_size": "8", + "cluster_size": 8, "comment": "", "tags": ["SQL", "managed", "column-oriented"], @@ -1972,7 +1972,7 @@ const data = [ "system": "Snowflake", "date": "2022-07-01", "machine": "M", - "cluster_size": "4", + "cluster_size": 4, "comment": "", "tags": ["SQL", "managed", "column-oriented"], @@ -2031,7 +2031,7 @@ const data = [ "system": "Snowflake", "date": "2022-07-01", "machine": "S", - "cluster_size": "2", + "cluster_size": 2, "comment": "", "tags": ["SQL", "managed", "column-oriented"], @@ -2090,7 +2090,7 @@ const data = [ "system": "Snowflake", "date": "2022-07-01", "machine": "XL", - "cluster_size": "16", + "cluster_size": 16, "comment": "", "tags": ["SQL", "managed", "column-oriented"], @@ -2149,7 +2149,7 @@ const data = [ "system": "Snowflake", "date": "2022-07-01", "machine": "XS", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "managed", "column-oriented"], @@ -2583,7 +2583,8 @@ unique_systems.map(elem => { selector.addEventListener('click', e => toggle(e, elem, selectors.machine)); }); -[... new Set(data.map(elem => elem.cluster_size ?? 1))].map(elem => { +[... new Set(data.map(elem => elem.cluster_size))].sort( + (a, b) => ((typeof(b) === 'number') - (typeof(a) === 'number')) || (a - b)).map(elem => { let selector = document.createElement('a'); selector.className = 'selector selector-active'; selector.appendChild(document.createTextNode(elem)); diff --git a/benchmark/infobright/results/c6a.4xlarge.json b/benchmark/infobright/results/c6a.4xlarge.json index 68c998c8043..11fbac77e56 100644 --- a/benchmark/infobright/results/c6a.4xlarge.json +++ b/benchmark/infobright/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "Infobright", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Only 90% of data successfully loaded. Some queries run for days.", "tags": ["SQL", "C++", "column-oriented", "MySQL compatible"], diff --git a/benchmark/mariadb-columnstore/results/c6a.4xlarge.json b/benchmark/mariadb-columnstore/results/c6a.4xlarge.json index 5dab5e5b6c0..7650be5d91d 100644 --- a/benchmark/mariadb-columnstore/results/c6a.4xlarge.json +++ b/benchmark/mariadb-columnstore/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "MariaDB ColumnStore", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Previous name: InfiniDB.", "tags": ["SQL", "C++", "column-oriented", "MySQL compatible"], diff --git a/benchmark/monetdb/results/c6a.4xlarge.json b/benchmark/monetdb/results/c6a.4xlarge.json index 7e7cc6c7bf5..d1e7c592303 100644 --- a/benchmark/monetdb/results/c6a.4xlarge.json +++ b/benchmark/monetdb/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "MonetDB", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C", "column-oriented"], diff --git a/benchmark/mysql-myisam/results/c6a.4xlarge.json b/benchmark/mysql-myisam/results/c6a.4xlarge.json index 5f7cce097d6..f937f393614 100644 --- a/benchmark/mysql-myisam/results/c6a.4xlarge.json +++ b/benchmark/mysql-myisam/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "MySQL (MyISAM)", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "row-oriented", "MySQL compatible"], diff --git a/benchmark/mysql/results/c6a.4xlarge.json b/benchmark/mysql/results/c6a.4xlarge.json index 4415d02fd75..36caf0fb646 100644 --- a/benchmark/mysql/results/c6a.4xlarge.json +++ b/benchmark/mysql/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "MySQL", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "row-oriented", "MySQL compatible"], diff --git a/benchmark/postgresql/results/c6a.4xlarge.json b/benchmark/postgresql/results/c6a.4xlarge.json index 76c1f84ce3e..7e1d3baef6e 100644 --- a/benchmark/postgresql/results/c6a.4xlarge.json +++ b/benchmark/postgresql/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "PostgreSQL", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C", "row-oriented", "PostgreSQL compatible"], diff --git a/benchmark/questdb/results/c6a.4xlarge.json b/benchmark/questdb/results/c6a.4xlarge.json index 5309216b39a..e8a00b03e5c 100644 --- a/benchmark/questdb/results/c6a.4xlarge.json +++ b/benchmark/questdb/results/c6a.4xlarge.json @@ -2,7 +2,7 @@ "system": "QuestDB", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "Many queries cannot run. It also crashes and hangs.", "tags": ["SQL", "Java", "time-series"], diff --git a/benchmark/redshift/results/4x.ra3.xplus.json b/benchmark/redshift/results/4x.ra3.xplus.json index 65e90ec6afa..c83558f210b 100644 --- a/benchmark/redshift/results/4x.ra3.xplus.json +++ b/benchmark/redshift/results/4x.ra3.xplus.json @@ -2,7 +2,7 @@ "system": "Redshift", "date": "2022-07-01", "machine": "ra3.xplus", - "cluster_size": "4", + "cluster_size": 4, "comment": "One query did not run due to overflow check and another due to missing regexp function.", "tags": ["SQL", "managed", "column-oriented"], diff --git a/benchmark/snowflake/results/2xl.json b/benchmark/snowflake/results/2xl.json index aae5f167b0f..4eca3044467 100644 --- a/benchmark/snowflake/results/2xl.json +++ b/benchmark/snowflake/results/2xl.json @@ -2,7 +2,7 @@ "system": "Snowflake", "date": "2022-07-01", "machine": "2XL", - "cluster_size": "32", + "cluster_size": 32, "comment": "", "tags": ["SQL", "managed", "column-oriented"], diff --git a/benchmark/snowflake/results/3xl.json b/benchmark/snowflake/results/3xl.json index 1541189ec65..70b58170934 100644 --- a/benchmark/snowflake/results/3xl.json +++ b/benchmark/snowflake/results/3xl.json @@ -2,7 +2,7 @@ "system": "Snowflake", "date": "2022-07-01", "machine": "3XL", - "cluster_size": "64", + "cluster_size": 64, "comment": "", "tags": ["SQL", "managed", "column-oriented"], diff --git a/benchmark/snowflake/results/4xl.json b/benchmark/snowflake/results/4xl.json index c9af42c58c4..7c57c0616af 100644 --- a/benchmark/snowflake/results/4xl.json +++ b/benchmark/snowflake/results/4xl.json @@ -2,7 +2,7 @@ "system": "Snowflake", "date": "2022-07-01", "machine": "4XL", - "cluster_size": "128", + "cluster_size": 128, "comment": "", "tags": ["SQL", "managed", "column-oriented"], diff --git a/benchmark/snowflake/results/l.json b/benchmark/snowflake/results/l.json index ff9dc720d5f..b61b7108d99 100644 --- a/benchmark/snowflake/results/l.json +++ b/benchmark/snowflake/results/l.json @@ -2,7 +2,7 @@ "system": "Snowflake", "date": "2022-07-01", "machine": "L", - "cluster_size": "8", + "cluster_size": 8, "comment": "", "tags": ["SQL", "managed", "column-oriented"], diff --git a/benchmark/snowflake/results/m.json b/benchmark/snowflake/results/m.json index 3c93a4a6da7..a126e975d62 100644 --- a/benchmark/snowflake/results/m.json +++ b/benchmark/snowflake/results/m.json @@ -2,7 +2,7 @@ "system": "Snowflake", "date": "2022-07-01", "machine": "M", - "cluster_size": "4", + "cluster_size": 4, "comment": "", "tags": ["SQL", "managed", "column-oriented"], diff --git a/benchmark/snowflake/results/s.json b/benchmark/snowflake/results/s.json index 16382d5ae12..58296761a6a 100644 --- a/benchmark/snowflake/results/s.json +++ b/benchmark/snowflake/results/s.json @@ -2,7 +2,7 @@ "system": "Snowflake", "date": "2022-07-01", "machine": "S", - "cluster_size": "2", + "cluster_size": 2, "comment": "", "tags": ["SQL", "managed", "column-oriented"], diff --git a/benchmark/snowflake/results/xl.json b/benchmark/snowflake/results/xl.json index 0f78ed1b404..6bc08cae044 100644 --- a/benchmark/snowflake/results/xl.json +++ b/benchmark/snowflake/results/xl.json @@ -2,7 +2,7 @@ "system": "Snowflake", "date": "2022-07-01", "machine": "XL", - "cluster_size": "16", + "cluster_size": 16, "comment": "", "tags": ["SQL", "managed", "column-oriented"], diff --git a/benchmark/snowflake/results/xs.json b/benchmark/snowflake/results/xs.json index bac069a6206..36e0e78aa91 100644 --- a/benchmark/snowflake/results/xs.json +++ b/benchmark/snowflake/results/xs.json @@ -2,7 +2,7 @@ "system": "Snowflake", "date": "2022-07-01", "machine": "XS", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "managed", "column-oriented"], From bb564b3ecb5bd0132c201080826119cc957dc450 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 09:38:11 +0200 Subject: [PATCH 346/627] Fix error --- benchmark/citus/results/c6a.4xlarge.json | 2 +- benchmark/index.html | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/benchmark/citus/results/c6a.4xlarge.json b/benchmark/citus/results/c6a.4xlarge.json index 8364e70cbc8..fab86db8ca6 100644 --- a/benchmark/citus/results/c6a.4xlarge.json +++ b/benchmark/citus/results/c6a.4xlarge.json @@ -1,5 +1,5 @@ { - "system": "BigQuery", + "system": "Citus", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", "cluster_size": 1, diff --git a/benchmark/index.html b/benchmark/index.html index 9966039dbbc..bdeb5bd13a9 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -494,7 +494,7 @@ const data = [ } , { - "system": "BigQuery", + "system": "Citus", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", "cluster_size": 1, @@ -2706,9 +2706,9 @@ function render() { let filtered_data = data.filter(elem => selectors.system[elem.system] && - selectors.machine[elem.machine] /*&& + selectors.machine[elem.machine] && selectors.cluster_size[elem.cluster_size] && - elem.types.filter(type => selectors.type[type]).length > 0*/); + elem.tags.filter(type => selectors.type[type]).length > 0); let nothing_selected_elem = document.getElementById('nothing-selected'); if (filtered_data.length == 0) { From 17a9257106593e3bb034292cc31003ff285471af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 09:44:17 +0200 Subject: [PATCH 347/627] Fix error --- .../results/c6a.4xlarge.partitioned.json | 2 +- .../clickhouse-local/results/c6a.4xlarge.single.json | 4 ++-- benchmark/clickhouse/results/c6a.4xlarge.json | 2 +- benchmark/clickhouse/results/c6a.metal.json | 2 +- benchmark/index.html | 10 +++++----- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json b/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json index 243c20aee20..34c5687d902 100644 --- a/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json +++ b/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless"], + "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], "load_time": 0, "data_size": 14737666736, diff --git a/benchmark/clickhouse-local/results/c6a.4xlarge.single.json b/benchmark/clickhouse-local/results/c6a.4xlarge.single.json index cfbb5886f99..f2805b2c2e5 100644 --- a/benchmark/clickhouse-local/results/c6a.4xlarge.single.json +++ b/benchmark/clickhouse-local/results/c6a.4xlarge.single.json @@ -1,11 +1,11 @@ { - "system": "clickhouse-local (partitioned)", + "system": "clickhouse-local (single)", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", "cluster_size": "1", "comment": "", - "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless"], + "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], "load_time": 0, "data_size": 14779976446, diff --git a/benchmark/clickhouse/results/c6a.4xlarge.json b/benchmark/clickhouse/results/c6a.4xlarge.json index cc6617717bb..df5693a2367 100644 --- a/benchmark/clickhouse/results/c6a.4xlarge.json +++ b/benchmark/clickhouse/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented"], + "tags": ["SQL", "C++", "column-oriented", "ClickHouse derivative"], "load_time": 475.529, "data_size": 14345515782, diff --git a/benchmark/clickhouse/results/c6a.metal.json b/benchmark/clickhouse/results/c6a.metal.json index 9e01abc5b26..27738fcae00 100644 --- a/benchmark/clickhouse/results/c6a.metal.json +++ b/benchmark/clickhouse/results/c6a.metal.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented"], + "tags": ["SQL", "C++", "column-oriented", "ClickHouse derivative"], "load_time": 136.869, "data_size": 14571706777, diff --git a/benchmark/index.html b/benchmark/index.html index bdeb5bd13a9..9e218167431 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -559,7 +559,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless"], + "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], "load_time": 0, "data_size": 14737666736, @@ -615,10 +615,10 @@ const data = [ "system": "clickhouse-local (partitioned)", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, + "cluster_size": "1", "comment": "", - "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless"], + "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], "load_time": 0, "data_size": 14779976446, @@ -677,7 +677,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented"], + "tags": ["SQL", "C++", "column-oriented", "ClickHouse derivative"], "load_time": 475.529, "data_size": 14345515782, @@ -736,7 +736,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented"], + "tags": ["SQL", "C++", "column-oriented", "ClickHouse derivative"], "load_time": 136.869, "data_size": 14571706777, From 1f46f48d7df7e37398bbf67cc993c3094ff05a0a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 7 Jul 2022 07:57:05 +0000 Subject: [PATCH 348/627] Fix: remove heeavy performance tests, introduced within this PR --- tests/performance/order_by_tuple.xml | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/performance/order_by_tuple.xml b/tests/performance/order_by_tuple.xml index bb6c4b205e3..74588e96869 100644 --- a/tests/performance/order_by_tuple.xml +++ b/tests/performance/order_by_tuple.xml @@ -1,5 +1,3 @@ select * from numbers(300000000) order by (1 - number , number + 1 , number) limit 10; - SELECT * FROM hits_100m_single ORDER BY (CounterID, EventDate) format Null - SELECT * FROM hits_100m_single ORDER BY (CounterID, EventDate, URL) format Null From cacc833da69318dd84554ab473c6a2b1e99f28f4 Mon Sep 17 00:00:00 2001 From: Vladimir Galunshchikov Date: Thu, 7 Jul 2022 11:15:59 +0300 Subject: [PATCH 349/627] Rename NUMBER_OF_DIMENSIONS_MISMATHED const to NUMBER_OF_DIMENSIONS_MISMATCHED --- src/Columns/ColumnObject.cpp | 4 ++-- src/Common/ErrorCodes.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index b52a5aab256..e76862ad988 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int ILLEGAL_COLUMN; extern const int DUPLICATE_COLUMN; - extern const int NUMBER_OF_DIMENSIONS_MISMATHED; + extern const int NUMBER_OF_DIMENSIONS_MISMATCHED; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int ARGUMENT_OUT_OF_BOUND; } @@ -298,7 +298,7 @@ void ColumnObject::Subcolumn::insert(Field field, FieldInfo info) value_dim = column_dim; if (value_dim != column_dim) - throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATHED, + throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATCHED, "Dimension of types mismatched between inserted value and column. " "Dimension of value: {}. Dimension of column: {}", value_dim, column_dim); diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 206f2061cde..904fee99214 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -613,7 +613,7 @@ M(642, CANNOT_PACK_ARCHIVE) \ M(643, CANNOT_UNPACK_ARCHIVE) \ M(644, REMOTE_FS_OBJECT_CACHE_ERROR) \ - M(645, NUMBER_OF_DIMENSIONS_MISMATHED) \ + M(645, NUMBER_OF_DIMENSIONS_MISMATCHED) \ M(646, CANNOT_BACKUP_DATABASE) \ M(647, CANNOT_BACKUP_TABLE) \ M(648, WRONG_DDL_RENAMING_SETTINGS) \ From 1869b7f4086e5201b3c0a3a4a033a1c755b17bd4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 6 Jul 2022 23:58:04 +0000 Subject: [PATCH 350/627] Add functions translate & translateUTF8 --- .../registerFunctionsStringRegexp.cpp | 2 + src/Functions/translate.cpp | 324 ++++++++++++++++++ .../0_stateless/02353_translate.reference | 14 + tests/queries/0_stateless/02353_translate.sql | 7 + 4 files changed, 347 insertions(+) create mode 100644 src/Functions/translate.cpp create mode 100644 tests/queries/0_stateless/02353_translate.reference create mode 100644 tests/queries/0_stateless/02353_translate.sql diff --git a/src/Functions/registerFunctionsStringRegexp.cpp b/src/Functions/registerFunctionsStringRegexp.cpp index 61853b19d11..df7e8f58396 100644 --- a/src/Functions/registerFunctionsStringRegexp.cpp +++ b/src/Functions/registerFunctionsStringRegexp.cpp @@ -9,6 +9,7 @@ void registerFunctionNotLike(FunctionFactory &); void registerFunctionNotILike(FunctionFactory &); void registerFunctionMatch(FunctionFactory &); void registerFunctionExtract(FunctionFactory &); +void registerFunctionTranslate(FunctionFactory &); void registerFunctionReplaceOne(FunctionFactory &); void registerFunctionReplaceAll(FunctionFactory &); void registerFunctionReplaceRegexpOne(FunctionFactory &); @@ -31,6 +32,7 @@ void registerFunctionsStringRegexp(FunctionFactory & factory) registerFunctionNotILike(factory); registerFunctionMatch(factory); registerFunctionExtract(factory); + registerFunctionTranslate(factory); registerFunctionReplaceOne(factory); registerFunctionReplaceAll(factory); registerFunctionReplaceRegexpOne(factory); diff --git a/src/Functions/translate.cpp b/src/Functions/translate.cpp new file mode 100644 index 00000000000..e509ea96a05 --- /dev/null +++ b/src/Functions/translate.cpp @@ -0,0 +1,324 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int BAD_ARGUMENTS; +} + +struct TranslateImpl +{ + static void fillMapWithValues( + std::array & map, + const std::string & map_from, + const std::string & map_to) + { + if (map_from.size() != map_to.size()) + throw Exception("Second and trird arguments must be the same size", ErrorCodes::BAD_ARGUMENTS); + + for (size_t i = 0; i < 256; ++i) + map[i] = i; + + for (size_t i = 0; i < map_from.size(); ++i) + { + if (!isASCII(map_from[i]) || !isASCII(map_to[i])) + throw Exception("Second and trird arguments must be ASCII strings", ErrorCodes::BAD_ARGUMENTS); + + map[map_from[i]] = map_to[i]; + } + } + + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + const std::string & map_from, + const std::string & map_to, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + std::array map; + fillMapWithValues(map, map_from, map_to); + + res_data.resize(data.size()); + res_offsets.assign(offsets); + + UInt8 * dst = res_data.data(); + + for (UInt64 i = 0; i < offsets.size(); ++i) + { + const UInt8 * src = data.data() + offsets[i - 1]; + const UInt8 * src_end = data.data() + offsets[i] - 1; + + while (src < src_end) + { + *dst = map[*src]; + + ++src; + ++dst; + } + + /// Technically '\0' can be mapped into other character, + /// so we need to process '\0' delimiter separately + *dst++ = 0; + } + } + + static void vectorFixed( + const ColumnString::Chars & data, + size_t /*n*/, + const std::string & map_from, + const std::string & map_to, + ColumnString::Chars & res_data) + { + std::array map; + fillMapWithValues(map, map_from, map_to); + + res_data.resize(data.size()); + + const UInt8 * src = data.data(); + const UInt8 * src_end = data.data() + data.size(); + UInt8 * dst = res_data.data(); + + while (src < src_end) + { + *dst = map[*src]; + + ++src; + ++dst; + } + } +}; + +struct TranslateUTF8Impl +{ + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + const std::string & map_from, + const std::string & map_to, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + auto map_from_size = UTF8::countCodePoints(reinterpret_cast(map_from.data()), map_from.size()); + auto map_to_size = UTF8::countCodePoints(reinterpret_cast(map_to.data()), map_to.size()); + + if (map_from_size != map_to_size) + throw Exception("Second and trird arguments must be the same size", ErrorCodes::BAD_ARGUMENTS); + + HashMap> map(map_from_size); + + const UInt8 * map_from_ptr = reinterpret_cast(map_from.data()); + const UInt8 * map_from_end = map_from_ptr + map_from.size(); + const UInt8 * map_to_ptr = reinterpret_cast(map_to.data()); + + while (map_from_ptr < map_from_end) + { + size_t len_a = UTF8::seqLength(*map_from_ptr); + auto res_a = UTF8::convertUTF8ToCodePoint(map_from_ptr, len_a); + + size_t len_b = UTF8::seqLength(*map_to_ptr); + auto res_b = UTF8::convertUTF8ToCodePoint(map_to_ptr, len_b); + + if (!res_a) + throw Exception("Second argument must be a valid UTF-8 string", ErrorCodes::BAD_ARGUMENTS); + + if (!res_b) + throw Exception("Third argument must be a valid UTF-8 string", ErrorCodes::BAD_ARGUMENTS); + + map[*res_a] = *res_b; + + map_from_ptr += len_a; + map_to_ptr += len_b; + } + + res_data.resize(data.size()); + res_offsets.resize(offsets.size()); + + UInt8 * dst = res_data.data(); + UInt64 data_size = 0; + + for (UInt64 i = 0; i < offsets.size(); ++i) + { + const UInt8 * src = data.data() + offsets[i - 1]; + const UInt8 * src_end = data.data() + offsets[i] - 1; + + while (src < src_end) + { + /// Maximum length of UTF-8 sequence is 4 byte + 1 zero byte + if (data_size + 5 > res_data.size()) + { + res_data.resize(data_size * 2 + 5); + dst = res_data.data() + data_size; + } + + size_t src_len = UTF8::seqLength(*src); + assert(0 < src_len && src_len <= 4); + + if (src + src_len <= src_end) + { + auto res = UTF8::convertUTF8ToCodePoint(src, src_len); + + if (res) + { + auto * it = map.find(*res); + if (it != map.end()) + { + size_t dst_len = UTF8::convertCodePointToUTF8(it->getMapped(), dst, 4); + assert(0 < dst_len && dst_len <= 4); + + src += src_len; + dst += dst_len; + data_size += dst_len; + continue; + } + } + } + else + { + src_len = src_end - src; + } + + memcpy(dst, src, src_len); + dst += src_len; + src += src_len; + data_size += src_len; + } + + /// Technically '\0' can be mapped into other character, + /// so we need to process '\0' delimiter separately + *dst++ = 0; + + ++data_size; + res_offsets[i] = data_size; + } + + res_data.resize(data_size); + } + + [[noreturn]] static void vectorFixed( + const ColumnString::Chars & /*data*/, + size_t /*n*/, + const std::string & /*map_from*/, + const std::string & /*map_to*/, + ColumnString::Chars & /*res_data*/) + { + throw Exception("Function translateUTF8 does not support FixedString argument", ErrorCodes::BAD_ARGUMENTS); + } + +private: + static constexpr auto ascii_upper_bound = '\x7f'; +}; + + +template +class FunctionTranslate : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 3; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isStringOrFixedString(arguments[1])) + throw Exception( + "Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isStringOrFixedString(arguments[2])) + throw Exception( + "Illegal type " + arguments[2]->getName() + " of third argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const ColumnPtr column_src = arguments[0].column; + const ColumnPtr column_map_from = arguments[1].column; + const ColumnPtr column_map_to = arguments[2].column; + + if (!isColumnConst(*column_map_from) || !isColumnConst(*column_map_to)) + throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants.", ErrorCodes::ILLEGAL_COLUMN); + + const IColumn * c1 = arguments[1].column.get(); + const IColumn * c2 = arguments[2].column.get(); + const ColumnConst * c1_const = typeid_cast(c1); + const ColumnConst * c2_const = typeid_cast(c2); + String map_from = c1_const->getValue(); + String map_to = c2_const->getValue(); + + if (map_from.empty()) + throw Exception("Length of the second argument of function " + getName() + " must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + if (const ColumnString * col = checkAndGetColumn(column_src.get())) + { + auto col_res = ColumnString::create(); + Impl::vector(col->getChars(), col->getOffsets(), map_from, map_to, col_res->getChars(), col_res->getOffsets()); + return col_res; + } + else if (const ColumnFixedString * col_fixed = checkAndGetColumn(column_src.get())) + { + auto col_res = ColumnFixedString::create(col_fixed->getN()); + Impl::vectorFixed(col_fixed->getChars(), col_fixed->getN(), map_from, map_to, col_res->getChars()); + return col_res; + } + else + throw Exception( + "Illegal column " + arguments[0].column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + + +namespace +{ + +struct NameTranslate +{ + static constexpr auto name = "translate"; +}; + +struct NameTranslateUTF8 +{ + static constexpr auto name = "translateUTF8"; +}; + +using FunctionTranslateASCII = FunctionTranslate; +using FunctionTranslateUTF8 = FunctionTranslate; + +} + +void registerFunctionTranslate(FunctionFactory & factory) +{ + factory.registerFunction(); + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02353_translate.reference b/tests/queries/0_stateless/02353_translate.reference new file mode 100644 index 00000000000..c4a06bb21c7 --- /dev/null +++ b/tests/queries/0_stateless/02353_translate.reference @@ -0,0 +1,14 @@ +Hello, world! +cagaacgttc +jihgfe +jihgff +jihgfg +jihgfh +jihgfi +HotelGenev +ードとは +¿йðՅন𐐏 +¿йðՅনন +¿йðՅনՅ +¿йðՅনð +¿йðՅনй diff --git a/tests/queries/0_stateless/02353_translate.sql b/tests/queries/0_stateless/02353_translate.sql new file mode 100644 index 00000000000..bc63cc71e69 --- /dev/null +++ b/tests/queries/0_stateless/02353_translate.sql @@ -0,0 +1,7 @@ +SELECT translate('Hello? world.', '.?', '!,'); +SELECT translate('gtcttgcaag', 'ACGTacgt', 'TGCAtgca'); +SELECT translate(toString(number), '0123456789', 'abcdefghij') FROM numbers(987654, 5); + +SELECT translateUTF8('HôtelGenèv', 'Ááéíóúôè', 'aaeiouoe'); +SELECT translateUTF8('中文内码', '久标准中文内码', 'ユニコードとは'); +SELECT translateUTF8(toString(number), '1234567890', 'ዩय𐑿𐐏নՅðй¿ค') FROM numbers(987654, 5); From fcb6cfb6dfa426f1c19afccc752def182d2c12b6 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 7 Jul 2022 08:31:55 +0000 Subject: [PATCH 351/627] Fix style issues --- docs/en/sql-reference/statements/select/index.md | 2 +- docs/en/sql-reference/statements/select/into-outfile.md | 2 +- docs/ru/sql-reference/statements/select/into-outfile.md | 2 +- src/Client/ClientBase.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 66ed8a71f94..e039548e50a 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -25,7 +25,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] [UNION ...] -[INTO OUTFILE filename [COMPRESSION type [LEVEL level] ] +[INTO OUTFILE filename [COMPRESSION type [LEVEL level]] ] [FORMAT format] ``` diff --git a/docs/en/sql-reference/statements/select/into-outfile.md b/docs/en/sql-reference/statements/select/into-outfile.md index 2e7ea8dceb1..f101c10ff60 100644 --- a/docs/en/sql-reference/statements/select/into-outfile.md +++ b/docs/en/sql-reference/statements/select/into-outfile.md @@ -6,7 +6,7 @@ sidebar_label: INTO OUTFILE `INTO OUTFILE` clause redirects the result of a `SELECT` query to a file on the **client** side. -Compressed files are supported. Compression type is detected by the extension of the file name (mode `'auto'` is used by default). Or it can be explicitly specified in a `COMPRESSION` clause. Compression level for a certain compression type can be specified in a `LEVEL` clause. +Compressed files are supported. Compression type is detected by the extension of the file name (mode `'auto'` is used by default). Or it can be explicitly specified in a `COMPRESSION` clause. The compression level for a certain compression type can be specified in a `LEVEL` clause. **Syntax** diff --git a/docs/ru/sql-reference/statements/select/into-outfile.md b/docs/ru/sql-reference/statements/select/into-outfile.md index 67344711977..81d48badbe6 100644 --- a/docs/ru/sql-reference/statements/select/into-outfile.md +++ b/docs/ru/sql-reference/statements/select/into-outfile.md @@ -6,7 +6,7 @@ sidebar_label: INTO OUTFILE Секция `INTO OUTFILE` перенаправляет результат запроса `SELECT` в файл на стороне **клиента**. -Поддерживаются сжатые файлы. Формат сжатия определяется по расширению файла (по умолчанию используется режим `'auto'`), либо он может быть задан явно в секции `COMPRESSION`. Уровень сжатия для конкретного алгоритма может быть зада в секции `LEVEL`. +Поддерживаются сжатые файлы. Формат сжатия определяется по расширению файла (по умолчанию используется режим `'auto'`), либо он может быть задан явно в секции `COMPRESSION`. Уровень сжатия для конкретного алгоритма может быть задан в секции `LEVEL`. **Синтаксис** diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 399975821bc..0e243f97aaf 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -539,8 +539,8 @@ try { const auto & compression_level_node = query_with_output->compression_level->as(); bool res = compression_level_node.value.tryGet(compression_level); - auto range = getCompressionLevelRange(compression_method); + if (!res || compression_level < range.first || compression_level > range.second) throw Exception( ErrorCodes::BAD_ARGUMENTS, From b48a626d028f81365c6d22752d7b301774c524f9 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 7 Jul 2022 08:45:13 +0000 Subject: [PATCH 352/627] Simplify test - complexity comes from dreaming about EXPLAIN queries as an ordinary ones returning scalar result. So they can be used in SQL tests instead of shell ones --- .../0_stateless/02353_order_by_tuple.reference | 4 +++- tests/queries/0_stateless/02353_order_by_tuple.sh | 12 ++++-------- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02353_order_by_tuple.reference b/tests/queries/0_stateless/02353_order_by_tuple.reference index d00491fd7e5..1637224ed18 100644 --- a/tests/queries/0_stateless/02353_order_by_tuple.reference +++ b/tests/queries/0_stateless/02353_order_by_tuple.reference @@ -1 +1,3 @@ -1 +SELECT number AS a, number % 2 AS b FROM numbers(10) ORDER BY a DESC NULLS FIRST WITH FILL FROM 2 TO 1 STEP -1, b DESC NULLS FIRST WITH FILL FROM 2 TO 1 STEP -1 +SELECT number AS a, number % 2 AS b FROM numbers(10) ORDER BY a DESC NULLS FIRST WITH FILL FROM 2 TO 1 STEP -1, b DESC NULLS FIRST WITH FILL FROM 2 TO 1 STEP -1 +OK diff --git a/tests/queries/0_stateless/02353_order_by_tuple.sh b/tests/queries/0_stateless/02353_order_by_tuple.sh index f7daee20cd1..78c7dfc9288 100755 --- a/tests/queries/0_stateless/02353_order_by_tuple.sh +++ b/tests/queries/0_stateless/02353_order_by_tuple.sh @@ -7,19 +7,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) [ ! -z "$CLICKHOUSE_CLIENT_REDEFINED" ] && CLICKHOUSE_CLIENT=$CLICKHOUSE_CLIENT_REDEFINED ################## -# checking that both queries have the same `EXPLAIN SYNTAX` output -# todo: `EXPLAIN SYTNAX` can be replaced by `EXPLAIN AST rewrite=1` when available +# check that both queries have the same AST after rewrite, EXPLAIN SYNTAX returns it in form of query ################## QUERY_ORDER_BY="SELECT number AS a, number % 2 AS b FROM numbers(10) ORDER BY a DESC NULLS FIRST WITH FILL FROM 2 TO 1 STEP -1, b DESC NULLS FIRST WITH FILL FROM 2 TO 1 STEP -1" QUERY_ORDER_BY_TUPLE="SELECT number AS a, number % 2 AS b FROM numbers(10) ORDER BY (a, b) DESC NULLS FIRST WITH FILL FROM 2 TO 1 STEP -1" EXPLAIN="EXPLAIN SYNTAX" OUTPUT_EXPLAIN_ORDER_BY=$($CLICKHOUSE_CLIENT -q "$EXPLAIN $QUERY_ORDER_BY") +echo $OUTPUT_EXPLAIN_ORDER_BY OUTPUT_EXPLAIN_ORDER_BY_TUPLE=$($CLICKHOUSE_CLIENT -q "$EXPLAIN $QUERY_ORDER_BY_TUPLE") +echo $OUTPUT_EXPLAIN_ORDER_BY_TUPLE -$CLICKHOUSE_CLIENT -q "drop table if exists order_by_syntax" -$CLICKHOUSE_CLIENT -q "create table order_by_syntax (explain String) engine=Memory" -$CLICKHOUSE_CLIENT -q "insert into order_by_syntax values('$OUTPUT_EXPLAIN_ORDER_BY')" -$CLICKHOUSE_CLIENT -q "insert into order_by_syntax values('$OUTPUT_EXPLAIN_ORDER_BY_TUPLE')" -$CLICKHOUSE_CLIENT -q "select count(distinct explain) from order_by_syntax" -$CLICKHOUSE_CLIENT -q "drop table if exists order_by_syntax" +[ "$OUTPUT_EXPLAIN_ORDER_BY" == "$OUTPUT_EXPLAIN_ORDER_BY_TUPLE" ] && echo "OK" From 5dcc271856bb3b605ed4bc6ed6702024cce498cf Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 7 Jul 2022 10:45:56 +0200 Subject: [PATCH 353/627] More careful destructors. --- src/Backups/BackupIO_Disk.cpp | 17 +++--------- src/Backups/BackupIO_File.cpp | 17 +++--------- src/Backups/BackupImpl.cpp | 49 +++++++++++++++++++++++------------ src/Backups/BackupsWorker.cpp | 8 +++--- 4 files changed, 45 insertions(+), 46 deletions(-) diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index 0734ddf0f57..a5c26bdbed6 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -1,9 +1,7 @@ #include -#include #include #include #include -#include namespace DB @@ -49,17 +47,10 @@ std::unique_ptr BackupWriterDisk::writeFile(const String & file_nam void BackupWriterDisk::removeFilesAfterFailure(const Strings & file_names) { - try - { - for (const auto & file_name : file_names) - disk->removeFileIfExists(path / file_name); - if (disk->isDirectory(path) && disk->isDirectoryEmpty(path)) - disk->removeDirectory(path); - } - catch (...) - { - LOG_WARNING(&Poco::Logger::get("BackupWriterDisk"), "RemoveFilesAfterFailure: {}", getCurrentExceptionMessage(false)); - } + for (const auto & file_name : file_names) + disk->removeFileIfExists(path / file_name); + if (disk->isDirectory(path) && disk->isDirectoryEmpty(path)) + disk->removeDirectory(path); } } diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 600c510183a..8e7bfb5b83e 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -1,8 +1,6 @@ #include -#include #include #include -#include namespace fs = std::filesystem; @@ -50,17 +48,10 @@ std::unique_ptr BackupWriterFile::writeFile(const String & file_nam void BackupWriterFile::removeFilesAfterFailure(const Strings & file_names) { - try - { - for (const auto & file_name : file_names) - fs::remove(path / file_name); - if (fs::is_directory(path) && fs::is_empty(path)) - fs::remove(path); - } - catch (...) - { - LOG_WARNING(&Poco::Logger::get("BackupWriterFile"), "RemoveFilesAfterFailure: {}", getCurrentExceptionMessage(false)); - } + for (const auto & file_name : file_names) + fs::remove(path / file_name); + if (fs::is_directory(path) && fs::is_empty(path)) + fs::remove(path); } } diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 5d4dba748e3..20b7bf37cfc 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -167,7 +167,14 @@ BackupImpl::BackupImpl( BackupImpl::~BackupImpl() { - close(); + try + { + close(); + } + catch (...) + { + DB::tryLogCurrentException(__PRETTY_FUNCTION__); + } } @@ -231,10 +238,11 @@ void BackupImpl::close() archive_writer = {"", nullptr}; if (!is_internal_backup && writer && !writing_finalized) - { - LOG_INFO(log, "Removing all files of backup {} after failure", backup_name); removeAllFilesAfterFailure(); - } + + writer.reset(); + reader.reset(); + coordination.reset(); } time_t BackupImpl::getTimestamp() const @@ -733,24 +741,33 @@ std::shared_ptr BackupImpl::getArchiveWriter(const String & suff void BackupImpl::removeAllFilesAfterFailure() { - Strings files_to_remove; - if (use_archives) + try { - files_to_remove.push_back(archive_params.archive_name); - for (const auto & suffix : coordination->getAllArchiveSuffixes()) + LOG_INFO(log, "Removing all files of backup {} after failure", backup_name); + + Strings files_to_remove; + if (use_archives) { - String archive_name_with_suffix = getArchiveNameWithSuffix(suffix); - files_to_remove.push_back(std::move(archive_name_with_suffix)); + files_to_remove.push_back(archive_params.archive_name); + for (const auto & suffix : coordination->getAllArchiveSuffixes()) + { + String archive_name_with_suffix = getArchiveNameWithSuffix(suffix); + files_to_remove.push_back(std::move(archive_name_with_suffix)); + } } + else + { + files_to_remove.push_back(".backup"); + for (const auto & file_info : coordination->getAllFileInfos()) + files_to_remove.push_back(file_info.data_file_name); + } + + writer->removeFilesAfterFailure(files_to_remove); } - else + catch (...) { - files_to_remove.push_back(".backup"); - for (const auto & file_info : coordination->getAllFileInfos()) - files_to_remove.push_back(file_info.data_file_name); + DB::tryLogCurrentException(__PRETTY_FUNCTION__); } - - writer->removeFilesAfterFailure(files_to_remove); } } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 85a997e24d3..ca89244cb9b 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -100,10 +100,10 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c /// Make a backup coordination. std::shared_ptr backup_coordination; - SCOPE_EXIT({ + SCOPE_EXIT_SAFE( if (backup_coordination && !backup_settings.internal) backup_coordination->drop(); - }); + ); ClusterPtr cluster; if (on_cluster) @@ -278,10 +278,10 @@ UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr conte /// Make a restore coordination. std::shared_ptr restore_coordination; - SCOPE_EXIT({ + SCOPE_EXIT_SAFE( if (restore_coordination && !restore_settings.internal) restore_coordination->drop(); - }); + ); if (on_cluster && restore_settings.coordination_zk_path.empty()) { From 2bc099e03ae39eb62abf479d3b02462ffec3a53d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 11:51:46 +0200 Subject: [PATCH 354/627] Add HTML page --- .../results/c6a.4xlarge.single.json | 2 +- benchmark/druid/benchmark.sh | 2 + benchmark/druid/results/c6a.4xlarge.json | 4 +- benchmark/index.html | 237 +++++++++++++----- 4 files changed, 179 insertions(+), 66 deletions(-) diff --git a/benchmark/clickhouse-local/results/c6a.4xlarge.single.json b/benchmark/clickhouse-local/results/c6a.4xlarge.single.json index f2805b2c2e5..a90e2442946 100644 --- a/benchmark/clickhouse-local/results/c6a.4xlarge.single.json +++ b/benchmark/clickhouse-local/results/c6a.4xlarge.single.json @@ -2,7 +2,7 @@ "system": "clickhouse-local (single)", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], diff --git a/benchmark/druid/benchmark.sh b/benchmark/druid/benchmark.sh index ad5634f6be9..03718557e66 100755 --- a/benchmark/druid/benchmark.sh +++ b/benchmark/druid/benchmark.sh @@ -29,6 +29,8 @@ gzip -d hits.tsv.gz ./apache-druid-${VERSION}/bin/post-index-task --file ingest.json --url http://localhost:8081 +# The loading time should be checked from the logs + # Run the queries ./run.sh diff --git a/benchmark/druid/results/c6a.4xlarge.json b/benchmark/druid/results/c6a.4xlarge.json index 8adaa163b26..acc8c215a0d 100644 --- a/benchmark/druid/results/c6a.4xlarge.json +++ b/benchmark/druid/results/c6a.4xlarge.json @@ -7,8 +7,8 @@ "tags": ["SQL", "Java", "column-oriented"], - "load_time": null, - "data_size": null, + "load_time": 19620, + "data_size": 45188608472, "result": [ [0.179351, 0.040782, 0.026180], diff --git a/benchmark/index.html b/benchmark/index.html index 9e218167431..bbede6b0e08 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -101,6 +101,7 @@ font-family: monospace; text-align: right; padding-left: 1rem; + white-space: nowrap; } th { @@ -612,10 +613,10 @@ const data = [ } , { - "system": "clickhouse-local (partitioned)", + "system": "clickhouse-local (single)", "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": "1", + "cluster_size": 1, "comment": "", "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], @@ -915,8 +916,8 @@ const data = [ "tags": ["SQL", "Java", "column-oriented"], - "load_time": null, - "data_size": null, + "load_time": 19620, + "data_size": 45188608472, "result": [ [0.179351, 0.040782, 0.026180], @@ -2446,7 +2447,7 @@ const data = [ @@ -2477,10 +2478,10 @@ const data = [
Metric: - Cold Run - Hot Run - Load Time - Storage Size + Cold Run + Hot Run + Load Time + Storage Size
@@ -2492,7 +2493,7 @@ const data = [ System & Machine - Relative time (lower is better) + Relative time (lower is better) @@ -2545,6 +2546,7 @@ function toggle(e, elem, selectors_map) { selectors_map[elem] = !selectors_map[elem]; e.target.className = selectors_map[elem] ? 'selector selector-active' : 'selector'; render(); + updateHistory(); } function toggleAll(e, selectors_map) { @@ -2554,6 +2556,7 @@ function toggleAll(e, selectors_map) { Object.keys(selectors_map).map(k => { selectors_map[k] = new_value }); render(); + updateHistory(); } unique_systems.map(elem => { @@ -2598,8 +2601,29 @@ document.getElementById('select-all-types').addEventListener('click', e => toggl document.getElementById('select-all-machines').addEventListener('click', e => toggleAll(e, selectors.machine)); document.getElementById('select-all-cluster-sizes').addEventListener('click', e => toggleAll(e, selectors.cluster_size)); +[...document.getElementById('selectors_run').querySelectorAll('a')].map(elem => elem.addEventListener('click', e => { + [...e.target.parentElement.querySelectorAll('a')].map(elem => { elem.className = elem == e.target ? 'selector selector-active' : 'selector' }); +})); + +document.getElementById('selector-metric-cold').addEventListener('click', e => { selectors.metric = 'cold'; render(); updateHistory(); }); +document.getElementById('selector-metric-hot').addEventListener('click', e => { selectors.metric = 'hot'; render(); updateHistory(); }); +document.getElementById('selector-metric-load').addEventListener('click', e => { selectors.metric = 'load'; render(); updateHistory(); }); +document.getElementById('selector-metric-size').addEventListener('click', e => { selectors.metric = 'size'; render(); updateHistory(); }); + selectors.queries = [...data[0].result.keys()].map(k => true); +function updateSelectors() { + [...systems.childNodes].map(elem => { elem.className = selectors.system[elem.innerText] ? 'selector selector-active' : 'selector' }); + [...types.childNodes].map(elem => { elem.className = selectors.type[elem.innerText] ? 'selector selector-active' : 'selector' }); + [...machines.childNodes].map(elem => { elem.className = selectors.machine[elem.innerText] ? 'selector selector-active' : 'selector' }); + [...cluster_sizes.childNodes].map(elem => { elem.className = selectors.cluster_size[elem.innerText] ? 'selector selector-active' : 'selector' }); + + [...document.getElementById('selectors_run').querySelectorAll('a')].map(elem => { + elem.className = elem.id == 'selector-metric-' + selectors.metric ? 'selector selector-active' : 'selector' }); + + [...document.querySelectorAll('.query-checkbox')].map((elem, i) => { elem.checked = selectors.queries[i] }); +} + function clearElement(elem) { while (elem.firstChild) { @@ -2630,28 +2654,39 @@ function renderSummary(filtered_data) { [...Array(3).keys()].map(run_num => Math.min(...filtered_data.map(elem => elem.result[query_num][run_num]).filter(x => x)))); - const summaries = filtered_data.map(elem => { - console.log(elem.system); + const min_load_time = Math.min(...filtered_data.map(elem => elem.load_time).filter(x => x)); + const min_data_size = Math.min(...filtered_data.map(elem => elem.data_size).filter(x => x)); - const fallback_timing = missing_result_penalty * Math.max(missing_result_time, ...elem.result.map(timings => selectRun(timings))); + let summaries; + if (selectors.metric == 'load') { + summaries = filtered_data.map(elem => elem.load_time / min_load_time); + document.getElementById('time-or-size').innerText = 'time'; + } else if (selectors.metric == 'size') { + summaries = filtered_data.map(elem => elem.data_size / min_data_size); + document.getElementById('time-or-size').innerText = 'size'; + } else { + summaries = filtered_data.map(elem => { + const fallback_timing = missing_result_penalty * Math.max(missing_result_time, ...elem.result.map(timings => selectRun(timings))); - let accumulator = 0; - let used_queries = 0; + let accumulator = 0; + let used_queries = 0; - const no_queries_selected = selectors.queries.filter(x => x).length == 0; + const no_queries_selected = selectors.queries.filter(x => x).length == 0; - for (let i = 0; i < num_queries; ++i) { - if (no_queries_selected || selectors.queries[i]) { - const curr_timing = selectRun(elem.result[i]) ?? fallback_timing; - const baseline_timing = selectRun(baseline_data[i]); - const ratio = (constant_time_add + curr_timing) / (constant_time_add + baseline_timing); - accumulator += Math.log(ratio); - ++used_queries; + for (let i = 0; i < num_queries; ++i) { + if (no_queries_selected || selectors.queries[i]) { + const curr_timing = selectRun(elem.result[i]) ?? fallback_timing; + const baseline_timing = selectRun(baseline_data[i]); + const ratio = (constant_time_add + curr_timing) / (constant_time_add + baseline_timing); + accumulator += Math.log(ratio); + ++used_queries; + } } - } - return Math.exp(accumulator / used_queries); - }); + return Math.exp(accumulator / used_queries); + }); + document.getElementById('time-or-size').innerText = 'time'; + } const sorted_indices = [...summaries.keys()].sort((a, b) => summaries[a] - summaries[b]); const max_ratio = summaries[sorted_indices[sorted_indices.length - 1]]; @@ -2676,7 +2711,12 @@ function renderSummary(filtered_data) { let td_number = document.createElement('td'); td_number.className = 'summary-number'; - td_number.appendChild(document.createTextNode(`×${ratio.toFixed(2)}`)); + + const text = selectors.metric == 'load' ? (elem.load_time ? `${Math.round(elem.load_time)}s (×${ratio.toFixed(2)})` : 'stateless') + : selectors.metric == 'size' ? `${(elem.data_size / 1024 / 1024 / 1024).toFixed(2)} GiB (×${ratio.toFixed(2)})` + : `×${ratio.toFixed(2)}`; + + td_number.appendChild(document.createTextNode(text)); let td_bar = document.createElement('td'); td_bar.className = 'summary-bar-cell'; @@ -2697,6 +2737,42 @@ function renderSummary(filtered_data) { return [sorted_indices, baseline_data]; } +function colorize(elem, ratio) { + let [r, g, b] = [0, 0, 0]; + + /// ratio less than 1 - green + /// ratio from 1 to 10 - green to orange + /// ratio from 10 to 100 - orange to red + /// ratio from 100 to 1000 to infinity - red to brown to black + + if (ratio !== null) { + if (ratio < 1) { + r = 232; + g = 255; + b = 232; + } else if (ratio <= 1) { + g = 255; + } else if (ratio <= 2) { + g = 255; + r = (ratio - 1) * 255; + } else if (ratio <= 10) { + g = (10 - ratio) / 9 * 255; + r = 255; + } else { + r = (1 - ((ratio - 10) / ((ratio - 10) + 1000))) * 255; + } + } + + elem.style.backgroundColor = `rgb(${r}, ${g}, ${b})`; + if (ratio === null || ratio > 10) { + elem.style.color = 'white'; + } + + if (ratio == 1) { + elem.style.fontWeight = 'bold'; + } +} + function render() { let details_head = document.getElementById('details_head'); let details_body = document.getElementById('details_body'); @@ -2732,6 +2808,7 @@ function render() { [...document.querySelectorAll('.query-checkbox')].map(elem => { elem.checked = e.target.checked }); selectors.queries.map((_, i) => { selectors.queries[i] = e.target.checked }); renderSummary(filtered_data); + updateHistory(); }); th_checkbox.appendChild(checkbox); details_head.appendChild(th_checkbox); @@ -2745,6 +2822,54 @@ function render() { details_head.appendChild(th); }); + { + let tr = document.createElement('tr'); + tr.className = 'shadow'; + + let td_title = document.createElement('td'); + td_title.colSpan = 2; + td_title.appendChild(document.createTextNode('Load time: ')); + tr.appendChild(td_title); + + sorted_indices.map(idx => { + const curr_timing = filtered_data[idx].load_time; + const baseline_timing = Math.min(...filtered_data.map(elem => elem.load_time).filter(x => x)); + const ratio = curr_timing / baseline_timing; + + let td = document.createElement('td'); + td.appendChild(document.createTextNode(curr_timing ? `${curr_timing.toFixed(2)} (×${ratio.toFixed(2)})` : '0')); + + colorize(td, ratio); + tr.appendChild(td); + }); + + details_body.appendChild(tr); + } + + { + let tr = document.createElement('tr'); + tr.className = 'shadow'; + + let td_title = document.createElement('td'); + td_title.colSpan = 2; + td_title.appendChild(document.createTextNode('Data size: ')); + tr.appendChild(td_title); + + sorted_indices.map(idx => { + const curr_size = filtered_data[idx].data_size; + const baseline_size = Math.min(...filtered_data.map(elem => elem.data_size).filter(x => x)); + const ratio = curr_size / baseline_size; + + let td = document.createElement('td'); + td.appendChild(document.createTextNode(curr_size ? `${(curr_size / 1024 / 1024 / 1024).toFixed(2)} GiB (×${ratio.toFixed(2)})` : '0')); + + colorize(td, ratio); + tr.appendChild(td); + }); + + details_body.appendChild(tr); + } + const num_queries = filtered_data[0].result.length; for (let query_num = 0; query_num < num_queries; ++query_num) { @@ -2759,57 +2884,24 @@ function render() { checkbox.addEventListener('change', e => { selectors.queries[query_num] = e.target.checked; renderSummary(filtered_data); + updateHistory(); }); td_checkbox.appendChild(checkbox); tr.appendChild(td_checkbox); let td_query_num = document.createElement('td'); td_query_num.appendChild(document.createTextNode(`Q${query_num}. `)); - tr.appendChild(td_query_num); sorted_indices.map(idx => { const curr_timing = selectRun(filtered_data[idx].result[query_num]); const baseline_timing = selectRun(baseline_data[query_num]); - const ratio = (constant_time_add + curr_timing) / (constant_time_add + baseline_timing); + const ratio = curr_timing !== null ? (constant_time_add + curr_timing) / (constant_time_add + baseline_timing) : null; let td = document.createElement('td'); td.appendChild(document.createTextNode(curr_timing !== null ? `${curr_timing.toFixed(2)} (×${ratio.toFixed(2)})` : '☠')); - let [r, g, b] = [0, 0, 0]; - - /// ratio less than 1 - green - /// ratio from 1 to 10 - green to orange - /// ratio from 10 to 100 - orange to red - /// ratio from 100 to 1000 to infinity - red to brown to black - - if (curr_timing !== null) { - if (ratio < 1) { - r = 232; - g = 255; - b = 232; - } else if (ratio <= 1) { - g = 255; - } else if (ratio <= 2) { - g = 255; - r = (ratio - 1) * 255; - } else if (ratio <= 10) { - g = (10 - ratio) / 9 * 255; - r = 255; - } else { - r = (1 - ((ratio - 10) / ((ratio - 10) + 1000))) * 255; - } - } - - td.style.backgroundColor = `rgb(${r}, ${g}, ${b})`; - if (curr_timing === null || ratio > 10) { - td.style.color = 'white'; - } - - if (ratio == 1) { - td.style.fontWeight = 'bold'; - } - + colorize(td, ratio); tr.appendChild(td); }); @@ -2822,7 +2914,26 @@ function render() { } } +function updateHistory() { + history.pushState(selectors, '', + window.location.pathname + (window.location.search || '') + '#' + btoa(JSON.stringify(selectors))); +} + +window.onpopstate = function(event) { + if (!event.state) { return; } + selectors = event.state; + render(); + updateSelectors(); +}; + +if (window.location.hash) { + try { + selectors = JSON.parse(atob(window.location.hash.substring(1))); + } catch {} +} + render(); +updateSelectors(); From 29d75e5a12bfc98446b0b60fd8fb9d79ebe2542a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 7 Jul 2022 08:21:01 -0300 Subject: [PATCH 355/627] Update column.md --- docs/ru/sql-reference/statements/alter/column.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index 2042553efe5..6bbee5479ce 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -75,8 +75,9 @@ DROP COLUMN [IF EXISTS] name Запрос удаляет данные из файловой системы. Так как это представляет собой удаление целых файлов, запрос выполняется почти мгновенно. -:::danger "Предупреждение" +:::warning "Предупреждение" Вы не можете удалить столбец, используемый в [материализованном представлениии](../../../sql-reference/statements/create/view.md#materialized). В противном случае будет ошибка. +::: Пример: From fdd2db4aceac392e92665aefc4c74180d2bf9ad0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 7 Jul 2022 15:24:36 +0300 Subject: [PATCH 356/627] Move check for denied allocations (#38858) * move check for denied allocations * remove include * make hardening softer --- src/Common/CurrentMemoryTracker.cpp | 69 +++++++++++-------- src/Common/CurrentMemoryTracker.h | 20 ++++-- src/Common/MemoryTracker.cpp | 26 ++----- src/Common/MemoryTracker.h | 25 ++----- src/Common/ThreadStatus.cpp | 5 +- .../MergeTree/MergeTreeBackgroundExecutor.cpp | 26 ++++++- 6 files changed, 89 insertions(+), 82 deletions(-) diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index bf0745e667e..e41cb258f08 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -4,6 +4,18 @@ #include +#ifdef MEMORY_TRACKER_DEBUG_CHECKS +thread_local bool memory_tracker_always_throw_logical_error_on_allocation = false; +#endif + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} +} + namespace { @@ -23,64 +35,66 @@ MemoryTracker * getMemoryTracker() } -namespace CurrentMemoryTracker -{ - using DB::current_thread; -namespace +void CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) { - void allocImpl(Int64 size, bool throw_if_memory_exceeded) +#ifdef MEMORY_TRACKER_DEBUG_CHECKS + if (unlikely(memory_tracker_always_throw_logical_error_on_allocation)) { - if (auto * memory_tracker = getMemoryTracker()) - { - if (current_thread) - { - current_thread->untracked_memory += size; + memory_tracker_always_throw_logical_error_on_allocation = false; + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Memory tracker: allocations not allowed."); + } +#endif - if (current_thread->untracked_memory > current_thread->untracked_memory_limit) - { - /// Zero untracked before track. If tracker throws out-of-limit we would be able to alloc up to untracked_memory_limit bytes - /// more. It could be useful to enlarge Exception message in rethrow logic. - Int64 tmp = current_thread->untracked_memory; - current_thread->untracked_memory = 0; - memory_tracker->allocImpl(tmp, throw_if_memory_exceeded); - } - } - /// total_memory_tracker only, ignore untracked_memory - else + if (auto * memory_tracker = getMemoryTracker()) + { + if (current_thread) + { + current_thread->untracked_memory += size; + + if (current_thread->untracked_memory > current_thread->untracked_memory_limit) { - memory_tracker->allocImpl(size, throw_if_memory_exceeded); + /// Zero untracked before track. If tracker throws out-of-limit we would be able to alloc up to untracked_memory_limit bytes + /// more. It could be useful to enlarge Exception message in rethrow logic. + Int64 tmp = current_thread->untracked_memory; + current_thread->untracked_memory = 0; + memory_tracker->allocImpl(tmp, throw_if_memory_exceeded); } } + /// total_memory_tracker only, ignore untracked_memory + else + { + memory_tracker->allocImpl(size, throw_if_memory_exceeded); + } } } -void check() +void CurrentMemoryTracker::check() { if (auto * memory_tracker = getMemoryTracker()) memory_tracker->allocImpl(0, true); } -void alloc(Int64 size) +void CurrentMemoryTracker::alloc(Int64 size) { bool throw_if_memory_exceeded = true; allocImpl(size, throw_if_memory_exceeded); } -void allocNoThrow(Int64 size) +void CurrentMemoryTracker::allocNoThrow(Int64 size) { bool throw_if_memory_exceeded = false; allocImpl(size, throw_if_memory_exceeded); } -void realloc(Int64 old_size, Int64 new_size) +void CurrentMemoryTracker::realloc(Int64 old_size, Int64 new_size) { Int64 addition = new_size - old_size; addition > 0 ? alloc(addition) : free(-addition); } -void free(Int64 size) +void CurrentMemoryTracker::free(Int64 size) { if (auto * memory_tracker = getMemoryTracker()) { @@ -101,4 +115,3 @@ void free(Int64 size) } } -} diff --git a/src/Common/CurrentMemoryTracker.h b/src/Common/CurrentMemoryTracker.h index 5f735b10be9..e125e4cbe4a 100644 --- a/src/Common/CurrentMemoryTracker.h +++ b/src/Common/CurrentMemoryTracker.h @@ -3,11 +3,17 @@ #include /// Convenience methods, that use current thread's memory_tracker if it is available. -namespace CurrentMemoryTracker +struct CurrentMemoryTracker { - void alloc(Int64 size); - void allocNoThrow(Int64 size); - void realloc(Int64 old_size, Int64 new_size); - void free(Int64 size); - void check(); -} + /// Call the following functions before calling of corresponding operations with memory allocators. + static void alloc(Int64 size); + static void allocNoThrow(Int64 size); + static void realloc(Int64 old_size, Int64 new_size); + + /// This function should be called after memory deallocation. + static void free(Int64 size); + static void check(); + +private: + static void allocImpl(Int64 size, bool throw_if_memory_exceeded); +}; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index ba0392e802e..5546dc6b7b3 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -19,10 +19,6 @@ #include -#ifdef MEMORY_TRACKER_DEBUG_CHECKS -thread_local bool memory_tracker_always_throw_logical_error_on_allocation = false; -#endif - namespace { @@ -170,14 +166,6 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } } -#ifdef MEMORY_TRACKER_DEBUG_CHECKS - if (unlikely(memory_tracker_always_throw_logical_error_on_allocation)) - { - memory_tracker_always_throw_logical_error_on_allocation = false; - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Memory tracker: allocations not allowed."); - } -#endif - std::bernoulli_distribution fault(fault_probability); if (unlikely(fault_probability && fault(thread_local_rng)) && memoryTrackerCanThrow(level, true) && throw_if_memory_exceeded) { @@ -271,16 +259,12 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT level == VariableContext::Process ? this : query_tracker); } -void MemoryTracker::alloc(Int64 size) +void MemoryTracker::adjustWithUntrackedMemory(Int64 untracked_memory) { - bool throw_if_memory_exceeded = true; - allocImpl(size, throw_if_memory_exceeded); -} - -void MemoryTracker::allocNoThrow(Int64 size) -{ - bool throw_if_memory_exceeded = false; - allocImpl(size, throw_if_memory_exceeded); + if (untracked_memory > 0) + allocImpl(untracked_memory, /*throw_if_memory_exceeded*/ false); + else + free(-untracked_memory); } bool MemoryTracker::updatePeak(Int64 will_be, bool log_memory_usage) diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 226e60bd06a..d9dd55a3a50 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -85,6 +85,10 @@ private: void setOrRaiseProfilerLimit(Int64 value); + /// allocImpl(...) and free(...) should not be used directly + friend struct CurrentMemoryTracker; + void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); + void free(Int64 size); public: static constexpr auto USAGE_EVENT_NAME = "MemoryTrackerUsage"; @@ -96,26 +100,7 @@ public: VariableContext level; - /** Call the following functions before calling of corresponding operations with memory allocators. - */ - void alloc(Int64 size); - - void allocNoThrow(Int64 size); - - void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); - - void realloc(Int64 old_size, Int64 new_size) - { - Int64 addition = new_size - old_size; - if (addition > 0) - alloc(addition); - else - free(-addition); - } - - /** This function should be called after memory deallocation. - */ - void free(Int64 size); + void adjustWithUntrackedMemory(Int64 untracked_memory); Int64 get() const { diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index a4f99c1be1a..4dd32f7ff10 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -148,10 +148,7 @@ ThreadStatus::ThreadStatus() ThreadStatus::~ThreadStatus() { - if (untracked_memory > 0) - memory_tracker.allocNoThrow(untracked_memory); - else - memory_tracker.free(-untracked_memory); + memory_tracker.adjustWithUntrackedMemory(untracked_memory); if (thread_group) { diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 3399d46f0bb..c3f83771338 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -118,6 +119,7 @@ void MergeTreeBackgroundExecutor::removeTasksCorrespondingToStorage(Stora template void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) { + /// FIXME Review exception-safety of this, remove NOEXCEPT_SCOPE and ALLOW_ALLOCATIONS_IN_SCOPE if possible DENY_ALLOCATIONS_IN_SCOPE; /// All operations with queues are considered no to do any allocations @@ -136,6 +138,8 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) } catch (const Exception & e) { + NOEXCEPT_SCOPE; + ALLOW_ALLOCATIONS_IN_SCOPE; if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); else @@ -143,6 +147,8 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) } catch (...) { + NOEXCEPT_SCOPE; + ALLOW_ALLOCATIONS_IN_SCOPE; tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -155,7 +161,11 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) erase_from_active(); /// This is significant to order the destructors. - item->task.reset(); + { + NOEXCEPT_SCOPE; + ALLOW_ALLOCATIONS_IN_SCOPE; + item->task.reset(); + } item->is_done.set(); item = nullptr; return; @@ -187,6 +197,8 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) } catch (const Exception & e) { + NOEXCEPT_SCOPE; + ALLOW_ALLOCATIONS_IN_SCOPE; if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); else @@ -194,15 +206,23 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) } catch (...) { + NOEXCEPT_SCOPE; + ALLOW_ALLOCATIONS_IN_SCOPE; tryLogCurrentException(__PRETTY_FUNCTION__); } + /// We have to call reset() under a lock, otherwise a race is possible. /// Imagine, that task is finally completed (last execution returned false), /// we removed the task from both queues, but still have pointer. /// The thread that shutdowns storage will scan queues in order to find some tasks to wait for, but will find nothing. /// So, the destructor of a task and the destructor of a storage will be executed concurrently. - item->task.reset(); + { + NOEXCEPT_SCOPE; + ALLOW_ALLOCATIONS_IN_SCOPE; + item->task.reset(); + } + item->is_done.set(); item = nullptr; } @@ -236,6 +256,8 @@ void MergeTreeBackgroundExecutor::threadFunction() } catch (...) { + NOEXCEPT_SCOPE; + ALLOW_ALLOCATIONS_IN_SCOPE; tryLogCurrentException(__PRETTY_FUNCTION__); } } From 1de5e9a7da63d940e12c4e570507f86d76488926 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Jul 2022 21:36:14 +0000 Subject: [PATCH 357/627] Avoid copy-ing array elements --- .../FunctionsMultiStringFuzzySearch.h | 11 +- src/Functions/FunctionsMultiStringSearch.h | 9 +- src/Functions/MultiMatchAllIndicesImpl.h | 37 ++-- src/Functions/MultiMatchAnyImpl.h | 163 ++++++++++++++---- src/Functions/MultiSearchFirstIndexImpl.h | 29 +++- src/Functions/MultiSearchFirstPositionImpl.h | 29 +++- src/Functions/MultiSearchImpl.h | 37 ++-- 7 files changed, 239 insertions(+), 76 deletions(-) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 34295b7e914..26cafa7d410 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -100,15 +100,20 @@ public: if (col_needles_const) { - Array needles_arr = col_needles_const->getValue(); Impl::vectorConstant( - col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needles_arr, vec_res, offsets_res, edit_distance, + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), + col_needles_const->getValue(), + vec_res, offsets_res, + edit_distance, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); } else { Impl::vectorVector( - col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), *col_needles_vector, vec_res, offsets_res, edit_distance, + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), + col_needles_vector->getData(), col_needles_vector->getOffsets(), + vec_res, offsets_res, + edit_distance, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); } diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index f18a7d0904c..fb800448bfa 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -100,15 +100,18 @@ public: if (col_needles_const) { - Array needles_arr = col_needles_const->getValue(); Impl::vectorConstant( - col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needles_arr, vec_res, offsets_res, + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), + col_needles_const->getValue(), + vec_res, offsets_res, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); } else { Impl::vectorVector( - col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), *col_needles_vector, vec_res, offsets_res, + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), + col_needles_vector->getData(), col_needles_vector->getOffsets(), + vec_res, offsets_res, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); } diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index 56ef0a828a9..1a8d4439be4 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -138,20 +138,22 @@ struct MultiMatchAllIndicesImpl static void vectorVector( const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, - const ColumnArray & needles_col, + const IColumn & needles_data, + const ColumnArray::Offsets & needles_offsets, PaddedPODArray & res, PaddedPODArray & offsets, bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length) { - vectorVector(haystack_data, haystack_offsets, needles_col, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); } static void vectorVector( const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, - const ColumnArray & needles_col, + const IColumn & needles_data, + const ColumnArray::Offsets & needles_offsets, PaddedPODArray & res, PaddedPODArray & offsets, std::optional edit_distance, @@ -164,15 +166,27 @@ struct MultiMatchAllIndicesImpl #if USE_VECTORSCAN offsets.resize(haystack_offsets.size()); size_t prev_haystack_offset = 0; + + const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); + const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); + + std::vector needles; + + size_t prev_needles_offsets_offset = 0; + size_t prev_needles_data_offset = 0; + for (size_t i = 0; i < haystack_offsets.size(); ++i) { - Field field = needles_col[i]; - const Array & needles_arr = DB::get(field); + needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); - std::vector needles; - needles.reserve(needles_arr.size()); - for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + { + const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; + auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; + needles.emplace_back(std::string_view(p, sz)); + prev_needles_data_offset = needles_data_string_offsets[j]; + } checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); @@ -216,11 +230,14 @@ struct MultiMatchAllIndicesImpl offsets[i] = res.size(); prev_haystack_offset = haystack_offsets[i]; + prev_needles_offsets_offset = needles_offsets[i]; + needles.clear(); } #else (void)haystack_data; (void)haystack_offsets; - (void)needles_col; + (void)needles_data; + (void)needles_offsets; (void)res; (void)offsets; (void)edit_distance; diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index fa56a5f0924..e0ca9e0cc03 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -14,6 +14,7 @@ # include #else # include "MatchImpl.h" + #include #endif @@ -163,26 +164,28 @@ struct MultiMatchAnyImpl static void vectorVector( const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, - const ColumnArray & needles_col, + const IColumn & needles_data, + const ColumnArray::Offsets & needles_offsets, PaddedPODArray & res, PaddedPODArray & offsets, bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length) { - vectorVector(haystack_data, haystack_offsets, needles_col, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); } static void vectorVector( - [[maybe_unused]] const ColumnString::Chars & haystack_data, - [[maybe_unused]] const ColumnString::Offsets & haystack_offsets, - [[maybe_unused]] const ColumnArray & needles_col, - [[maybe_unused]] PaddedPODArray & res, + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const IColumn & needles_data, + const ColumnArray::Offsets & needles_offsets, + PaddedPODArray & res, PaddedPODArray & /*offsets*/, - [[maybe_unused]] std::optional edit_distance, + std::optional edit_distance, bool allow_hyperscan, - [[maybe_unused]] size_t max_hyperscan_regexp_length, - [[maybe_unused]] size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_length, + size_t max_hyperscan_regexp_total_length) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -190,15 +193,27 @@ struct MultiMatchAnyImpl res.resize(haystack_offsets.size()); #if USE_VECTORSCAN size_t prev_haystack_offset = 0; + + const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); + const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); + + std::vector needles; + + size_t prev_needles_offsets_offset = 0; + size_t prev_needles_data_offset = 0; + for (size_t i = 0; i < haystack_offsets.size(); ++i) { - Field field = needles_col[i]; - const Array & needles_arr = DB::get(field); + needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); - std::vector needles; - needles.reserve(needles_arr.size()); - for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + { + const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; + auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; + needles.emplace_back(std::string_view(p, sz)); + prev_needles_data_offset = needles_data_string_offsets[j]; + } checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); @@ -245,51 +260,129 @@ struct MultiMatchAnyImpl throw Exception("Failed to scan with vectorscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT); prev_haystack_offset = haystack_offsets[i]; + prev_needles_offsets_offset = needles_offsets[i]; + needles.clear(); } #else /// fallback if vectorscan is not compiled + /// -- the code is copypasted from vectorVector() in MatchImpl.h and quite complex code ... all of it can be removed once vectorscan is + /// enabled on all platforms (#38906) if constexpr (WithEditDistance) throw Exception( "Edit distance multi-search is not implemented when vectorscan is off", ErrorCodes::NOT_IMPLEMENTED); + (void)edit_distance; + memset(res.data(), 0, res.size() * sizeof(res.front())); size_t prev_haystack_offset = 0; + const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); + const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); + + std::vector needles; + + size_t prev_needles_offsets_offset = 0; + size_t prev_needles_data_offset = 0; + for (size_t i = 0; i < haystack_offsets.size(); ++i) { const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset]; const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; - Field field; - needles_col.get(i, field); - Array & needles_arr = DB::get(field); + needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); - std::vector needles; - needles.reserve(needles_arr.size()); - for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + { + const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; + auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; + needles.emplace_back(std::string_view(p, sz)); + prev_needles_data_offset = needles_data_string_offsets[j]; + } checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); - for (size_t j = 0; j < needles_arr.size(); ++j) + for (size_t j = 0; j < needles.size(); ++j) { - String needle = needles_arr[j].safeGet(); + String needle(needles[j]); + const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); - const bool match = regexp.getRE2()->Match( - {reinterpret_cast(cur_haystack_data), cur_haystack_length}, - 0, - cur_haystack_length, - re2_st::RE2::UNANCHORED, - nullptr, - 0); - if constexpr (FindAny) - res[i] |= match; - else if (FindAnyIndex && match) - res[i] = j + 1; + + String required_substr; + bool is_trivial; + bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + + regexp.getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix); + + if (required_substr.empty()) + { + if (!regexp.getRE2()) /// An empty regexp. Always matches. + { + if constexpr (FindAny) + res[i] |= 1; + else if (FindAnyIndex) + res[i] = j + 1; + } + else + { + const bool match = regexp.getRE2()->Match( + {reinterpret_cast(cur_haystack_data), cur_haystack_length}, + 0, + cur_haystack_length, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + if constexpr (FindAny) + res[i] |= match; + else if (FindAnyIndex && match) + res[i] = j + 1; + } + } + else + { + Volnitsky searcher(required_substr.data(), required_substr.size(), cur_haystack_length); + const auto * match = searcher.search(cur_haystack_data, cur_haystack_length); + + if (match == cur_haystack_data + cur_haystack_length) + { + /// no match + } + else + { + if (is_trivial) + { + /// no wildcards in pattern + if constexpr (FindAny) + res[i] |= 1; + else if (FindAnyIndex) + res[i] = j + 1; + } + else + { + const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0; + const size_t end_pos = cur_haystack_length; + + const bool match2 = regexp.getRE2()->Match( + {reinterpret_cast(cur_haystack_data), cur_haystack_length}, + start_pos, + end_pos, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + if constexpr (FindAny) + res[i] |= match2; + else if (FindAnyIndex && match2) + res[i] = j + 1; + } + } + } } + prev_haystack_offset = haystack_offsets[i]; + prev_needles_offsets_offset = needles_offsets[i]; + needles.clear(); } #endif // USE_VECTORSCAN } diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 2f781de3214..06003c7f31f 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -72,7 +72,8 @@ struct MultiSearchFirstIndexImpl static void vectorVector( const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, - const ColumnArray & needles_col, + const IColumn & needles_data, + const ColumnArray::Offsets & needles_offsets, PaddedPODArray & res, PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, @@ -84,15 +85,26 @@ struct MultiSearchFirstIndexImpl size_t prev_offset = 0; + const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); + const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); + + std::vector needles; + + size_t prev_needles_offsets_offset = 0; + size_t prev_needles_data_offset = 0; + for (size_t i = 0; i < haystack_size; ++i) { - Field field = needles_col[i]; - const Array & needles_arr = DB::get(field); + needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); - std::vector needles; - needles.reserve(needles_arr.size()); - for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + { + const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; + auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; + needles.emplace_back(std::string_view(p, sz)); + prev_needles_data_offset = needles_data_string_offsets[j]; + } auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal @@ -112,7 +124,10 @@ struct MultiSearchFirstIndexImpl { res[i] = 0; } + prev_offset = haystack_offsets[i]; + prev_needles_offsets_offset = needles_offsets[i]; + needles.clear(); } } }; diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index 8029fc703c4..0b81ce88065 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -81,7 +81,8 @@ struct MultiSearchFirstPositionImpl static void vectorVector( const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, - const ColumnArray & needles_col, + const IColumn & needles_data, + const ColumnArray::Offsets & needles_offsets, PaddedPODArray & res, PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, @@ -93,6 +94,15 @@ struct MultiSearchFirstPositionImpl size_t prev_offset = 0; + const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); + const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); + + std::vector needles; + + size_t prev_needles_offsets_offset = 0; + size_t prev_needles_data_offset = 0; + auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 { return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); @@ -100,13 +110,15 @@ struct MultiSearchFirstPositionImpl for (size_t i = 0; i < haystack_size; ++i) { - Field field = needles_col[i]; - const Array & needles_arr = DB::get(field); + needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); - std::vector needles; - needles.reserve(needles_arr.size()); - for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + { + const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; + auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; + needles.emplace_back(std::string_view(p, sz)); + prev_needles_data_offset = needles_data_string_offsets[j]; + } auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal @@ -134,7 +146,10 @@ struct MultiSearchFirstPositionImpl { res[i] = 0; } + prev_offset = haystack_offsets[i]; + prev_needles_offsets_offset = needles_offsets[i]; + needles.clear(); } } }; diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 9a7801a586d..ad789004db0 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -71,7 +71,8 @@ struct MultiSearchImpl static void vectorVector( const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, - const ColumnArray & needles_col, + const IColumn & needles_data, + const ColumnArray::Offsets & needles_offsets, PaddedPODArray & res, PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, @@ -83,23 +84,34 @@ struct MultiSearchImpl size_t prev_offset = 0; + const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); + const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); + + std::vector needles; + + size_t prev_needles_offsets_offset = 0; + size_t prev_needles_data_offset = 0; + for (size_t i = 0; i < haystack_size; ++i) { + needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); + + for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + { + const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; + auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; + needles.emplace_back(std::string_view(p, sz)); + prev_needles_data_offset = needles_data_string_offsets[j]; + } + const auto * const haystack = &haystack_data[prev_offset]; const size_t haystack_length = haystack_offsets[i] - prev_offset - 1; - Field field = needles_col[i]; - const Array & needles_arr = DB::get(field); - - std::vector needles; - needles.reserve(needles_arr.size()); - for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); - size_t iteration = 0; - for (size_t j = 0; j < needles_arr.size(); ++j) + for (const auto & needle : needles) { - auto searcher = Impl::createSearcherInSmallHaystack(needles[j].data(), needles[j].size()); + auto searcher = Impl::createSearcherInSmallHaystack(needle.data(), needle.size()); if (iteration == 0 || !res[i]) { const auto * match = searcher.search(haystack, haystack_length); @@ -109,7 +121,10 @@ struct MultiSearchImpl } if (iteration == 0) res[i] = 0; + prev_offset = haystack_offsets[i]; + prev_needles_offsets_offset = needles_offsets[i]; + needles.clear(); } } }; From 8df2d756f59d1b00cb052e54ae4247a32c987980 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 7 Jul 2022 16:42:29 +0300 Subject: [PATCH 358/627] updated tests --- tests/queries/0_stateless/02346_non_negative_derivative.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02346_non_negative_derivative.sql b/tests/queries/0_stateless/02346_non_negative_derivative.sql index 673106e100d..265a8afb2cb 100644 --- a/tests/queries/0_stateless/02346_non_negative_derivative.sql +++ b/tests/queries/0_stateless/02346_non_negative_derivative.sql @@ -10,6 +10,8 @@ ORDER BY id; INSERT INTO nnd VALUES (1, toDateTime64('1979-12-12 21:21:21.123456788', 9, 'UTC'), 1.1), (2, toDateTime64('1979-12-12 21:21:21.123456789', 9, 'UTC'), 2.34), (3, toDateTime64('1979-12-12 21:21:21.127', 3, 'UTC'), 3.7); INSERT INTO nnd VALUES (4, toDateTime64('1979-12-12 21:21:21.129', 3, 'UTC'), 2.1), (5, toDateTime('1979-12-12 21:21:22', 'UTC'), 1.3345), (6, toDateTime('1979-12-12 21:21:23', 'UTC'), 1.54), (7, toDateTime('1979-12-12 21:21:23', 'UTC'), 1.54); +OPTIMIZE TABLE nnd; + -- shall work for precise intervals -- INTERVAL 1 SECOND shall be default SELECT ( From 33ffb71453eec129fd856cb65f85ee152bd240c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Jul 2022 18:59:53 +0200 Subject: [PATCH 359/627] Avoid 'Node exists' errors when creating Ephemeral Locks --- .../MergeTree/EphemeralLockInZooKeeper.cpp | 49 ++++++++++++++----- .../MergeTree/EphemeralLockInZooKeeper.h | 15 ++++-- src/Storages/StorageReplicatedMergeTree.cpp | 30 +----------- 3 files changed, 48 insertions(+), 46 deletions(-) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 3bed534e090..7abed125b7a 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -12,30 +12,53 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -EphemeralLockInZooKeeper::EphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, Coordination::Requests * precheck_ops) - : zookeeper(&zookeeper_), path_prefix(path_prefix_) +EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & holder_path_) + : zookeeper(&zookeeper_), path_prefix(path_prefix_), holder_path(holder_path_) +{ + /// Write the path to the secondary node in the main node. + path = zookeeper->create(path_prefix, holder_path, zkutil::CreateMode::EphemeralSequential); + if (path.size() <= path_prefix.size()) + throw Exception("Logical error: name of the main node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR); +} + +std::optional createEphemeralLockInZooKeeper( + const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path) { /// The /abandonable_lock- name is for backward compatibility. String holder_path_prefix = temp_path + "/abandonable_lock-"; + String holder_path; /// Let's create an secondary ephemeral node. - if (!precheck_ops || precheck_ops->empty()) + if (deduplication_path.empty()) { - holder_path = zookeeper->create(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential); + holder_path = zookeeper_.create(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential); } else { - precheck_ops->emplace_back(zkutil::makeCreateRequest(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential)); - Coordination::Responses op_results = zookeeper->multi(*precheck_ops); - holder_path = dynamic_cast(*op_results.back()).path_created; + /// Check for duplicates in advance, to avoid superfluous block numbers allocation + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(deduplication_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1)); + ops.emplace_back(zkutil::makeCreateRequest(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential)); + Coordination::Responses responses; + Coordination::Error e = zookeeper_.tryMulti(ops, responses); + if (e != Coordination::Error::ZOK) + { + if (responses[0]->error == Coordination::Error::ZNODEEXISTS) + { + return {}; + } + else + { + zkutil::KeeperMultiException::check(e, ops, responses); // This should always throw the proper exception + throw Exception("Unable to handle error {} when acquiring ephemeral lock in ZK", ErrorCodes::LOGICAL_ERROR); + } + } + + holder_path = dynamic_cast(responses.back().get())->path_created; } - /// Write the path to the secondary node in the main node. - path = zookeeper->create(path_prefix, holder_path, zkutil::CreateMode::EphemeralSequential); - - if (path.size() <= path_prefix.size()) - throw Exception("Logical error: name of the main node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR); + return EphemeralLockInZooKeeper{path_prefix_, zookeeper_, holder_path}; } void EphemeralLockInZooKeeper::unlock() diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index 007768aea3a..276ffab5254 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -24,12 +24,14 @@ namespace ErrorCodes /// it would be simpler to allocate block numbers for all partitions in one ZK directory). class EphemeralLockInZooKeeper : public boost::noncopyable { + friend std::optional createEphemeralLockInZooKeeper( + const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); + +protected: + EphemeralLockInZooKeeper() = delete; + EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & holder_path_); + public: - EphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, Coordination::Requests * precheck_ops = nullptr); - - EphemeralLockInZooKeeper() = default; - EphemeralLockInZooKeeper(EphemeralLockInZooKeeper && rhs) noexcept { *this = std::move(rhs); @@ -90,6 +92,9 @@ private: String holder_path; }; +std::optional createEphemeralLockInZooKeeper( + const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); + /// Acquires block number locks in all partitions. class EphemeralLocksInAllPartitions : public boost::noncopyable diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0af5d75f8b0..b2b2aa816d0 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5199,14 +5199,6 @@ StorageReplicatedMergeTree::allocateBlockNumber( else zookeeper_table_path = zookeeper_path_prefix; - /// Lets check for duplicates in advance, to avoid superfluous block numbers allocation - Coordination::Requests deduplication_check_ops; - if (!zookeeper_block_id_path.empty()) - { - deduplication_check_ops.emplace_back(zkutil::makeCreateRequest(zookeeper_block_id_path, "", zkutil::CreateMode::Persistent)); - deduplication_check_ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_block_id_path, -1)); - } - String block_numbers_path = fs::path(zookeeper_table_path) / "block_numbers"; String partition_path = fs::path(block_numbers_path) / partition_id; @@ -5225,26 +5217,8 @@ StorageReplicatedMergeTree::allocateBlockNumber( zkutil::KeeperMultiException::check(code, ops, responses); } - EphemeralLockInZooKeeper lock; - /// 2 RTT - try - { - lock = EphemeralLockInZooKeeper( - fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", *zookeeper, &deduplication_check_ops); - } - catch (const zkutil::KeeperMultiException & e) - { - if (e.code == Coordination::Error::ZNODEEXISTS && e.getPathForFirstFailedOp() == zookeeper_block_id_path) - return {}; - - throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION); - } - catch (const Coordination::Exception & e) - { - throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION); - } - - return {std::move(lock)}; + return createEphemeralLockInZooKeeper( + fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", *zookeeper, zookeeper_block_id_path); } From c3935000a85f46020cbcc8d019fb2846b345ef7b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Jul 2022 13:44:32 +0200 Subject: [PATCH 360/627] Avoid reporting ZK EEXIST errors on checkPartChecksumsAndCommit --- src/Common/ZooKeeper/KeeperException.h | 4 +-- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 33 ++++++++++++--------- 3 files changed, 21 insertions(+), 18 deletions(-) diff --git a/src/Common/ZooKeeper/KeeperException.h b/src/Common/ZooKeeper/KeeperException.h index 6498aca809c..4442c41a19d 100644 --- a/src/Common/ZooKeeper/KeeperException.h +++ b/src/Common/ZooKeeper/KeeperException.h @@ -24,9 +24,7 @@ public: static void check(Coordination::Error code, const Coordination::Requests & requests, const Coordination::Responses & responses); KeeperMultiException(Coordination::Error code, const Coordination::Requests & requests, const Coordination::Responses & responses); - -private: - static size_t getFailedOpIndex(Coordination::Error code, const Coordination::Responses & responses); }; +size_t getFailedOpIndex(Coordination::Error code, const Coordination::Responses & responses); } diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 5a0be0f76ff..9346497de6a 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1227,7 +1227,7 @@ void ZooKeeper::setZooKeeperLog(std::shared_ptr zk_log_) } -size_t KeeperMultiException::getFailedOpIndex(Coordination::Error exception_code, const Coordination::Responses & responses) +size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination::Responses & responses) { if (responses.empty()) throw DB::Exception("Responses for multi transaction is empty", DB::ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b2b2aa816d0..1fe81c6e296 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1435,23 +1435,28 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd try { - zookeeper->multi(ops); - return transaction.commit(); - } - catch (const zkutil::KeeperMultiException & e) - { - size_t num_check_ops = 2 * absent_part_paths_on_replicas.size(); - size_t failed_op_index = e.failed_op_index; + Coordination::Responses responses; + Coordination::Error e = zookeeper->tryMulti(ops, responses); + if (e == Coordination::Error::ZOK) + return transaction.commit(); - if (failed_op_index < num_check_ops && e.code == Coordination::Error::ZNODEEXISTS) + if (e == Coordination::Error::ZNODEEXISTS) { - LOG_INFO(log, "The part {} on a replica suddenly appeared, will recheck checksums", e.getPathForFirstFailedOp()); - } - else - { - unlockSharedData(*part); - throw; + size_t num_check_ops = 2 * absent_part_paths_on_replicas.size(); + size_t failed_op_index = zkutil::getFailedOpIndex(e, responses); + if (failed_op_index < num_check_ops) + { + LOG_INFO(log, "The part {} on a replica suddenly appeared, will recheck checksums", ops[failed_op_index]->getPath()); + continue; + } } + + throw zkutil::KeeperException(e); + } + catch (const std::exception &) + { + unlockSharedData(*part); + throw; } } } From 9b0c7714e561d682bdd508f5c72bf92c61a3721a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Jul 2022 17:15:23 +0200 Subject: [PATCH 361/627] Style --- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1fe81c6e296..520b5534fe3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -146,7 +146,6 @@ namespace ErrorCodes extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS; extern const int PART_IS_TEMPORARILY_LOCKED; extern const int CANNOT_ASSIGN_OPTIMIZE; - extern const int KEEPER_EXCEPTION; extern const int ALL_REPLICAS_LOST; extern const int REPLICA_STATUS_CHANGED; extern const int CANNOT_ASSIGN_ALTER; From 7630fc76aebb6f6261812def7e73ea2afbae4fcc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 7 Jul 2022 17:16:42 +0200 Subject: [PATCH 362/627] avoid weird exception --- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- src/Common/ZooKeeper/ZooKeeperCommon.h | 1 + src/Coordination/KeeperDispatcher.cpp | 3 ++- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 5a0be0f76ff..6e072d273e0 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -885,7 +885,7 @@ void ZooKeeper::waitForEphemeralToDisappearIfAny(const std::string & path) if (!tryGet(path, content, nullptr, eph_node_disappeared)) return; - int32_t timeout_ms = 2 * session_timeout_ms; + int32_t timeout_ms = 3 * session_timeout_ms; if (!eph_node_disappeared->tryWait(timeout_ms)) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Ephemeral node {} still exists after {}s, probably it's owned by someone else. " diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 09f797fb47b..53fabf651fa 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -134,6 +134,7 @@ struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse OpNum getOpNum() const override { + chassert(false); throw Exception("OpNum for watch response doesn't exist", Error::ZRUNTIMEINCONSISTENCY); } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 213c924af6b..b086a277d86 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -215,7 +215,8 @@ void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKe /// Session was disconnected, just skip this response if (session_response_callback == session_to_response_callback.end()) { - LOG_TEST(log, "Cannot write response xid={}, op={}, session {} disconnected", response->xid, response->getOpNum(), session_id); + LOG_TEST(log, "Cannot write response xid={}, op={}, session {} disconnected", + response->xid, response->xid == Coordination::WATCH_XID ? "Watch" : toString(response->getOpNum()), session_id); return; } From 5f7002eafcf55edd91eb9cbd09af0a7ff6dfdab2 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Thu, 7 Jul 2022 09:08:46 -0700 Subject: [PATCH 363/627] Include boringssl header files --- contrib/thrift-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index e4ed25b27b8..3454b1b87e3 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -92,5 +92,5 @@ include_directories("${CMAKE_CURRENT_BINARY_DIR}") add_library(_thrift ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) add_library(ch_contrib::thrift ALIAS _thrift) -target_include_directories(_thrift SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src" ${CMAKE_CURRENT_BINARY_DIR}) +target_include_directories(_thrift SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src" ${CMAKE_CURRENT_BINARY_DIR} "${ClickHouse_SOURCE_DIR}/contrib/boringssl/include") target_link_libraries (_thrift PUBLIC boost::headers_only) From 1ac675e1411185a4fb945fa8cfae92f9b1ef8672 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 7 Jul 2022 16:48:38 +0000 Subject: [PATCH 364/627] Update version_date.tsv after v22.3.8.39-lts --- utils/list-versions/version_date.tsv | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 64e2050f683..9282b807800 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v22.6.3.35-stable 2022-07-06 v22.6.2.12-stable 2022-06-29 v22.6.1.1985-stable 2022-06-16 v22.5.1.2079-stable 2022-05-19 @@ -5,6 +6,7 @@ v22.4.5.9-stable 2022-05-06 v22.4.4.7-stable 2022-04-29 v22.4.3.3-stable 2022-04-26 v22.4.2.1-stable 2022-04-22 +v22.3.8.39-lts 2022-07-07 v22.3.7.28-lts 2022-06-20 v22.3.6.5-lts 2022-05-06 v22.3.5.5-lts 2022-04-29 From 2e84e22bed3556d310ae507f9a510feae8afb119 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Thu, 7 Jul 2022 17:05:11 +0000 Subject: [PATCH 365/627] Add Pinot --- benchmark/pinot/benchmark.sh | 44 +++ benchmark/pinot/local.yaml | 38 +++ benchmark/pinot/offline_table.json | 17 + benchmark/pinot/queries.sql | 43 +++ benchmark/pinot/results/c6a.4xlarge.txt | 43 +++ benchmark/pinot/run.sh | 15 + benchmark/pinot/schema.json | 437 ++++++++++++++++++++++++ benchmark/pinot/splitted.yaml | 37 ++ 8 files changed, 674 insertions(+) create mode 100755 benchmark/pinot/benchmark.sh create mode 100644 benchmark/pinot/local.yaml create mode 100644 benchmark/pinot/offline_table.json create mode 100644 benchmark/pinot/queries.sql create mode 100644 benchmark/pinot/results/c6a.4xlarge.txt create mode 100755 benchmark/pinot/run.sh create mode 100644 benchmark/pinot/schema.json create mode 100644 benchmark/pinot/splitted.yaml diff --git a/benchmark/pinot/benchmark.sh b/benchmark/pinot/benchmark.sh new file mode 100755 index 00000000000..fffcfe6702d --- /dev/null +++ b/benchmark/pinot/benchmark.sh @@ -0,0 +1,44 @@ +#!/bin/bash + +sudo apt-get update +sudo apt install openjdk-11-jdk jq -y +sudo update-alternatives --config java + +# Install + +PINOT_VERSION=0.10.0 + +wget https://downloads.apache.org/pinot/apache-pinot-$PINOT_VERSION/apache-pinot-$PINOT_VERSION-bin.tar.gz +tar -zxvf apache-pinot-$PINOT_VERSION-bin.tar.gz + +./apache-pinot-$PINOT_VERSION-bin/bin/pinot-admin.sh QuickStart -type batch & +sleep 30 +./apache-pinot-$PINOT_VERSION-bin/bin/pinot-admin.sh AddTable -tableConfigFile offline_table.json -schemaFile schema.json -exec + +# Load the data + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz + +# Pinot was unable to load data as a single file wihout any errors returned. We have to split the data +split -d --additional-suffix .tsv --verbose -n l/100 hits.tsv parts + +# Pinot can't load value '"tatuirovarki_redmond' so we need to fix this row to make it work +sed parts93.tsv -e 's "tatuirovarki_redmond tatuirovarki_redmond g' -i + +# Fix path to local directory +sed splitted.yaml 's PWD_DIR_PLACEHOLDER '$PWD' g' -i +sed local.yaml 's PWD_DIR_PLACEHOLDER '$PWD' g' -i + +# Load data +./apache-pinot-$PINOT_VERSION-bin/bin/pinot-admin.sh LaunchDataIngestionJob -jobSpecFile splitted.yaml + +# After upload it shows 94465149 rows instead of 99997497 in the dataset + +# Run the queries +./run.sh + +# stop Druid services +kill %1 + +du -bcs ./batch diff --git a/benchmark/pinot/local.yaml b/benchmark/pinot/local.yaml new file mode 100644 index 00000000000..e2bae09526f --- /dev/null +++ b/benchmark/pinot/local.yaml @@ -0,0 +1,38 @@ +executionFrameworkSpec: + name: 'standalone' + segmentGenerationJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentGenerationJobRunner' + segmentTarPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentTarPushJobRunner' + segmentUriPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentUriPushJobRunner' + segmentMetadataPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentMetadataPushJobRunner' +jobType: SegmentCreationAndTarPush +inputDirURI: 'PWD_DIR_PLACEHOLDER' +includeFileNamePattern: 'glob:PWD_DIR_PLACEHOLDER/hits.tsv' +outputDirURI: 'batch/hits/segments' +overwriteOutput: true +segmentCreationJobParallelism: 10 +pinotFSSpecs: + - scheme: file + className: org.apache.pinot.spi.filesystem.LocalPinotFS +recordReaderSpec: + dataFormat: 'csv' + className: 'org.apache.pinot.plugin.inputformat.csv.CSVRecordReader' + configClassName: 'org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig' + configs: + fileFormat: 'default' + delimiter: ' ' + multiValueDelimiter: '' + header: 'WatchID JavaEnable Title GoodEvent EventTime EventDate CounterID ClientIP RegionID UserID CounterClass OS UserAgent URL Referer IsRefresh RefererCategoryID RefererRegionID URLCategoryID URLRegionID ResolutionWidth ResolutionHeight ResolutionDepth FlashMajor FlashMinor FlashMinor2 NetMajor NetMinor UserAgentMajor UserAgentMinor CookieEnable JavascriptEnable IsMobile MobilePhone MobilePhoneModel Params IPNetworkID TraficSourceID SearchEngineID SearchPhrase AdvEngineID IsArtifical WindowClientWidth WindowClientHeight ClientTimeZone ClientEventTime SilverlightVersion1 SilverlightVersion2 SilverlightVersion3 SilverlightVersion4 PageCharset CodeVersion IsLink IsDownload IsNotBounce FUniqID OriginalURL HID IsOldCounter IsEvent IsParameter DontCountHits WithHash HitColor LocalEventTime Age Sex Income Interests Robotness RemoteIP WindowName OpenerName HistoryLength BrowserLanguage BrowserCountry SocialNetwork SocialAction HTTPError SendTiming DNSTiming ConnectTiming ResponseStartTiming ResponseEndTiming FetchTiming SocialSourceNetworkID SocialSourcePage ParamPrice ParamOrderID ParamCurrency ParamCurrencyID OpenstatServiceName OpenstatCampaignID OpenstatAdID OpenstatSourceID UTMSource UTMMedium UTMCampaign UTMContent UTMTerm FromTag HasGCLID RefererHash URLHash' + + +tableSpec: + tableName: 'hits' + schemaURI: 'http://localhost:9000/tables/hits/schema' + tableConfigURI: 'http://localhost:9000/tables/hits' + +pinotClusterSpecs: + - controllerURI: 'http://localhost:9000' + +pushJobSpec: + pushAttempts: 10 + pushRetryIntervalMillis: 1000 + segmentUriPrefix: 'file://' diff --git a/benchmark/pinot/offline_table.json b/benchmark/pinot/offline_table.json new file mode 100644 index 00000000000..da386d53d38 --- /dev/null +++ b/benchmark/pinot/offline_table.json @@ -0,0 +1,17 @@ +{ + "tableName": "hits", + "tableType": "OFFLINE", + "segmentsConfig": { + "segmentPushType": "APPEND", + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy", + "schemaName": "hitsSchema", + "replication": "1" + }, + "tableIndexConfig":{}, + "metadata":{}, + "tenants": + { + "server": "DefaultTenant", + "broker": "DefaultTenant" + } +} diff --git a/benchmark/pinot/queries.sql b/benchmark/pinot/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/pinot/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/pinot/results/c6a.4xlarge.txt b/benchmark/pinot/results/c6a.4xlarge.txt new file mode 100644 index 00000000000..aef31a6bd8e --- /dev/null +++ b/benchmark/pinot/results/c6a.4xlarge.txt @@ -0,0 +1,43 @@ +[0.002, 0.001, 0.001], +[0.176, 0.176, 0.175], +[0.237, 0.261, 0.244], +[0.449, 0.265, 0.225], +[3.691, 3.453, 3.432], +[28.785, 13.874, 14.104], +["-", "-", "-"], +[0.128, 0.127, 0.14], +[2.871, 2.741, 2.775], +[2.984, 3.034, 3.047], +[3.984, 3.965, 4.141], +[3.916, 3.896, 3.893], +[2.824, 2.971, 3.004], +[6.048, 6.505, 6.021], +[3.065, 3.165, 2.956], +[4.829, 4.749, 4.857], +[5.204, 4.987, 4.966], +[0.817, 0.809, 0.783], +["-", "-", "-"], +[0.016, 0.014, 0.014], +[51.341, 18.48, 18.07], +["-", "-", "-"], +[72.358, 70.585, 13.441], +[7.029, 5.451, 5.545], +[0.355, 0.309, 0.27], +[7.264, 0.373, 1.21], +[3.244, 0.471, 0.472], +[26.148, 2.246, 2.261], +["-", "-", "-"], +[2.098, 2.104, 2.047], +[4.668, 4.382, 4.312], +[5.329, 5.073, 4.73], +[5.002, 4.729, 5.061], +[4.988, 4.922, 4.823], +[5.886, 5.893, 6.032], +[5.591, 5.605, 5.606], +[0.246, 0.191, 0.198], +[0.344, 0.068, 0.065], +[0.04, 0.032, 0.033], +[1.401, 0.648, 0.615], +[0.107, 0.067, 0.075], +[0.04, 0.048, 0.035], +["-", "-", "-"] diff --git a/benchmark/pinot/run.sh b/benchmark/pinot/run.sh new file mode 100755 index 00000000000..9434c6ee8a7 --- /dev/null +++ b/benchmark/pinot/run.sh @@ -0,0 +1,15 @@ +#!/bin/bash + +TRIES=3 +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + echo -n "[" + for i in $(seq 1 $TRIES); do + echo "{\"sql\":\"$query option(timeoutMs=300000)\"}"| tr -d ';' > query.json + RES=$(curl -s -XPOST -H'Content-Type: application/json' http://localhost:8000/query/sql/ -d @query.json | jq 'if .exceptions == [] then .timeUsedMs/1000 else "-" end' ) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + done + echo "]," +done diff --git a/benchmark/pinot/schema.json b/benchmark/pinot/schema.json new file mode 100644 index 00000000000..cc603fb45a4 --- /dev/null +++ b/benchmark/pinot/schema.json @@ -0,0 +1,437 @@ +{ + "metricFieldSpecs": [ +{ + "name": "AdvEngineID", + "dataType": "LONG" + }, + { + "name": "ClientIP", + "dataType": "LONG" + }, + { + "name": "RegionID", + "dataType": "LONG" + }, + { + "name": "IsRefresh", + "dataType": "LONG" + }, + { + "name": "ResolutionWidth", + "dataType": "LONG" + }, + { + "name": "ResolutionHeight", + "dataType": "LONG" + }, + { + "name": "WindowClientWidth", + "dataType": "LONG" + }, + { + "name": "WindowClientHeight", + "dataType": "LONG" + }, + { + "name": "URLHash", + "dataType": "LONG" + }, + { + "name": "JavaEnable", + "dataType": "LONG" + }, + { + "name": "GoodEvent", + "dataType": "LONG" + }, + { + "name": "CounterClass", + "dataType": "LONG" + }, + { + "name": "OS", + "dataType": "LONG" + }, + { + "name": "UserAgent", + "dataType": "LONG" + }, + { + "name": "RefererCategoryID", + "dataType": "LONG" + }, + { + "name": "RefererRegionID", + "dataType": "LONG" + }, + { + "name": "URLCategoryID", + "dataType": "LONG" + }, + { + "name": "URLRegionID", + "dataType": "LONG" + }, + { + "name": "ResolutionDepth", + "dataType": "LONG" + }, + { + "name": "FlashMajor", + "dataType": "LONG" + }, + { + "name": "FlashMinor", + "dataType": "LONG" + }, + { + "name": "NetMajor", + "dataType": "LONG" + }, + { + "name": "NetMinor", + "dataType": "LONG" + }, + { + "name": "UserAgentMajor", + "dataType": "LONG" + }, + { + "name": "CookieEnable", + "dataType": "LONG" + }, + { + "name": "JavascriptEnable", + "dataType": "LONG" + }, + { + "name": "IsMobile", + "dataType": "LONG" + }, + { + "name": "MobilePhone", + "dataType": "LONG" + }, + { + "name": "IPNetworkID", + "dataType": "LONG" + }, + { + "name": "TraficSourceID", + "dataType": "LONG" + }, + { + "name": "SearchEngineID", + "dataType": "LONG" + }, + { + "name": "IsArtifical", + "dataType": "LONG" + }, + { + "name": "ClientTimeZone", + "dataType": "LONG" + }, + { + "name": "SilverlightVersion1", + "dataType": "LONG" + }, + { + "name": "SilverlightVersion2", + "dataType": "LONG" + }, + { + "name": "SilverlightVersion3", + "dataType": "LONG" + }, + { + "name": "SilverlightVersion4", + "dataType": "LONG" + }, + { + "name": "CodeVersion", + "dataType": "LONG" + }, + { + "name": "IsLink", + "dataType": "LONG" + }, + { + "name": "IsDownload", + "dataType": "LONG" + }, + { + "name": "IsNotBounce", + "dataType": "LONG" + }, + { + "name": "FUniqID", + "dataType": "LONG" + }, + { + "name": "HID", + "dataType": "LONG" + }, + { + "name": "IsOldCounter", + "dataType": "LONG" + }, + { + "name": "IsEvent", + "dataType": "LONG" + }, + { + "name": "IsParameter", + "dataType": "LONG" + }, + { + "name": "DontCountHits", + "dataType": "LONG" + }, + { + "name": "WithHash", + "dataType": "LONG" + }, + { + "name": "Age", + "dataType": "LONG" + }, + { + "name": "Sex", + "dataType": "LONG" + }, + { + "name": "Income", + "dataType": "LONG" + }, + { + "name": "Interests", + "dataType": "LONG" + }, + { + "name": "Robotness", + "dataType": "LONG" + }, + { + "name": "RemoteIP", + "dataType": "LONG" + }, + { + "name": "WindowName", + "dataType": "LONG" + }, + { + "name": "OpenerName", + "dataType": "LONG" + }, + { + "name": "HistoryLength", + "dataType": "LONG" + }, + { + "name": "HTTPError", + "dataType": "LONG" + }, + { + "name": "SendTiming", + "dataType": "LONG" + }, + { + "name": "DNSTiming", + "dataType": "LONG" + }, + { + "name": "ConnectTiming", + "dataType": "LONG" + }, + { + "name": "ResponseStartTiming", + "dataType": "LONG" + }, + { + "name": "ResponseEndTiming", + "dataType": "LONG" + }, + { + "name": "FetchTiming", + "dataType": "LONG" + }, + { + "name": "SocialSourceNetworkID", + "dataType": "LONG" + }, + { + "name": "ParamPrice", + "dataType": "LONG" + }, + { + "name": "ParamCurrencyID", + "dataType": "LONG" + }, + { + "name": "HasGCLID", + "dataType": "LONG" + }, + { + "name": "RefererHash", + "dataType": "LONG" + }, + { + "name": "CLID", + "dataType": "LONG" + } + + ], + "dimensionFieldSpecs": [ + { + "name": "CounterID", + "dataType": "LONG" + }, + { + "name": "UserID", + "dataType": "LONG" + }, + { + "name": "URL", + "dataType": "STRING" + }, + { + "name": "OpenstatServiceName", + "dataType": "STRING" + }, + { + "name": "OpenstatCampaignID", + "dataType": "STRING" + }, + { + "name": "OpenstatAdID", + "dataType": "STRING" + }, + { + "name": "UserAgentMinor", + "dataType": "STRING" + }, + { + "name": "OpenstatSourceID", + "dataType": "STRING" + }, + { + "name": "MobilePhoneModel", + "dataType": "STRING" + }, + { + "name": "Params", + "dataType": "STRING" + }, + { + "name": "UTMSource", + "dataType": "STRING" + }, + { + "name": "PageCharset", + "dataType": "STRING" + }, + { + "name": "SearchPhrase", + "dataType": "STRING" + }, + { + "name": "UTMMedium", + "dataType": "STRING" + }, + { + "name": "OriginalURL", + "dataType": "STRING" + }, + { + "name": "UTMCampaign", + "dataType": "STRING" + }, + { + "name": "ClientEventTime", + "dataType": "STRING" + }, + { + "name": "ParamOrderID", + "dataType": "STRING" + }, + { + "name": "ParamCurrency", + "dataType": "STRING" + }, + { + "name": "UTMContent", + "dataType": "STRING" + }, + { + "name": "UTMTerm", + "dataType": "STRING" + }, + { + "name": "FlashMinor2", + "dataType": "STRING" + }, + { + "name": "FromTag", + "dataType": "STRING" + }, + + { + "name": "Referer", + "dataType": "STRING" + }, + { + "name": "Title", + "dataType": "STRING" + }, + { + "name": "HitColor", + "dataType": "STRING" + }, + { + "name": "LocalEventTime", + "dataType": "STRING" + }, + { + "name": "BrowserLanguage", + "dataType": "STRING" + }, + { + "name": "SocialSourcePage", + "dataType": "STRING" + }, + { + "name": "BrowserCountry", + "dataType": "STRING" + }, + { + "name": "SocialNetwork", + "dataType": "STRING" + }, + { + "name": "SocialAction", + "dataType": "STRING" + }, + + { + "name": "WatchID", + "dataType": "LONG" + } + + ], + "dateTimeFieldSpecs": [ + { + "name": "EventTime", + "dataType": "STRING", + "format": "1:SECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss", + "granularity": "1:SECONDS" + }, + { + "name": "EventDate", + "dataType": "STRING", + "format": "1:DAYS:SIMPLE_DATE_FORMAT:yyyy-MM-dd", + "granularity": "1:DAYS" + } + ], + "schemaName": "hitsSchema" +} diff --git a/benchmark/pinot/splitted.yaml b/benchmark/pinot/splitted.yaml new file mode 100644 index 00000000000..4f2b1eec62c --- /dev/null +++ b/benchmark/pinot/splitted.yaml @@ -0,0 +1,37 @@ +executionFrameworkSpec: + name: 'standalone' + segmentGenerationJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentGenerationJobRunner' + segmentTarPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentTarPushJobRunner' + segmentUriPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentUriPushJobRunner' + segmentMetadataPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentMetadataPushJobRunner' +jobType: SegmentCreationAndTarPush +inputDirURI: 'PWD_DIR_PLACEHOLDER' +includeFileNamePattern: 'glob:PWD_DIR_PLACEHOLDER/parts??.tsv' +outputDirURI: 'batch/hits/segments' +overwriteOutput: true +segmentCreationJobParallelism: 16 +pinotFSSpecs: + - scheme: file + className: org.apache.pinot.spi.filesystem.LocalPinotFS +recordReaderSpec: + dataFormat: 'csv' + className: 'org.apache.pinot.plugin.inputformat.csv.CSVRecordReader' + configClassName: 'org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig' + configs: + fileFormat: 'default' + delimiter: ' ' + multiValueDelimiter: '' + header: 'WatchID JavaEnable Title GoodEvent EventTime EventDate CounterID ClientIP RegionID UserID CounterClass OS UserAgent URL Referer IsRefresh RefererCategoryID RefererRegionID URLCategoryID URLRegionID ResolutionWidth ResolutionHeight ResolutionDepth FlashMajor FlashMinor FlashMinor2 NetMajor NetMinor UserAgentMajor UserAgentMinor CookieEnable JavascriptEnable IsMobile MobilePhone MobilePhoneModel Params IPNetworkID TraficSourceID SearchEngineID SearchPhrase AdvEngineID IsArtifical WindowClientWidth WindowClientHeight ClientTimeZone ClientEventTime SilverlightVersion1 SilverlightVersion2 SilverlightVersion3 SilverlightVersion4 PageCharset CodeVersion IsLink IsDownload IsNotBounce FUniqID OriginalURL HID IsOldCounter IsEvent IsParameter DontCountHits WithHash HitColor LocalEventTime Age Sex Income Interests Robotness RemoteIP WindowName OpenerName HistoryLength BrowserLanguage BrowserCountry SocialNetwork SocialAction HTTPError SendTiming DNSTiming ConnectTiming ResponseStartTiming ResponseEndTiming FetchTiming SocialSourceNetworkID SocialSourcePage ParamPrice ParamOrderID ParamCurrency ParamCurrencyID OpenstatServiceName OpenstatCampaignID OpenstatAdID OpenstatSourceID UTMSource UTMMedium UTMCampaign UTMContent UTMTerm FromTag HasGCLID RefererHash URLHash' + + +tableSpec: + tableName: 'hits' + schemaURI: 'http://localhost:9000/tables/hits/schema' + tableConfigURI: 'http://localhost:9000/tables/hits' + +pinotClusterSpecs: + - controllerURI: 'http://localhost:9000' + +pushJobSpec: + pushAttempts: 10 + pushRetryIntervalMillis: 1000 From 4d4f29508140aa1e97f97c40e0e41e4889b41dc5 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 7 Jul 2022 17:27:45 +0000 Subject: [PATCH 366/627] Rmove tag no-backward-compatibility-check for specific versions --- tests/queries/0_stateless/01160_table_dependencies.sh | 1 - .../queries/0_stateless/01414_mutations_and_errors_zookeeper.sh | 1 - tests/queries/0_stateless/01825_type_json_parallel_insert.sql | 2 +- tests/queries/0_stateless/01942_create_table_with_sample.sql | 2 -- tests/queries/0_stateless/01943_non_deterministic_order_key.sql | 2 -- tests/queries/0_stateless/02227_union_match_by_name.sql | 2 -- .../0_stateless/02242_optimize_to_subcolumns_no_storage.sql | 1 - .../queries/0_stateless/02245_format_string_stack_overflow.sql | 1 - .../0_stateless/02248_nullable_custom_types_to_string.sql | 1 - .../02260_alter_compact_part_drop_nested_column.sql.j2 | 2 -- .../02268_json_wrong_root_type_in_schema_inference.sql | 2 -- .../02271_fix_column_matcher_and_column_transformer.sql | 1 - .../0_stateless/02304_grouping_sets_with_rollup_cube.sql | 2 -- .../0_stateless/02311_create_table_with_unknown_format.sql | 2 +- tests/queries/0_stateless/02324_map_combinator_bug.sql | 2 -- .../queries/0_stateless/02342_window_view_different_struct.sql | 2 -- tests/queries/0_stateless/02351_Map_combinator_dist.sql | 2 -- 17 files changed, 2 insertions(+), 26 deletions(-) diff --git a/tests/queries/0_stateless/01160_table_dependencies.sh b/tests/queries/0_stateless/01160_table_dependencies.sh index 4cfad526f24..a0a3f05c6a9 100755 --- a/tests/queries/0_stateless/01160_table_dependencies.sh +++ b/tests/queries/0_stateless/01160_table_dependencies.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-backward-compatibility-check:21.12.1.8761 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh 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 d58d57d4e52..f92ab265eb8 100755 --- a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh +++ b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh @@ -76,4 +76,3 @@ $CLICKHOUSE_CLIENT --query "ALTER TABLE replicated_mutation_table MODIFY COLUMN $CLICKHOUSE_CLIENT --query "SELECT distinct(value) FROM replicated_mutation_table ORDER BY value" -$CLICKHOUSE_CLIENT --query "DROP TABLE replicated_mutation_table" diff --git a/tests/queries/0_stateless/01825_type_json_parallel_insert.sql b/tests/queries/0_stateless/01825_type_json_parallel_insert.sql index e443c9455d5..654c0ecb3c3 100644 --- a/tests/queries/0_stateless/01825_type_json_parallel_insert.sql +++ b/tests/queries/0_stateless/01825_type_json_parallel_insert.sql @@ -1,4 +1,4 @@ --- Tags: long, no-backward-compatibility-check:22.3.2.1 +-- Tags: long DROP TABLE IF EXISTS t_json_parallel; SET allow_experimental_object_type = 1, max_insert_threads = 20, max_threads = 20, min_insert_block_size_rows = 65536; diff --git a/tests/queries/0_stateless/01942_create_table_with_sample.sql b/tests/queries/0_stateless/01942_create_table_with_sample.sql index c4ededae4ca..6320edd7a31 100644 --- a/tests/queries/0_stateless/01942_create_table_with_sample.sql +++ b/tests/queries/0_stateless/01942_create_table_with_sample.sql @@ -1,5 +1,3 @@ --- Tags: no-backward-compatibility-check:21.9.1.1 - CREATE TABLE IF NOT EXISTS sample_incorrect (`x` UUID) ENGINE = MergeTree diff --git a/tests/queries/0_stateless/01943_non_deterministic_order_key.sql b/tests/queries/0_stateless/01943_non_deterministic_order_key.sql index 8b6abebe4da..200a88ec677 100644 --- a/tests/queries/0_stateless/01943_non_deterministic_order_key.sql +++ b/tests/queries/0_stateless/01943_non_deterministic_order_key.sql @@ -1,5 +1,3 @@ --- Tags: no-backward-compatibility-check:21.9.1.1 - CREATE TABLE a (number UInt64) ENGINE = MergeTree ORDER BY if(now() > toDateTime('2020-06-01 13:31:40'), toInt64(number), -number); -- { serverError 36 } CREATE TABLE b (number UInt64) ENGINE = MergeTree ORDER BY now() > toDateTime(number); -- { serverError 36 } CREATE TABLE c (number UInt64) ENGINE = MergeTree ORDER BY now(); -- { serverError 36 } diff --git a/tests/queries/0_stateless/02227_union_match_by_name.sql b/tests/queries/0_stateless/02227_union_match_by_name.sql index 8ff33b6193d..cc0ab8ba5aa 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.sql +++ b/tests/queries/0_stateless/02227_union_match_by_name.sql @@ -1,5 +1,3 @@ --- Tags: no-backward-compatibility-check:22.5.1.2079 - -- { echo } EXPLAIN header = 1, optimize = 0 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); diff --git a/tests/queries/0_stateless/02242_optimize_to_subcolumns_no_storage.sql b/tests/queries/0_stateless/02242_optimize_to_subcolumns_no_storage.sql index 8f8485eb58f..e6e4663c5aa 100644 --- a/tests/queries/0_stateless/02242_optimize_to_subcolumns_no_storage.sql +++ b/tests/queries/0_stateless/02242_optimize_to_subcolumns_no_storage.sql @@ -1,4 +1,3 @@ --- Tags: no-backward-compatibility-check:22.3.2.1 SET optimize_functions_to_subcolumns = 1; SELECT count(*) FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 WHERE (n1.number = n2.number) AND (n2.number = n3.number); diff --git a/tests/queries/0_stateless/02245_format_string_stack_overflow.sql b/tests/queries/0_stateless/02245_format_string_stack_overflow.sql index 40053fd0d9b..1ee3606d3a6 100644 --- a/tests/queries/0_stateless/02245_format_string_stack_overflow.sql +++ b/tests/queries/0_stateless/02245_format_string_stack_overflow.sql @@ -1,2 +1 @@ --- Tags: no-backward-compatibility-check:22.3 select format('{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}', toString(number)) str from numbers(1); diff --git a/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql b/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql index 605500ee840..313f703fd03 100644 --- a/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql +++ b/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql @@ -1,4 +1,3 @@ --- Tags: no-backward-compatibility-check:22.3.4.44 select toString(toNullable(true)); select toString(CAST(NULL, 'Nullable(Bool)')); select toString(toNullable(toIPv4('0.0.0.0'))); diff --git a/tests/queries/0_stateless/02260_alter_compact_part_drop_nested_column.sql.j2 b/tests/queries/0_stateless/02260_alter_compact_part_drop_nested_column.sql.j2 index dce6c0b5b9a..57db8618f8d 100644 --- a/tests/queries/0_stateless/02260_alter_compact_part_drop_nested_column.sql.j2 +++ b/tests/queries/0_stateless/02260_alter_compact_part_drop_nested_column.sql.j2 @@ -1,5 +1,3 @@ --- Tags: no-backward-compatibility-check:22.4 - {# force compact parts and wide #} {% for min_bytes_for_wide_part in [100000, 0] %} DROP TABLE IF EXISTS compact_alter_{{ min_bytes_for_wide_part }}; diff --git a/tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.sql b/tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.sql index 2e66635a752..9d435820ce2 100644 --- a/tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.sql +++ b/tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.sql @@ -1,5 +1,3 @@ --- Tags: no-backward-compatibility-check:22.4.1.1 - insert into function file('02268_data.jsonl', 'TSV') select 1; select * from file('02268_data.jsonl'); --{serverError CANNOT_EXTRACT_TABLE_STRUCTURE} diff --git a/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql b/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql index 593fbb06599..f0c0e2bae46 100644 --- a/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql +++ b/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql @@ -1,4 +1,3 @@ --- Tags: no-backward-compatibility-check:22.4.2 DROP TABLE IF EXISTS github_events; CREATE TABLE github_events diff --git a/tests/queries/0_stateless/02304_grouping_sets_with_rollup_cube.sql b/tests/queries/0_stateless/02304_grouping_sets_with_rollup_cube.sql index 8a53ccb5121..25263edc980 100644 --- a/tests/queries/0_stateless/02304_grouping_sets_with_rollup_cube.sql +++ b/tests/queries/0_stateless/02304_grouping_sets_with_rollup_cube.sql @@ -1,5 +1,3 @@ --- Tags: no-backward-compatibility-check:22.5.1 - SELECT number FROM diff --git a/tests/queries/0_stateless/02311_create_table_with_unknown_format.sql b/tests/queries/0_stateless/02311_create_table_with_unknown_format.sql index 54e388c3cf0..51baf3a4918 100644 --- a/tests/queries/0_stateless/02311_create_table_with_unknown_format.sql +++ b/tests/queries/0_stateless/02311_create_table_with_unknown_format.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, use-hdfs, no-backward-compatibility-check:22.5 +-- Tags: no-fasttest, use-hdfs create table test_02311 (x UInt32) engine=File(UnknownFormat); -- {serverError UNKNOWN_FORMAT} create table test_02311 (x UInt32) engine=URL('http://some/url', UnknownFormat); -- {serverError UNKNOWN_FORMAT} diff --git a/tests/queries/0_stateless/02324_map_combinator_bug.sql b/tests/queries/0_stateless/02324_map_combinator_bug.sql index b4e039b95ab..aa9eefaa9ba 100644 --- a/tests/queries/0_stateless/02324_map_combinator_bug.sql +++ b/tests/queries/0_stateless/02324_map_combinator_bug.sql @@ -1,5 +1,3 @@ --- Tags: no-backward-compatibility-check:22.6 - DROP TABLE IF EXISTS segfault; DROP TABLE IF EXISTS segfault_mv; diff --git a/tests/queries/0_stateless/02342_window_view_different_struct.sql b/tests/queries/0_stateless/02342_window_view_different_struct.sql index 0b21e39d3c9..c5bf8899cae 100644 --- a/tests/queries/0_stateless/02342_window_view_different_struct.sql +++ b/tests/queries/0_stateless/02342_window_view_different_struct.sql @@ -1,5 +1,3 @@ --- Tags: no-backward-compatibility-check:22.6 - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS data_02342; diff --git a/tests/queries/0_stateless/02351_Map_combinator_dist.sql b/tests/queries/0_stateless/02351_Map_combinator_dist.sql index 30816061338..937afa5480e 100644 --- a/tests/queries/0_stateless/02351_Map_combinator_dist.sql +++ b/tests/queries/0_stateless/02351_Map_combinator_dist.sql @@ -1,5 +1,3 @@ --- Tags: no-backward-compatibility-check:22.6 - -- https://github.com/ClickHouse/ClickHouse/issues/35359 -- sumMap From 4cfc0eb7449272a6b03ad57f3f89f7cd68d97125 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 21:13:24 +0300 Subject: [PATCH 367/627] Update 02353_simdjson_buffer_overflow.sql --- tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql index 0861ae48f37..0572536e19e 100644 --- a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql +++ b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql @@ -1,4 +1,5 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-msan +-- Tag: no-msan: fuzzer can make this query very memory hungry, and under MSan, the MemoryTracker cannot account for the additional memory used by sanitizer, and OOM happens. SET max_execution_time = 3; SET timeout_overflow_mode = 'break'; From 9d6afaf0e6c60d422e190e2e6509150f4d655330 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 7 Jul 2022 21:59:01 +0200 Subject: [PATCH 368/627] add Hetzner benchmark --- website/benchmark/hardware/index.html | 1 + .../hardware/results/hetzner_ex62.json | 54 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 website/benchmark/hardware/results/hetzner_ex62.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 9c9b14b56da..05cc923a206 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -97,6 +97,7 @@ Results for AWS instance type i3.2xlarge are from Ananth Gundabattula (Da Results for 2x EPYC 7702 on ZFS mirror NVME are from Alibek A.
Results for Intel 11th Gen Core i9-11900KF are from Tim Xian.
Results for AWS instance type m5a.4xlarge are from Daniel Chimeno.
+Results for Hetzner EX62-NVME are from Talles Airan.

diff --git a/website/benchmark/hardware/results/hetzner_ex62.json b/website/benchmark/hardware/results/hetzner_ex62.json new file mode 100644 index 00000000000..2a23b3749ff --- /dev/null +++ b/website/benchmark/hardware/results/hetzner_ex62.json @@ -0,0 +1,54 @@ +[ + { + "system": "Hetzner EX62", + "system_full": "Hetzner EX62 (Intel(R) Core(TM) i9-9900K CPU @ 3.60GHz, 128 RAM DDR4 2666, NVMe)", + "time": "2022-04-28 00:00:00", + "kind": "server", + "result": + [ + [0.006, 0.001, 0.015], + [0.012, 0.033, 0.018], + [0.038, 0.030, 0.033], + [0.065, 0.044, 0.044], + [0.133, 0.117, 0.109], + [0.330, 0.281, 0.296], + [0.002, 0.001, 0.001], + [0.010, 0.011, 0.011], + [0.596, 0.564, 0.552], + [0.676, 0.629, 0.629], + [0.175, 0.154, 0.148], + [0.214, 0.190, 0.182], + [0.932, 0.891, 0.882], + [1.191, 1.131, 1.122], + [0.996, 0.877, 0.874], + [1.144, 1.062, 1.075], + [2.787, 2.687, 2.691], + [1.502, 1.455, 1.457], + [4.826, 4.703, 4.727], + [0.071, 0.050, 0.047], + [1.179, 0.934, 0.922], + [1.242, 1.021, 0.989], + [2.876, 2.389, 2.411], + [1.629, 1.050, 1.034], + [0.314, 0.254, 0.253], + [0.250, 0.231, 0.226], + [0.328, 0.269, 0.249], + [1.219, 0.975, 0.993], + [1.686, 1.517, 1.485], + [3.458, 3.455, 3.443], + [0.771, 0.654, 0.677], + [1.208, 1.067, 1.053], + [6.915, 6.738, 6.770], + [4.771, 4.656, 4.669], + [4.851, 4.701, 4.671], + [1.451, 1.341, 1.345], + [0.153, 0.126, 0.120], + [0.053, 0.056, 0.047], + [0.054, 0.046, 0.043], + [0.276, 0.250, 0.258], + [0.021, 0.018, 0.019], + [0.018, 0.015, 0.015], + [0.004, 0.003, 0.003] + ] + } +] From 7cef35da7067519a50f811737e1e9b924924b3d9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 7 Jul 2022 20:05:14 +0000 Subject: [PATCH 369/627] Update version_date.tsv after v22.4.6.53-stable --- utils/list-versions/version_date.tsv | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 64e2050f683..de9e4cfa0af 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,10 +1,14 @@ +v22.6.3.35-stable 2022-07-06 v22.6.2.12-stable 2022-06-29 v22.6.1.1985-stable 2022-06-16 +v22.5.2.53-stable 2022-07-07 v22.5.1.2079-stable 2022-05-19 +v22.4.6.53-stable 2022-07-07 v22.4.5.9-stable 2022-05-06 v22.4.4.7-stable 2022-04-29 v22.4.3.3-stable 2022-04-26 v22.4.2.1-stable 2022-04-22 +v22.3.8.39-lts 2022-07-07 v22.3.7.28-lts 2022-06-20 v22.3.6.5-lts 2022-05-06 v22.3.5.5-lts 2022-04-29 From c45482d6c6566987b5f284d01650497f85cecde8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 7 Jul 2022 21:46:17 +0200 Subject: [PATCH 370/627] disable instrumentation of sanitizer death callback --- base/base/defines.h | 1 + src/Daemon/BaseDaemon.cpp | 7 ++++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/base/base/defines.h b/base/base/defines.h index a707e965675..5a646f4dca2 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -93,6 +93,7 @@ # define NO_SANITIZE_ADDRESS __attribute__((__no_sanitize__("address"))) # define NO_SANITIZE_THREAD __attribute__((__no_sanitize__("thread"))) # define ALWAYS_INLINE_NO_SANITIZE_UNDEFINED __attribute__((__always_inline__, __no_sanitize__("undefined"))) +# define DISABLE_SANITIZER_INSTRUMENTATION __attribute__((disable_sanitizer_instrumentation)) #else /// It does not work in GCC. GCC 7 cannot recognize this attribute and GCC 8 simply ignores it. # define NO_SANITIZE_UNDEFINED # define NO_SANITIZE_ADDRESS diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 445aa4463bd..23835df87ea 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -395,7 +395,12 @@ private: #if defined(SANITIZER) extern "C" void __sanitizer_set_death_callback(void (*)()); -static void sanitizerDeathCallback() +/// Sanitizers may not expect some function calls from death callback. +/// Let's try to disable instrumentation to avoid possible issues. +/// However, this callback may call other functions that are still instrumented. +/// We can try [[clang::always_inline]] attribute for statements in future (available in clang-15) +/// See https://github.com/google/sanitizers/issues/1543 and https://github.com/google/sanitizers/issues/1549. +static DISABLE_SANITIZER_INSTRUMENTATION void sanitizerDeathCallback() { DENY_ALLOCATIONS_IN_SCOPE; /// Also need to send data via pipe. Otherwise it may lead to deadlocks or failures in printing diagnostic info. From 49348b833a49cbe101bcbbd2c0714f3c7f108443 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 7 Jul 2022 20:25:26 +0000 Subject: [PATCH 371/627] Simplify --- src/Functions/MultiMatchAllIndicesImpl.h | 17 +++------- src/Functions/MultiMatchAnyImpl.h | 34 ++++++-------------- src/Functions/MultiSearchFirstIndexImpl.h | 33 ++++++++----------- src/Functions/MultiSearchFirstPositionImpl.h | 33 ++++++++----------- src/Functions/MultiSearchImpl.h | 33 ++++++++----------- 5 files changed, 54 insertions(+), 96 deletions(-) diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index 1a8d4439be4..c6a8a477648 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -166,26 +166,19 @@ struct MultiMatchAllIndicesImpl #if USE_VECTORSCAN offsets.resize(haystack_offsets.size()); size_t prev_haystack_offset = 0; + size_t prev_needles_offset = 0; const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); - const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); - const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); std::vector needles; - size_t prev_needles_offsets_offset = 0; - size_t prev_needles_data_offset = 0; - for (size_t i = 0; i < haystack_offsets.size(); ++i) { - needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); + needles.reserve(needles_offsets[i] - prev_needles_offset); - for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; - auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; - needles.emplace_back(std::string_view(p, sz)); - prev_needles_data_offset = needles_data_string_offsets[j]; + needles.emplace_back(needles_data_string->getDataAt(j).toView()); } checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); @@ -230,7 +223,7 @@ struct MultiMatchAllIndicesImpl offsets[i] = res.size(); prev_haystack_offset = haystack_offsets[i]; - prev_needles_offsets_offset = needles_offsets[i]; + prev_needles_offset = needles_offsets[i]; needles.clear(); } #else diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index e0ca9e0cc03..38b1e37eca5 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -193,26 +193,19 @@ struct MultiMatchAnyImpl res.resize(haystack_offsets.size()); #if USE_VECTORSCAN size_t prev_haystack_offset = 0; + size_t prev_needles_offset = 0; const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); - const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); - const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); std::vector needles; - size_t prev_needles_offsets_offset = 0; - size_t prev_needles_data_offset = 0; - for (size_t i = 0; i < haystack_offsets.size(); ++i) { - needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); + needles.reserve(needles_offsets[i] - prev_needles_offset); - for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; - auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; - needles.emplace_back(std::string_view(p, sz)); - prev_needles_data_offset = needles_data_string_offsets[j]; + needles.emplace_back(needles_data_string->getDataAt(j).toView()); } checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); @@ -260,7 +253,7 @@ struct MultiMatchAnyImpl throw Exception("Failed to scan with vectorscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT); prev_haystack_offset = haystack_offsets[i]; - prev_needles_offsets_offset = needles_offsets[i]; + prev_needles_offset = needles_offsets[i]; needles.clear(); } #else @@ -277,29 +270,22 @@ struct MultiMatchAnyImpl memset(res.data(), 0, res.size() * sizeof(res.front())); size_t prev_haystack_offset = 0; + size_t prev_needles_offset = 0; const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); - const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); - const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); std::vector needles; - size_t prev_needles_offsets_offset = 0; - size_t prev_needles_data_offset = 0; - for (size_t i = 0; i < haystack_offsets.size(); ++i) { const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset]; const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; - needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); + needles.reserve(needles_offsets[i] - prev_needles_offset); - for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; - auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; - needles.emplace_back(std::string_view(p, sz)); - prev_needles_data_offset = needles_data_string_offsets[j]; + needles.emplace_back(needles_data_string->getDataAt(j).toView()); } checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); @@ -381,7 +367,7 @@ struct MultiMatchAnyImpl } prev_haystack_offset = haystack_offsets[i]; - prev_needles_offsets_offset = needles_offsets[i]; + prev_needles_offset = needles_offsets[i]; needles.clear(); } #endif // USE_VECTORSCAN diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 06003c7f31f..f108f2ca41e 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -53,15 +53,15 @@ struct MultiSearchFirstIndexImpl size_t iteration = 0; while (searcher.hasMoreToSearch()) { - size_t prev_offset = 0; + size_t prev_haystack_offset = 0; for (size_t j = 0; j < haystack_size; ++j) { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + const auto * haystack = &haystack_data[prev_haystack_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset - 1; /// hasMoreToSearch traverse needles in increasing order if (iteration == 0 || res[j] == 0) res[j] = searcher.searchOneFirstIndex(haystack, haystack_end); - prev_offset = haystack_offsets[j]; + prev_haystack_offset = haystack_offsets[j]; } ++iteration; } @@ -83,33 +83,26 @@ struct MultiSearchFirstIndexImpl const size_t haystack_size = haystack_offsets.size(); res.resize(haystack_size); - size_t prev_offset = 0; + size_t prev_haystack_offset = 0; + size_t prev_needles_offset = 0; const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); - const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); - const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); std::vector needles; - size_t prev_needles_offsets_offset = 0; - size_t prev_needles_data_offset = 0; - for (size_t i = 0; i < haystack_size; ++i) { - needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); + needles.reserve(needles_offsets[i] - prev_needles_offset); - for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; - auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; - needles.emplace_back(std::string_view(p, sz)); - prev_needles_data_offset = needles_data_string_offsets[j]; + needles.emplace_back(needles_data_string->getDataAt(j).toView()); } auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal - const auto * const haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[i] - prev_offset - 1; + const auto * const haystack = &haystack_data[prev_haystack_offset]; + const auto * haystack_end = haystack + haystack_offsets[i] - prev_haystack_offset - 1; size_t iteration = 0; while (searcher.hasMoreToSearch()) @@ -125,8 +118,8 @@ struct MultiSearchFirstIndexImpl res[i] = 0; } - prev_offset = haystack_offsets[i]; - prev_needles_offsets_offset = needles_offsets[i]; + prev_haystack_offset = haystack_offsets[i]; + prev_needles_offset = needles_offsets[i]; needles.clear(); } } diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index 0b81ce88065..69cab478d13 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -57,11 +57,11 @@ struct MultiSearchFirstPositionImpl size_t iteration = 0; while (searcher.hasMoreToSearch()) { - size_t prev_offset = 0; + size_t prev_haystack_offset = 0; for (size_t j = 0; j < haystack_size; ++j) { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + const auto * haystack = &haystack_data[prev_haystack_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset - 1; if (iteration == 0 || res[j] == 0) res[j] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); else @@ -70,7 +70,7 @@ struct MultiSearchFirstPositionImpl if (result != 0) res[j] = std::min(result, res[j]); } - prev_offset = haystack_offsets[j]; + prev_haystack_offset = haystack_offsets[j]; } ++iteration; } @@ -92,17 +92,13 @@ struct MultiSearchFirstPositionImpl const size_t haystack_size = haystack_offsets.size(); res.resize(haystack_size); - size_t prev_offset = 0; + size_t prev_haystack_offset = 0; + size_t prev_needles_offset = 0; const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); - const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); - const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); std::vector needles; - size_t prev_needles_offsets_offset = 0; - size_t prev_needles_data_offset = 0; - auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 { return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); @@ -110,20 +106,17 @@ struct MultiSearchFirstPositionImpl for (size_t i = 0; i < haystack_size; ++i) { - needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); + needles.reserve(needles_offsets[i] - prev_needles_offset); - for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; - auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; - needles.emplace_back(std::string_view(p, sz)); - prev_needles_data_offset = needles_data_string_offsets[j]; + needles.emplace_back(needles_data_string->getDataAt(j).toView()); } auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal - const auto * const haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[i] - prev_offset - 1; + const auto * const haystack = &haystack_data[prev_haystack_offset]; + const auto * haystack_end = haystack + haystack_offsets[i] - prev_haystack_offset - 1; size_t iteration = 0; while (searcher.hasMoreToSearch()) @@ -147,8 +140,8 @@ struct MultiSearchFirstPositionImpl res[i] = 0; } - prev_offset = haystack_offsets[i]; - prev_needles_offsets_offset = needles_offsets[i]; + prev_haystack_offset = haystack_offsets[i]; + prev_needles_offset = needles_offsets[i]; needles.clear(); } } diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index ad789004db0..b9ce0293234 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -53,14 +53,14 @@ struct MultiSearchImpl size_t iteration = 0; while (searcher.hasMoreToSearch()) { - size_t prev_offset = 0; + size_t prev_haystack_offset = 0; for (size_t j = 0; j < haystack_size; ++j) { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + const auto * haystack = &haystack_data[prev_haystack_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset - 1; if (iteration == 0 || !res[j]) res[j] = searcher.searchOne(haystack, haystack_end); - prev_offset = haystack_offsets[j]; + prev_haystack_offset = haystack_offsets[j]; } ++iteration; } @@ -82,31 +82,24 @@ struct MultiSearchImpl const size_t haystack_size = haystack_offsets.size(); res.resize(haystack_size); - size_t prev_offset = 0; + size_t prev_haystack_offset = 0; + size_t prev_needles_offset = 0; const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); - const ColumnString::Offsets & needles_data_string_offsets = needles_data_string->getOffsets(); - const ColumnString::Chars & needles_data_string_chars = needles_data_string->getChars(); std::vector needles; - size_t prev_needles_offsets_offset = 0; - size_t prev_needles_data_offset = 0; - for (size_t i = 0; i < haystack_size; ++i) { - needles.reserve(needles_offsets[i] - prev_needles_offsets_offset); + needles.reserve(needles_offsets[i] - prev_needles_offset); - for (size_t j = prev_needles_offsets_offset; j < needles_offsets[i]; ++j) + for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - const auto * p = reinterpret_cast(needles_data_string_chars.data()) + prev_needles_data_offset; - auto sz = needles_data_string_offsets[j] - prev_needles_data_offset - 1; - needles.emplace_back(std::string_view(p, sz)); - prev_needles_data_offset = needles_data_string_offsets[j]; + needles.emplace_back(needles_data_string->getDataAt(j).toView()); } - const auto * const haystack = &haystack_data[prev_offset]; - const size_t haystack_length = haystack_offsets[i] - prev_offset - 1; + const auto * const haystack = &haystack_data[prev_haystack_offset]; + const size_t haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; size_t iteration = 0; for (const auto & needle : needles) @@ -122,8 +115,8 @@ struct MultiSearchImpl if (iteration == 0) res[i] = 0; - prev_offset = haystack_offsets[i]; - prev_needles_offsets_offset = needles_offsets[i]; + prev_haystack_offset = haystack_offsets[i]; + prev_needles_offset = needles_offsets[i]; needles.clear(); } } From 4d2aa0bb768f4cf2b9e4088f3acce7bbf66d96db Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 7 Jul 2022 22:28:31 +0200 Subject: [PATCH 372/627] add ryzen 9 5950 benchmark --- website/benchmark/hardware/index.html | 2 + .../hardware/results/amd_ryzen_9_5950x.json | 54 +++++++++++++++++++ 2 files changed, 56 insertions(+) create mode 100644 website/benchmark/hardware/results/amd_ryzen_9_5950x.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 9c9b14b56da..31ea61d5ffd 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -97,6 +97,8 @@ Results for AWS instance type i3.2xlarge are from Ananth Gundabattula (Da Results for 2x EPYC 7702 on ZFS mirror NVME are from Alibek A.
Results for Intel 11th Gen Core i9-11900KF are from Tim Xian.
Results for AWS instance type m5a.4xlarge are from Daniel Chimeno.
+Results for AMD Ryzen 9 5950X are from Stefan.
+

diff --git a/website/benchmark/hardware/results/amd_ryzen_9_5950x.json b/website/benchmark/hardware/results/amd_ryzen_9_5950x.json new file mode 100644 index 00000000000..0f7f4194960 --- /dev/null +++ b/website/benchmark/hardware/results/amd_ryzen_9_5950x.json @@ -0,0 +1,54 @@ +[ + { + "system": "AMD Ryzen 9 (2022)", + "system_full": " AMD Ryzen 9 5950X 16-Core Processor, 125Gi RAM", + "time": "2022-05-03 00:00:00", + "kind": "desktop", + "result": + [ + [0.001, 0.001, 0.010], + [0.008, 0.006, 0.006], + [0.017, 0.014, 0.014], + [0.039, 0.022, 0.028], + [0.074, 0.064, 0.074], + [0.150, 0.128, 0.133], + [0.001, 0.001, 0.001], + [0.006, 0.007, 0.006], + [0.215, 0.195, 0.197], + [0.247, 0.221, 0.221], + [0.100, 0.089, 0.098], + [0.112, 0.111, 0.111], + [0.467, 0.445, 0.444], + [0.588, 0.563, 0.554], + [0.521, 0.482, 0.485], + [0.503, 0.616, 0.619], + [1.465, 1.636, 1.630], + [0.907, 0.881, 0.893], + [3.401, 2.832, 2.822], + [0.040, 0.025, 0.022], + [0.465, 0.322, 0.309], + [0.501, 0.356, 0.358], + [1.207, 0.944, 0.938], + [0.869, 0.415, 0.401], + [0.143, 0.101, 0.099], + [0.101, 0.090, 0.090], + [0.139, 0.096, 0.096], + [0.467, 0.332, 0.317], + [0.558, 0.468, 0.464], + [2.288, 2.128, 2.058], + [0.322, 0.285, 0.283], + [0.768, 0.545, 0.537], + [4.126, 4.078, 4.155], + [2.730, 2.511, 2.510], + [2.658, 2.536, 2.566], + [0.877, 0.732, 0.747], + [0.096, 0.085, 0.082], + [0.038, 0.035, 0.036], + [0.038, 0.034, 0.034], + [0.228, 0.218, 0.219], + [0.014, 0.014, 0.016], + [0.016, 0.013, 0.010], + [0.003, 0.006, 0.003] + ] + } +] \ No newline at end of file From c403ade4a7f98b293635686461d4aacb8570af3b Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 7 Jul 2022 22:52:16 +0200 Subject: [PATCH 373/627] add macbook pro core i7 2014 benchmark --- website/benchmark/hardware/index.html | 1 + .../results/macbook_pro_core_i7_2014.json | 54 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 website/benchmark/hardware/results/macbook_pro_core_i7_2014.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 9c9b14b56da..256a90b61a3 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -97,6 +97,7 @@ Results for AWS instance type i3.2xlarge are from Ananth Gundabattula (Da Results for 2x EPYC 7702 on ZFS mirror NVME are from Alibek A.
Results for Intel 11th Gen Core i9-11900KF are from Tim Xian.
Results for AWS instance type m5a.4xlarge are from Daniel Chimeno.
+Results for Macbook Pro Intel Core i7 (2014) are from Vladislav.

diff --git a/website/benchmark/hardware/results/macbook_pro_core_i7_2014.json b/website/benchmark/hardware/results/macbook_pro_core_i7_2014.json new file mode 100644 index 00000000000..2c861028698 --- /dev/null +++ b/website/benchmark/hardware/results/macbook_pro_core_i7_2014.json @@ -0,0 +1,54 @@ +[ + { + "system": "MacBook Pro 2014", + "system_full": "MacBook Pro 2014, .5 GHz Quad-Core Intel Core i7, 16 GiB RAM", + "time": "2020-04-04 00:00:00", + "kind": "laptop", + "result": + [ + [0.030, 0.003, 0.003], + [0.078, 0.020, 0.020], + [0.176, 0.056, 0.056], + [0.358, 0.082, 0.082], + [0.451, 0.254, 0.208], + [0.887, 0.582, 0.583], + [0.040, 0.004, 0.002], + [0.079, 0.023, 0.024], + [1.213, 1.100, 1.109], + [1.839, 1.250, 1.529], + [0.590, 0.304, 0.370], + [0.645, 0.502, 0.489], + [1.793, 1.418, 1.531], + [2.803, 1.953, 2.333], + [2.101, 1.871, 1.718], + [1.875, 1.508, 1.493], + [5.053, 5.682, 5.334], + [3.484, 4.643, 3.188], + [10.762, 10.994, 10.125], + [0.517, 0.241, 0.166], + [3.898, 1.701, 1.828], + [4.394, 2.155, 1.987], + [8.082, 4.622, 5.137], + [6.218, 2.413, 2.131], + [1.099, 0.531, 0.550], + [0.766, 0.436, 0.712], + [1.094, 0.585, 0.559], + [4.207, 1.628, 1.818], + [3.969, 2.775, 2.579], + [2.604, 2.441, 2.449], + [1.773, 1.262, 1.165], + [3.059, 1.803, 1.833], + [19.756, 17.851, 13.698], + [10.651, 8.640, 7.184], + [10.125, 8.230, 7.775], + [2.865, 2.256, 2.196], + [0.292, 0.226, 0.249], + [0.194, 0.084, 0.070], + [0.162, 0.063, 0.064], + [0.515, 0.404, 0.423], + [0.127, 0.024, 0.025], + [0.099, 0.021, 0.018], + [0.045, 0.007, 0.007] + ] + } +] From 002be94d2714c04b5836ce0dcb8862252a6eecef Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 7 Jul 2022 23:15:19 +0200 Subject: [PATCH 374/627] add Huawie TaiShan 920 Benchmark --- website/benchmark/hardware/index.html | 1 + .../hardware/results/huawei_taishan_920.json | 54 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 website/benchmark/hardware/results/huawei_taishan_920.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 9c9b14b56da..ffa75dc4617 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -97,6 +97,7 @@ Results for AWS instance type i3.2xlarge are from Ananth Gundabattula (Da Results for 2x EPYC 7702 on ZFS mirror NVME are from Alibek A.
Results for Intel 11th Gen Core i9-11900KF are from Tim Xian.
Results for AWS instance type m5a.4xlarge are from Daniel Chimeno.
+Results for Huawei Taishan 920 are from Yu ZiChange at EioTek.

diff --git a/website/benchmark/hardware/results/huawei_taishan_920.json b/website/benchmark/hardware/results/huawei_taishan_920.json new file mode 100644 index 00000000000..61a37d338d5 --- /dev/null +++ b/website/benchmark/hardware/results/huawei_taishan_920.json @@ -0,0 +1,54 @@ +[ + { + "system": "Huawei TaiShan", + "system_full": "Huawei TaiShan Kunpeng 920, 96 vCPU, 256GB RAM", + "time": "2022-06-18 00:00:00", + "kind": "server", + "result": + [ + [0.018, 0.001, 0.002], + [0.106, 0.019, 0.019], + [0.835, 0.027, 0.026], + [3.314, 0.031, 0.030], + [1.034, 0.130, 0.131], + [3.221, 0.166, 0.166], + [0.002, 0.002, 0.002], + [0.061, 0.023, 0.022], + [3.989, 0.191, 0.189], + [3.233, 0.215, 0.213], + [2.774, 0.112, 0.106], + [1.999, 0.111, 0.109], + [3.377, 0.259, 0.262], + [5.308, 0.324, 0.326], + [3.505, 0.267, 0.272], + [2.382, 0.316, 0.311], + [5.633, 0.648, 0.656], + [5.505, 0.446, 0.449], + [8.911, 1.509, 1.487], + [2.116, 0.042, 0.032], + [21.597, 0.308, 0.287], + [28.341, 0.347, 0.354], + [53.912, 0.907, 0.901], + [52.521, 1.923, 0.904], + [7.845, 0.100, 0.100], + [3.213, 0.082, 0.083], + [6.985, 0.102, 0.099], + [21.502, 0.404, 0.406], + [20.771, 0.505, 0.498], + [0.739, 0.666, 0.664], + [6.025, 0.228, 0.224], + [12.865, 0.410, 0.408], + [10.248, 2.236, 2.222], + [21.545, 1.226, 1.232], + [26.602, 1.218, 1.247], + [1.696, 0.435, 0.431], + [0.353, 0.232, 0.227], + [0.193, 0.103, 0.100], + [0.305, 0.086, 0.087], + [0.640, 0.476, 0.483], + [0.190, 0.028, 0.030], + [0.137, 0.023, 0.024], + [0.064, 0.005, 0.005] + ] + } +] From 8981c38ad6fbf694fc11ef723d738a5e47ff5daf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 00:25:21 +0300 Subject: [PATCH 375/627] Update macbook_pro_core_i7_2014.json --- .../benchmark/hardware/results/macbook_pro_core_i7_2014.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/benchmark/hardware/results/macbook_pro_core_i7_2014.json b/website/benchmark/hardware/results/macbook_pro_core_i7_2014.json index 2c861028698..011401ed3e5 100644 --- a/website/benchmark/hardware/results/macbook_pro_core_i7_2014.json +++ b/website/benchmark/hardware/results/macbook_pro_core_i7_2014.json @@ -1,7 +1,7 @@ [ { "system": "MacBook Pro 2014", - "system_full": "MacBook Pro 2014, .5 GHz Quad-Core Intel Core i7, 16 GiB RAM", + "system_full": "MacBook Pro 2014, 2.5 GHz Quad-Core Intel Core i7, 16 GiB RAM", "time": "2020-04-04 00:00:00", "kind": "laptop", "result": From 551b0e62ca992204dc6d08a90f56811a4bd6551f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 00:41:45 +0300 Subject: [PATCH 376/627] Update build-cross-arm.md --- docs/en/development/build-cross-arm.md | 32 ++------------------------ 1 file changed, 2 insertions(+), 30 deletions(-) diff --git a/docs/en/development/build-cross-arm.md b/docs/en/development/build-cross-arm.md index ed968ee8e77..346fa909567 100644 --- a/docs/en/development/build-cross-arm.md +++ b/docs/en/development/build-cross-arm.md @@ -5,34 +5,6 @@ sidebar_label: Build on Linux for AARCH64 (ARM64) # How to Build ClickHouse on Linux for AARCH64 (ARM64) Architecture -This is for the case when you have Linux machine and want to use it to build `clickhouse` binary that will run on another Linux machine with AARCH64 CPU architecture. -This is intended for continuous integration checks that run on Linux servers. +If you use AArch64 machine and want to build ClickHouse for AArch64, build as usual. -The cross-build for AARCH64 is based on the [Build instructions](../development/build.md), follow them first. - -## Install Clang-14 or newer - -Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup or do -``` -sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" -``` - -## Install Cross-Compilation Toolset {#install-cross-compilation-toolset} - -``` bash -cd ClickHouse -mkdir -p build-aarch64/cmake/toolchain/linux-aarch64 -wget 'https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.03/binrel/gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz?revision=2e88a73f-d233-4f96-b1f4-d8b36e9bb0b9&la=en' -O gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz -tar xJf gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz -C build-aarch64/cmake/toolchain/linux-aarch64 --strip-components=1 -``` - -## Build ClickHouse {#build-clickhouse} - -``` bash -cd ClickHouse -mkdir build-arm64 -CC=clang-14 CXX=clang++-14 cmake . -Bbuild-arm64 -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-aarch64.cmake -ninja -C build-arm64 -``` - -The resulting binary will run only on Linux with the AARCH64 CPU architecture. +If you use x86_64 machine and want cross-compile for AArch64, add the following flag to `cmake`: `-DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-aarch64.cmake` From 60a126cd12a7bcb1f0018a452be862b48a5e6255 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 06:15:34 +0200 Subject: [PATCH 377/627] Add README, 70% --- benchmark/README.md | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/benchmark/README.md b/benchmark/README.md index 94ba78914d1..19c93dec8f9 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -131,6 +131,35 @@ The results can be used for comparison of various systems, but always take them We allow but do not recommend creating scoreboards from this benchmark or tell that one system is better (faster, cheaper, etc) than another. +There is a web page to navigate across benchmark results and present a summary report. It allows to filter out some systems, setups, or queries. E.g. if you found some subset of the 43 queries irrelevant, you can simply exclude them from the calculation and share the report without these queries. + +You can select the summary metric from one of the following: "Cold Run", "Hot Run", "Load Time", and "Data Size". If you select the "Load Time" or "Data Size", the entries will be simply ordered from best to worst, and additionally, the ratio to the best non-zero result will be shown (how times one system is worse than the best system in this metric). Load time can be zero for stateless query engines like `clickhouse-local` or `Amazon Athena`. + +If you select "Cold Run" or "Hot Run", the aggregation across the queries is performed in the following way: + +1. For Cold Run, the first run for every query is selected. For Hot Run, the minimum from 2nd and 3rd run time is selected, if both runs are successful, or null if some were unsuccessful. + +By default, the "Hot Run" metric is selected, because it's not always possible to obtain a cold runtime for managed services, while for on-premise a quite slow EBS volume is used by default which makes the comparison slightly less interesting. + +2. For every query, find a system that demonstrated the best (fastest) query time and take it as a baseline. + +This gives us a point of comparison. Alternatively, we can take a benchmark entry like "ClickHouse on c6a.metal" as a baseline and divide all queries time by the times from a baseline. But it would be quite arbitrary and asymmetric. Instead, we take the best result for every query separately. + +3. For every query, if the result is present, calculate the ratio to the baseline, but add constant 10ms to the nominator and denominator, so the formula will be: `(10ms + query_time) / (10ms + baseline_query_time)`. This formula gives a value >= 1, which is equal to 1 for the best benchmark entry on this query. + +We are interested in relative query run times, not absolute. The benchmark has a broad set of queries, and there can be queries that typically run in 100ms (e.g. for interactive dashboards) and some queries that typically run in a minute (e.g. complex ad-hoc queries). And we want to treat these queries equally important in the benchmark, that's why we need relative values. + +The constant shift is needed to make the formula well-defined when query time approaches zero. E.g. some systems can get query results in 0 ms using table metadata lookup, and another in 10 ms by range scan. But this should not be treated as the infinite advantage of one system over the other. With the constant shift, we will treat it as only two times an advantage. + +4. For every query, if the result is not present, substitute it with a "penalty" calculated as follows: take the maximum query runtime for this benchmark entry across other queries that have a result, but if it is less than 300 seconds, put it 300 seconds. Then multiply the value by 2. Then calculate the ratio as explained above. + +For example, one system is immature and crashed while trying to run a query. Or does not run a query due to limitations. If this system shows run times like 1..1000 sec. on other queries, we will substitute 2000 sec. instead of this missing result. + +5. Take the geometric mean of the ratios across the queries. It will be the summary rating. + +Why geometric mean? The ratios can only be naturally averaged in this way. Imagine there are two queries and two systems. The first system ran the first query in 1s and the second query in 20s. The second system ran the first query in 2s and the second query in 10s. So, the first system is two times faster on the first query and two times slower on the second query and vice-versa. The final score should be identical for these systems. + + ## History and Motivation The benchmark has been created in October 2013 for evaluating various DBMS to use for a web analytics system. It has been made by taking a 1/50th of one week of production pageviews (a.k.a. "hits") data and taking the first one billion, one hundred million, and ten million records from it. It has been run on a 3-node cluster of Xeon E2650v2 with 128 GiB RAM, 8x6TB HDD in md-RAID-6, and 10 Gbit network in a private datacenter in Finland. From 24927f86fc78af24d912c33848d3caa9ba36fca4 Mon Sep 17 00:00:00 2001 From: HeenaBansal2009 Date: Thu, 7 Jul 2022 21:22:00 -0700 Subject: [PATCH 378/627] Fix flaky test --- .../02297_regex_parsing_file_names.sh | 34 +++++++++---------- 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/02297_regex_parsing_file_names.sh b/tests/queries/0_stateless/02297_regex_parsing_file_names.sh index 2db4ae8044c..43f4166f86b 100755 --- a/tests/queries/0_stateless/02297_regex_parsing_file_names.sh +++ b/tests/queries/0_stateless/02297_regex_parsing_file_names.sh @@ -13,26 +13,24 @@ CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/ -echo '{"obj": "aaa", "id": 1, "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_0.json -echo '{"id": 2, "obj": "bbb", "s": "bar"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_1.json -echo '{"id": 3, "obj": "ccc", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_2.json -echo '{"id": 4, "obj": "ddd", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_3.json -echo '{"id": 5, "obj": "eee", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_4.json -echo '{"id": 6, "obj": "fff", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_5.json -echo '{"id": 7, "obj": "ggg", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_6.json -echo '{"id": 8, "obj": "hhh", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_7.json -echo '{"id": 9, "obj": "iii", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_8.json -echo '{"id": 10, "obj":"jjj", "s": "foo"}' >> ${CLICKHOUSE_USER_FILES_PATH}/file_9.json -echo '{"id": 11, "obj": "kkk", "s": "foo"}'>> ${CLICKHOUSE_USER_FILES_PATH}/file_10.json +echo '{"obj": "aaa", "id": 1, "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_0.json +echo '{"id": 2, "obj": "bbb", "s": "bar"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_1.json +echo '{"id": 3, "obj": "ccc", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_2.json +echo '{"id": 4, "obj": "ddd", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_3.json +echo '{"id": 5, "obj": "eee", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_4.json +echo '{"id": 6, "obj": "fff", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_5.json +echo '{"id": 7, "obj": "ggg", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_6.json +echo '{"id": 8, "obj": "hhh", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_7.json +echo '{"id": 9, "obj": "iii", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_8.json +echo '{"id": 10, "obj":"jjj", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_9.json +echo '{"id": 11, "obj": "kkk", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_10.json +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_regex;" -${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_regex" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_regex (id UInt64, obj String, s String) ENGINE = MergeTree() order by id;" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_regex (id UInt64, obj String, s String) ENGINE =File(JSONEachRow)" ; - - -${CLICKHOUSE_CLIENT} -q "INSERT INTO t_regex SELECT * FROM file('file_{0..10}.json','JSONEachRow')"; -${CLICKHOUSE_CLIENT} -q "SELECT count() from t_regex" +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_regex SELECT * FROM file('file_{0..10}.json','JSONEachRow');" +${CLICKHOUSE_CLIENT} -q "SELECT count() from t_regex;" rm -rf ${CLICKHOUSE_USER_FILES_PATH}/file_*.json; -${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_regex" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_regex;" From 08635d9d9589e0844b2a569fdc51e01750c8dc2c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 06:30:25 +0200 Subject: [PATCH 379/627] Add Pinot --- benchmark/README.md | 4 +- benchmark/pinot/results/c6a.4xlarge.json | 58 ++++++++++++++++++++++++ benchmark/pinot/results/c6a.4xlarge.txt | 43 ------------------ 3 files changed, 60 insertions(+), 45 deletions(-) create mode 100644 benchmark/pinot/results/c6a.4xlarge.json delete mode 100644 benchmark/pinot/results/c6a.4xlarge.txt diff --git a/benchmark/README.md b/benchmark/README.md index 19c93dec8f9..ae9e7894920 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -149,7 +149,7 @@ This gives us a point of comparison. Alternatively, we can take a benchmark entr We are interested in relative query run times, not absolute. The benchmark has a broad set of queries, and there can be queries that typically run in 100ms (e.g. for interactive dashboards) and some queries that typically run in a minute (e.g. complex ad-hoc queries). And we want to treat these queries equally important in the benchmark, that's why we need relative values. -The constant shift is needed to make the formula well-defined when query time approaches zero. E.g. some systems can get query results in 0 ms using table metadata lookup, and another in 10 ms by range scan. But this should not be treated as the infinite advantage of one system over the other. With the constant shift, we will treat it as only two times an advantage. +The constant shift is needed to make the formula well-defined when query time approaches zero. E.g. some systems can get query results in 0 ms using table metadata lookup, and another in 10 ms by range scan. But this should not be treated as the infinite advantage of one system over the other. With the constant shift, we will treat it as only two times an advantage. 4. For every query, if the result is not present, substitute it with a "penalty" calculated as follows: take the maximum query runtime for this benchmark entry across other queries that have a result, but if it is less than 300 seconds, put it 300 seconds. Then multiply the value by 2. Then calculate the ratio as explained above. @@ -210,7 +210,7 @@ We also introduced the [Hardware Benchmark](https://clickhouse.com/benchmark/har - [x] Databend - [ ] Doris/PALO - [x] Druid -- [ ] Pinot +- [x] Pinot - [x] CrateDB - [ ] Spark SQL - [ ] Starrocks diff --git a/benchmark/pinot/results/c6a.4xlarge.json b/benchmark/pinot/results/c6a.4xlarge.json new file mode 100644 index 00000000000..2edf14c920e --- /dev/null +++ b/benchmark/pinot/results/c6a.4xlarge.json @@ -0,0 +1,58 @@ +{ + "system": "Pinot", + "date": "2022-07-01", + "machine": "c6a.4xlarge, 500gb gp2", + "cluster_size": 1, + "comment": "It successfully loaded only 94465149 out of 99997497 records. Some queries returned NullPointerException. The loading process is painful - splitting to 100 pieces required. It does not correctly report errors on data loading, the results may be incorrect.", + + "tags": ["SQL", "Java", "column-oriented"], + + "load_time": 2032, + "data_size": 9245538172, + + "result": [ +[0.002, 0.001, 0.001], +[0.186, 0.186, 0.185], +[0.251, 0.276, 0.258], +[0.475, 0.281, 0.238], +[3.907, 3.655, 3.633], +[30.471, 14.687, 14.93], +[null, null, null], +[0.135, 0.134, 0.148], +[3.039, 2.902, 2.938], +[3.159, 3.212, 3.225], +[4.217, 4.197, 4.384], +[4.145, 4.124, 4.121], +[2.989, 3.145, 3.18], +[6.402, 6.886, 6.374], +[3.245, 3.35, 3.129], +[5.112, 5.027, 5.141], +[5.509, 5.279, 5.257], +[0.865, 0.856, 0.829], +[null, null, null], +[0.017, 0.015, 0.015], +[54.348, 19.562, 19.128], +[null, null, null], +[76.596, 74.719, 14.228], +[7.441, 5.77, 5.87], +[0.376, 0.327, 0.286], +[7.689, 0.395, 1.281], +[3.434, 0.499, 0.5], +[27.679, 2.378, 2.393], +[null, null, null], +[2.221, 2.227, 2.167], +[4.941, 4.639, 4.565], +[5.641, 5.37, 5.007], +[5.295, 5.006, 5.357], +[5.28, 5.21, 5.105], +[6.231, 6.238, 6.385], +[5.918, 5.933, 5.934], +[0.26, 0.202, 0.21], +[0.364, 0.072, 0.069], +[0.042, 0.034, 0.035], +[1.483, 0.686, 0.651], +[0.113, 0.071, 0.079], +[0.042, 0.051, 0.037], +[null, null, null] +] +} diff --git a/benchmark/pinot/results/c6a.4xlarge.txt b/benchmark/pinot/results/c6a.4xlarge.txt deleted file mode 100644 index aef31a6bd8e..00000000000 --- a/benchmark/pinot/results/c6a.4xlarge.txt +++ /dev/null @@ -1,43 +0,0 @@ -[0.002, 0.001, 0.001], -[0.176, 0.176, 0.175], -[0.237, 0.261, 0.244], -[0.449, 0.265, 0.225], -[3.691, 3.453, 3.432], -[28.785, 13.874, 14.104], -["-", "-", "-"], -[0.128, 0.127, 0.14], -[2.871, 2.741, 2.775], -[2.984, 3.034, 3.047], -[3.984, 3.965, 4.141], -[3.916, 3.896, 3.893], -[2.824, 2.971, 3.004], -[6.048, 6.505, 6.021], -[3.065, 3.165, 2.956], -[4.829, 4.749, 4.857], -[5.204, 4.987, 4.966], -[0.817, 0.809, 0.783], -["-", "-", "-"], -[0.016, 0.014, 0.014], -[51.341, 18.48, 18.07], -["-", "-", "-"], -[72.358, 70.585, 13.441], -[7.029, 5.451, 5.545], -[0.355, 0.309, 0.27], -[7.264, 0.373, 1.21], -[3.244, 0.471, 0.472], -[26.148, 2.246, 2.261], -["-", "-", "-"], -[2.098, 2.104, 2.047], -[4.668, 4.382, 4.312], -[5.329, 5.073, 4.73], -[5.002, 4.729, 5.061], -[4.988, 4.922, 4.823], -[5.886, 5.893, 6.032], -[5.591, 5.605, 5.606], -[0.246, 0.191, 0.198], -[0.344, 0.068, 0.065], -[0.04, 0.032, 0.033], -[1.401, 0.648, 0.615], -[0.107, 0.067, 0.075], -[0.04, 0.048, 0.035], -["-", "-", "-"] From a14a585f8e408362fd4e3e6083626a52f3da2cd7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 06:42:56 +0200 Subject: [PATCH 380/627] Update generator --- benchmark/generate-results.sh | 25 ++++++++++---- benchmark/index.html | 61 ++++++++++++++++++++++++++++++++++- 2 files changed, 79 insertions(+), 7 deletions(-) diff --git a/benchmark/generate-results.sh b/benchmark/generate-results.sh index a7d35538717..87d67bcc81c 100755 --- a/benchmark/generate-results.sh +++ b/benchmark/generate-results.sh @@ -1,7 +1,20 @@ -#!/bin/bash +#!/bin/bash -e -ls -1 */results/*.json | while read file -do - cat "${file}" - echo ',' -done +( + sed '/^const data = \[$/q' index.html + + FIRST=1 + ls -1 */results/*.json | while read file + do + [ "${FIRST}" = "0" ] && echo ',' + cat "${file}" + FIRST=0 + done + + echo ']; // end of data' + sed '0,/^\]; \/\/ end of data$/d' index.html + +) > index.html.new + +mv index.html index.html.bak +mv index.html.new index.html diff --git a/benchmark/index.html b/benchmark/index.html index bbede6b0e08..895fe581d76 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -1438,6 +1438,65 @@ const data = [ ] } , +{ + "system": "Pinot", + "date": "2022-07-01", + "machine": "c6a.4xlarge, 500gb gp2", + "cluster_size": 1, + "comment": "It successfully loaded only 94465149 out of 99997497 records. Some queries returned NullPointerException. The loading process is painful - splitting to 100 pieces required. It does not correctly report errors on data loading, the results may be incorrect.", + + "tags": ["SQL", "Java", "column-oriented"], + + "load_time": 2032, + "data_size": 9245538172, + + "result": [ +[0.002, 0.001, 0.001], +[0.186, 0.186, 0.185], +[0.251, 0.276, 0.258], +[0.475, 0.281, 0.238], +[3.907, 3.655, 3.633], +[30.471, 14.687, 14.93], +[null, null, null], +[0.135, 0.134, 0.148], +[3.039, 2.902, 2.938], +[3.159, 3.212, 3.225], +[4.217, 4.197, 4.384], +[4.145, 4.124, 4.121], +[2.989, 3.145, 3.18], +[6.402, 6.886, 6.374], +[3.245, 3.35, 3.129], +[5.112, 5.027, 5.141], +[5.509, 5.279, 5.257], +[0.865, 0.856, 0.829], +[null, null, null], +[0.017, 0.015, 0.015], +[54.348, 19.562, 19.128], +[null, null, null], +[76.596, 74.719, 14.228], +[7.441, 5.77, 5.87], +[0.376, 0.327, 0.286], +[7.689, 0.395, 1.281], +[3.434, 0.499, 0.5], +[27.679, 2.378, 2.393], +[null, null, null], +[2.221, 2.227, 2.167], +[4.941, 4.639, 4.565], +[5.641, 5.37, 5.007], +[5.295, 5.006, 5.357], +[5.28, 5.21, 5.105], +[6.231, 6.238, 6.385], +[5.918, 5.933, 5.934], +[0.26, 0.202, 0.21], +[0.364, 0.072, 0.069], +[0.042, 0.034, 0.035], +[1.483, 0.686, 0.651], +[0.113, 0.071, 0.079], +[0.042, 0.051, 0.037], +[null, null, null] +] +} +, { "system": "PostgreSQL", "date": "2022-07-01", @@ -2440,7 +2499,7 @@ const data = [ [0.106915,0.077122,0.076134] ] } -]; +]; // end of data From 504941dcb43d825c7e6b7a109058df82e6b42b35 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 06:47:52 +0200 Subject: [PATCH 381/627] Remove bogus tag --- benchmark/athena/results/partitioned.json | 2 +- benchmark/athena/results/single.json | 2 +- benchmark/aurora-mysql/results/16acu.json | 2 +- .../aurora-postgresql/results/16acu.json | 2 +- benchmark/citus/results/c6a.4xlarge.json | 2 +- .../results/c6a.4xlarge.partitioned.json | 2 +- .../results/c6a.4xlarge.single.json | 2 +- benchmark/clickhouse/results/c6a.4xlarge.json | 2 +- benchmark/clickhouse/results/c6a.metal.json | 2 +- benchmark/cratedb/results/c6a.4xlarge.json | 2 +- benchmark/databend/results/c6a.4xlarge.json | 2 +- benchmark/druid/results/c6a.4xlarge.json | 2 +- benchmark/duckdb/results/c6a.4xlarge.json | 2 +- benchmark/greenplum/results/c6a.4xlarge.json | 2 +- benchmark/heavyai/results/c6a.4xlarge.json | 2 +- benchmark/index.html.bak | 2999 +++++++++++++++++ benchmark/infobright/results/c6a.4xlarge.json | 2 +- .../results/c6a.4xlarge.json | 2 +- benchmark/monetdb/results/c6a.4xlarge.json | 2 +- .../mysql-myisam/results/c6a.4xlarge.json | 2 +- benchmark/mysql/results/c6a.4xlarge.json | 2 +- benchmark/pinot/results/c6a.4xlarge.json | 2 +- benchmark/postgresql/results/c6a.4xlarge.json | 2 +- benchmark/questdb/results/c6a.4xlarge.json | 2 +- .../results/serverless.json | 2 +- benchmark/redshift/results/4x.ra3.xplus.json | 2 +- .../singlestore/results/c6a.4xlarge.json | 2 +- benchmark/snowflake/results/2xl.json | 2 +- benchmark/snowflake/results/3xl.json | 2 +- benchmark/snowflake/results/4xl.json | 2 +- benchmark/snowflake/results/l.json | 2 +- benchmark/snowflake/results/m.json | 2 +- benchmark/snowflake/results/s.json | 2 +- benchmark/snowflake/results/xl.json | 2 +- benchmark/snowflake/results/xs.json | 2 +- benchmark/sqlite/results/c6a.4xlarge.json | 2 +- .../results/c6a.4xlarge.compression.json | 2 +- .../timescaledb/results/c6a.4xlarge.json | 2 +- 38 files changed, 3036 insertions(+), 37 deletions(-) create mode 100644 benchmark/index.html.bak diff --git a/benchmark/athena/results/partitioned.json b/benchmark/athena/results/partitioned.json index dc244aa60c3..8a67c09bb47 100644 --- a/benchmark/athena/results/partitioned.json +++ b/benchmark/athena/results/partitioned.json @@ -5,7 +5,7 @@ "cluster_size": "serverless", "comment": "", - "tags": ["stateless", "managed", "SQL", "Java", "column-oriented"], + "tags": ["stateless", "managed", "Java", "column-oriented"], "load_time": 0, "data_size": 13800000000, diff --git a/benchmark/athena/results/single.json b/benchmark/athena/results/single.json index e9f56ef5c5b..d2b84187f11 100644 --- a/benchmark/athena/results/single.json +++ b/benchmark/athena/results/single.json @@ -5,7 +5,7 @@ "cluster_size": "serverless", "comment": "", - "tags": ["stateless", "managed", "SQL", "Java", "column-oriented"], + "tags": ["stateless", "managed", "Java", "column-oriented"], "load_time": 0, "data_size": 13800000000, diff --git a/benchmark/aurora-mysql/results/16acu.json b/benchmark/aurora-mysql/results/16acu.json index 12abda3b307..ef00d213842 100644 --- a/benchmark/aurora-mysql/results/16acu.json +++ b/benchmark/aurora-mysql/results/16acu.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "Some queries cannot run due to ERROR 1114 (HY000) at line 1: The table '/rdsdbdata/tmp/#sqlaff_e5_0' is full", - "tags": ["managed", "SQL", "C++", "MySQL compatible", "row-oriented"], + "tags": ["managed", "C++", "MySQL compatible", "row-oriented"], "load_time": 7687.318, "data_size": 89614492631, diff --git a/benchmark/aurora-postgresql/results/16acu.json b/benchmark/aurora-postgresql/results/16acu.json index a18f4c54271..194329d7f4f 100644 --- a/benchmark/aurora-postgresql/results/16acu.json +++ b/benchmark/aurora-postgresql/results/16acu.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["managed", "SQL", "C", "PostgreSQL compatible", "row-oriented"], + "tags": ["managed", "C", "PostgreSQL compatible", "row-oriented"], "load_time": 2127, "data_size": 52183852646, diff --git a/benchmark/citus/results/c6a.4xlarge.json b/benchmark/citus/results/c6a.4xlarge.json index fab86db8ca6..23d170aa869 100644 --- a/benchmark/citus/results/c6a.4xlarge.json +++ b/benchmark/citus/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "PostgreSQL compatible", "column-oriented"], + "tags": ["C", "PostgreSQL compatible", "column-oriented"], "load_time": 1579, "data_size": 18980918899, diff --git a/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json b/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json index 34c5687d902..c44114d12c1 100644 --- a/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json +++ b/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], + "tags": ["C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], "load_time": 0, "data_size": 14737666736, diff --git a/benchmark/clickhouse-local/results/c6a.4xlarge.single.json b/benchmark/clickhouse-local/results/c6a.4xlarge.single.json index a90e2442946..8d19a6d0ce4 100644 --- a/benchmark/clickhouse-local/results/c6a.4xlarge.single.json +++ b/benchmark/clickhouse-local/results/c6a.4xlarge.single.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], + "tags": ["C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], "load_time": 0, "data_size": 14779976446, diff --git a/benchmark/clickhouse/results/c6a.4xlarge.json b/benchmark/clickhouse/results/c6a.4xlarge.json index df5693a2367..4bc9508ea6e 100644 --- a/benchmark/clickhouse/results/c6a.4xlarge.json +++ b/benchmark/clickhouse/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented", "ClickHouse derivative"], + "tags": ["C++", "column-oriented", "ClickHouse derivative"], "load_time": 475.529, "data_size": 14345515782, diff --git a/benchmark/clickhouse/results/c6a.metal.json b/benchmark/clickhouse/results/c6a.metal.json index 27738fcae00..92727d881b8 100644 --- a/benchmark/clickhouse/results/c6a.metal.json +++ b/benchmark/clickhouse/results/c6a.metal.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented", "ClickHouse derivative"], + "tags": ["C++", "column-oriented", "ClickHouse derivative"], "load_time": 136.869, "data_size": 14571706777, diff --git a/benchmark/cratedb/results/c6a.4xlarge.json b/benchmark/cratedb/results/c6a.4xlarge.json index b558c24ebe7..0e469b6ca27 100644 --- a/benchmark/cratedb/results/c6a.4xlarge.json +++ b/benchmark/cratedb/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "For some queries it gives \"Data too large\".", - "tags": ["SQL", "Java", "column-oriented"], + "tags": ["Java", "column-oriented"], "load_time": 10687, "data_size": 109636633416, diff --git a/benchmark/databend/results/c6a.4xlarge.json b/benchmark/databend/results/c6a.4xlarge.json index a820a90803b..f0c678f210f 100644 --- a/benchmark/databend/results/c6a.4xlarge.json +++ b/benchmark/databend/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "Only 90% of data successfully loaded. For some queries it gives \"Data too large\".", - "tags": ["SQL", "Rust", "column-oriented", "ClickHouse derivative"], + "tags": ["Rust", "column-oriented", "ClickHouse derivative"], "load_time": 484, "data_size": 43016643271, diff --git a/benchmark/druid/results/c6a.4xlarge.json b/benchmark/druid/results/c6a.4xlarge.json index acc8c215a0d..a52d1afecb6 100644 --- a/benchmark/druid/results/c6a.4xlarge.json +++ b/benchmark/druid/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "Java", "column-oriented"], + "tags": ["Java", "column-oriented"], "load_time": 19620, "data_size": 45188608472, diff --git a/benchmark/duckdb/results/c6a.4xlarge.json b/benchmark/duckdb/results/c6a.4xlarge.json index a59ca62863e..40b9d88bc71 100644 --- a/benchmark/duckdb/results/c6a.4xlarge.json +++ b/benchmark/duckdb/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "Many queries triggered OOM", - "tags": ["SQL", "C", "column-oriented", "embedded"], + "tags": ["C", "column-oriented", "embedded"], "load_time": 4217, "data_size": 27241492480, diff --git a/benchmark/greenplum/results/c6a.4xlarge.json b/benchmark/greenplum/results/c6a.4xlarge.json index 1b6ddb22ec9..68052fdd12b 100644 --- a/benchmark/greenplum/results/c6a.4xlarge.json +++ b/benchmark/greenplum/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "column-oriented", "PostgreSQL compatible"], + "tags": ["C", "column-oriented", "PostgreSQL compatible"], "load_time": 1080, "data_size": 42000000000, diff --git a/benchmark/heavyai/results/c6a.4xlarge.json b/benchmark/heavyai/results/c6a.4xlarge.json index 5c84f71fc24..61307fec311 100644 --- a/benchmark/heavyai/results/c6a.4xlarge.json +++ b/benchmark/heavyai/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "Previous names: OmniSci, mapD. Many queries cannot run due to errors and limitations.", - "tags": ["SQL", "C++", "column-oriented"], + "tags": ["C++", "column-oriented"], "load_time": 572.633, "data_size": 50887437386, diff --git a/benchmark/index.html.bak b/benchmark/index.html.bak new file mode 100644 index 00000000000..895fe581d76 --- /dev/null +++ b/benchmark/index.html.bak @@ -0,0 +1,2999 @@ + + + + + ClickBench — a Benchmark For Analytical DBMS + + + + + + + + + + +
+

ClickBench — a Benchmark For Analytical DBMS

+ Methodology | Reproduce and Validate the Results | Add a System | Report Mistake | Hardware Benchmark +
+ + + + + + + + + + + + + + + + + + + + + + +
System: + All +
Type: + All +
Machine: + All +
Cluster size: + All +
Metric: + Cold Run + Hot Run + Load Time + Storage Size +
+ + + + + + + + + + +
+ System & Machine + + Relative time (lower is better) +
+ +
Nothing selected
+ +
+

Detailed Comparison

+
+ + + + + + + + +
+ +
+ + + + diff --git a/benchmark/infobright/results/c6a.4xlarge.json b/benchmark/infobright/results/c6a.4xlarge.json index 11fbac77e56..c9157d0b101 100644 --- a/benchmark/infobright/results/c6a.4xlarge.json +++ b/benchmark/infobright/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "Only 90% of data successfully loaded. Some queries run for days.", - "tags": ["SQL", "C++", "column-oriented", "MySQL compatible"], + "tags": ["C++", "column-oriented", "MySQL compatible"], "load_time": 2317, "data_size": 13760341294, diff --git a/benchmark/mariadb-columnstore/results/c6a.4xlarge.json b/benchmark/mariadb-columnstore/results/c6a.4xlarge.json index 7650be5d91d..e318b64da41 100644 --- a/benchmark/mariadb-columnstore/results/c6a.4xlarge.json +++ b/benchmark/mariadb-columnstore/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "Previous name: InfiniDB.", - "tags": ["SQL", "C++", "column-oriented", "MySQL compatible"], + "tags": ["C++", "column-oriented", "MySQL compatible"], "load_time": 2507.8, "data_size": 19712857022, diff --git a/benchmark/monetdb/results/c6a.4xlarge.json b/benchmark/monetdb/results/c6a.4xlarge.json index d1e7c592303..eecb21b37d7 100644 --- a/benchmark/monetdb/results/c6a.4xlarge.json +++ b/benchmark/monetdb/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "column-oriented"], + "tags": ["C", "column-oriented"], "load_time": 939, "data_size": 49696606499, diff --git a/benchmark/mysql-myisam/results/c6a.4xlarge.json b/benchmark/mysql-myisam/results/c6a.4xlarge.json index f937f393614..b4493f1fb4a 100644 --- a/benchmark/mysql-myisam/results/c6a.4xlarge.json +++ b/benchmark/mysql-myisam/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "row-oriented", "MySQL compatible"], + "tags": ["C++", "row-oriented", "MySQL compatible"], "load_time": 2512, "data_size": 121588958061, diff --git a/benchmark/mysql/results/c6a.4xlarge.json b/benchmark/mysql/results/c6a.4xlarge.json index 36caf0fb646..18b8509215f 100644 --- a/benchmark/mysql/results/c6a.4xlarge.json +++ b/benchmark/mysql/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "row-oriented", "MySQL compatible"], + "tags": ["C++", "row-oriented", "MySQL compatible"], "load_time": 9472, "data_size": 171953585825, diff --git a/benchmark/pinot/results/c6a.4xlarge.json b/benchmark/pinot/results/c6a.4xlarge.json index 2edf14c920e..04855bd604a 100644 --- a/benchmark/pinot/results/c6a.4xlarge.json +++ b/benchmark/pinot/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "It successfully loaded only 94465149 out of 99997497 records. Some queries returned NullPointerException. The loading process is painful - splitting to 100 pieces required. It does not correctly report errors on data loading, the results may be incorrect.", - "tags": ["SQL", "Java", "column-oriented"], + "tags": ["Java", "column-oriented"], "load_time": 2032, "data_size": 9245538172, diff --git a/benchmark/postgresql/results/c6a.4xlarge.json b/benchmark/postgresql/results/c6a.4xlarge.json index 7e1d3baef6e..0a0a6731021 100644 --- a/benchmark/postgresql/results/c6a.4xlarge.json +++ b/benchmark/postgresql/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "row-oriented", "PostgreSQL compatible"], + "tags": ["C", "row-oriented", "PostgreSQL compatible"], "load_time": 2342, "data_size": 77797067741, diff --git a/benchmark/questdb/results/c6a.4xlarge.json b/benchmark/questdb/results/c6a.4xlarge.json index e8a00b03e5c..95b9c382464 100644 --- a/benchmark/questdb/results/c6a.4xlarge.json +++ b/benchmark/questdb/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "Many queries cannot run. It also crashes and hangs.", - "tags": ["SQL", "Java", "time-series"], + "tags": ["Java", "time-series"], "load_time": 1667.5, "data_size": 126680518680, diff --git a/benchmark/redshift-serverless/results/serverless.json b/benchmark/redshift-serverless/results/serverless.json index 53180668e67..3ba94e542dc 100644 --- a/benchmark/redshift-serverless/results/serverless.json +++ b/benchmark/redshift-serverless/results/serverless.json @@ -5,7 +5,7 @@ "cluster_size": "serverless", "comment": "One query did not run due to overflow check and another due to missing regexp function.", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 1933, "data_size": 30300000000, diff --git a/benchmark/redshift/results/4x.ra3.xplus.json b/benchmark/redshift/results/4x.ra3.xplus.json index c83558f210b..fecc515e475 100644 --- a/benchmark/redshift/results/4x.ra3.xplus.json +++ b/benchmark/redshift/results/4x.ra3.xplus.json @@ -5,7 +5,7 @@ "cluster_size": 4, "comment": "One query did not run due to overflow check and another due to missing regexp function.", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2136, "data_size": 30794579968, diff --git a/benchmark/singlestore/results/c6a.4xlarge.json b/benchmark/singlestore/results/c6a.4xlarge.json index e432c1917dc..d434ddafdb1 100644 --- a/benchmark/singlestore/results/c6a.4xlarge.json +++ b/benchmark/singlestore/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "Previous name: MemSQL. Some queries did not run due to memory limits", - "tags": ["SQL", "MySQL compatible", "column-oriented"], + "tags": ["MySQL compatible", "column-oriented"], "load_time": 690, "data_size": 29836263469, diff --git a/benchmark/snowflake/results/2xl.json b/benchmark/snowflake/results/2xl.json index 4eca3044467..294eb66e65e 100644 --- a/benchmark/snowflake/results/2xl.json +++ b/benchmark/snowflake/results/2xl.json @@ -5,7 +5,7 @@ "cluster_size": 32, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, diff --git a/benchmark/snowflake/results/3xl.json b/benchmark/snowflake/results/3xl.json index 70b58170934..841d570b249 100644 --- a/benchmark/snowflake/results/3xl.json +++ b/benchmark/snowflake/results/3xl.json @@ -5,7 +5,7 @@ "cluster_size": 64, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, diff --git a/benchmark/snowflake/results/4xl.json b/benchmark/snowflake/results/4xl.json index 7c57c0616af..48c671277d4 100644 --- a/benchmark/snowflake/results/4xl.json +++ b/benchmark/snowflake/results/4xl.json @@ -5,7 +5,7 @@ "cluster_size": 128, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, diff --git a/benchmark/snowflake/results/l.json b/benchmark/snowflake/results/l.json index b61b7108d99..05fd7793646 100644 --- a/benchmark/snowflake/results/l.json +++ b/benchmark/snowflake/results/l.json @@ -5,7 +5,7 @@ "cluster_size": 8, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, diff --git a/benchmark/snowflake/results/m.json b/benchmark/snowflake/results/m.json index a126e975d62..fb98b643c9b 100644 --- a/benchmark/snowflake/results/m.json +++ b/benchmark/snowflake/results/m.json @@ -5,7 +5,7 @@ "cluster_size": 4, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, diff --git a/benchmark/snowflake/results/s.json b/benchmark/snowflake/results/s.json index 58296761a6a..7a686b95204 100644 --- a/benchmark/snowflake/results/s.json +++ b/benchmark/snowflake/results/s.json @@ -5,7 +5,7 @@ "cluster_size": 2, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, diff --git a/benchmark/snowflake/results/xl.json b/benchmark/snowflake/results/xl.json index 6bc08cae044..9b417b24a42 100644 --- a/benchmark/snowflake/results/xl.json +++ b/benchmark/snowflake/results/xl.json @@ -5,7 +5,7 @@ "cluster_size": 16, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, diff --git a/benchmark/snowflake/results/xs.json b/benchmark/snowflake/results/xs.json index 36e0e78aa91..32fbfeb0dff 100644 --- a/benchmark/snowflake/results/xs.json +++ b/benchmark/snowflake/results/xs.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, diff --git a/benchmark/sqlite/results/c6a.4xlarge.json b/benchmark/sqlite/results/c6a.4xlarge.json index f21fd3d2424..e442dcadc80 100644 --- a/benchmark/sqlite/results/c6a.4xlarge.json +++ b/benchmark/sqlite/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "embedded", "row-oriented"], + "tags": ["C", "embedded", "row-oriented"], "load_time": 2608, "data_size": 75776589824, diff --git a/benchmark/timescaledb/results/c6a.4xlarge.compression.json b/benchmark/timescaledb/results/c6a.4xlarge.compression.json index f28f1cf4f44..457038eb591 100644 --- a/benchmark/timescaledb/results/c6a.4xlarge.compression.json +++ b/benchmark/timescaledb/results/c6a.4xlarge.compression.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "PostgreSQL compatible", "column-oriented", "time-series"], + "tags": ["C", "PostgreSQL compatible", "column-oriented", "time-series"], "load_time": 4605, "data_size": 20333747165, diff --git a/benchmark/timescaledb/results/c6a.4xlarge.json b/benchmark/timescaledb/results/c6a.4xlarge.json index f928b8e83e8..63d57a74975 100644 --- a/benchmark/timescaledb/results/c6a.4xlarge.json +++ b/benchmark/timescaledb/results/c6a.4xlarge.json @@ -5,7 +5,7 @@ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "PostgreSQL compatible", "row-oriented", "time-series"], + "tags": ["C", "PostgreSQL compatible", "row-oriented", "time-series"], "load_time": 1620, "data_size": 72882392030, From 9c0ae5a824f194ffe7ae2c83e7545df7838e55df Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 07:20:45 +0200 Subject: [PATCH 382/627] Correct Pinot result --- benchmark/generate-results.sh | 3 + benchmark/index.html | 89 +++++++++++++----------- benchmark/pinot/results/c6a.4xlarge.json | 2 +- 3 files changed, 51 insertions(+), 43 deletions(-) diff --git a/benchmark/generate-results.sh b/benchmark/generate-results.sh index 87d67bcc81c..833f2a88935 100755 --- a/benchmark/generate-results.sh +++ b/benchmark/generate-results.sh @@ -1,5 +1,8 @@ #!/bin/bash -e +# This script will substitute the benchmark results into the HTML page. +# Note: editing HTML with sed may look strange, but at least we avoid using node.js and npm, and that's good. + ( sed '/^const data = \[$/q' index.html diff --git a/benchmark/index.html b/benchmark/index.html index 895fe581d76..90fbf47ec40 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -206,7 +206,7 @@ const data = [ "cluster_size": "serverless", "comment": "", - "tags": ["stateless", "managed", "SQL", "Java", "column-oriented"], + "tags": ["stateless", "managed", "Java", "column-oriented"], "load_time": 0, "data_size": 13800000000, @@ -265,7 +265,7 @@ const data = [ "cluster_size": "serverless", "comment": "", - "tags": ["stateless", "managed", "SQL", "Java", "column-oriented"], + "tags": ["stateless", "managed", "Java", "column-oriented"], "load_time": 0, "data_size": 13800000000, @@ -324,7 +324,7 @@ const data = [ "cluster_size": 1, "comment": "Some queries cannot run due to ERROR 1114 (HY000) at line 1: The table '/rdsdbdata/tmp/#sqlaff_e5_0' is full", - "tags": ["managed", "SQL", "C++", "MySQL compatible", "row-oriented"], + "tags": ["managed", "C++", "MySQL compatible", "row-oriented"], "load_time": 7687.318, "data_size": 89614492631, @@ -383,7 +383,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["managed", "SQL", "C", "PostgreSQL compatible", "row-oriented"], + "tags": ["managed", "C", "PostgreSQL compatible", "row-oriented"], "load_time": 2127, "data_size": 52183852646, @@ -442,7 +442,7 @@ const data = [ "cluster_size": "serverless", "comment": "Looks like it shows uncompressed (billable) data size, and it is higher than CSV size", - "tags": ["managed", "SQL", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 519.485, "data_size": 94410000000, @@ -501,7 +501,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "PostgreSQL compatible", "column-oriented"], + "tags": ["C", "PostgreSQL compatible", "column-oriented"], "load_time": 1579, "data_size": 18980918899, @@ -560,7 +560,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], + "tags": ["C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], "load_time": 0, "data_size": 14737666736, @@ -619,7 +619,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], + "tags": ["C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], "load_time": 0, "data_size": 14779976446, @@ -678,7 +678,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented", "ClickHouse derivative"], + "tags": ["C++", "column-oriented", "ClickHouse derivative"], "load_time": 475.529, "data_size": 14345515782, @@ -737,7 +737,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "column-oriented", "ClickHouse derivative"], + "tags": ["C++", "column-oriented", "ClickHouse derivative"], "load_time": 136.869, "data_size": 14571706777, @@ -796,7 +796,7 @@ const data = [ "cluster_size": 1, "comment": "For some queries it gives \"Data too large\".", - "tags": ["SQL", "Java", "column-oriented"], + "tags": ["Java", "column-oriented"], "load_time": 10687, "data_size": 109636633416, @@ -855,7 +855,7 @@ const data = [ "cluster_size": 1, "comment": "Only 90% of data successfully loaded. For some queries it gives \"Data too large\".", - "tags": ["SQL", "Rust", "column-oriented", "ClickHouse derivative"], + "tags": ["Rust", "column-oriented", "ClickHouse derivative"], "load_time": 484, "data_size": 43016643271, @@ -914,7 +914,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "Java", "column-oriented"], + "tags": ["Java", "column-oriented"], "load_time": 19620, "data_size": 45188608472, @@ -973,7 +973,7 @@ const data = [ "cluster_size": 1, "comment": "Many queries triggered OOM", - "tags": ["SQL", "C", "column-oriented", "embedded"], + "tags": ["C", "column-oriented", "embedded"], "load_time": 4217, "data_size": 27241492480, @@ -1032,7 +1032,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "column-oriented", "PostgreSQL compatible"], + "tags": ["C", "column-oriented", "PostgreSQL compatible"], "load_time": 1080, "data_size": 42000000000, @@ -1091,7 +1091,7 @@ const data = [ "cluster_size": 1, "comment": "Previous names: OmniSci, mapD. Many queries cannot run due to errors and limitations.", - "tags": ["SQL", "C++", "column-oriented"], + "tags": ["C++", "column-oriented"], "load_time": 572.633, "data_size": 50887437386, @@ -1150,7 +1150,7 @@ const data = [ "cluster_size": 1, "comment": "Only 90% of data successfully loaded. Some queries run for days.", - "tags": ["SQL", "C++", "column-oriented", "MySQL compatible"], + "tags": ["C++", "column-oriented", "MySQL compatible"], "load_time": 2317, "data_size": 13760341294, @@ -1209,7 +1209,7 @@ const data = [ "cluster_size": 1, "comment": "Previous name: InfiniDB.", - "tags": ["SQL", "C++", "column-oriented", "MySQL compatible"], + "tags": ["C++", "column-oriented", "MySQL compatible"], "load_time": 2507.8, "data_size": 19712857022, @@ -1268,7 +1268,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "column-oriented"], + "tags": ["C", "column-oriented"], "load_time": 939, "data_size": 49696606499, @@ -1327,7 +1327,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "row-oriented", "MySQL compatible"], + "tags": ["C++", "row-oriented", "MySQL compatible"], "load_time": 2512, "data_size": 121588958061, @@ -1386,7 +1386,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C++", "row-oriented", "MySQL compatible"], + "tags": ["C++", "row-oriented", "MySQL compatible"], "load_time": 9472, "data_size": 171953585825, @@ -1445,10 +1445,10 @@ const data = [ "cluster_size": 1, "comment": "It successfully loaded only 94465149 out of 99997497 records. Some queries returned NullPointerException. The loading process is painful - splitting to 100 pieces required. It does not correctly report errors on data loading, the results may be incorrect.", - "tags": ["SQL", "Java", "column-oriented"], + "tags": ["Java", "column-oriented"], "load_time": 2032, - "data_size": 9245538172, + "data_size": null, "result": [ [0.002, 0.001, 0.001], @@ -1504,7 +1504,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "row-oriented", "PostgreSQL compatible"], + "tags": ["C", "row-oriented", "PostgreSQL compatible"], "load_time": 2342, "data_size": 77797067741, @@ -1563,7 +1563,7 @@ const data = [ "cluster_size": 1, "comment": "Many queries cannot run. It also crashes and hangs.", - "tags": ["SQL", "Java", "time-series"], + "tags": ["Java", "time-series"], "load_time": 1667.5, "data_size": 126680518680, @@ -1622,7 +1622,7 @@ const data = [ "cluster_size": 4, "comment": "One query did not run due to overflow check and another due to missing regexp function.", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2136, "data_size": 30794579968, @@ -1681,7 +1681,7 @@ const data = [ "cluster_size": "serverless", "comment": "One query did not run due to overflow check and another due to missing regexp function.", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 1933, "data_size": 30300000000, @@ -1740,7 +1740,7 @@ const data = [ "cluster_size": 1, "comment": "Previous name: MemSQL. Some queries did not run due to memory limits", - "tags": ["SQL", "MySQL compatible", "column-oriented"], + "tags": ["MySQL compatible", "column-oriented"], "load_time": 690, "data_size": 29836263469, @@ -1799,7 +1799,7 @@ const data = [ "cluster_size": 32, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, @@ -1858,7 +1858,7 @@ const data = [ "cluster_size": 64, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, @@ -1917,7 +1917,7 @@ const data = [ "cluster_size": 128, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, @@ -1976,7 +1976,7 @@ const data = [ "cluster_size": 8, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, @@ -2035,7 +2035,7 @@ const data = [ "cluster_size": 4, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, @@ -2094,7 +2094,7 @@ const data = [ "cluster_size": 2, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, @@ -2153,7 +2153,7 @@ const data = [ "cluster_size": 16, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, @@ -2212,7 +2212,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "managed", "column-oriented"], + "tags": ["managed", "column-oriented"], "load_time": 2524, "data_size": 12300000000, @@ -2271,7 +2271,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "embedded", "row-oriented"], + "tags": ["C", "embedded", "row-oriented"], "load_time": 2608, "data_size": 75776589824, @@ -2330,7 +2330,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "PostgreSQL compatible", "column-oriented", "time-series"], + "tags": ["C", "PostgreSQL compatible", "column-oriented", "time-series"], "load_time": 4605, "data_size": 20333747165, @@ -2389,7 +2389,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "C", "PostgreSQL compatible", "row-oriented", "time-series"], + "tags": ["C", "PostgreSQL compatible", "row-oriented", "time-series"], "load_time": 1620, "data_size": 72882392030, @@ -2448,7 +2448,7 @@ const data = [ "cluster_size": 1, "comment": "", - "tags": ["SQL", "column-oriented"], + "tags": ["column-oriented"], "load_time": 12239, "data_size": 22597367016, @@ -2752,6 +2752,9 @@ function renderSummary(filtered_data) { sorted_indices.map(idx => { const elem = filtered_data[idx]; + + if (selectors.metric == 'size' && elem.data_size === null) { return; } + let tr = document.createElement('tr'); tr.className = 'summary-row'; @@ -2881,6 +2884,7 @@ function render() { details_head.appendChild(th); }); + /// Load times { let tr = document.createElement('tr'); tr.className = 'shadow'; @@ -2905,6 +2909,7 @@ function render() { details_body.appendChild(tr); } + /// Data sizes { let tr = document.createElement('tr'); tr.className = 'shadow'; @@ -2917,10 +2922,10 @@ function render() { sorted_indices.map(idx => { const curr_size = filtered_data[idx].data_size; const baseline_size = Math.min(...filtered_data.map(elem => elem.data_size).filter(x => x)); - const ratio = curr_size / baseline_size; + const ratio = curr_size !== null ? curr_size / baseline_size : null; let td = document.createElement('td'); - td.appendChild(document.createTextNode(curr_size ? `${(curr_size / 1024 / 1024 / 1024).toFixed(2)} GiB (×${ratio.toFixed(2)})` : '0')); + td.appendChild(document.createTextNode(curr_size !== null ? `${(curr_size / 1024 / 1024 / 1024).toFixed(2)} GiB (×${ratio.toFixed(2)})` : '☠')); colorize(td, ratio); tr.appendChild(td); diff --git a/benchmark/pinot/results/c6a.4xlarge.json b/benchmark/pinot/results/c6a.4xlarge.json index 04855bd604a..8be94900f42 100644 --- a/benchmark/pinot/results/c6a.4xlarge.json +++ b/benchmark/pinot/results/c6a.4xlarge.json @@ -8,7 +8,7 @@ "tags": ["Java", "column-oriented"], "load_time": 2032, - "data_size": 9245538172, + "data_size": null, "result": [ [0.002, 0.001, 0.001], From afcfe939ca50ffdb46c6e2807e82e33c6b9631c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 08:11:01 +0200 Subject: [PATCH 383/627] Better highlighting --- benchmark/index.html | 26 +++++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/benchmark/index.html b/benchmark/index.html index 90fbf47ec40..624fe4768f5 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -108,7 +108,12 @@ padding-bottom: 0.5rem; } - .summary-row:hover { + .th-entry-hilite { + background: #EEE; + font-weight: bold; + } + + .summary-row:hover, .summary-row-hilite { background: #EEE !important; font-weight: bold; } @@ -2625,6 +2630,18 @@ unique_systems.map(elem => { systems.appendChild(selector); if (!(elem in selectors.system)) { selectors.system[elem] = true; } selector.addEventListener('click', e => toggle(e, elem, selectors.system)); + + /// Highlighting summary rows and table columns on hovering over the system selector. + selector.addEventListener('mouseover', e => { + [...document.querySelectorAll('.summary-row')].map(row => { + row.className = row.dataset.system == elem ? 'summary-row summary-row-hilite' : 'summary-row' }); + [...document.querySelectorAll('.th-entry')].map(th => { + th.className = th.dataset.system == elem ? 'th-entry th-entry-hilite' : 'th-entry' }); + }); + selector.addEventListener('mouseout', e => { + [...document.querySelectorAll('.summary-row')].map(row => { row.className = 'summary-row' }); + [...document.querySelectorAll('.th-entry')].map(row => { row.className = 'th-entry' }); + }); }); [... new Set(data.map(elem => elem.tags).flat())].map(elem => { @@ -2758,6 +2775,8 @@ function renderSummary(filtered_data) { let tr = document.createElement('tr'); tr.className = 'summary-row'; + tr.dataset.system = elem.system; + let td_name = document.createElement('td'); td_name.className = 'summary-name'; @@ -2861,6 +2880,7 @@ function render() { /// Generate details + /// Global checkbox { let th_checkbox = document.createElement('th'); let checkbox = document.createElement('input'); @@ -2877,10 +2897,13 @@ function render() { details_head.appendChild(document.createElement('th')); } + /// Table header sorted_indices.map(idx => { const elem = filtered_data[idx]; let th = document.createElement('th'); th.appendChild(document.createTextNode(`${elem.system}\n(${elem.machine})`)); + th.className = 'th-entry'; + th.dataset.system = elem.system; details_head.appendChild(th); }); @@ -2934,6 +2957,7 @@ function render() { details_body.appendChild(tr); } + /// Query runtimes const num_queries = filtered_data[0].result.length; for (let query_num = 0; query_num < num_queries; ++query_num) { From fca47ec0f879f2e6b4d84700619eff938dfe3351 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 09:17:47 +0200 Subject: [PATCH 384/627] Add tooltips --- benchmark/databend/results/c6a.4xlarge.json | 2 +- benchmark/index.html | 59 ++++++++++++++++++++- 2 files changed, 58 insertions(+), 3 deletions(-) diff --git a/benchmark/databend/results/c6a.4xlarge.json b/benchmark/databend/results/c6a.4xlarge.json index f0c678f210f..a26bad5b49f 100644 --- a/benchmark/databend/results/c6a.4xlarge.json +++ b/benchmark/databend/results/c6a.4xlarge.json @@ -3,7 +3,7 @@ "date": "2022-07-01", "machine": "c6a.4xlarge, 500gb gp2", "cluster_size": 1, - "comment": "Only 90% of data successfully loaded. For some queries it gives \"Data too large\".", + "comment": "Only 90% of data successfully loaded.", "tags": ["Rust", "column-oriented", "ClickHouse derivative"], diff --git a/benchmark/index.html b/benchmark/index.html index 624fe4768f5..56cb496890c 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -154,6 +154,43 @@ font-weight: bold; color: #CCC; } + + .note { + position: relative; + display: inline-block; + } + + .tooltip { + position: absolute; + bottom: calc(100% + 0.5rem); + left: calc(50% - 0.25rem); + visibility: hidden; + background-color: black; + color: white; + box-shadow: 0 0 1rem gray; + padding: 0.5rem 0.75rem; + border-radius: 0.5rem; + z-index: 1; + width: 20rem; + margin-left: -10rem; + text-align: left; + white-space: normal; + } + + .note:hover .tooltip { + visibility: visible; + } + + .tooltip::after { + content: " "; + position: absolute; + top: 100%; + left: 50%; + margin-left: -1rem; + border-width: 0.5rem; + border-style: solid; + border-color: black transparent transparent transparent; + } diff --git a/benchmark/index.html.bak b/benchmark/index.html.bak deleted file mode 100644 index 895fe581d76..00000000000 --- a/benchmark/index.html.bak +++ /dev/null @@ -1,2999 +0,0 @@ - - - - - ClickBench — a Benchmark For Analytical DBMS - - - - - - - - - - -
-

ClickBench — a Benchmark For Analytical DBMS

- Methodology | Reproduce and Validate the Results | Add a System | Report Mistake | Hardware Benchmark -
- - - - - - - - - - - - - - - - - - - - - - -
System: - All -
Type: - All -
Machine: - All -
Cluster size: - All -
Metric: - Cold Run - Hot Run - Load Time - Storage Size -
- - - - - - - - - - -
- System & Machine - - Relative time (lower is better) -
- -
Nothing selected
- -
-

Detailed Comparison

-
- - - - - - - - -
- -
- - - - From 23dbdb29166ab276ebdba8a92fe0d87ccc0b7b4b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 09:55:14 +0200 Subject: [PATCH 387/627] Proper links --- benchmark/index.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmark/index.html b/benchmark/index.html index 3536806fd0f..79a495fc4c8 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -571,7 +571,7 @@ function renderSummary(filtered_data) { let link = document.createElement('a'); link.appendChild(document.createTextNode(`${elem.system} (${elem.machine})`)); - link.href = "https://github.com/ClickHouse/ClickBench/" + elem.system; + link.href = "https://github.com/ClickHouse/ClickBench/" + elem.source; td_name.appendChild(link); if (elem.comment) { td_name.appendChild(addNote(elem.comment)); } From 640421d4322b4bc76b452844f85199e40cb45340 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 10:57:26 +0200 Subject: [PATCH 388/627] Additional data size points --- benchmark/index.html | 31 +++++++++++++++++++++++++------ 1 file changed, 25 insertions(+), 6 deletions(-) diff --git a/benchmark/index.html b/benchmark/index.html index 79a495fc4c8..5e2cad02f5d 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -281,6 +281,16 @@ const data = [ ,{"system":"TimescaleDB","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","PostgreSQL compatible","row-oriented","time-series"],"load_time":1620,"data_size":72882392030,"result":[[437.7,215.793,176.42],[327.026,259.568,244.578],[262.978,263.09,263.083],[262.807,263.046,266.847],[337.497,334.964,330.852],[355.689,356.801,362.894],[262.762,263.012,262.968],[263.055,263.016,263.028],[319.928,319.388,320.704],[323.584,322.224,322.488],[265.979,265.465,265.375],[266.019,265.543,265.462],[277.018,276.3,276.595],[280.352,279.251,279.572],[279.915,279.896,279.674],[296.377,298.506,297.659],[314.448,314.605,312.57],[302.668,302.672,303.039],[325.81,324.061,324.376],[262.447,262.698,262.704],[267.581,267.467,267.482],[268.085,267.466,267.696],[263.391,263.097,263.126],[38.291,0.435,0.335],[0.127,0.005,0.005],[263.138,263.1,263.092],[0.889,0.341,0.339],[267.586,267.498,267.491],[289.086,290.012,290.093],[263.22,263.071,263.109],[274.78,273.995,273.998],[282.217,281.39,281.47],[429.273,426.588,439.431],[448.808,418.724,418.207],[455.196,422.75,423.142],[299.263,296.937,297.261],[18.693,1.552,1.481],[18.125,0.945,0.937],[18.528,1.062,0.902],[18.268,1.779,1.77],[19.615,1.965,1.966],[18.97,1.435,1.43],[18.33,1.153,0.952]],"source":"timescaledb/results/c6a.4xlarge.json"} ,{"system":"Vertica","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["column-oriented"],"load_time":12239,"data_size":22597367016,"result":[[0.031357,0.020759,0.019373],[0.156612,0.033543,0.032149],[0.404881,0.124383,0.124252],[0.220711,0.074036,0.070379],[2.19663,2.18441,2.1763],[11.5901,11.4847,11.4573],[0.160286,0.063093,0.064844],[0.116135,0.047536,0.046322],[2.36592,2.15174,2.15675],[4.42049,4.37322,4.38596],[1.24771,1.17476,1.21011],[1.30116,1.24332,1.24521],[14.0523,13.9236,13.9943],[39.8961,39.7911,39.771],[14.6776,14.5524,14.5899],[2.82581,2.71968,2.72123],[48.4932,48.4037,48.4298],[36.8767,36.7432,36.9178],[108.743,108.81,108.671],[0.02734,0.018601,0.018892],[8.98822,4.2958,4.38323],[10.0097,4.31009,4.26843],[14.2354,5.48257,5.47171],[3.29109,0.405847,0.337797],[0.088318,0.079666,0.091852],[1.05864,1.26979,1.81786],[0.087436,0.092269,0.087506],[15.0424,15.0873,15.2561],[22.1566,22.1208,21.9509],[2.59797,2.46053,2.46406],[3.63938,3.68086,3.68988],[6.58816,4.84546,4.8853],[20.7209,20.4806,20.5823],[77.3779,78.1301,77.4366],[77.2494,77.646,77.4976],[6.79571,6.81133,6.67973],[0.887148,0.782822,0.753053],[0.369373,0.339676,0.338677],[0.099038,0.089598,0.099019],[2.15693,2.01152,2.00868],[0.096018,0.052062,0.053813],[0.058647,0.046352,0.045415],[0.106915,0.077122,0.076134]],"source":"vertica/results/c6a.4xlarge.json"} ]; // end of data + +const additional_data_size_points = [ +{"fake": true, "system": "hits.tsv", "data_size": 74807831229}, +{"fake": true, "system": "hits.csv", "data_size": 81136059858}, +{"fake": true, "system": "hits.json", "data_size": 232733025002}, +{"fake": true, "system": "hits.parquet", "data_size": 14779976446}, +{"fake": true, "system": "hits.tsv.gz", "data_size": 16298506510}, +{"fake": true, "system": "hits.csv.gz", "data_size": 16608960810}, +{"fake": true, "system": "hits.json.gz", "data_size": 23728268670} +]; @@ -517,7 +527,7 @@ function renderSummary(filtered_data) { const baseline_data = [...filtered_data[0].result.keys()].map(query_num => [...Array(3).keys()].map(run_num => - Math.min(...filtered_data.map(elem => elem.result[query_num][run_num]).filter(x => x)))); + Math.min(...filtered_data.filter(elem => !elem.fake).map(elem => elem.result[query_num][run_num]).filter(x => x)))); const min_load_time = Math.min(...filtered_data.map(elem => elem.load_time).filter(x => x)); const min_data_size = Math.min(...filtered_data.map(elem => elem.data_size).filter(x => x)); @@ -569,11 +579,15 @@ function renderSummary(filtered_data) { let td_name = document.createElement('td'); td_name.className = 'summary-name'; - let link = document.createElement('a'); - link.appendChild(document.createTextNode(`${elem.system} (${elem.machine})`)); - link.href = "https://github.com/ClickHouse/ClickBench/" + elem.source; + if (!elem.fake) { + let link = document.createElement('a'); + link.appendChild(document.createTextNode(`${elem.system} (${elem.machine})`)); + link.href = "https://github.com/ClickHouse/ClickBench/" + elem.source; + td_name.appendChild(link); + } else { + td_name.appendChild(document.createTextNode(elem.system)); + } - td_name.appendChild(link); if (elem.comment) { td_name.appendChild(addNote(elem.comment)); } td_name.appendChild(document.createTextNode(': ')); @@ -670,7 +684,12 @@ function render() { nothing_selected_elem.style.display = 'none'; [...document.querySelectorAll('.comparison')].map(e => e.style.display = 'block'); - const [sorted_indices, baseline_data] = renderSummary(filtered_data); + if (selectors.metric == 'size') { + filtered_data = [...filtered_data, ...additional_data_size_points]; + } + + let [sorted_indices, baseline_data] = renderSummary(filtered_data); + sorted_indices = sorted_indices.filter(idx => !filtered_data[idx].fake); /// Generate details From d4784203b7f3f139ace8d761a33da92a9a4d8b78 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 8 Jul 2022 12:51:30 +0300 Subject: [PATCH 389/627] Revert "Fix toHour() monotonicity which can lead to incorrect query result (incorrect index analysis)" --- .../FunctionDateOrDateTimeToSomething.h | 18 ++++++++---------- .../02346_to_hour_monotonicity_fix.reference | 1 - .../02346_to_hour_monotonicity_fix.sql | 8 -------- 3 files changed, 8 insertions(+), 19 deletions(-) delete mode 100644 tests/queries/0_stateless/02346_to_hour_monotonicity_fix.reference delete mode 100644 tests/queries/0_stateless/02346_to_hour_monotonicity_fix.sql diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index d734c7f87c1..5269eecea37 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -148,10 +148,8 @@ public: const IFunction::Monotonicity is_monotonic = { .is_monotonic = true }; const IFunction::Monotonicity is_not_monotonic; - const DateLUTImpl * date_lut = &DateLUT::instance(); - if (const auto * timezone = dynamic_cast(&type)) - date_lut = &timezone->getTimeZone(); - + /// This method is called only if the function has one argument. Therefore, we do not care about the non-local time zone. + const DateLUTImpl & date_lut = DateLUT::instance(); if (left.isNull() || right.isNull()) return is_not_monotonic; @@ -159,20 +157,20 @@ public: if (checkAndGetDataType(&type)) { - return Transform::FactorTransform::execute(UInt16(left.get()), *date_lut) - == Transform::FactorTransform::execute(UInt16(right.get()), *date_lut) + return Transform::FactorTransform::execute(UInt16(left.get()), date_lut) + == Transform::FactorTransform::execute(UInt16(right.get()), date_lut) ? is_monotonic : is_not_monotonic; } else if (checkAndGetDataType(&type)) { - return Transform::FactorTransform::execute(Int32(left.get()), *date_lut) - == Transform::FactorTransform::execute(Int32(right.get()), *date_lut) + return Transform::FactorTransform::execute(Int32(left.get()), date_lut) + == Transform::FactorTransform::execute(Int32(right.get()), date_lut) ? is_monotonic : is_not_monotonic; } else { - return Transform::FactorTransform::execute(UInt32(left.get()), *date_lut) - == Transform::FactorTransform::execute(UInt32(right.get()), *date_lut) + return Transform::FactorTransform::execute(UInt32(left.get()), date_lut) + == Transform::FactorTransform::execute(UInt32(right.get()), date_lut) ? is_monotonic : is_not_monotonic; } } diff --git a/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.reference b/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.reference deleted file mode 100644 index 5d49984b71b..00000000000 --- a/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.reference +++ /dev/null @@ -1 +0,0 @@ -5 8 42 diff --git a/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.sql b/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.sql deleted file mode 100644 index cad7df63eb5..00000000000 --- a/tests/queries/0_stateless/02346_to_hour_monotonicity_fix.sql +++ /dev/null @@ -1,8 +0,0 @@ -drop table if exists test_tz_hour; - -create table test_tz_hour(t DateTime, x String) engine MergeTree partition by toYYYYMMDD(t) order by x; -insert into test_tz_hour select toDateTime('2021-06-01 00:00:00') + number * 600, 'x' from numbers(1e3); - -select toHour(toTimeZone(t, 'UTC')) as toHour_UTC, toHour(toTimeZone(t, 'Asia/Jerusalem')) as toHour_Israel, count() from test_tz_hour where toHour_Israel = 8 group by toHour_UTC, toHour_Israel; - -drop table test_tz_hour; From a4e3fac8638ffb047909e4f5b4a46eb3a3644f1e Mon Sep 17 00:00:00 2001 From: Constantine Peresypkin Date: Fri, 8 Jul 2022 11:54:00 +0200 Subject: [PATCH 390/627] fix wrong OSX building instructions It was a mess of `clang-13` and `clang-14` or `aarch64` and `x86_64` --- docs/en/development/build-cross-osx.md | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/docs/en/development/build-cross-osx.md b/docs/en/development/build-cross-osx.md index d09552e06e9..0072c3253cf 100644 --- a/docs/en/development/build-cross-osx.md +++ b/docs/en/development/build-cross-osx.md @@ -25,6 +25,7 @@ sudo apt-get install clang-14 Let’s remember the path where we install `cctools` as ${CCTOOLS} ``` bash +export CCTOOLS=$(cd ~/cctools && pwd) mkdir ${CCTOOLS} cd ${CCTOOLS} @@ -43,10 +44,8 @@ make install Also, we need to download macOS X SDK into the working tree. ``` bash -cd ClickHouse -wget 'https://github.com/phracker/MacOSX-SDKs/releases/download/10.15/MacOSX10.15.sdk.tar.xz' -mkdir -p build-darwin/cmake/toolchain/darwin-x86_64 -tar xJf MacOSX10.15.sdk.tar.xz -C build-darwin/cmake/toolchain/darwin-x86_64 --strip-components=1 +cd ClickHouse/cmake/toolchain/darwin-x86_64 +curl -L 'https://github.com/phracker/MacOSX-SDKs/releases/download/10.15/MacOSX10.15.sdk.tar.xz' | tar xJ --strip-components=1 ``` ## Build ClickHouse {#build-clickhouse} @@ -55,7 +54,7 @@ tar xJf MacOSX10.15.sdk.tar.xz -C build-darwin/cmake/toolchain/darwin-x86_64 --s cd ClickHouse mkdir build-darwin cd build-darwin -CC=clang-13 CXX=clang++-13 cmake -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/aarch64-apple-darwin-ar -DCMAKE_INSTALL_NAME_TOOL=${CCTOOLS}/bin/aarch64-apple-darwin-install_name_tool -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/aarch64-apple-darwin-ranlib -DLINKER_NAME=${CCTOOLS}/bin/aarch64-apple-darwin-ld -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake .. +CC=clang-14 CXX=clang++-14 cmake -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar -DCMAKE_INSTALL_NAME_TOOL=${CCTOOLS}/bin/x86_64-apple-darwin-install_name_tool -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake .. ninja ``` From 53d019a94bdfb4f23a3c33ae0390bc7a201a0def Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jul 2022 11:33:09 +0300 Subject: [PATCH 391/627] check-style: improve ZooKeeper path check for Replicated.*MergeTree tables - detech all Replicated.*MergeTree tables - allow default_path_test/uuid in zookeeper path - improve pattern for filtering tests Signed-off-by: Azat Khuzhin --- utils/check-style/check-style | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index adae3068dcd..c99d20730d2 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -201,7 +201,7 @@ done tests_with_replicated_merge_tree=( $( find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | - xargs grep --with-filename -e ReplicatedMergeTree | cut -d: -f1 | sort -u + xargs grep --with-filename -e "Replicated.*MergeTree[ ]*(.*" | cut -d: -f1 | sort -u ) ) for test_case in "${tests_with_replicated_merge_tree[@]}"; do case "$test_case" in @@ -209,11 +209,11 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do ;; *.sh) test_case_zk_prefix="\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" - grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" + grep -q -e "Replicated.*MergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "Replicated.*MergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.sql|*.sql.j2) - test_case_zk_prefix="\({database}\|currentDatabase()\)" - grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" + test_case_zk_prefix="\({database}\|currentDatabase()\|{uuid}\|{default_path_test}\)" + grep -q -e "Replicated.*MergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "Replicated.*MergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.py) # Right now there is not such tests anyway From 2e6a9d5272696ca660533cbba93a1d646c38562d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jul 2022 12:25:04 +0200 Subject: [PATCH 392/627] Preparation for dark theme --- benchmark/index.html | 112 +++++++++++++++++++++++++++++++++++-------- 1 file changed, 92 insertions(+), 20 deletions(-) diff --git a/benchmark/index.html b/benchmark/index.html index 5e2cad02f5d..0a92490e06d 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -9,7 +9,65 @@ + + + +
+ 🌚🌞 +

ClickBench — a Benchmark For Analytical DBMS

+ Methodology | Reproduce and Validate the Results | Add a System | Report Mistake | Hardware Benchmark
-
-
+ + + + + + + + + + + + + + + + + + + + + +
System: + All +
Type: + All +
Machine: + All +
Cluster size: + All +
Metric: + Cold Run + Hot Run + Load Time + Storage Size +
+ + + + + + + + + + +
+ System & Machine + + Relative time (lower is better) +
+ +
Nothing selected
+ +
+

Detailed Comparison

-
-
-

Relative query processing time (lower is better)

-
-
-
+ + + + + + + +
-
-
-

Full results

-
-
-
+ + + diff --git a/website/benchmark/dbms/queries.js b/website/benchmark/dbms/queries.js deleted file mode 100644 index 3f77544b5a2..00000000000 --- a/website/benchmark/dbms/queries.js +++ /dev/null @@ -1,135 +0,0 @@ -var current_data_size = 100000000; -var current_systems = ["ClickHouse", "Vertica", "Greenplum"]; - -var queries = - [ - { - "query": "SELECT count() FROM hits" - }, - { - "query": "SELECT count() FROM hits WHERE AdvEngineID != 0" - }, - { - "query": "SELECT sum(AdvEngineID), count(), avg(ResolutionWidth) FROM hits" - }, - { - "query": "SELECT sum(UserID) FROM hits" - }, - { - "query": "SELECT uniq(UserID) FROM hits" - }, - { - "query": "SELECT uniq(SearchPhrase) FROM hits" - }, - { - "query": "SELECT min(EventDate), max(EventDate) FROM hits" - }, - { - "query": "SELECT AdvEngineID, count() FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count() DESC" - }, - { - "query": "SELECT RegionID, uniq(UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10" - }, - { - "query": "SELECT RegionID, sum(AdvEngineID), count() AS c, avg(ResolutionWidth), uniq(UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10" - }, - { - "query": "SELECT MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10" - }, - { - "query": "SELECT MobilePhone, MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10" - }, - { - "query": "SELECT SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10" - }, - { - "query": "SELECT SearchPhrase, uniq(UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10" - }, - { - "query": "SELECT SearchEngineID, SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10" - }, - { - "query": "SELECT UserID, count() FROM hits GROUP BY UserID ORDER BY count() DESC LIMIT 10" - }, - { - "query": "SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase ORDER BY count() DESC LIMIT 10" - }, - { - "query": "SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase LIMIT 10" - }, - { - "query": "SELECT UserID, toMinute(EventTime) AS m, SearchPhrase, count() FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count() DESC LIMIT 10" - }, - { - "query": "SELECT UserID FROM hits WHERE UserID = 12345678901234567890" - }, - { - "query": "SELECT count() FROM hits WHERE URL LIKE '%metrika%'" - }, - { - "query": "SELECT SearchPhrase, any(URL), count() AS c FROM hits WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10" - }, - { - "query": "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" - }, - { - "query": "SELECT * FROM hits WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10" - }, - { - "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10" - }, - { - "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10" - }, - { - "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10" - }, - { - "query": "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" - }, - { - "query": "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" - }, - { - "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" - }, - { - "query": "SELECT SearchEngineID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10" - }, - { - "query": "SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10" - }, - { - "query": "SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10" - }, - { - "query": "SELECT URL, count() AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10" - }, - { - "query": "SELECT 1, URL, count() AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10" - }, - { - "query": "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" - }, - { - "query": "SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND notEmpty(URL) GROUP BY URL ORDER BY PageViews DESC LIMIT 10" - }, - { - "query": "SELECT Title, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND notEmpty(Title) GROUP BY Title ORDER BY PageViews DESC LIMIT 10" - }, - { - "query": "SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000" - }, - { - "query": "SELECT TraficSourceID, SearchEngineID, AdvEngineID, ((SearchEngineID = 0 AND AdvEngineID = 0) ? Referer : '') AS Src, URL AS Dst, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000" - }, - { - "query": "SELECT URLHash, EventDate, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = halfMD5('http://yandex.ru/') GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100" - }, - { - "query": "SELECT WindowClientWidth, WindowClientHeight, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = halfMD5('http://yandex.ru/') GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000;" - }, - { - "query": "SELECT toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM hits WHERE CounterID = 34 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute;" - } - ] diff --git a/website/benchmark/dbms/results/.gitignore b/website/benchmark/dbms/results/.gitignore deleted file mode 100644 index c128402e106..00000000000 --- a/website/benchmark/dbms/results/.gitignore +++ /dev/null @@ -1,4 +0,0 @@ -# De-Witt Clause: - -vertica* -bigquery* diff --git a/website/benchmark/dbms/results/athena.partitioned.json b/website/benchmark/dbms/results/athena.partitioned.json deleted file mode 100644 index e77974b51c4..00000000000 --- a/website/benchmark/dbms/results/athena.partitioned.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "athena (partitioned)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[2.777,3.275,2.925], -[1.503,3.136,4.003], -[4.544,3.833,3.64], -[3.9,2.514,3.522], -[3.46,2.186,3.244], -[3.624,2.742,3.185], -[2.21,1.984,3.123], -[3.207,2.403,2.685], -[2.936,2.014,3.869], -[8.333,7.102,4.434], -[7.401,4.697,3.155], -[4.214,3.065,4.748], -[6.207,4.213,2.576], -[3.428,3.085,3.401], -[2.92,3.3,3.278], -[2.205,2.558,2.419], -[4.641,3.888,2.155], -[3.219,2.822,3.292], -[3.23,3.579,4.31], -[2.288,3.543,3.95], -[3.032,2.859,2.807], -[3.926,3.247,2.928], -[4.477,4.048,4.392], -[7.407,6.375,6.123], -[2.611,2.872,2.827], -[2.566,2.567,3.6], -[3.673,3.733,2.925], -[2.426,3.218,2.78], -[5.125,3.778,4.25], -[4.565,4.03,4.066], -[3.628,3.219,2.953], -[6.207,5.973,3.158], -[4.339,5.601,4.234], -[2.618,3.107,3.433], -[4.661,2.79,2.846], -[2.373,1.629,2.734], -[2.721,2.15,1.962], -[3.207,2.154,2.186], -[2.453,2.477,3.217], -[2.691,4.732,3.584], -[2.589,2.613,3.231], -[1.926,3.617,1.82], -[1.506,2.404,2.343] -]}] diff --git a/website/benchmark/dbms/results/athena.single.json b/website/benchmark/dbms/results/athena.single.json deleted file mode 100644 index 7c004934c80..00000000000 --- a/website/benchmark/dbms/results/athena.single.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "athena (single)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[2.268,1.327,2.137], -[3.427,2.248,3.605], -[3.254,2.548,2.316], -[3.025,2.314,3.003], -[2.264,2.876,4.213], -[3.044,2.745,2.698], -[2.732,2.199,2.659], -[2.022,3.692,3.072], -[2.746,2.477,2.785], -[3.53,2.782,4.031], -[2.709,2.047,2.853], -[2.318,1.969,3.4], -[2.635,1.935,2.707], -[3.049,3.38,3.071], -[3.661,2.387,2.476], -[2.479,2.591,2.21], -[3.093,3.698,4.351], -[3.479,3.236,2.274], -[4.36,2.97,3.457], -[2.525,2.384,3.328], -[3.34,3.174,3.409], -[3.163,2.971,3.034], -[2.999,3.539,2.906], -[6.454,7.597,7.858], -[2.754,1.951,2.645], -[2.852,3.018,2.718], -[2.513,2.678,2.417], -[3.293,2.521,2.771], -[4.392,3.863,3.981], -[3.658,4.246,4.027], -[3.028,3.87,2.337], -[2.923,3.635,3.591], -[3.142,4.105,3.15], -[3.66,3.187,4.745], -[2.652,2.695,2.742], -[2.262,2.776,1.815], -[1.881,2.212,2.053], -[1.934,2.551,1.524], -[2.069,2.26,1.805], -[2.626,2.902,2.793], -[1.791,2.082,2.481], -[3.757,2.6,1.946], -[2.608,1.994,3.967] -]}] diff --git a/website/benchmark/dbms/results/aurora-mysql.16acu.json b/website/benchmark/dbms/results/aurora-mysql.16acu.json deleted file mode 100644 index d0e35ffbc1a..00000000000 --- a/website/benchmark/dbms/results/aurora-mysql.16acu.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "aurora-mysql (16acu)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[740.42,739.91,746.65], -[828.2,835.67,832.87], -[830.08,830.98,832.38], -[829.88,832.83,830.87], -[845.99,842.4,843.21], -[869.51,870.69,869.75], -[823.77,829.08,825.54], -[827.74,832.87,829.25], -[916.26,909.46,929.17], -[946.49,939.27,932.32], -[852.37,857.69,854.74], -[857.99,864.05,825.14], -[null,null,null], -[863.37,860.2,865.62], -[null,null,null], -[891.84,895.28,893.68], -[null,null,null], -[null,null,null], -[1420.12,1419.34,1445.08], -[28.94,0.21,0.21], -[917.64,917.56,916.92], -[923.47,921.7,923.82], -[919.95,918.37,920.17], -[1002.19,1002.07,1001.2], -[902.23,902.65,901.8], -[901.17,900.02,898.3], -[900.04,898.89,903.35], -[901.78,902.71,901.28], -[null,null,null], -[1153.29,1154,1156.46], -[862.57,863.35,859.69], -[923.14,921.1,923.92], -[1370.78,1401.72,1401.44], -[1454.67,1455.55,1458.79], -[1463.31,1466.75,1461.83], -[941.03,944.07,937.23], -[7.42,2.80,2.77], -[2.57,2.52,2.59], -[1.50,1.52,1.59], -[3.62,3.57,3.61], -[0.95,0.94,0.94], -[0.90,0.92,0.91], -[1.69,1.72,1.69] -]}] diff --git a/website/benchmark/dbms/results/aurora-postgresql.16acu.json b/website/benchmark/dbms/results/aurora-postgresql.16acu.json deleted file mode 100644 index af9fca3d4f1..00000000000 --- a/website/benchmark/dbms/results/aurora-postgresql.16acu.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "aurora-postgresql (16acu)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[12.8361,5.71812,5.8241], -[61.2565,62.1402,63.7173], -[68.0578,68.1218,67.609], -[7.83207,5.90193,6.0461], -[48.7194,48.0233,48.2198], -[289.492,304.639,282.436], -[6.30572,6.31857,6.21598], -[53.644,53.8931,53.5307], -[131.526,131.45,131.102], -[137.724,136.921,137.758], -[57.2079,56.2775,56.2152], -[56.5349,56.2048,55.9569], -[82.3897,82.8866,83.534], -[97.0569,97.1392,96.4731], -[85.6557,86.7783,86.2804], -[49.4325,42.4309,42.5743], -[111.537,114.59,111.807], -[88.4322,89.3756,87.7899], -[160.781,163.866,161.394], -[1025.04,2.10165,2.10065], -[106.741,56.2731,56.1535], -[59.2681,59.5272,59.536], -[58.6083,57.6054,57.3935], -[54.8271,55.1397,56.3487], -[54.718,52.469,53.271], -[53.5387,53.1926,52.4008], -[52.0042,51.9581,52.2453], -[60.1317,59.9695,59.2187], -[244.608,242.954,243.815], -[91.8674,92.4165,91.5884], -[63.7122,64.277,64.2783], -[69.2596,68.9535,69.4508], -[234.222,241.138,240.316], -[488.169,462.257,460.466], -[472.929,471.809,476.635], -[103.664,116.131,103.467], -[16.8124,3.34058,3.37782], -[0.852414,0.832073,0.859857], -[0.305464,0.31166,0.306694], -[4.55625,4.54098,4.58501], -[0.299746,0.297532,0.30334], -[0.275732,0.279817,0.27766], -[0.332107,0.324387,0.320099] -]}] diff --git a/website/benchmark/dbms/results/citus.c6a.4xlarge.json b/website/benchmark/dbms/results/citus.c6a.4xlarge.json deleted file mode 100644 index 0197ba4eb74..00000000000 --- a/website/benchmark/dbms/results/citus.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "citus (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[7.58503,6.70447,6.52499], -[6.33941,5.06063,5.00238], -[11.7488,9.86417,9.93223], -[12.6306,9.36305,9.17061], -[40.6101,39.0803,38.1187], -[117.654,113.912,113.441], -[10.3404,8.08936,7.70732], -[6.31542,4.72821,4.72989], -[82.5425,77.2124,76.9219], -[91.1776,83.4492,82.4727], -[14.5474,10.0815,10.3873], -[15.4899,11.2922,11.1877], -[19.9794,15.5002,17.4492], -[76.9216,72.5172,72.7915], -[21.5446,17.5691,18.561], -[56.9438,54.6387,53.5745], -[75.0977,69.7842,70.0259], -[31.3299,27.0267,26.3216], -[129.417,122.956,121.182], -[3.73386,2.14148,2.12737], -[34.6021,27.9727,28.6878], -[37.152,29.6193,29.2966], -[52.2157,37.8589,37.6994], -[181.955,149.08,148.471], -[15.4687,11.3138,10.3856], -[10.2779,8.46868,8.8324], -[14.4687,10.4076,11.4263], -[47.009,40.2969,39.6888], -[749.946,742.979,744.461], -[69.4383,67.5636,67.2128], -[27.0317,21.4008,20.9524], -[36.6675,25.6347,26.4408], -[140.424,130.546,129.738], -[106.959,92.033,90.1609], -[110.98,94.4787,96.2656], -[64.4474,60.1853,60.6816], -[6.17549,6.25376,5.87004], -[1.99153,1.81776,1.80596], -[1.00141,0.800271,0.801975], -[7.91778,7.70928,8.33299], -[0.929845,0.642076,0.638478], -[0.866536,0.683567,0.680218], -[0.937823,0.784747,0.765929] -]}] diff --git a/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.partitioned.json b/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.partitioned.json deleted file mode 100644 index 604192f26bf..00000000000 --- a/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.partitioned.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "clickhouse-local (c6a.4xlarge.partitioned)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.850, 0.102, 0.114], -[1.342, 0.090, 0.099], -[2.547, 0.179, 0.189], -[1.681, 0.245, 0.252], -[2.704, 1.680, 1.648], -[2.194, 1.460, 1.489], -[0.832, 0.113, 0.102], -[1.371, 0.106, 0.101], -[2.240, 0.790, 0.825], -[4.548, 1.021, 1.026], -[3.094, 0.552, 0.552], -[3.088, 0.623, 0.630], -[2.017, 1.170, 1.165], -[4.319, 1.677, 1.708], -[2.157, 1.496, 1.500], -[1.629, 1.138, 1.139], -[5.026, 3.267, 3.241], -[4.142, 2.303, 2.319], -[8.295, 5.569, 5.629], -[1.331, 0.255, 0.252], -[10.712, 3.668, 3.786], -[13.053, 4.185, 4.202], -[24.170, 7.935, 8.008], -[55.965, 23.933, 23.071], -[4.417, 0.947, 0.974], -[1.793, 0.698, 0.690], -[4.376, 0.955, 0.956], -[11.731, 4.385, 4.321], -[11.403, 8.549, 8.288], -[2.764, 2.754, 2.735], -[5.096, 1.262, 1.273], -[9.515, 1.682, 1.688], -[10.325, 6.745, 6.608], -[11.686, 6.261, 6.242], -[11.769, 6.301, 6.364], -[1.675, 1.490, 1.495], -[14.937, 3.631, 3.604], -[14.187, 3.609, 3.631], -[14.842, 3.769, 3.741], -[22.222, 6.355, 6.263], -[7.212, 0.836, 0.838], -[7.863, 0.716, 0.718], -[5.120, 0.587, 0.574] -]}] diff --git a/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.single.json b/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.single.json deleted file mode 100644 index e47a265585d..00000000000 --- a/website/benchmark/dbms/results/clickhouse-local.c6a.4xlarge.single.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "clickhouse-local (c6a.4xlarge.single)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[1.176, 0.251, 0.249], -[1.037, 0.134, 0.111], -[1.609, 0.403, 0.369], -[1.616, 0.372, 0.370], -[3.008, 2.338, 2.266], -[9.061, 7.537, 7.535], -[1.206, 0.191, 0.187], -[0.882, 0.144, 0.135], -[4.610, 3.406, 3.256], -[6.712, 4.479, 4.469], -[4.081, 2.413, 2.394], -[3.894, 2.719, 2.691], -[7.651, 6.436, 6.243], -[10.765, 8.043, 7.894], -[9.860, 8.945, 8.235], -[7.159, 5.815, 5.814], -[20.916, 18.159, 18.013], -[20.952, 17.862, 17.850], -[37.585, 32.649, 32.487], -[1.767, 0.401, 0.393], -[23.713, 15.687, 15.755], -[28.700, 19.241, 19.198], -[50.740, 33.161, 33.011], -[152.485, 117.417, 118.178], -[7.606, 4.491, 5.326], -[4.331, 4.214, 3.587], -[6.743, 4.486, 5.357], -[22.910, 15.043, 15.183], -[43.342, 37.167, 36.842], -[11.807, 4.490, 4.546], -[9.557, 6.349, 6.263], -[13.964, 8.493, 8.464], -[38.110, 33.642, 33.996], -[41.266, 35.080, 27.073], -[34.056, 26.814, 26.902], -[8.855, 7.548, 7.475], -[22.596, 12.615, 12.669], -[23.217, 13.956, 13.831], -[22.528, 21.601, 13.207], -[37.890, 23.115, 22.955], -[6.490, 1.548, 1.522], -[6.413, 1.474, 1.416], -[3.796, 1.339, 1.316] -]}] diff --git a/website/benchmark/dbms/results/clickhouse.c6a.4xlarge.json b/website/benchmark/dbms/results/clickhouse.c6a.4xlarge.json deleted file mode 100644 index 54f3eef41be..00000000000 --- a/website/benchmark/dbms/results/clickhouse.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "clickhouse (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.027, 0.001, 0.001], -[0.035, 0.015, 0.021], -[0.083, 0.034, 0.033], -[0.171, 0.044, 0.045], -[1.552, 1.495, 1.574], -[1.270, 1.075, 1.063], -[0.045, 0.026, 0.025], -[0.032, 0.016, 0.015], -[0.717, 0.615, 0.607], -[0.843, 0.821, 0.747], -[0.293, 0.219, 0.216], -[0.312, 0.226, 0.235], -[0.804, 0.694, 0.702], -[1.476, 1.047, 1.029], -[1.013, 0.898, 0.911], -[1.043, 0.964, 1.453], -[3.632, 2.715, 2.711], -[1.867, 1.750, 1.714], -[5.187, 4.797, 4.953], -[0.112, 0.068, 0.041], -[8.637, 1.761, 1.212], -[9.902, 0.902, 0.869], -[18.831, 2.067, 1.829], -[41.903, 4.476, 3.486], -[1.801, 0.254, 0.238], -[0.627, 0.214, 0.207], -[2.181, 0.241, 0.246], -[8.868, 0.748, 0.733], -[9.674, 6.891, 5.770], -[2.620, 2.355, 2.368], -[1.395, 0.533, 0.525], -[4.454, 0.730, 0.712], -[5.453, 4.990, 5.922], -[9.955, 3.968, 4.096], -[9.987, 4.035, 4.476], -[1.695, 1.236, 1.241], -[0.142, 0.079, 0.103], -[0.066, 0.033, 0.040], -[0.065, 0.030, 0.033], -[0.246, 0.207, 0.192], -[0.044, 0.019, 0.020], -[0.030, 0.023, 0.012], -[0.030, 0.018, 0.013] -]}] diff --git a/website/benchmark/dbms/results/clickhouse.c6a.metal.json b/website/benchmark/dbms/results/clickhouse.c6a.metal.json deleted file mode 100644 index e246b7df816..00000000000 --- a/website/benchmark/dbms/results/clickhouse.c6a.metal.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "clickhouse (c6a.metal)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.011, 0.001, 0.001], -[0.040, 0.015, 0.013], -[0.045, 0.021, 0.023], -[0.090, 0.023, 0.023], -[1.922, 1.565, 1.576], -[0.961, 0.737, 0.739], -[0.040, 0.023, 0.018], -[0.032, 0.028, 0.028], -[0.321, 0.287, 0.275], -[0.632, 0.284, 0.287], -[0.166, 0.124, 0.118], -[0.235, 0.100, 0.102], -[1.006, 0.182, 0.159], -[1.637, 0.216, 0.213], -[0.871, 0.174, 0.177], -[0.258, 0.148, 0.148], -[1.804, 0.370, 0.358], -[1.235, 0.275, 0.278], -[3.143, 0.854, 0.815], -[0.071, 0.024, 0.016], -[8.816, 0.215, 0.155], -[10.239, 0.203, 0.173], -[19.179, 0.388, 0.357], -[43.152, 0.824, 0.823], -[1.821, 0.059, 0.052], -[0.992, 0.045, 0.051], -[2.539, 0.063, 0.058], -[9.258, 0.300, 0.278], -[7.923, 0.961, 0.936], -[0.445, 0.431, 0.428], -[1.367, 0.131, 0.113], -[4.819, 0.205, 0.175], -[3.808, 0.739, 0.726], -[8.935, 0.607, 0.600], -[8.988, 0.634, 0.615], -[0.242, 0.220, 0.226], -[0.075, 0.058, 0.056], -[0.038, 0.028, 0.026], -[0.043, 0.028, 0.021], -[0.172, 0.127, 0.119], -[0.028, 0.018, 0.017], -[0.027, 0.019, 0.014], -[0.018, 0.026, 0.015] -]}] diff --git a/website/benchmark/dbms/results/cratedb.c6a.4xlarge.json b/website/benchmark/dbms/results/cratedb.c6a.4xlarge.json deleted file mode 100644 index 4a7bebcf51d..00000000000 --- a/website/benchmark/dbms/results/cratedb.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "cratedb (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.008162,0.005118,0.002553], -[0.350014,0.39977,0.133775], -[2.58426,2.47192,2.59779], -[2.12939,0.532981,0.507246], -[null,null,null], -[null,null,null], -[1.18488,1.06603,1.07219], -[0.209264,0.073284,0.067912], -[null,null,null], -[null,null,null], -[1.68892,1.2866,1.47428], -[1.62976,1.43073,1.26904], -[12.7517,13.0334,13.2685], -[18.8587,null,18.6951], -[11.2982,11.2108,11.577], -[20.2964,20.4035,19.1076], -[null,null,null], -[null,null,null], -[null,null,null], -[0.202044,0.010009,0.005566], -[9.22964,4.54606,0.774149], -[1.41673,1.09885,0.789775], -[12.3933,8.06911,1.69671], -[1.45018,0.969528,0.979718], -[0.357589,0.14887,0.153326], -[0.189282,0.133963,0.130279], -[0.153222,0.140756,0.139861], -[27.5195,19.6862,20.1825], -[72.7575,68.2,67.1238], -[144.533,146.579,152.144], -[8.76866,9.00563,8.46917], -[17.6652,16.6755,16.0558], -[null,null,null], -[null,null,null], -[null,null,null], -[42.2967,44.9621,44.4386], -[0.786911,0.4904,0.508416], -[0.602075,0.226261,0.182399], -[0.131407,0.058958,0.054518], -[0.954736,1.1361,1.14233], -[0.23764,0.139109,0.134472], -[0.110253,0.057695,0.056073], -[0.124285,0.150479,0.066226] -]}] diff --git a/website/benchmark/dbms/results/databend.c6a.4xlarge.json b/website/benchmark/dbms/results/databend.c6a.4xlarge.json deleted file mode 100644 index 3904d77a03c..00000000000 --- a/website/benchmark/dbms/results/databend.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "databend (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.010087, 0.002961, 0.003271], -[0.127964, 0.080012, 0.075741], -[0.162388, 0.143967, 0.144762], -[0.252904, 0.217471, 0.217369], -[34.281026, 34.844158, 34.526942], -[25.290307, 25.793068, 25.620563], -[0.112484, 0.093867, 0.090891], -[0.086604, 0.07796, 0.076448], -[20.723203, 20.7483, 20.354869], -[20.81994, 20.72446, 20.696573], -[1.964378, 1.93559, 1.893824], -[1.846866, 1.789111, 1.763664], -[4.468158, 4.407959, 4.438036], -[19.947276, 19.8859, 19.853514], -[5.478573, 5.474461, 5.460604], -[5.509521, 5.513413, 5.363123], -[15.430359, 15.5406, 15.461211], -[14.905998, 15.029721, 15.019642], -[31.069663, 30.811763, 30.737336], -[0.281067, 0.220021, 0.217741], -[8.89374, 4.12692, 4.131689], -[10.38448, 4.603694, 4.571757], -[19.980572, 8.836322, 8.892694], -[59.786474, 52.452881, 39.941988], -[2.804019, 0.994794, 0.958224], -[0.765299, 0.730434, 0.723964], -[2.784648, 0.94665, 0.936684], -[8.905027, 5.418438, 5.386109], -[12.187652, 12.230066, 12.164123], -[3.35748, 3.395991, 3.319434], -[4.309389, 3.854977, 3.772506], -[9.958201, 7.027432, 6.888253], -[50.200569, 50.535126, 50.283066], -[24.469412, 21.222713, 21.010188], -[26.115852, 23.93507, 24.835342], -[7.511517, 7.296179, 7.324549], -[2.156784, 1.298258, 1.278441], -[2.155447, 1.314499, 1.331237], -[2.007053, 1.181676, 1.155612], -[null, null, null], -[0.485363, 0.420291, 0.416819], -[0.372131, 0.322068, 0.323578], -[null, null, null] -]}] diff --git a/website/benchmark/dbms/results/duckdb.c6a.4xlarge.json b/website/benchmark/dbms/results/duckdb.c6a.4xlarge.json deleted file mode 100644 index 0b0d67edac7..00000000000 --- a/website/benchmark/dbms/results/duckdb.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "duckdb (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.005694353996659629,0.003944558004150167,0.003837226002360694], -[0.16991353100456763,0.03919722700084094,0.03835860399703961], -[0.44898432699847035,0.04947217500011902,0.04852217998995911], -[0.07586832098604646,0.07051395199960098,0.07007493599667214], -[9.554053236002801,8.153356187991449,8.73448242500308], -[7.66042533799191,6.931124911992811,7.103380946995458], -[0.030703739990713075,0.027668555994750932,0.027583695002249442], -[0.1778664360026596,0.03942437999648973,0.03882004099432379], -[8.53439180701389,8.869582625004114,9.020313234999776], -[10.40215514000738,11.125320470004226,8.941559945000336], -[1.1747649609897053,1.04221136700653,1.004799570000614], -[1.2380354650085792,1.1211603130068397,2.4278587239969056], -[3.1751541379926493,0.9360461989999749,0.8868292279948946], -[6.855684430003748,7.300301584007684,5.712960822012974], -[3.70588762400439,1.0249276379909134,0.9473389159975341], -[2.1037107890006155,1.6215517020027619,1.5671920729946578], -[null,null,null], -[null,null,null], -[null,null,null], -[0.0002772739971987903,0.00016792300448287278,0.0001574420020915568], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[2.9310110910009826,0.19020285899750888,0.1736805049877148], -[2.939304119994631,0.18754731099761557,0.18073286200524308], -[2.8706370779982535,0.18822155400994234,0.17905898999015335], -[null,null,null], -[null,null,null], -[0.884408778991201,0.714329167996766,0.7135983259940986], -[5.3762675570033025,0.8803737630078103,0.8728962720051641], -[7.249190265996731,2.9648747390019707,2.866687831003219], -[null,null,null], -[null,null,null], -[null,null,null], -[4.515183198003797,4.030519469000865,4.014251719010645], -[0.11604027298744768,0.040539135996368714,0.04280066800129134], -[0.0457908230018802,0.021069509006338194,0.019683108999743126], -[0.0680370800109813,0.011889394998434,0.01056639499438461], -[0.22029169600864407,0.08547276000899728,0.09095505000732373], -[0.03759863799496088,0.008373684002435766,0.007633563989656977], -[0.025631797994719818,0.008081699008471332,0.007858585988287814], -[0.034359957004198804,0.025543516996549442,0.02533275399764534] -]}] diff --git a/website/benchmark/dbms/results/greenplum.c6a.4xlarge.json b/website/benchmark/dbms/results/greenplum.c6a.4xlarge.json deleted file mode 100644 index 67db7aedd6c..00000000000 --- a/website/benchmark/dbms/results/greenplum.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "greenplum (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[2.20009,1.45883,1.13573], -[1.28661,1.07897,1.01856], -[2.2582,2.42193,2.59693], -[1.89371,1.9246,1.73866], -[9.54471,9.71956,10.0357], -[4.55957,4.10112,4.70458], -[1.65167,1.89767,1.97051], -[1.02537,0.999323,1.00524], -[7.89792,7.75718,8.01219], -[11.3779,11.6003,11.2552], -[2.11955,2.05003,1.9553], -[2.01997,1.97932,1.98284], -[2.88666,2.92828,2.96822], -[4.06272,4.01771,4.03011], -[3.46303,3.24779,3.06475], -[8.43717,8.18641,7.99202], -[9.02176,8.91572,8.92748], -[8.66982,8.39332,8.62497], -[15.1321,14.9502,15.2344], -[1.00877,0.934589,0.975342], -[11.0582,3.25807,3.17687], -[12.2634,3.5188,3.93705], -[24.1522,4.86328,4.88764], -[67.1517,20.4,20.261], -[2.5799,1.66114,1.90432], -[1.72831,1.89536,1.2515], -[2.44154,1.46595,1.88238], -[11.1175,6.09551,6.02292], -[83.041,82.9784,82.8678], -[82.5441,82.5427,83.9836], -[3.0962,3.22225,3.19431], -[5.77576,4.05973,4.00366], -[43.6826,40.4375,40.1076], -[18.0978,13.2741,12.8894], -[17.3187,13.5411,13.5927], -[9.72405,9.90029,10.0177], -[0.294344,0.169606,0.173804], -[0.162524,0.117489,0.115532], -[0.145205,0.098342,0.097275], -[0.334809,0.275365,0.265053], -[0.154522,0.107654,0.10529], -[0.158957,0.117284,0.119068], -[0.193756,0.144787,0.145485] -]}] diff --git a/website/benchmark/dbms/results/heavyai.c6a.4xlarge.json b/website/benchmark/dbms/results/heavyai.c6a.4xlarge.json deleted file mode 100644 index 4f22107b3c9..00000000000 --- a/website/benchmark/dbms/results/heavyai.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "heavyai (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[6.525,0.022,0.029], -[0.301,0.042,0.04], -[0.287,0.095,0.093], -[2.572,0.039,0.04], -[null,null,null], -[null,null,null], -[7.327,0.093,0.097], -[0.244,0.043,0.038], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[2.939,0.295,0.294], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[4.716,3.91,3.955], -[null,null,null], -[0.154,0.083,0.106], -[14.426,0.07,0.071], -[null,null,null], -[null,null,null], -[null,null,null], -[2.276,0.258,0.272], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[1.832,1.64,1.602], -[null,null,null], -[null,null,null], -[null,null,null], -[14.811,0.494,0.497], -[null,null,null], -[null,null,null], -[1.941,0.255,0.255], -[5.457,0.172,0.283], -[0.476,0.269,0.256], -[14.239,0.179,0.178], -[3.992,0.112,0.112], -[1.031,0.116,0.116], -[1.365,0.089,0.088] -]}] diff --git a/website/benchmark/dbms/results/mariadb-columnstore.c6a.4xlarge.json b/website/benchmark/dbms/results/mariadb-columnstore.c6a.4xlarge.json deleted file mode 100644 index 0a2d10f98f2..00000000000 --- a/website/benchmark/dbms/results/mariadb-columnstore.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "mariadb-columnstore (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.151,0.158,0.148], -[0.100,0.101,0.106], -[1.221,1.233,1.226], -[0.739,0.736,0.741], -[2.025,2.046,2.004], -[3.725,4.801,3.755], -[0.871,0.749,0.736], -[0.118,0.108,0.103], -[2.108,2.029,2.029], -[4.225,4.271,4.288], -[1.711,1.402,1.407], -[1.526,1.435,1.420], -[5.339,4.172,3.610], -[4.692,4.729,4.960], -[4.013,3.860,3.918], -[3.236,2.680,2.629], -[null,null,null], -[null,null,null], -[null,null,null], -[0.137,0.109,0.147], -[12.331,6.069,8.619], -[2.162,2.178,2.192], -[16.849,30.463,26.639], -[92,90.208,92.814], -[3.042,1.763,1.791], -[1.779,1.772,1.749], -[1.793,1.821,1.888], -[13.036,10.747,9.590], -[null,null,null], -[78.224,79.141,77.806], -[2.837,2.654,2.675], -[5.833,4.552,3.678], -[null,null,null], -[null,null,null], -[null,null,null], -[3.626,3.546,3.709], -[1.719,1.787,1.876], -[1.345,0.906,0.910], -[0.117,0.091,0.093], -[1.217,1.133,1.133], -[0.114,0.063,0.062], -[0.100,0.062,0.061], -[null,null,null] -]}] diff --git a/website/benchmark/dbms/results/monetdb.c6a.4xlarge.json b/website/benchmark/dbms/results/monetdb.c6a.4xlarge.json deleted file mode 100644 index 73b1113bd4d..00000000000 --- a/website/benchmark/dbms/results/monetdb.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "monetdb (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.000218,0.000157,0.000155], -[0.101903,0.019908,0.018439], -[0.282431,0.035987,0.034938], -[2.868,0.029387,0.029207], -[4.675,4.515,4.511], -[6.584,4.269,4.650], -[0.528827,0.063135,0.065742], -[0.506863,0.020966,0.021687], -[8.343,4.457,4.408], -[7.224,6.548,7.576], -[0.267003,0.233353,0.230444], -[0.347206,0.28358,0.266085], -[5.389,3.099,3.074], -[7.653,7.759,8.596], -[3.276,3.326,3.292], -[5.310,3.465,3.578], -[9.341,9.143,9.536], -[9.584,9.604,9.419], -[19.539,19.783,19.611], -[0.004509,0.000702,0.000643], -[20.801,1.570,1.603], -[2.752,0.418221,0.395884], -[14.717,0.800894,0.395477], -[14.429,1.804,1.869], -[1.386,0.159602,0.156426], -[0.189736,0.167664,0.168781], -[0.164681,0.176666,0.17126], -[3.005,3.113,3.882], -[null,null,null], -[2.751,2.846,2.676], -[7.937,2.579,2.447], -[5.120,3.492,3.467], -[22.862,22.567,23.211], -[33.437,18.889,19.043], -[18.898,19.583,19.047], -[14.774,12.984,13.803], -[3.865,0.322143,0.323117], -[0.192149,0.177791,0.175984], -[0.194173,0.159398,0.165201], -[0.680778,0.592252,0.560738], -[0.106465,0.10638,0.102692], -[0.154871,0.153752,0.155782], -[0.11459,0.09639,0.095594] -]}] diff --git a/website/benchmark/dbms/results/mysql-myisam.c6a.4xlarge.json b/website/benchmark/dbms/results/mysql-myisam.c6a.4xlarge.json deleted file mode 100644 index 88364415e90..00000000000 --- a/website/benchmark/dbms/results/mysql-myisam.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "mysql-myisam (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.00,0.00,0.00], -[283.32,276.83,274.52], -[276.93,278.29,283.27], -[28.83,23.63,21.55], -[46.41,40.81,40.93], -[467.04,467.39,469.08], -[31.02,25.89,24.20], -[277.89,275.3,277.3], -[329.34,325.8,325.35], -[342.86,338.43,336.95], -[282.03,279.87,281.22], -[277.74,282.68,282], -[335.66,334.83,336.44], -[305.24,310.39,307.3], -[337.41,338.52,342.94], -[308.66,307.34,306.27], -[738.38,748.44,740.75], -[738.75,734.01,738.25], -[867.01,872.92,868.84], -[25.65,20.61,18.46], -[312.39,313.67,306.66], -[301.66,305.12,308.01], -[298.12,298.44,312.4], -[311.34,309.9,311.85], -[281.87,278.5,275], -[277.46,277.46,277.46], -[280.75,278.04,281.76], -[263.9,417.39,406.88], -[707.21,711.96,705], -[668.1,668.33,665.96], -[330.31,333.36,331.94], -[506.57,506.18,500.53], -[2604.49,2681.96,2703.12], -[830.65,832.88,831.14], -[831.98,830.46,833.41], -[608.49,608.51,613.68], -[4.56,4.13,4.16], -[3.80,3.80,3.70], -[1.65,1.45,1.46], -[6.33,5.14,6.15], -[1.60,1.41,1.41], -[1.56,1.42,1.39], -[7.04,1.17,1.13] -]}] diff --git a/website/benchmark/dbms/results/mysql.c6a.4xlarge.json b/website/benchmark/dbms/results/mysql.c6a.4xlarge.json deleted file mode 100644 index b8619c12848..00000000000 --- a/website/benchmark/dbms/results/mysql.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "mysql (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[339.77,339.88,339.77], -[364.91,371.86,367.55], -[366.2,368.91,389.66], -[364.39,377.53,571.45], -[377.69,390.02,384.86], -[569.48,576.51,574.68], -[367.4,368.23,370.41], -[371.29,384.02,613.22], -[478.85,683.22,495.68], -[489.9,635.96,662.43], -[386.07,396.49,640.15], -[389.13,412.55,444.12], -[447.97,455.54,448.06], -[423.22,845.44,813.6], -[452.48,460.07,453.98], -[577.54,623.21,586.49], -[852.07,856.36,862.66], -[838.09,848.92,851.12], -[1006.37,1011.16,1023.17], -[369.76,375.61,415.28], -[412.45,419.9,456.62], -[411.65,432.88,482.2], -[412.73,420.73,429.5], -[551.16,577.62,545.45], -[382.89,394.76,386.37], -[380.9,391.4,385.05], -[385.3,394.67,460.32], -[388.95,394.7,387.21], -[800.33,807.90,807.11], -[706.03,745.27,718.9], -[450.9,489.59,530.97], -[625.5,651.93,647.32], -[2721.13,2792.12,2819.26], -[945.9,954.94,957.54], -[945.42,953.78,965.16], -[684.36,716.29,708.75], -[10.01,3.79,3.77], -[7.48,3.32,3.27], -[5.09,0.98,0.96], -[8.70,4.77,4.68], -[4.82,0.76,0.74], -[4.46,0.77,0.75], -[7.04,1.17,1.13] -]}] diff --git a/website/benchmark/dbms/results/postgresql.c6a.4xlarge.json b/website/benchmark/dbms/results/postgresql.c6a.4xlarge.json deleted file mode 100644 index a1d17e20065..00000000000 --- a/website/benchmark/dbms/results/postgresql.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "postgresql (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[439.753,309.785,282.017], -[317.874,254.238,254.941], -[262.883,263.072,263.090], -[32.421,5.310,5.060], -[57.134,42.648,42.334], -[358.423,356.315,358.342], -[31.524,5.350,4.994], -[263.145,263.193,263.165], -[323.659,322.858,321.918], -[327.395,326.170,326.231], -[265.983,265.681,265.912], -[269.984,265.336,265.379], -[284.096,284.560,282.234], -[277.250,279.455,280.035], -[285.660,286.200,283.611], -[66.605,32.023,38.282], -[312.452,304.431,305.391], -[289.209,290.449,287.578], -[331.706,327.485,334.428], -[24.646,2.543,2.263], -[267.561,267.496,267.524], -[267.729,267.690,268.184], -[263.074,263.120,267.040], -[267.602,267.488,267.494], -[263.141,263.859,263.137], -[262.923,263.102,263.113], -[262.885,263.088,263.114], -[267.864,269.127,268.204], -[303.376,306.925,308.664], -[263.221,263.119,263.148], -[270.814,270.575,270.294], -[278.342,275.925,276.224], -[584.599,576.932,591.502], -[462.576,446.962,439.779], -[429.930,417.696,416.704], -[296.875,297.283,295.140], -[3.461,0.842,0.794], -[2.179,0.564,0.558], -[2.258,0.566,0.416], -[2.805,1.311,1.317], -[2.936,0.820,0.615], -[2.197,0.736,0.535], -[1.983,0.320,0.312] -]}] diff --git a/website/benchmark/dbms/results/questdb.c6a.4xlarge.json b/website/benchmark/dbms/results/questdb.c6a.4xlarge.json deleted file mode 100644 index 4bbed6f1108..00000000000 --- a/website/benchmark/dbms/results/questdb.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "questdb (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.0155878,0.000283657,0.000328587], -[0.586734,0.0486348,0.0485346], -[6.41881,5.57954,6.25638], -[6.8554,3.11997,3.11813], -[47.8723,54.5425,53.1198], -[null,null,null], -[5.63439,3.70619,3.92043], -[0.699764,0.130462,0.126272], -[55.7179,62.8433,62.0045], -[null,59.6416,64.7753], -[63.7134,null,39.8928], -[46.759,46.8544,null], -[16.7181,10.9064,10.0735], -[79.9885,null,52.5617], -[16.1015,12.1835,13.1469], -[17.0225,11.7342,10.6194], -[24.1167,15.6749,16.8193], -[23.0848,15.3149,17.375], -[42.0965,26.2421,25.7593], -[0.0356335,0.027459,0.0289404], -[70.7679,71.5825,71.4573], -[null,null,null], -[null,null,null], -[77.0625,13.2257,12.7578], -[11.0775,2.90421,2.33398], -[3.31611,3.38837,3.35419], -[3.13233,5.2785,3.07075], -[null,null,null], -[null,null,null], -[186.032,185.627,180.963], -[26.7279,16.4799,18.8758], -[63.8785,32.1097,32.1561], -[64.4635,31.4538,35.654], -[118.897,null,119.015], -[null,null,null], -[19.6853,17.4427,16.7998], -[1.50985,0.877967,0.885536], -[0.805639,0.577352,0.534731], -[0.509284,0.448942,0.467679], -[1.37609,0.966942,0.912858], -[0.567887,0.394619,0.438952], -[0.508977,0.441015,0.40528], -[null,null,null] -]}] diff --git a/website/benchmark/dbms/results/redshift-serverless.serverless.json b/website/benchmark/dbms/results/redshift-serverless.serverless.json deleted file mode 100644 index cd47d4a8c84..00000000000 --- a/website/benchmark/dbms/results/redshift-serverless.serverless.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "redshift-serverless (serverless)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.436955,0.390225,0.387823], -[2.74529,0.280571,0.198074], -[7.24569,0.295612,0.386568], -[null,null,null], -[3.98825,0.488446,0.325645], -[4.24601,0.538891,0.690124], -[7.25492,0.288505,0.207515], -[5.99594,0.212732,0.281278], -[6.17534,0.681868,0.586073], -[6.37779,1.37865,1.31614], -[6.41405,0.549735,0.421345], -[6.64688,0.445678,0.477323], -[4.2294,0.361772,0.520471], -[4.73701,0.788258,0.743465], -[4.49038,0.670446,0.52727], -[6.19886,0.663381,0.583588], -[5.9867,0.679607,0.672772], -[2.76661,0.567555,0.51494], -[6.20219,0.816422,0.760568], -[2.51526,0.053,0.025066], -[2.96003,0.282737,0.226979], -[6.79648,0.57495,0.400798], -[6.37854,0.92746,1.05793], -[25.8462,0.902664,0.905365], -[5.00521,0.247895,0.308836], -[6.38373,0.308781,0.244082], -[4.25427,0.229966,0.247201], -[5.98382,0.398218,0.455249], -[null,null,null], -[6.52367,1.35877,1.30562], -[7.36935,0.536226,0.582304], -[7.05948,0.621982,0.639653], -[4.62901,0.954522,0.908651], -[4.95273,1.03062,1.10289], -[4.71404,1.06378,1.04157], -[4.8201,0.499996,0.575546], -[5.32757,0.566517,0.562058], -[5.37681,0.626458,0.658628], -[5.23137,0.470622,0.540079], -[6.09326,0.561312,0.574978], -[4.86561,0.595546,0.534209], -[4.34256,0.433804,0.414541], -[4.19814,0.288269,0.415328] -]}] diff --git a/website/benchmark/dbms/results/redshift.4x.ra3.xplus.json b/website/benchmark/dbms/results/redshift.4x.ra3.xplus.json deleted file mode 100644 index e902175ddf7..00000000000 --- a/website/benchmark/dbms/results/redshift.4x.ra3.xplus.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "redshift (4x.ra3.xplus)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.081437,0.022376,0.022491], -[2.54868,0.024112,0.024072], -[2.62053,0.049948,0.049768], -[null,null,null], -[0.62714,0.447655,0.43262], -[1.14153,1.09479,1.09441], -[0.153399,0.053941,0.048224], -[3.76153,0.032235,0.030949], -[4.43439,0.746808,0.723719], -[6.13424,2.20458,2.20031], -[4.35338,0.220122,0.228684], -[5.15139,0.231856,0.230993], -[5.00006,1.10841,1.12871], -[5.80125,1.86531,1.90209], -[5.16246,1.21239,1.14848], -[0.579428,0.535859,0.546178], -[6.05764,2.17455,2.18286], -[5.86612,2.52634,2.48672], -[7.80075,3.35512,3.41153], -[2.54112,0.036378,0.035944], -[6.47189,2.26909,2.2673], -[6.95344,2.61929,2.65637], -[12.9508,6.85457,6.99], -[25.3022,11.8857,11.8493], -[4.37592,0.452737,0.452867], -[4.17199,0.469457,0.476302], -[6.24746,0.470935,0.482502], -[6.00065,2.08332,2.08059], -[null,null,null], -[22.2567,18.5376,18.3441], -[5.37492,0.868068,0.849486], -[7.29067,1.06155,1.11209], -[7.7832,4.07132,4.00384], -[8.95385,5.15488,5.21863], -[9.2232,5.32052,5.73207], -[4.99205,0.664347,0.618918], -[6.82279,1.89738,1.89398], -[7.09077,2.19008,2.26612], -[5.29731,0.19626,0.204603], -[6.07138,0.276315,0.267161], -[4.03108,0.054134,0.058568], -[4.35647,0.061157,0.053367], -[5.23605,0.037217,0.036335] -]}] diff --git a/website/benchmark/dbms/results/singlestore.c6a.4xlarge.json b/website/benchmark/dbms/results/singlestore.c6a.4xlarge.json deleted file mode 100644 index 8b7b7ccd4e9..00000000000 --- a/website/benchmark/dbms/results/singlestore.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "singlestore (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.09,0.00,0.00], -[0.23,0.04,0.01], -[0.47,0.15,0.15], -[0.63,0.09,0.08], -[1.65,1.23,1.20], -[7.96,2.79,2.63], -[0.10,0.00,0.00], -[0.17,0.02,0.02], -[1.90,1.40,1.39], -[4.79,3.52,3.48], -[0.94,0.22,0.23], -[0.89,0.24,0.23], -[5.82,2.26,2.25], -[6.97,4.62,4.66], -[3.05,2.28,2.31], -[3.92,2.70,2.28], -[5.83,4.48,4.42], -[4.76,4.13,4.12], -[14.03,null,null], -[0.57,0.05,0.04], -[18.18,1.74,1.94], -[20.85,2.17,0.98], -[31.98,3.12,1.22], -[78.96,3.35,108.85], -[2.82,0.39,0.32], -[1.83,0.44,0.35], -[2.81,0.33,0.32], -[18.33,2.57,1.15], -[null,null,null], -[3.56,2.40,2.40], -[3.83,1.11,1.11], -[7.35,1.73,1.70], -[null,null,null], -[null,null,null], -[null,null,null], -[2.53,1.92,1.84], -[0.92,0.23,0.19], -[0.84,0.15,0.08], -[0.70,0.05,0.05], -[3.12,0.38,0.36], -[0.29,0.03,0.03], -[0.22,0.06,0.02], -[0.27,0.11,0.12] -]}] diff --git a/website/benchmark/dbms/results/snowflake.2xl.json b/website/benchmark/dbms/results/snowflake.2xl.json deleted file mode 100644 index 39054eb345f..00000000000 --- a/website/benchmark/dbms/results/snowflake.2xl.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "snowflake (2xl)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.177,0.052,0.090], -[0.903,0.324,0.782], -[0.458,2.909,0.275], -[0.881,0.316,0.166], -[0.404,0.257,0.256], -[0.481,0.325,0.339], -[0.056,0.062,0.060], -[0.183,0.324,0.280], -[0.444,0.314,0.320], -[0.408,0.426,0.417], -[0.345,0.241,0.253], -[0.406,0.245,0.235], -[0.521,1.259,0.326], -[0.466,0.493,0.526], -[0.447,0.349,0.362], -[0.327,0.322,0.302], -[0.462,0.508,0.461], -[0.489,0.481,0.455], -[0.731,0.659,0.674], -[0.151,0.156,0.160], -[0.832,0.339,0.312], -[0.289,0.292,0.340], -[0.591,0.484,0.419], -[2.661,0.716,0.696], -[0.190,0.199,0.377], -[0.181,0.182,0.194], -[0.220,0.209,0.195], -[0.368,0.330,0.347], -[0.677,0.645,0.646], -[0.877,0.886,0.871], -[0.415,0.282,0.301], -[1.265,0.404,0.550], -[0.786,0.750,0.757], -[0.905,0.835,0.841], -[0.864,0.865,0.823], -[0.352,0.360,0.364], -[0.201,0.186,0.200], -[0.143,0.137,0.276], -[0.290,0.132,0.146], -[0.310,0.301,0.304], -[0.212,0.197,0.170], -[0.223,0.174,0.177], -[0.172,0.172,0.172] -]}] diff --git a/website/benchmark/dbms/results/snowflake.3xl.json b/website/benchmark/dbms/results/snowflake.3xl.json deleted file mode 100644 index c4890b6b407..00000000000 --- a/website/benchmark/dbms/results/snowflake.3xl.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "snowflake (3xl)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.165,0.061,0.049], -[1.356,1.252,1.502], -[1.287,0.470,0.325], -[0.627,0.331,0.181], -[0.265,0.265,0.483], -[0.887,0.450,0.298], -[0.054,0.066,0.062], -[0.182,0.222,0.194], -[0.408,0.438,0.319], -[0.434,0.462,0.411], -[1.357,0.247,0.412], -[0.343,0.250,0.517], -[0.273,0.289,0.600], -[0.404,0.405,0.392], -[0.379,0.304,0.455], -[0.275,0.271,0.264], -[0.418,0.386,0.388], -[0.417,0.434,0.567], -[0.753,0.759,0.506], -[0.291,0.307,0.400], -[0.954,0.423,0.280], -[0.568,0.562,0.248], -[0.568,0.477,0.496], -[1.458,0.492,0.514], -[0.179,0.300,0.184], -[0.165,0.169,0.176], -[0.197,0.186,0.190], -[0.289,0.547,0.397], -[0.513,0.544,0.632], -[0.766,0.754,0.775], -[0.389,0.374,0.383], -[0.484,0.297,0.286], -[0.505,0.505,0.734], -[0.656,0.598,0.621], -[0.634,0.646,0.609], -[0.309,0.298,0.370], -[0.192,0.219,0.212], -[0.840,0.174,0.139], -[0.172,0.163,0.151], -[0.323,0.296,0.347], -[0.200,0.154,0.144], -[0.191,0.121,0.125], -[0.137,0.233,0.148] -]}] diff --git a/website/benchmark/dbms/results/snowflake.4xl.json b/website/benchmark/dbms/results/snowflake.4xl.json deleted file mode 100644 index 713ed9e445c..00000000000 --- a/website/benchmark/dbms/results/snowflake.4xl.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "snowflake (4xl)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.164,0.061,0.078], -[2.471,2.436,1.927], -[1.656,0.222,0.639], -[0.336,1.244,0.206], -[0.435,0.414,0.373], -[0.520,0.495,0.326], -[0.052,0.051,0.057], -[0.244,0.515,0.358], -[0.473,0.477,0.659], -[0.706,0.523,0.499], -[0.619,0.361,0.303], -[0.463,0.367,0.290], -[0.385,0.319,0.376], -[0.661,0.436,0.452], -[0.446,0.317,0.336], -[0.504,0.283,0.395], -[0.478,1.395,0.350], -[0.555,1.041,0.993], -[0.565,0.558,1.473], -[0.310,0.684,0.278], -[0.637,1.202,0.249], -[0.467,0.294,0.436], -[0.671,0.478,0.611], -[6.262,0.449,0.425], -[0.476,0.213,0.184], -[0.552,0.241,0.201], -[0.458,0.415,0.402], -[0.339,0.357,0.322], -[0.732,0.549,0.483], -[0.914,0.757,0.743], -[0.718,0.310,0.606], -[0.605,0.363,0.425], -[0.468,0.860,0.784], -[0.868,1.787,0.713], -[0.807,0.691,0.544], -[0.485,0.338,0.672], -[0.263,0.221,0.230], -[0.147,0.145,0.139], -[0.135,0.239,0.136], -[0.322,0.378,0.348], -[0.236,0.138,0.132], -[0.193,0.124,0.139], -[0.146,0.145,0.139] -]}] diff --git a/website/benchmark/dbms/results/snowflake.l.json b/website/benchmark/dbms/results/snowflake.l.json deleted file mode 100644 index a95126636ec..00000000000 --- a/website/benchmark/dbms/results/snowflake.l.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "snowflake (l)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.067,0.054,0.051], -[1.158,1.409,0.190], -[1.470,0.566,0.715], -[0.892,0.534,0.264], -[0.474,0.499,0.489], -[0.950,0.650,0.699], -[0.069,0.055,0.110], -[0.317,0.307,0.321], -[0.949,0.593,0.654], -[0.713,1.099,0.860], -[0.622,0.370,0.404], -[0.457,0.369,0.414], -[0.610,0.566,0.653], -[0.970,1.006,0.976], -[1.517,0.636,0.603], -[0.532,0.541,0.533], -[1.018,1.001,1.022], -[0.942,0.996,0.940], -[2.246,1.596,1.560], -[0.181,0.184,0.200], -[1.135,0.788,0.609], -[0.669,0.528,0.524], -[1.164,0.827,0.882], -[3.545,2.214,2.107], -[0.559,0.431,0.426], -[0.340,0.296,0.383], -[0.695,0.314,0.368], -[0.628,0.658,0.637], -[1.511,1.385,1,440], -[1.390,1.418,1.322], -[1.107,0.687,0.537], -[1.026,0.737,0.659], -[1.712,1.681,1.728], -[2.141,2.130,2.225], -[2.163,2.157,2.110], -[0.650,0.619,0.627], -[0.204,0.195,0.225], -[0.159,0.152,0.156], -[0.146,0.136,0.150], -[0.359,0.290,0.364], -[0.196,0.129,0.227], -[0.201,0.128,0.143], -[0.176,0.129,0.146] -]}] diff --git a/website/benchmark/dbms/results/snowflake.m.json b/website/benchmark/dbms/results/snowflake.m.json deleted file mode 100644 index 3b0543d7b6b..00000000000 --- a/website/benchmark/dbms/results/snowflake.m.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "snowflake (m)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.054,0.062,0.064], -[0.698,0.563,0.407], -[0.586,0.412,0.266], -[1.386,0.363,0.301], -[0.814,0.825,0.984], -[1.303,1.024,1.048], -[0.069,0.066,0.062], -[0.334,0.254,0.270], -[1.043,0.952,0.993], -[1.210,1.209,1.171], -[0.667,0.483,0.456], -[0.543,0.495,0.500], -[1.005,0.889,0.888], -[1.644,1.646,1.652], -[1.054,1.044,0.966], -[0.893,0.874,0.907], -[1.737,1.779,1.837], -[1.518,1.539,1.526], -[3.082,2.818,2.842], -[0.309,0.286,0.256], -[1.594,1.017,0.993], -[0.781,0.853,0.735], -[1.461,1.226,1.080], -[5.308,2.974,2.642], -[0.511,0.625,0.467], -[0.405,0.382,0.439], -[0.601,0.535,0.471], -[0.947,1.624,1.192], -[2.631,2.486,2.490], -[1.938,1.960,1.954], -[1.930,0.830,0.835], -[1.359,1.140,1.062], -[3.599,3.623,3.621], -[3.619,3.741,3.663], -[3.725,3.614,3.786], -[1.149,1.126,1.055], -[0.202,0.207,0.196], -[0.152,0.139,0.145], -[0.149,0.144,0.148], -[0.383,0.287,0.294], -[0.203,0.137,0.119], -[0.200,0.312,0.137], -[0.149,0.130,0.214] -]}] diff --git a/website/benchmark/dbms/results/snowflake.s.json b/website/benchmark/dbms/results/snowflake.s.json deleted file mode 100644 index d6d096cb711..00000000000 --- a/website/benchmark/dbms/results/snowflake.s.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "snowflake (s)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.186,0.060,0.062], -[0.980,0.574,0.311], -[0.977,0.554,0.426], -[0.686,0.573,0.404], -[1.386,1.384,1.349], -[1.871,1.697,1.704], -[0.052,0.059,0.227], -[0.309,0.536,0.508], -[1.768,1.631,1.635], -[2.039,2.219,1.908], -[0.807,0.647,0.587], -[0.763,0.690,0.631], -[1.403,1.586,1.404], -[2.593,2.584,2.554], -[1.670,1.538,1.653], -[1.659,1.509,1.514], -[2.875,2.990,2.998], -[2.605,2.549,2.598], -[6.120,5.894,5.766], -[0.320,0.431,0.416], -[2.406,1.703,1.609], -[1.189,1.186,1.163], -[2.104,1.441,1.370], -[7.144,5.174,4.139], -[0.839,0.659,0.641], -[0.527,0.518,0.509], -[0.633,0.621,0.695], -[1.491,1.509,1.514], -[4.848,4.485,4.571], -[3.067,3.106,3.098], -[1.521,1.224,1.236], -[1.839,1.690,1.497], -[5.692,5.751,6.087], -[6.733,6.755,6.712], -[6.722,6.709,6.676], -[1.704,1.686,1.676], -[0.203,0.231,0.218], -[0.151,0.134,0.214], -[0.140,0.156,0.163], -[0.317,0.328,0.319], -[0.166,0.133,0.141], -[0.166,0.120,0.140], -[0.120,0.119,0.126] -]}] diff --git a/website/benchmark/dbms/results/snowflake.xl.json b/website/benchmark/dbms/results/snowflake.xl.json deleted file mode 100644 index 5b688792378..00000000000 --- a/website/benchmark/dbms/results/snowflake.xl.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "snowflake (xl)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.071,0.053,0.057], -[0.998,0.610,0.240], -[0.420,1.138,1.051], -[0.653,0.264,0.178], -[0.352,0.312,0.349], -[1.126,0.431,0.420], -[0.067,0.057,0.054], -[0.225,0.217,0.200], -[0.617,0.366,0.371], -[1.006,0.541,0.498], -[0.463,0.425,0.293], -[0.431,0.360,0.339], -[0.392,0.371,0.386], -[0.588,0.581,0.590], -[0.634,0.414,0.400], -[0.368,0.410,0.388], -[0.594,0.639,0.663], -[0.616,0.581,0.569], -[1.092,0.933,0.901], -[0.493,0.213,0.160], -[0.886,0.480,0.442], -[0.448,0.337,0.399], -[0.840,0.572,0.505], -[2.251,1.230,0.959], -[0.295,0.253,0.241], -[0.214,0.239,0.278], -[0.261,0.232,0.314], -[0.422,0.429,0.403], -[0.892,0.934,0.883], -[1.041,1.017,1.009], -[0.715,0.442,0.363], -[0.845,0.413,0.461], -[1.101,1.085,1.102], -[1.294,1.272,1.339], -[1.839,1.327,1.241], -[0.439,0.399,0.393], -[0.199,0.211,0.190], -[0.157,0.143,0.140], -[0.145,0.157,0.141], -[0.331,0.291,0.333], -[0.173,0.214,0.138], -[0.189,0.150,0.159], -[0.135,0.149,0.138] -]}] diff --git a/website/benchmark/dbms/results/snowflake.xs.json b/website/benchmark/dbms/results/snowflake.xs.json deleted file mode 100644 index 5cf9e5aac47..00000000000 --- a/website/benchmark/dbms/results/snowflake.xs.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "snowflake (xs)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[0.169,0.055,0.056], -[1.184,0.582,0.386], -[1.350,0.560,0.568], -[1.270,0.554,0.538], -[2.516,2.564,2.506], -[2.935,2.649,2.670], -[0.052,0.050,0.064], -[0.383,0.387,0.397], -[3.249,2.993,3.014], -[3.589,3.627,3.887], -[1.243,0.986,0.966], -[1.325,1.080,1.073], -[2.038,2.046,2.035], -[3.738,3.626,3.718], -[2.318,2.159,2.176], -[2.733,2.637,2.668], -[5.607,5.683,5.667], -[3.978,3.923,3.879], -[10.085,9.871,9.844], -[0.450,0.375,0.469], -[5.474,3.103,3.060], -[2.012,1.982,1.971], -[3.365,2.471,2.501], -[11.960,10.619,9.518], -[1.074,1.059,1.026], -[0.856,0.846,0.879], -[1.100,1.085,1.083], -[3.057,3.228,3.117], -[9.406,9.019,9.158], -[6.196,6.243,6.911], -[2.906,2.343,2.017], -[2.954,2.666,2.565], -[9.459,9.565,9.557], -[9.555,9.529,9.368], -[9.409,9.185,9.294], -[2.796,2.880,2.685], -[0.299,0.249,0.262], -[0.156,0.145,0.180], -[0.147,0.146,0.160], -[0.371,0.357,0.356], -[0.166,0.133,0.155], -[0.218,0.140,0.135], -[0.140,0.152,0.158] -]}] diff --git a/website/benchmark/dbms/results/sqlite.c6a.4xlarge.json b/website/benchmark/dbms/results/sqlite.c6a.4xlarge.json deleted file mode 100644 index 49cce70b560..00000000000 --- a/website/benchmark/dbms/results/sqlite.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "sqlite (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[752.739,2.003,1.2], -[304.302,291.521,286.965], -[293.964,287.619,287.219], -[758.302,5.879,5.65], -[836.393,48.593,48.452], -[362.605,344.884,356.245], -[763.993,11.602,10.795], -[296.348,286.879,287.557], -[365.816,360.339,354.126], -[374.403,365.196,362.261], -[302.989,293.888,298.432], -[303.64,291.729,295.347], -[316.824,298.18,301.006], -[320.665,301.15,305.227], -[313.593,301.021,301.626], -[794.881,47,47.225], -[355.346,344.615,342.442], -[316.499,305.971,305.007], -[398.177,380.383,385.571], -[751.82,5.082,4.913], -[295.522,286.573,287.368], -[298.58,287.182,288.303], -[296.474,288.747,288.638], -[296.579,287.127,287.361], -[304.709,286.865,287.56], -[300.391,290.633,288.587], -[294.605,286.91,287.799], -[305.986,312.111,305.626], -[null,null,null], -[411.225,397.614,394.253], -[307.711,295.181,300.266], -[312.472,299.079,298.19], -[386.674,378.347,376.963], -[409.742,409.554,420.273], -[468.73,453.709,458.149], -[366.015,347.446,346.728], -[2.911,0.781,0.757], -[1.599,0.609,0.587], -[1.288,0.256,0.238], -[2.469,1.582,1.52], -[1.274,0.303,0.283], -[1.322,0.317,0.314], -[1.498,0.602,0.613] -]}] diff --git a/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.compression.json b/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.compression.json deleted file mode 100644 index 148b06847a4..00000000000 --- a/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.compression.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "timescaledb (c6a.4xlarge.compression)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[2.28686,1.63642,1.64263], -[32.6848,1.63476,1.40052], -[60.8633,3.70484,3.59342], -[36.4029,2.87091,2.80739], -[110.391,38.9688,38.0549], -[147.379,66.2513,65.6379], -[33.0294,2.92031,2.84375], -[33.0221,1.2984,1.19227], -[115.694,47.4651,47.0125], -[156.347,51.577,51.2694], -[68.3301,4.75521,4.68007], -[77.4356,5.55128,5.56577], -[49.7741,11.2911,11.3265], -[81.1014,14.9111,14.9541], -[82.9569,14.6156,14.6331], -[62.0338,26.399,26.3351], -[103.259,36.4122,36.6076], -[92.8828,26.2395,25.8991], -[144.281,63.5102,63.7661], -[7.00679,0.573073,0.536283], -[75.0203,7.86344,7.90495], -[81.2825,9.15868,9.01775], -[104.084,13.9528,13.8435], -[132.531,81.522,82.1561], -[80.6965,3.28231,3.16574], -[39.7693,2.51443,2.43849], -[80.4245,3.26941,3.13916], -[104.015,13.7044,13.5313], -[307.26,253.127,252.147], -[42.8549,22.4187,22.0325], -[137.601,14.9592,14.6804], -[136.767,22.8007,22.131], -[263.005,168.551,163.355], -[156.919,92.6308,91.702], -[160.842,96.0512,97.1773], -[62.8357,28.0336,28.7397], -[1.75869,0.561604,0.541215], -[0.46607,0.191863,0.19021], -[0.303671,0.137579,0.136615], -[2.32031,1.49223,1.52369], -[0.563764,0.14192,0.138234], -[0.372428,0.122989,0.123709], -[0.448574,0.159092,0.154687] -]}] diff --git a/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.json b/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.json deleted file mode 100644 index 67bb1a09555..00000000000 --- a/website/benchmark/dbms/results/timescaledb.c6a.4xlarge.json +++ /dev/null @@ -1,45 +0,0 @@ -[{"system": "timescaledb (c6a.4xlarge)", "version": "", "data_size": 100000000, "time": "2022-07-01 00:00:00", "comments": "", "result": [ -[437.700,215.793,176.420], -[327.026,259.568,244.578], -[262.978,263.090,263.083], -[262.807,263.046,266.847], -[337.497,334.964,330.852], -[355.689,356.801,362.894], -[262.762,263.012,262.968], -[263.055,263.016,263.028], -[319.928,319.388,320.704], -[323.584,322.224,322.488], -[265.979,265.465,265.375], -[266.019,265.543,265.462], -[277.018,276.300,276.595], -[280.352,279.251,279.572], -[279.915,279.896,279.674], -[296.377,298.506,297.659], -[314.448,314.605,312.570], -[302.668,302.672,303.039], -[325.810,324.061,324.376], -[262.447,262.698,262.704], -[267.581,267.467,267.482], -[268.085,267.466,267.696], -[263.391,263.097,263.126], -[38.291,0.435,0.335], -[0.127,0.005,0.005], -[263.138,263.100,263.092], -[0.889,0.341,0.339], -[267.586,267.498,267.491], -[289.086,290.012,290.093], -[263.220,263.071,263.109], -[274.780,273.995,273.998], -[282.217,281.390,281.470], -[429.273,426.588,439.431], -[448.808,418.724,418.207], -[455.196,422.750,423.142], -[299.263,296.937,297.261], -[18.693,1.552,1.481], -[18.125,0.945,0.937], -[18.528,1.062,0.902], -[18.268,1.779,1.770], -[19.615,1.965,1.966], -[18.970,1.435,1.430], -[18.330,1.153,0.952] -]}] From fd9d897df2cec949a9806540ed6dcdeb22a03eb3 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Jul 2022 23:18:44 +0000 Subject: [PATCH 470/627] Fix ActionsDAG for window expressions --- src/Interpreters/ActionsVisitor.cpp | 32 +++---------------- src/Interpreters/ActionsVisitor.h | 9 ------ src/Interpreters/ExpressionAnalyzer.cpp | 31 +++++++++++------- src/Interpreters/ExpressionAnalyzer.h | 1 + src/Interpreters/GetAggregatesVisitor.cpp | 10 +++--- ...sion_with_aggregation_expression.reference | 2 ++ ...expression_with_aggregation_expression.sql | 12 +++++++ 7 files changed, 44 insertions(+), 53 deletions(-) create mode 100644 tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.reference create mode 100644 tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.sql diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index e25a6260787..f5d08efe83a 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -963,40 +963,18 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & // Don't need to do anything more for window functions here -- the // resulting column is added in ExpressionAnalyzer, similar to the // aggregate functions. - if (data.window_dependancy_state == WindowDependancyState::MAY_DEPEND) - data.window_function_in_subtree = true; return; } else if (node.compute_after_window_functions) { - // In this case we have window function call in subtree - // Add this function to actions index only if Data::build_expression_with_window_functions is set. - data.window_dependancy_state = WindowDependancyState::MAY_DEPEND; - for (const auto & arg : node.arguments->children) - { - data.window_function_in_subtree = false; - visit(arg, data); - // There is no point to check value of window_function_in_subtree here, - // because after window functions are computed, this variable is always false. - } - data.window_dependancy_state = WindowDependancyState::NONE; if (!data.build_expression_with_window_functions) - return; - } - else if (data.window_dependancy_state == WindowDependancyState::MAY_DEPEND) - { - // This function may depend on evaluation of window function. - // We need to check it and add it to the index only if Data::build_expression_with_window_functions is set. - bool subtree_contains_window_call = false; - for (const auto & arg : node.arguments->children) { - data.window_function_in_subtree = false; - visit(arg, data); - subtree_contains_window_call = subtree_contains_window_call || data.window_function_in_subtree; - } - data.window_function_in_subtree = subtree_contains_window_call; - if (subtree_contains_window_call && !data.build_expression_with_window_functions) + for (const auto & arg : node.arguments->children) + { + visit(arg, data); + } return; + } } // An aggregate function can also be calculated as a window function, but we diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index afdf2948d47..5b5a3d31da2 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -121,12 +121,6 @@ class ActionsMatcher public: using Visitor = ConstInDepthNodeVisitor; - enum class WindowDependancyState - { - NONE, - MAY_DEPEND, - }; - struct Data : public WithContext { SizeLimits set_size_limit; @@ -150,9 +144,6 @@ public: */ int next_unique_suffix; - WindowDependancyState window_dependancy_state = WindowDependancyState::NONE; - bool window_function_in_subtree = false; - Data( ContextPtr context_, SizeLimits set_size_limit_, diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b79dc9c80e9..74d4e026cce 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1497,6 +1497,24 @@ void SelectQueryExpressionAnalyzer::appendExpressionsAfterWindowFunctions(Expres } } +void SelectQueryExpressionAnalyzer::appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node) +{ + if (auto * function = node->as()) + { + // Skip window function columns here -- they are calculated after + // other SELECT expressions by a special step. + if (function->is_window_function) + return; + if (function->compute_after_window_functions) + { + for (auto & arg : function->arguments->children) + appendSelectSkipWindowExpressions(step, arg); + return; + } + } + step.addRequiredOutput(node->getColumnName()); +} + bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types) { const auto * select_query = getAggregatingQuery(); @@ -1521,18 +1539,7 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, getRootActions(select_query->select(), only_types, step.actions()); for (const auto & child : select_query->select()->children) - { - if (const auto * function = typeid_cast(child.get()); - function - && (function->is_window_function || function->compute_after_window_functions)) - { - // Skip window function columns here -- they are calculated after - // other SELECT expressions by a special step. - continue; - } - - step.addRequiredOutput(child->getColumnName()); - } + appendSelectSkipWindowExpressions(step, child); } ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 167c3dfd918..aae45482a97 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -410,6 +410,7 @@ private: void appendWindowFunctionsArguments(ExpressionActionsChain & chain, bool only_types); void appendExpressionsAfterWindowFunctions(ExpressionActionsChain & chain, bool only_types); + void appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node); /// After aggregation: bool appendHaving(ExpressionActionsChain & chain, bool only_types); diff --git a/src/Interpreters/GetAggregatesVisitor.cpp b/src/Interpreters/GetAggregatesVisitor.cpp index a9d96a6d15a..b20b088b492 100644 --- a/src/Interpreters/GetAggregatesVisitor.cpp +++ b/src/Interpreters/GetAggregatesVisitor.cpp @@ -17,8 +17,7 @@ struct WindowExpressionsCollectorChildInfo // evaluation of window functions. Expression is collected only if // it's not a part of another expression. // -// Also all collected AST nodes are marked as dependent on window function. -// This information is used during ActionsDAG building process. +// Information about window function dependancy is used during ActionsDAG building process. struct WindowExpressionsCollectorMatcher { using ChildInfo = WindowExpressionsCollectorChildInfo; @@ -63,11 +62,12 @@ struct WindowExpressionsCollectorMatcher result.update(subtree_result); } - // We mark functions only on the top of AST - if ((!parent || !parent->as()) && result.window_function_in_subtree) + // We mark functions if they should be computed after WindowStep + if (result.window_function_in_subtree) { - expressions_with_window_functions.push_back(func); func->compute_after_window_functions = true; + if ((!parent || !parent->as())) + expressions_with_window_functions.push_back(func); } return result; diff --git a/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.reference b/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.reference new file mode 100644 index 00000000000..e58e9764b39 --- /dev/null +++ b/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.reference @@ -0,0 +1,2 @@ +100 +100 diff --git a/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.sql b/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.sql new file mode 100644 index 00000000000..f92053d47ae --- /dev/null +++ b/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.sql @@ -0,0 +1,12 @@ +SELECT + sum(a)*100/sum(sum(a)) OVER (PARTITION BY b) AS r +FROM +( + SELECT 1 AS a, 2 AS b + UNION ALL + SELECT 3 AS a, 4 AS b + UNION ALL + SELECT 5 AS a, 2 AS b + +) AS t +GROUP BY b; From d046132265a63fe5a639636bb19036da42db1b06 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Jul 2022 23:47:46 +0000 Subject: [PATCH 471/627] Allow to use higher-order functions in window expressions --- src/Interpreters/ActionsVisitor.cpp | 6 ++++++ src/Interpreters/ExpressionAnalyzer.cpp | 3 ++- src/Interpreters/GetAggregatesVisitor.cpp | 2 +- ...expression_with_aggregation_expression.reference | 13 +++++++++++++ ...indow_expression_with_aggregation_expression.sql | 4 ++++ 5 files changed, 26 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index f5d08efe83a..b62690b7a3a 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -971,6 +971,12 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & { for (const auto & arg : node.arguments->children) { + if (auto const * function = arg->as(); + function && function->name == "lambda") + { + // Lambda function is a special case. It shouldn't be visited here. + continue; + } visit(arg, data); } return; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 74d4e026cce..483ffad67b7 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1503,7 +1503,8 @@ void SelectQueryExpressionAnalyzer::appendSelectSkipWindowExpressions(Expression { // Skip window function columns here -- they are calculated after // other SELECT expressions by a special step. - if (function->is_window_function) + // Also skipping lambda functions because they can't be explicitly evaluated. + if (function->is_window_function || function->name == "lambda") return; if (function->compute_after_window_functions) { diff --git a/src/Interpreters/GetAggregatesVisitor.cpp b/src/Interpreters/GetAggregatesVisitor.cpp index b20b088b492..dd958693d89 100644 --- a/src/Interpreters/GetAggregatesVisitor.cpp +++ b/src/Interpreters/GetAggregatesVisitor.cpp @@ -17,7 +17,7 @@ struct WindowExpressionsCollectorChildInfo // evaluation of window functions. Expression is collected only if // it's not a part of another expression. // -// Information about window function dependancy is used during ActionsDAG building process. +// Information about window function dependency is used during ActionsDAG building process. struct WindowExpressionsCollectorMatcher { using ChildInfo = WindowExpressionsCollectorChildInfo; diff --git a/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.reference b/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.reference index e58e9764b39..494a1a0cd49 100644 --- a/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.reference +++ b/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.reference @@ -1,2 +1,15 @@ 100 100 +-- { echoOn } +SELECT arrayMap(x -> (x + 1), groupArray(number) OVER ()) AS result +FROM numbers(10); +[1,2,3,4,5,6,7,8,9,10] +[1,2,3,4,5,6,7,8,9,10] +[1,2,3,4,5,6,7,8,9,10] +[1,2,3,4,5,6,7,8,9,10] +[1,2,3,4,5,6,7,8,9,10] +[1,2,3,4,5,6,7,8,9,10] +[1,2,3,4,5,6,7,8,9,10] +[1,2,3,4,5,6,7,8,9,10] +[1,2,3,4,5,6,7,8,9,10] +[1,2,3,4,5,6,7,8,9,10] diff --git a/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.sql b/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.sql index f92053d47ae..21da3c5f403 100644 --- a/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.sql +++ b/tests/queries/0_stateless/02354_window_expression_with_aggregation_expression.sql @@ -10,3 +10,7 @@ FROM ) AS t GROUP BY b; + +-- { echoOn } +SELECT arrayMap(x -> (x + 1), groupArray(number) OVER ()) AS result +FROM numbers(10); From b67405915e6c41de92a3098a87086744455b2137 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jul 2022 02:01:41 +0000 Subject: [PATCH 472/627] keep LowCardinality type in tuple() function --- src/Functions/tuple.cpp | 1 + .../0_stateless/02354_tuple_lowcardinality.reference | 4 ++++ tests/queries/0_stateless/02354_tuple_lowcardinality.sql | 6 ++++++ 3 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/02354_tuple_lowcardinality.reference create mode 100644 tests/queries/0_stateless/02354_tuple_lowcardinality.sql diff --git a/src/Functions/tuple.cpp b/src/Functions/tuple.cpp index 4238b12157a..5e85984bee0 100644 --- a/src/Functions/tuple.cpp +++ b/src/Functions/tuple.cpp @@ -55,6 +55,7 @@ public: /// tuple(..., Nothing, ...) -> Tuple(..., Nothing, ...) bool useDefaultImplementationForNothing() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { diff --git a/tests/queries/0_stateless/02354_tuple_lowcardinality.reference b/tests/queries/0_stateless/02354_tuple_lowcardinality.reference new file mode 100644 index 00000000000..435417d7f2b --- /dev/null +++ b/tests/queries/0_stateless/02354_tuple_lowcardinality.reference @@ -0,0 +1,4 @@ +Tuple(LowCardinality(String), LowCardinality(UInt8)) +Tuple(LowCardinality(String), LowCardinality(UInt8)) +Tuple(LowCardinality(String), LowCardinality(UInt8)) +Tuple(LowCardinality(String), LowCardinality(UInt8)) diff --git a/tests/queries/0_stateless/02354_tuple_lowcardinality.sql b/tests/queries/0_stateless/02354_tuple_lowcardinality.sql new file mode 100644 index 00000000000..44b64aab327 --- /dev/null +++ b/tests/queries/0_stateless/02354_tuple_lowcardinality.sql @@ -0,0 +1,6 @@ +SET allow_suspicious_low_cardinality_types = 1; + +SELECT toTypeName(tuple(toLowCardinality('1'), toLowCardinality(1))); +SELECT toTypeName(tuple(materialize(toLowCardinality('1')), toLowCardinality(1))); +SELECT toTypeName(tuple(toLowCardinality('1'), materialize(toLowCardinality(1)))); +SELECT toTypeName(tuple(materialize(toLowCardinality('1')), materialize(toLowCardinality(1)))); From 72fe4ce680379e8ac908065713ae4492a7b26268 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jul 2022 03:12:42 +0000 Subject: [PATCH 473/627] fix function toColumnTypeName with LowCardinality --- src/Functions/toColumnTypeName.cpp | 2 ++ tests/queries/0_stateless/02355_column_type_name_lc.reference | 1 + tests/queries/0_stateless/02355_column_type_name_lc.sql | 1 + 3 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02355_column_type_name_lc.reference create mode 100644 tests/queries/0_stateless/02355_column_type_name_lc.sql diff --git a/src/Functions/toColumnTypeName.cpp b/src/Functions/toColumnTypeName.cpp index 466a8ba17c5..d8013f13340 100644 --- a/src/Functions/toColumnTypeName.cpp +++ b/src/Functions/toColumnTypeName.cpp @@ -28,6 +28,8 @@ public: bool useDefaultImplementationForNothing() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } size_t getNumberOfArguments() const override diff --git a/tests/queries/0_stateless/02355_column_type_name_lc.reference b/tests/queries/0_stateless/02355_column_type_name_lc.reference new file mode 100644 index 00000000000..234a072299f --- /dev/null +++ b/tests/queries/0_stateless/02355_column_type_name_lc.reference @@ -0,0 +1 @@ +ColumnLowCardinality diff --git a/tests/queries/0_stateless/02355_column_type_name_lc.sql b/tests/queries/0_stateless/02355_column_type_name_lc.sql new file mode 100644 index 00000000000..13a6393aaf2 --- /dev/null +++ b/tests/queries/0_stateless/02355_column_type_name_lc.sql @@ -0,0 +1 @@ +SELECT toColumnTypeName(toLowCardinality(materialize('foo'))); From 1f049e8bac6858b59c5c786645ade1428d839d22 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 3 Jul 2022 20:32:46 +0300 Subject: [PATCH 474/627] Bump jemalloc to fix possible assertion This patch also updates the jemalloc version. Note, that I've enabled page_id for jemalloc PR_SET_VMA/PR_SET_VMA_ANON_NAME, that requires linux 5.17+ (but ignores EINVAL anyway). v2: add -isystem to fix reserved name for JEMALLOC_OVERRIDE___LIBC_PVALLOC Refs: https://github.com/jemalloc/jemalloc/pull/2304 Refs: https://github.com/ClickHouse/ClickHouse/issues/31531 Signed-off-by: Azat Khuzhin --- contrib/jemalloc | 2 +- contrib/jemalloc-cmake/CMakeLists.txt | 3 ++- .../jemalloc-cmake/include/jemalloc/jemalloc_defs.h | 4 ++++ .../jemalloc-cmake/include/jemalloc/jemalloc_protos.h | 6 ++++++ .../include/jemalloc/jemalloc_protos_jet.h | 6 ++++++ .../jemalloc/internal/jemalloc_internal_defs.h.in | 10 ++++++++++ .../jemalloc/internal/jemalloc_internal_defs.h.in | 10 ++++++++++ .../jemalloc/internal/jemalloc_internal_defs.h.in | 10 ++++++++++ .../jemalloc/internal/jemalloc_internal_defs.h.in | 10 ++++++++++ .../jemalloc/internal/jemalloc_internal_defs.h.in | 10 ++++++++++ .../jemalloc/internal/jemalloc_internal_defs.h.in | 10 ++++++++++ .../jemalloc/internal/jemalloc_internal_defs.h.in | 10 ++++++++++ .../jemalloc/internal/jemalloc_internal_defs.h.in | 10 ++++++++++ .../jemalloc/internal/jemalloc_internal_defs.h.in | 10 ++++++++++ 14 files changed, 109 insertions(+), 2 deletions(-) diff --git a/contrib/jemalloc b/contrib/jemalloc index 78b58379c85..41a859ef732 160000 --- a/contrib/jemalloc +++ b/contrib/jemalloc @@ -1 +1 @@ -Subproject commit 78b58379c854a639df79beb3289351129d863d4b +Subproject commit 41a859ef7325569c6c25f92d294d45123bb81355 diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index fdb0fd0e8af..d5ea69d4926 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -117,10 +117,11 @@ if (OS_DARWIN) endif () add_library(_jemalloc ${SRCS}) + # First include jemalloc-cmake files, to override anything that jemalloc has. # (for example if you were trying to build jemalloc inside contrib/jemalloc you # will have some files that may be out of date) -target_include_directories(_jemalloc PUBLIC include) +target_include_directories(_jemalloc SYSTEM PUBLIC include) target_include_directories(_jemalloc PRIVATE "${LIBRARY_DIR}/include") set (JEMALLOC_INCLUDE_PREFIX) diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h index 47569b16a88..c20ce45b684 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h @@ -27,6 +27,10 @@ #if !defined(USE_MUSL) #define JEMALLOC_OVERRIDE_MEMALIGN #define JEMALLOC_OVERRIDE_VALLOC + #if defined(__linux__) + #define JEMALLOC_OVERRIDE_PVALLOC + #define JEMALLOC_OVERRIDE___LIBC_PVALLOC + #endif #endif /* diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h index e1e5b0575f3..1a6c624496b 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h @@ -82,3 +82,9 @@ JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW *je_valloc(size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc); #endif + +#ifdef JEMALLOC_OVERRIDE_PVALLOC +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN + void JEMALLOC_SYS_NOTHROW *je_pvalloc(size_t size) JEMALLOC_CXX_THROW + JEMALLOC_ATTR(malloc); +#endif diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos_jet.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos_jet.h index 72182727a6f..5f16e3704a7 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos_jet.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos_jet.h @@ -69,3 +69,9 @@ JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW *jet_valloc(size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc); #endif + +#ifdef JEMALLOC_OVERRIDE_PVALLOC +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN + void JEMALLOC_SYS_NOTHROW *je_pvalloc(size_t size) JEMALLOC_CXX_THROW + JEMALLOC_ATTR(malloc); +#endif diff --git a/contrib/jemalloc-cmake/include_darwin_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_darwin_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in index 8ad95c51560..496cf02a521 100644 --- a/contrib/jemalloc-cmake/include_darwin_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_darwin_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -36,6 +36,7 @@ #define CPU_SPINWAIT /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ #define HAVE_CPU_SPINWAIT 0 +/* #undef HAVE_RDTSCP */ /* * Number of significant bits in virtual addresses. This may be less than the @@ -162,6 +163,12 @@ /* Use gcc intrinsics for profile backtracing if defined. */ /* #undef JEMALLOC_PROF_GCC */ +/* JEMALLOC_PAGEID enabled page id */ +/* #undef JEMALLOC_PAGEID */ + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +/* #undef JEMALLOC_HAVE_PRCTL */ + /* * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage * segment (DSS). @@ -422,4 +429,7 @@ /* Darwin VM_MAKE_TAG support */ #define JEMALLOC_HAVE_VM_MAKE_TAG +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +/* #undef JEMALLOC_ZERO_REALLOC_DEFAULT_FREE */ + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_darwin_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_darwin_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in index 8671da5db69..db62d2c48ec 100644 --- a/contrib/jemalloc-cmake/include_darwin_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_darwin_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -36,6 +36,7 @@ #define CPU_SPINWAIT __asm__ volatile("pause") /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ #define HAVE_CPU_SPINWAIT 1 +#define HAVE_RDTSCP 1 /* * Number of significant bits in virtual addresses. This may be less than the @@ -162,6 +163,12 @@ /* Use gcc intrinsics for profile backtracing if defined. */ /* #undef JEMALLOC_PROF_GCC */ +/* JEMALLOC_PAGEID enabled page id */ +/* #undef JEMALLOC_PAGEID */ + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +/* #undef JEMALLOC_HAVE_PRCTL */ + /* * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage * segment (DSS). @@ -422,4 +429,7 @@ /* Darwin VM_MAKE_TAG support */ #define JEMALLOC_HAVE_VM_MAKE_TAG +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +/* #undef JEMALLOC_ZERO_REALLOC_DEFAULT_FREE */ + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in index 0f61417d65f..d58dce1faca 100644 --- a/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -36,6 +36,7 @@ #define CPU_SPINWAIT /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ #define HAVE_CPU_SPINWAIT 0 +/* #undef HAVE_RDTSCP */ /* * Number of significant bits in virtual addresses. This may be less than the @@ -164,6 +165,12 @@ /* Use gcc intrinsics for profile backtracing if defined. */ /* #undef JEMALLOC_PROF_GCC */ +/* JEMALLOC_PAGEID enabled page id */ +/* #undef JEMALLOC_PAGEID */ + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +/* #undef JEMALLOC_HAVE_PRCTL */ + /* * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage * segment (DSS). @@ -424,4 +431,7 @@ /* Darwin VM_MAKE_TAG support */ /* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +/* #undef JEMALLOC_ZERO_REALLOC_DEFAULT_FREE */ + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_freebsd_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_freebsd_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in index 32cad025f5f..bd72723c5f3 100644 --- a/contrib/jemalloc-cmake/include_freebsd_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_freebsd_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -36,6 +36,7 @@ #define CPU_SPINWAIT __asm__ volatile("pause") /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ #define HAVE_CPU_SPINWAIT 1 +#define HAVE_RDTSCP 1 /* * Number of significant bits in virtual addresses. This may be less than the @@ -164,6 +165,12 @@ /* Use gcc intrinsics for profile backtracing if defined. */ /* #undef JEMALLOC_PROF_GCC */ +/* JEMALLOC_PAGEID enabled page id */ +/* #undef JEMALLOC_PAGEID */ + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +/* #undef JEMALLOC_HAVE_PRCTL */ + /* * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage * segment (DSS). @@ -424,4 +431,7 @@ /* Darwin VM_MAKE_TAG support */ /* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +/* #undef JEMALLOC_ZERO_REALLOC_DEFAULT_FREE */ + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in index ad535e6d773..597f2d59933 100644 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -38,6 +38,7 @@ #define CPU_SPINWAIT /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ #define HAVE_CPU_SPINWAIT 0 +/* #undef HAVE_RDTSCP */ /* * Number of significant bits in virtual addresses. This may be less than the @@ -164,6 +165,12 @@ /* Use gcc intrinsics for profile backtracing if defined. */ /* #undef JEMALLOC_PROF_GCC */ +/* JEMALLOC_PAGEID enabled page id */ +#define JEMALLOC_PAGEID + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +#define JEMALLOC_HAVE_PRCTL + /* * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage * segment (DSS). @@ -424,4 +431,7 @@ /* Darwin VM_MAKE_TAG support */ /* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +#define JEMALLOC_ZERO_REALLOC_DEFAULT_FREE + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in index 12890f80ef1..0a72e71496b 100644 --- a/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in @@ -38,6 +38,7 @@ #define CPU_SPINWAIT /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ #define HAVE_CPU_SPINWAIT 0 +/* #undef HAVE_RDTSCP */ /* * Number of significant bits in virtual addresses. This may be less than the @@ -164,6 +165,12 @@ /* Use gcc intrinsics for profile backtracing if defined. */ /* #undef JEMALLOC_PROF_GCC */ +/* JEMALLOC_PAGEID enabled page id */ +#define JEMALLOC_PAGEID + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +#define JEMALLOC_HAVE_PRCTL + /* * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage * segment (DSS). @@ -424,4 +431,7 @@ /* Darwin VM_MAKE_TAG support */ /* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +#define JEMALLOC_ZERO_REALLOC_DEFAULT_FREE + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_riscv64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_riscv64/jemalloc/internal/jemalloc_internal_defs.h.in index ad535e6d773..597f2d59933 100644 --- a/contrib/jemalloc-cmake/include_linux_riscv64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_riscv64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -38,6 +38,7 @@ #define CPU_SPINWAIT /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ #define HAVE_CPU_SPINWAIT 0 +/* #undef HAVE_RDTSCP */ /* * Number of significant bits in virtual addresses. This may be less than the @@ -164,6 +165,12 @@ /* Use gcc intrinsics for profile backtracing if defined. */ /* #undef JEMALLOC_PROF_GCC */ +/* JEMALLOC_PAGEID enabled page id */ +#define JEMALLOC_PAGEID + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +#define JEMALLOC_HAVE_PRCTL + /* * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage * segment (DSS). @@ -424,4 +431,7 @@ /* Darwin VM_MAKE_TAG support */ /* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +#define JEMALLOC_ZERO_REALLOC_DEFAULT_FREE + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in index 99ab2d53ca9..d21098a4dcc 100644 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -38,6 +38,7 @@ #define CPU_SPINWAIT __asm__ volatile("pause") /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ #define HAVE_CPU_SPINWAIT 1 +#define HAVE_RDTSCP 1 /* * Number of significant bits in virtual addresses. This may be less than the @@ -164,6 +165,12 @@ /* Use gcc intrinsics for profile backtracing if defined. */ /* #undef JEMALLOC_PROF_GCC */ +/* JEMALLOC_PAGEID enabled page id */ +#define JEMALLOC_PAGEID + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +#define JEMALLOC_HAVE_PRCTL + /* * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage * segment (DSS). @@ -424,4 +431,7 @@ /* Darwin VM_MAKE_TAG support */ /* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +#define JEMALLOC_ZERO_REALLOC_DEFAULT_FREE + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_x86_64_musl/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_x86_64_musl/jemalloc/internal/jemalloc_internal_defs.h.in index 684d4debb14..ff97d297d8f 100644 --- a/contrib/jemalloc-cmake/include_linux_x86_64_musl/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_x86_64_musl/jemalloc/internal/jemalloc_internal_defs.h.in @@ -38,6 +38,7 @@ #define CPU_SPINWAIT __asm__ volatile("pause") /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ #define HAVE_CPU_SPINWAIT 1 +#define HAVE_RDTSCP 1 /* * Number of significant bits in virtual addresses. This may be less than the @@ -165,6 +166,12 @@ /* Use gcc intrinsics for profile backtracing if defined. */ /* #undef JEMALLOC_PROF_GCC */ +/* JEMALLOC_PAGEID enabled page id */ +#define JEMALLOC_PAGEID + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +#define JEMALLOC_HAVE_PRCTL + /* * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage * segment (DSS). @@ -425,4 +432,7 @@ /* Darwin VM_MAKE_TAG support */ /* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +#define JEMALLOC_ZERO_REALLOC_DEFAULT_FREE + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ From 6dcdf0fcb1ccf3be848442a12776ad6e1ded9539 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 12 Jul 2022 06:41:58 +0000 Subject: [PATCH 475/627] Throw CORRUPTED_DATA in debug mode for invalid checksum --- src/Coordination/KeeperStateManager.cpp | 21 ++++++++++++++----- src/Coordination/tests/gtest_coordination.cpp | 4 ++++ 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 71dfa2cc55f..3d7f5f2fb34 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int RAFT_ERROR; + extern const int CORRUPTED_DATA; } namespace @@ -330,7 +331,7 @@ nuraft::ptr KeeperStateManager::read_state() if (content_size == 0) return nullptr; - uint64_t read_checksum; + uint64_t read_checksum{0}; readIntBinary(read_checksum, read_buf); uint8_t version; @@ -347,22 +348,32 @@ nuraft::ptr KeeperStateManager::read_state() if (read_checksum != hash.get64()) { - LOG_ERROR( - logger, + const auto error_string = fmt::format( "Invalid checksum while reading state from {}. Got {}, expected {}", path.generic_string(), hash.get64(), read_checksum); +#ifdef NDEBUG + LOG_ERROR(logger, fmt::runtime(error_string)); return nullptr; +#else + throw Exception(ErrorCodes::CORRUPTED_DATA, error_string); +#endif } auto state = nuraft::srv_state::deserialize(*state_buf); LOG_INFO(logger, "Read state from {}", path.generic_string()); return state; } - catch (...) + catch (const std::exception & e) { - LOG_WARNING(logger, "Failed to deserialize state from {}", path.generic_string()); + if (const auto * exception = dynamic_cast(&e); + exception != nullptr && exception->code() == ErrorCodes::CORRUPTED_DATA) + { + throw; + } + + LOG_ERROR(logger, "Failed to deserialize state from {}", path.generic_string()); return nullptr; } }; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 801275b1ff8..20c33880139 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2092,7 +2092,11 @@ TEST_P(CoordinationTest, TestDurableState) write_buf.sync(); write_buf.close(); reload_state_manager(); +#ifdef NDEBUG ASSERT_EQ(state_manager->read_state(), nullptr); +#else + ASSERT_THROW(state_manager->read_state(), DB::Exception); +#endif } { From 631be9219bf67444411aceda7948c8a1336184fe Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Jul 2022 08:28:22 +0000 Subject: [PATCH 476/627] Properly remove projection from part in case it was removed from table metadata. --- .../MergeTree/DataPartStorageOnDisk.cpp | 41 +++++++++++++++++-- ...clear_projection_and_part_remove.reference | 0 ...01701_clear_projection_and_part_remove.sql | 19 +++++++++ 3 files changed, 57 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01701_clear_projection_and_part_remove.reference create mode 100644 tests/queries/0_stateless/01701_clear_projection_and_part_remove.sql diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 2cc28d020bb..91d24057f72 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -268,9 +268,10 @@ void DataPartStorageOnDisk::remove( // Record existing projection directories so we don't remove them twice std::unordered_set projection_directories; + std::string proj_suffix = ".proj"; for (const auto & projection : projections) { - std::string proj_dir_name = projection.name + ".proj"; + std::string proj_dir_name = projection.name + proj_suffix; projection_directories.emplace(proj_dir_name); clearDirectory( @@ -278,6 +279,40 @@ void DataPartStorageOnDisk::remove( can_remove_shared_data, names_not_to_remove, projection.checksums, {}, log, true); } + /// It is possible that we are removing the part which have a written but not loaded projection. + /// Such a part can appear server was restarted after DROP PROJECTION but before old part was removed. + /// In this case, the old part will load only projections from metadata. + /// See test 01701_clear_projection_and_part. + for (const auto & [name, _] : checksums.files) + { + if (endsWith(name, proj_suffix) && !projection_directories.contains(name) && disk->isDirectory(fs::path(to) / name)) + { + + /// If we have a directory with suffix '.proj' it is likely a projection. + /// Try to load checksums for it (to avoid recusrive removing fallback). + std::string checksum_path = fs::path(to) / name / "checksums.txt"; + if (disk->exists(checksum_path)) + { + try + { + MergeTreeDataPartChecksums tmp_checksums; + auto in = disk->readFile(checksum_path, {}); + tmp_checksums.read(*in); + + projection_directories.emplace(name); + + clearDirectory( + fs::path(to) / name, + can_remove_shared_data, names_not_to_remove, tmp_checksums, {}, log, true); + } + catch (...) + { + LOG_ERROR(log, "Cannot load checksums from {}", checksum_path); + } + } + } + } + clearDirectory(to, can_remove_shared_data, names_not_to_remove, checksums, projection_directories, log, false); } @@ -343,8 +378,8 @@ void DataPartStorageOnDisk::clearDirectory( /// Recursive directory removal does many excessive "stat" syscalls under the hood. LOG_ERROR(log, "Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {}", fullPath(disk, dir), getCurrentExceptionMessage(false)); - - disk->removeSharedRecursive(fs::path(dir) / "", !can_remove_shared_data, names_not_to_remove); + throw; + //disk->removeSharedRecursive(fs::path(dir) / "", !can_remove_shared_data, names_not_to_remove); } } diff --git a/tests/queries/0_stateless/01701_clear_projection_and_part_remove.reference b/tests/queries/0_stateless/01701_clear_projection_and_part_remove.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01701_clear_projection_and_part_remove.sql b/tests/queries/0_stateless/01701_clear_projection_and_part_remove.sql new file mode 100644 index 00000000000..e6f2142cac5 --- /dev/null +++ b/tests/queries/0_stateless/01701_clear_projection_and_part_remove.sql @@ -0,0 +1,19 @@ +drop table if exists tp_1; +-- In this test, we are going to create an old part with written projection which does not exist in table metadata +create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = MergeTree order by y partition by intDiv(y, 100) settings old_parts_lifetime=1; +insert into tp_1 select number, number from numbers(3); +set mutations_sync = 2; +alter table tp_1 add projection pp (select x, count() group by x); +insert into tp_1 select number, number from numbers(4); +-- Here we have a part with written projection pp +alter table tp_1 detach partition '0'; +-- Move part to detached +alter table tp_1 clear projection pp; +-- Remove projection from table metadata +alter table tp_1 drop projection pp; +-- Now, we don't load projection pp for attached part, bit it is written on disk +alter table tp_1 attach partition '0'; +-- Make this part obsolete +optimize table tp_1 final; +-- Now, DROP TABLE triggers part removal +drop table tp_1; From 03d93014255a1d630d5fe6d77daa04584b031045 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Jul 2022 08:31:09 +0000 Subject: [PATCH 477/627] Fix log. --- src/Storages/MergeTree/DataPartStorageOnDisk.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 91d24057f72..5df90835ee3 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -378,8 +378,7 @@ void DataPartStorageOnDisk::clearDirectory( /// Recursive directory removal does many excessive "stat" syscalls under the hood. LOG_ERROR(log, "Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {}", fullPath(disk, dir), getCurrentExceptionMessage(false)); - throw; - //disk->removeSharedRecursive(fs::path(dir) / "", !can_remove_shared_data, names_not_to_remove); + disk->removeSharedRecursive(fs::path(dir) / "", !can_remove_shared_data, names_not_to_remove); } } From d10c665b716d6cde0a0bb8d804f63e70fd3faf1a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 11:31:23 +0200 Subject: [PATCH 478/627] Add results for MariaDB --- benchmark/mariadb/benchmark.sh | 8 +++ benchmark/mariadb/results/c6a.4xlarge.json | 58 ++++++++++++++++++++++ 2 files changed, 66 insertions(+) create mode 100644 benchmark/mariadb/results/c6a.4xlarge.json diff --git a/benchmark/mariadb/benchmark.sh b/benchmark/mariadb/benchmark.sh index 612a6ac39ab..9baf1360af4 100755 --- a/benchmark/mariadb/benchmark.sh +++ b/benchmark/mariadb/benchmark.sh @@ -19,3 +19,11 @@ time sudo mariadb test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" # 2:23:45 elapsed ./run.sh 2>&1 | tee log.txt + +sudo du -bcs /var/lib/mysql + +cat log.txt | + grep -P 'rows? in set|Empty set|^ERROR' | + sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) days? )?(([0-9.]+) hours? )?(([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \4 \6 \7/' | + awk '{ if ($4) { print $1 * 86400 + $2 * 3600 + $3 * 60 + $4 } else if ($3) { print $1 * 3600 + $2 * 60 + $3 } else if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/mariadb/results/c6a.4xlarge.json b/benchmark/mariadb/results/c6a.4xlarge.json new file mode 100644 index 00000000000..7a2869285e9 --- /dev/null +++ b/benchmark/mariadb/results/c6a.4xlarge.json @@ -0,0 +1,58 @@ +{ + "system": "MariaDB", + "date": "2022-07-12", + "machine": "c6a.4xlarge, 500gb gp2", + "cluster_size": 1, + "comment": "For some reason many queries run in almost exactly one hour. Most likely a coincidence.", + + "tags": ["C++", "row-oriented", "MySQL compatible"], + + "load_time": 8625, + "data_size": 107252505530, + + "result": [ +[7910.23,10229.8,70.543], +[3595.79,3595.59,3595.21], +[3596.12,3595.34,3595.49], +[3597.22,3595.7,3595.09], +[60.569,3596.08,3595.82], +[3596.57,3597.76,3598.47], +[3595.94,3595.36,3596.2], +[3595.95,3595.38,3595.45], +[109.147,84.264,83.707], +[105.386,3763.75,90.706], +[4519.77,4552.84,4552.7], +[4628.21,4629.55,4628.16], +[3773.52,3774.44,3779.05], +[12198,12192.4,12192.6], +[3779.67,3774.97,3777.86], +[4277.99,4302.56,4275.29], +[4697.19,4367.3,4372.73], +[4401.2,4340.43,4310.13], +[5335.93,5373.04,5361.51], +[3596.91,3598.61,3598.71], +[3598.76,3598.56,3598.68], +[3598.76,3598.63,3599.39], +[64.281,63.709,63.665], +[3598.83,3598.2,3598.19], +[3599.11,3598.1,3598.17], +[3599.04,3598.46,3598.13], +[3598.76,3598.71,3598.22], +[3599.02,3599.76,3598.18], +[104.124,3716.36,3725.92], +[3933.26,3854.55,3784.74], +[3392.13,3402.26,62.153], +[3717.3,3735.06,3723.07], +[6126.66,6135.63,6193.45], +[4029.78,3998.89,3952.7], +[3955.34,4029.28,3941.84], +[4881.07,4782.53,4900.72], +[24.455,23.862,25.173], +[24.195,23.529,23.473], +[20.719,21.629,21.582], +[24.577,25.877,25.953], +[20.686,20.787,21.199], +[20.036,17.973,22.759], +[null,null,null] +] +} From 7cf98f8f31a649264ab76b5042b72831274723e4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 11:32:18 +0200 Subject: [PATCH 479/627] Regenerate HTML --- benchmark/index.html | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/benchmark/index.html b/benchmark/index.html index ff27c28e870..417394fd45b 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -342,12 +342,13 @@ const data = [ ,{"system":"ClickHouse","date":"2022-07-01","machine":"c6a.metal, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","column-oriented","ClickHouse derivative"],"load_time":136.869,"data_size":14571706777,"result":[[0.011,0.001,0.001],[0.04,0.015,0.013],[0.045,0.021,0.023],[0.09,0.023,0.023],[1.922,1.565,1.576],[0.961,0.737,0.739],[0.04,0.023,0.018],[0.032,0.028,0.028],[0.321,0.287,0.275],[0.632,0.284,0.287],[0.166,0.124,0.118],[0.235,0.1,0.102],[1.006,0.182,0.159],[1.637,0.216,0.213],[0.871,0.174,0.177],[0.258,0.148,0.148],[1.804,0.37,0.358],[1.235,0.275,0.278],[3.143,0.854,0.815],[0.071,0.024,0.016],[8.816,0.215,0.155],[10.239,0.203,0.173],[19.179,0.388,0.357],[43.152,0.824,0.823],[1.821,0.059,0.052],[0.992,0.045,0.051],[2.539,0.063,0.058],[9.258,0.3,0.278],[7.923,0.961,0.936],[0.445,0.431,0.428],[1.367,0.131,0.113],[4.819,0.205,0.175],[3.808,0.739,0.726],[8.935,0.607,0.6],[8.988,0.634,0.615],[0.242,0.22,0.226],[0.075,0.058,0.056],[0.038,0.028,0.026],[0.043,0.028,0.021],[0.172,0.127,0.119],[0.028,0.018,0.017],[0.027,0.019,0.014],[0.018,0.026,0.015]],"source":"clickhouse/results/c6a.metal.json"} ,{"system":"CrateDB","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"For some queries it gives \"Data too large\".","tags":["Java","column-oriented"],"load_time":10687,"data_size":109636633416,"result":[[0.008162,0.005118,0.002553],[0.350014,0.39977,0.133775],[2.58426,2.47192,2.59779],[2.12939,0.532981,0.507246],[null,null,null],[null,null,null],[1.18488,1.06603,1.07219],[0.209264,0.073284,0.067912],[null,null,null],[null,null,null],[1.68892,1.2866,1.47428],[1.62976,1.43073,1.26904],[12.7517,13.0334,13.2685],[18.8587,null,18.6951],[11.2982,11.2108,11.577],[20.2964,20.4035,19.1076],[null,null,null],[null,null,null],[null,null,null],[0.202044,0.010009,0.005566],[9.22964,4.54606,0.774149],[1.41673,1.09885,0.789775],[12.3933,8.06911,1.69671],[1.45018,0.969528,0.979718],[0.357589,0.14887,0.153326],[0.189282,0.133963,0.130279],[0.153222,0.140756,0.139861],[27.5195,19.6862,20.1825],[72.7575,68.2,67.1238],[144.533,146.579,152.144],[8.76866,9.00563,8.46917],[17.6652,16.6755,16.0558],[null,null,null],[null,null,null],[null,null,null],[42.2967,44.9621,44.4386],[0.786911,0.4904,0.508416],[0.602075,0.226261,0.182399],[0.131407,0.058958,0.054518],[0.954736,1.1361,1.14233],[0.23764,0.139109,0.134472],[0.110253,0.057695,0.056073],[0.124285,0.150479,0.066226]],"source":"cratedb/results/c6a.4xlarge.json"} ,{"system":"Databend","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Only 90% of data successfully loaded.","tags":["Rust","column-oriented","ClickHouse derivative"],"load_time":484,"data_size":43016643271,"result":[[0.010087,0.002961,0.003271],[0.127964,0.080012,0.075741],[0.162388,0.143967,0.144762],[0.252904,0.217471,0.217369],[34.281026,34.844158,34.526942],[25.290307,25.793068,25.620563],[0.112484,0.093867,0.090891],[0.086604,0.07796,0.076448],[20.723203,20.7483,20.354869],[20.81994,20.72446,20.696573],[1.964378,1.93559,1.893824],[1.846866,1.789111,1.763664],[4.468158,4.407959,4.438036],[19.947276,19.8859,19.853514],[5.478573,5.474461,5.460604],[5.509521,5.513413,5.363123],[15.430359,15.5406,15.461211],[14.905998,15.029721,15.019642],[31.069663,30.811763,30.737336],[0.281067,0.220021,0.217741],[8.89374,4.12692,4.131689],[10.38448,4.603694,4.571757],[19.980572,8.836322,8.892694],[59.786474,52.452881,39.941988],[2.804019,0.994794,0.958224],[0.765299,0.730434,0.723964],[2.784648,0.94665,0.936684],[8.905027,5.418438,5.386109],[12.187652,12.230066,12.164123],[3.35748,3.395991,3.319434],[4.309389,3.854977,3.772506],[9.958201,7.027432,6.888253],[50.200569,50.535126,50.283066],[24.469412,21.222713,21.010188],[26.115852,23.93507,24.835342],[7.511517,7.296179,7.324549],[2.156784,1.298258,1.278441],[2.155447,1.314499,1.331237],[2.007053,1.181676,1.155612],[null,null,null],[0.485363,0.420291,0.416819],[0.372131,0.322068,0.323578],[null,null,null]],"source":"databend/results/c6a.4xlarge.json"} -,{"system":"Druid","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["Java","column-oriented"],"load_time":19620,"data_size":45188608472,"result":[[0.02261,0.015202,0.015416],[0.408549,0.275624,0.275657],[0.365362,0.159893,0.151254],[1.380429,0.136963,0.13113],[3.126491,2.462627,2.431463],[9.350598,7.418928,7.213518],[null,null,null],[0.532669,0.27882,0.270483],[7.541045,6.631755,6.55507],[8.63431,7.615905,7.573183],[1.650839,0.83276,0.881859],[1.680002,0.504994,0.512728],[2.406642,1.939866,1.667841],[8.632186,7.264837,7.358885],[61.095584,59.243483,59.111986],[0.006551,0.005535,0.004936],[0.010023,0.010525,0.009739],[0.009157,0.008836,0.008432],[null,null,null],[0.005462,0.004723,0.004469],[0.006079,0.005037,0.005726],[null,null,null],[null,null,null],[0.009679,0.009139,0.011461],[0.006833,0.005752,0.005962],[null,null,null],[null,null,null],[0.011282,0.009306,0.009612],[null,null,null],[0.04285,0.04703,0.040549],[0.013658,0.013335,0.013686],[0.018262,0.012727,0.012357],[0.013259,0.011685,0.011177],[0.006448,0.005744,0.005489],[0.006924,0.008225,0.006115],[60.5054,59.499392,59.221125],[0.010123,0.008507,0.00899],[0.010177,0.008739,0.009381],[0.011097,0.009424,0.013203],[null,null,null],[0.014453,0.013647,0.013853],[0.017409,0.013077,0.012445],[0.010489,0.009483,0.009735]],"source":"druid/results/c6a.4xlarge.json"} +,{"system":"Druid","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Druid is killed and restarted after every query. Otherwise some queries make Druid degraded and results are incorrect. For example after Q13 even SELECT 1 works for 7 seconds","tags":["Java","column-oriented"],"load_time":19620,"data_size":45188608472,"result":[[0.032365,0.016716,0.016921],[0.400766,0.275591,0.271057],[0.38235,0.152084,0.151652],[1.379492,0.137201,0.128902],[3.278731,2.527105,2.515128],[8.576382,6.54654,6.503001],[null,null,null],[0.563852,0.273795,0.275086],[11.509993,10.636571,10.597993],[13.357647,12.42121,12.337247],[1.636875,0.8213,0.900056],[1.692544,0.512066,0.440511],[2.453274,1.769806,1.807207],[8.503408,7.261406,7.334872],[61.056041,59.251083,59.500549],[8.62067,8.236657,8.22538],[164.840762,null,null],[24.165797,22.308466,null],[null,null,null],[null,null,null],[25.973369,25.597864,25.602509],[null,null,null],[null,null,null],[7.805347,6.629776,6.947366],[0.257845,0.020327,0.016976],[null,null,null],[null,null,null],[32.948154,31.04677,29.221959],[null,null,null],[7.230865,7.033713,6.972421],[20.54625,19.237428,19.258469],[54.065945,52.451318,52.466653],[null,null,null],[17.499267,null,null],[null,null,null],[60.478315,60.05494,60.458946],[1.698088,1.490317,1.461969],[1.409572,0.939003,0.907252],[0.866729,0.329539,0.287435],[null,null,null],[0.932473,0.420781,0.359095],[0.723142,0.3253,0.296865],[0.603483,0.150892,0.140716]],"source":"druid/results/c6a.4xlarge.json"} ,{"system":"DuckDB","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Many queries triggered OOM","tags":["C","column-oriented","embedded"],"load_time":4217,"data_size":27241492480,"result":[[0.005694353996659629,0.003944558004150167,0.003837226002360694],[0.16991353100456763,0.03919722700084094,0.03835860399703961],[0.44898432699847035,0.04947217500011902,0.04852217998995911],[0.07586832098604646,0.07051395199960098,0.07007493599667214],[9.554053236002801,8.153356187991449,8.73448242500308],[7.66042533799191,6.931124911992811,7.103380946995458],[0.030703739990713075,0.027668555994750932,0.027583695002249442],[0.1778664360026596,0.03942437999648973,0.03882004099432379],[8.53439180701389,8.869582625004114,9.020313234999776],[10.40215514000738,11.125320470004226,8.941559945000336],[1.1747649609897053,1.04221136700653,1.004799570000614],[1.2380354650085792,1.1211603130068397,2.4278587239969056],[3.1751541379926493,0.9360461989999749,0.8868292279948946],[6.855684430003748,7.300301584007684,5.712960822012974],[3.70588762400439,1.0249276379909134,0.9473389159975341],[2.1037107890006155,1.6215517020027619,1.5671920729946578],[null,null,null],[null,null,null],[null,null,null],[0.0002772739971987903,0.00016792300448287278,0.0001574420020915568],[null,null,null],[null,null,null],[null,null,null],[null,null,null],[2.9310110910009826,0.19020285899750888,0.1736805049877148],[2.939304119994631,0.18754731099761557,0.18073286200524308],[2.8706370779982535,0.18822155400994234,0.17905898999015335],[null,null,null],[null,null,null],[0.884408778991201,0.714329167996766,0.7135983259940986],[5.3762675570033025,0.8803737630078103,0.8728962720051641],[7.249190265996731,2.9648747390019707,2.866687831003219],[null,null,null],[null,null,null],[null,null,null],[4.515183198003797,4.030519469000865,4.014251719010645],[0.11604027298744768,0.040539135996368714,0.04280066800129134],[0.0457908230018802,0.021069509006338194,0.019683108999743126],[0.0680370800109813,0.011889394998434,0.01056639499438461],[0.22029169600864407,0.08547276000899728,0.09095505000732373],[0.03759863799496088,0.008373684002435766,0.007633563989656977],[0.025631797994719818,0.008081699008471332,0.007858585988287814],[0.034359957004198804,0.025543516996549442,0.02533275399764534]],"source":"duckdb/results/c6a.4xlarge.json"} ,{"system":"Greenplum","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","column-oriented","PostgreSQL compatible"],"load_time":1080,"data_size":42000000000,"result":[[2.20009,1.45883,1.13573],[1.28661,1.07897,1.01856],[2.2582,2.42193,2.59693],[1.89371,1.9246,1.73866],[9.54471,9.71956,10.0357],[4.55957,4.10112,4.70458],[1.65167,1.89767,1.97051],[1.02537,0.999323,1.00524],[7.89792,7.75718,8.01219],[11.3779,11.6003,11.2552],[2.11955,2.05003,1.9553],[2.01997,1.97932,1.98284],[2.88666,2.92828,2.96822],[4.06272,4.01771,4.03011],[3.46303,3.24779,3.06475],[8.43717,8.18641,7.99202],[9.02176,8.91572,8.92748],[8.66982,8.39332,8.62497],[15.1321,14.9502,15.2344],[1.00877,0.934589,0.975342],[11.0582,3.25807,3.17687],[12.2634,3.5188,3.93705],[24.1522,4.86328,4.88764],[67.1517,20.4,20.261],[2.5799,1.66114,1.90432],[1.72831,1.89536,1.2515],[2.44154,1.46595,1.88238],[11.1175,6.09551,6.02292],[83.041,82.9784,82.8678],[82.5441,82.5427,83.9836],[3.0962,3.22225,3.19431],[5.77576,4.05973,4.00366],[43.6826,40.4375,40.1076],[18.0978,13.2741,12.8894],[17.3187,13.5411,13.5927],[9.72405,9.90029,10.0177],[0.294344,0.169606,0.173804],[0.162524,0.117489,0.115532],[0.145205,0.098342,0.097275],[0.334809,0.275365,0.265053],[0.154522,0.107654,0.10529],[0.158957,0.117284,0.119068],[0.193756,0.144787,0.145485]],"source":"greenplum/results/c6a.4xlarge.json"} ,{"system":"HeavyAI","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Previous names: OmniSci, mapD. Many queries cannot run due to errors and limitations.","tags":["C++","column-oriented"],"load_time":572.633,"data_size":50887437386,"result":[[6.525,0.022,0.029],[0.301,0.042,0.04],[0.287,0.095,0.093],[2.572,0.039,0.04],[null,null,null],[null,null,null],[7.327,0.093,0.097],[0.244,0.043,0.038],[null,null,null],[null,null,null],[null,null,null],[null,null,null],[2.939,0.295,0.294],[null,null,null],[null,null,null],[null,null,null],[null,null,null],[4.716,3.91,3.955],[null,null,null],[0.154,0.083,0.106],[14.426,0.07,0.071],[null,null,null],[null,null,null],[null,null,null],[2.276,0.258,0.272],[null,null,null],[null,null,null],[null,null,null],[null,null,null],[1.832,1.64,1.602],[null,null,null],[null,null,null],[null,null,null],[14.811,0.494,0.497],[null,null,null],[null,null,null],[1.941,0.255,0.255],[5.457,0.172,0.283],[0.476,0.269,0.256],[14.239,0.179,0.178],[3.992,0.112,0.112],[1.031,0.116,0.116],[1.365,0.089,0.088]],"source":"heavyai/results/c6a.4xlarge.json"} ,{"system":"Infobright","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Only 90% of data successfully loaded. Some queries run for days.","tags":["C++","column-oriented","MySQL compatible"],"load_time":2317,"data_size":13760341294,"result":[[0.01,0,0],[2.39,2.4,2.44],[0,0,0],[7.21,6.04,6.91],[16.09,16.86,15.69],[48.8,42.37,48.63],[0,0,0],[3.48,2.42,2.42],[23.56,24.78,22.21],[32.87,31.71,34.48],[14.8,14.83,14.11],[16.7,16.53,17.37],[1752.91,1999.88,1961.4],[1193.43,1167,1220.47],[2184.81,2316.12,2224.14],[32.58,30.69,31.58],[300.17,16221.33,16168.44],[122.4,120.49,124.67],[78927.44,79250.44,78504.89],[3.38,1.22,1.21],[289.73,302.3,285.83],[415.82,389.23,403.28],[573.82,590.81,575.06],[300.13,293.96,285.64],[41.42,37.48,39.64],[75.2,75.37,72.07],[39.22,41.52,40.11],[449.56,445.03,448.68],[null,null,null],[450.87,488.3,453.83],[58.69,59.29,58.07],[84.47,78.92,79.38],[517.97,520.29,504.96],[182468.89,182468.89,182468.89],[182468.89,182468.89,182468.89],[68.43,66.93,67.68],[8.3,3.62,3.61],[1.04,0.62,0.62],[0.22,0.18,0.18],[567.78,566.52,563.02],[0.29,0.14,0.11],[0.17,0.08,0.08],[1.37,1.34,1.32]],"source":"infobright/results/c6a.4xlarge.json"} ,{"system":"MariaDB ColumnStore","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Previous name: InfiniDB.","tags":["C++","column-oriented","MySQL compatible"],"load_time":2507.8,"data_size":19712857022,"result":[[0.151,0.158,0.148],[0.1,0.101,0.106],[1.221,1.233,1.226],[0.739,0.736,0.741],[2.025,2.046,2.004],[3.725,4.801,3.755],[0.871,0.749,0.736],[0.118,0.108,0.103],[2.108,2.029,2.029],[4.225,4.271,4.288],[1.711,1.402,1.407],[1.526,1.435,1.42],[5.339,4.172,3.61],[4.692,4.729,4.96],[4.013,3.86,3.918],[3.236,2.68,2.629],[null,null,null],[null,null,null],[null,null,null],[0.137,0.109,0.147],[12.331,6.069,8.619],[2.162,2.178,2.192],[16.849,30.463,26.639],[92,90.208,92.814],[3.042,1.763,1.791],[1.779,1.772,1.749],[1.793,1.821,1.888],[13.036,10.747,9.59],[null,null,null],[78.224,79.141,77.806],[2.837,2.654,2.675],[5.833,4.552,3.678],[null,null,null],[null,null,null],[null,null,null],[3.626,3.546,3.709],[1.719,1.787,1.876],[1.345,0.906,0.91],[0.117,0.091,0.093],[1.217,1.133,1.133],[0.114,0.063,0.062],[0.1,0.062,0.061],[null,null,null]],"source":"mariadb-columnstore/results/c6a.4xlarge.json"} +,{"system":"MariaDB","date":"2022-07-12","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"For some reason many queries run in almost exactly one hour. Most likely a coincidence.","tags":["C++","row-oriented","MySQL compatible"],"load_time":8625,"data_size":107252505530,"result":[[7910.23,10229.8,70.543],[3595.79,3595.59,3595.21],[3596.12,3595.34,3595.49],[3597.22,3595.7,3595.09],[60.569,3596.08,3595.82],[3596.57,3597.76,3598.47],[3595.94,3595.36,3596.2],[3595.95,3595.38,3595.45],[109.147,84.264,83.707],[105.386,3763.75,90.706],[4519.77,4552.84,4552.7],[4628.21,4629.55,4628.16],[3773.52,3774.44,3779.05],[12198,12192.4,12192.6],[3779.67,3774.97,3777.86],[4277.99,4302.56,4275.29],[4697.19,4367.3,4372.73],[4401.2,4340.43,4310.13],[5335.93,5373.04,5361.51],[3596.91,3598.61,3598.71],[3598.76,3598.56,3598.68],[3598.76,3598.63,3599.39],[64.281,63.709,63.665],[3598.83,3598.2,3598.19],[3599.11,3598.1,3598.17],[3599.04,3598.46,3598.13],[3598.76,3598.71,3598.22],[3599.02,3599.76,3598.18],[104.124,3716.36,3725.92],[3933.26,3854.55,3784.74],[3392.13,3402.26,62.153],[3717.3,3735.06,3723.07],[6126.66,6135.63,6193.45],[4029.78,3998.89,3952.7],[3955.34,4029.28,3941.84],[4881.07,4782.53,4900.72],[24.455,23.862,25.173],[24.195,23.529,23.473],[20.719,21.629,21.582],[24.577,25.877,25.953],[20.686,20.787,21.199],[20.036,17.973,22.759],[null,null,null]],"source":"mariadb/results/c6a.4xlarge.json"} ,{"system":"MonetDB","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","column-oriented"],"load_time":939,"data_size":49696606499,"result":[[0.000218,0.000157,0.000155],[0.101903,0.019908,0.018439],[0.282431,0.035987,0.034938],[2.868,0.029387,0.029207],[4.675,4.515,4.511],[6.584,4.269,4.65],[0.528827,0.063135,0.065742],[0.506863,0.020966,0.021687],[8.343,4.457,4.408],[7.224,6.548,7.576],[0.267003,0.233353,0.230444],[0.347206,0.28358,0.266085],[5.389,3.099,3.074],[7.653,7.759,8.596],[3.276,3.326,3.292],[5.31,3.465,3.578],[9.341,9.143,9.536],[9.584,9.604,9.419],[19.539,19.783,19.611],[0.004509,0.000702,0.000643],[20.801,1.57,1.603],[2.752,0.418221,0.395884],[14.717,0.800894,0.395477],[14.429,1.804,1.869],[1.386,0.159602,0.156426],[0.189736,0.167664,0.168781],[0.164681,0.176666,0.17126],[3.005,3.113,3.882],[null,null,null],[2.751,2.846,2.676],[7.937,2.579,2.447],[5.12,3.492,3.467],[22.862,22.567,23.211],[33.437,18.889,19.043],[18.898,19.583,19.047],[14.774,12.984,13.803],[3.865,0.322143,0.323117],[0.192149,0.177791,0.175984],[0.194173,0.159398,0.165201],[0.680778,0.592252,0.560738],[0.106465,0.10638,0.102692],[0.154871,0.153752,0.155782],[0.11459,0.09639,0.095594]],"source":"monetdb/results/c6a.4xlarge.json"} ,{"system":"MySQL (MyISAM)","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","row-oriented","MySQL compatible"],"load_time":2512,"data_size":121588958061,"result":[[0,0,0],[283.32,276.83,274.52],[276.93,278.29,283.27],[28.83,23.63,21.55],[46.41,40.81,40.93],[467.04,467.39,469.08],[31.02,25.89,24.2],[277.89,275.3,277.3],[329.34,325.8,325.35],[342.86,338.43,336.95],[282.03,279.87,281.22],[277.74,282.68,282],[335.66,334.83,336.44],[305.24,310.39,307.3],[337.41,338.52,342.94],[308.66,307.34,306.27],[738.38,748.44,740.75],[738.75,734.01,738.25],[867.01,872.92,868.84],[25.65,20.61,18.46],[312.39,313.67,306.66],[301.66,305.12,308.01],[298.12,298.44,312.4],[311.34,309.9,311.85],[281.87,278.5,275],[277.46,277.46,277.46],[280.75,278.04,281.76],[263.9,417.39,406.88],[707.21,711.96,705],[668.1,668.33,665.96],[330.31,333.36,331.94],[506.57,506.18,500.53],[2604.49,2681.96,2703.12],[830.65,832.88,831.14],[831.98,830.46,833.41],[608.49,608.51,613.68],[4.56,4.13,4.16],[3.8,3.8,3.7],[1.65,1.45,1.46],[6.33,5.14,6.15],[1.6,1.41,1.41],[1.56,1.42,1.39],[7.04,1.17,1.13]],"source":"mysql-myisam/results/c6a.4xlarge.json"} ,{"system":"MySQL","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","row-oriented","MySQL compatible"],"load_time":9472,"data_size":171953585825,"result":[[339.77,339.88,339.77],[364.91,371.86,367.55],[366.2,368.91,389.66],[364.39,377.53,571.45],[377.69,390.02,384.86],[569.48,576.51,574.68],[367.4,368.23,370.41],[371.29,384.02,613.22],[478.85,683.22,495.68],[489.9,635.96,662.43],[386.07,396.49,640.15],[389.13,412.55,444.12],[447.97,455.54,448.06],[423.22,845.44,813.6],[452.48,460.07,453.98],[577.54,623.21,586.49],[852.07,856.36,862.66],[838.09,848.92,851.12],[1006.37,1011.16,1023.17],[369.76,375.61,415.28],[412.45,419.9,456.62],[411.65,432.88,482.2],[412.73,420.73,429.5],[551.16,577.62,545.45],[382.89,394.76,386.37],[380.9,391.4,385.05],[385.3,394.67,460.32],[388.95,394.7,387.21],[800.33,807.9,807.11],[706.03,745.27,718.9],[450.9,489.59,530.97],[625.5,651.93,647.32],[2721.13,2792.12,2819.26],[945.9,954.94,957.54],[945.42,953.78,965.16],[684.36,716.29,708.75],[10.01,3.79,3.77],[7.48,3.32,3.27],[5.09,0.98,0.96],[8.7,4.77,4.68],[4.82,0.76,0.74],[4.46,0.77,0.75],[7.04,1.17,1.13]],"source":"mysql/results/c6a.4xlarge.json"} @@ -687,7 +688,7 @@ function renderSummary(filtered_data) { if (!elem.fake) { let link = document.createElement('a'); link.appendChild(document.createTextNode(`${elem.system} (${elem.machine})`)); - link.href = "https://github.com/ClickHouse/ClickBench/" + elem.source; + link.href = "https://github.com/ClickHouse/ClickBench/blob/main/" + elem.source; td_name.appendChild(link); } else { td_name.appendChild(document.createTextNode(elem.system)); From cd4211298e37ddd11b4c98a6c4cfae4916745343 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Jul 2022 09:37:00 +0000 Subject: [PATCH 480/627] Fix typo. --- src/Storages/MergeTree/DataPartStorageOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 5df90835ee3..5b9b3eec998 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -289,7 +289,7 @@ void DataPartStorageOnDisk::remove( { /// If we have a directory with suffix '.proj' it is likely a projection. - /// Try to load checksums for it (to avoid recusrive removing fallback). + /// Try to load checksums for it (to avoid recursive removing fallback). std::string checksum_path = fs::path(to) / name / "checksums.txt"; if (disk->exists(checksum_path)) { From 84064672bde0f90da770390cc6baef058850af65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 12:01:37 +0200 Subject: [PATCH 481/627] Fix error --- benchmark/infobright/benchmark.sh | 4 ++-- benchmark/mariadb/benchmark.sh | 4 ++-- benchmark/mariadb/results/c6a.4xlarge.json | 16 ++++++++-------- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/benchmark/infobright/benchmark.sh b/benchmark/infobright/benchmark.sh index 8075894248c..5d80afed863 100755 --- a/benchmark/infobright/benchmark.sh +++ b/benchmark/infobright/benchmark.sh @@ -33,6 +33,6 @@ sudo docker exec mysql_ib du -bcs /mnt/mysql_data/ /usr/local/infobright-4.0.7-x cat log.txt | grep -P 'rows? in set|Empty set|^ERROR' | - sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) days? )?(([0-9.]+) hours? )?(([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \4 \6 \7/' | - awk '{ if ($4) { print $1 * 86400 + $2 * 3600 + $3 * 60 + $4 } else if ($3) { print $1 * 3600 + $2 * 60 + $3 } else if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) days? )?(([0-9.]+) hours? )?(([0-9.]+) min )?([0-9.]+) sec\).*?$/\2,\4,\6,\7/' | + awk -F, '{ if ($1 == "null") { print } else { print $1 * 86400 + $2 * 3600 + $3 * 60 + $4 } }' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/mariadb/benchmark.sh b/benchmark/mariadb/benchmark.sh index 9baf1360af4..33f69f45c39 100755 --- a/benchmark/mariadb/benchmark.sh +++ b/benchmark/mariadb/benchmark.sh @@ -24,6 +24,6 @@ sudo du -bcs /var/lib/mysql cat log.txt | grep -P 'rows? in set|Empty set|^ERROR' | - sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) days? )?(([0-9.]+) hours? )?(([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \4 \6 \7/' | - awk '{ if ($4) { print $1 * 86400 + $2 * 3600 + $3 * 60 + $4 } else if ($3) { print $1 * 3600 + $2 * 60 + $3 } else if ($2) { print $1 * 60 + $2 } else { print $1 } }' | + sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) days? )?(([0-9.]+) hours? )?(([0-9.]+) min )?([0-9.]+) sec\).*?$/\2,\4,\6,\7/' | + awk -F, '{ if ($1 == "null") { print } else { print $1 * 86400 + $2 * 3600 + $3 * 60 + $4 } }' | awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/mariadb/results/c6a.4xlarge.json b/benchmark/mariadb/results/c6a.4xlarge.json index 7a2869285e9..e402032e20c 100644 --- a/benchmark/mariadb/results/c6a.4xlarge.json +++ b/benchmark/mariadb/results/c6a.4xlarge.json @@ -3,7 +3,7 @@ "date": "2022-07-12", "machine": "c6a.4xlarge, 500gb gp2", "cluster_size": 1, - "comment": "For some reason many queries run in almost exactly one hour. Most likely a coincidence.", + "comment": "For some reason many queries run in almost exactly one hour. Most likely a coincidence. It is installed from Debian packages, which might explain the abysmal performance.", "tags": ["C++", "row-oriented", "MySQL compatible"], @@ -11,16 +11,16 @@ "data_size": 107252505530, "result": [ -[7910.23,10229.8,70.543], +[7910.23,10229.8,3610.54], [3595.79,3595.59,3595.21], [3596.12,3595.34,3595.49], [3597.22,3595.7,3595.09], -[60.569,3596.08,3595.82], +[3600.57,3596.08,3595.82], [3596.57,3597.76,3598.47], [3595.94,3595.36,3596.2], [3595.95,3595.38,3595.45], -[109.147,84.264,83.707], -[105.386,3763.75,90.706], +[3649.15,3624.26,3623.71], +[3645.39,3763.75,3630.71], [4519.77,4552.84,4552.7], [4628.21,4629.55,4628.16], [3773.52,3774.44,3779.05], @@ -33,15 +33,15 @@ [3596.91,3598.61,3598.71], [3598.76,3598.56,3598.68], [3598.76,3598.63,3599.39], -[64.281,63.709,63.665], +[3604.28,3603.71,3603.66], [3598.83,3598.2,3598.19], [3599.11,3598.1,3598.17], [3599.04,3598.46,3598.13], [3598.76,3598.71,3598.22], [3599.02,3599.76,3598.18], -[104.124,3716.36,3725.92], +[3644.12,3716.36,3725.92], [3933.26,3854.55,3784.74], -[3392.13,3402.26,62.153], +[3392.13,3402.26,3602.15], [3717.3,3735.06,3723.07], [6126.66,6135.63,6193.45], [4029.78,3998.89,3952.7], From 1a0c75bf725059c7fbea7b65b79b0e32a8d39555 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 12:01:58 +0200 Subject: [PATCH 482/627] Generate HTML --- benchmark/index.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmark/index.html b/benchmark/index.html index 417394fd45b..eb63373bb61 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -348,7 +348,7 @@ const data = [ ,{"system":"HeavyAI","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Previous names: OmniSci, mapD. Many queries cannot run due to errors and limitations.","tags":["C++","column-oriented"],"load_time":572.633,"data_size":50887437386,"result":[[6.525,0.022,0.029],[0.301,0.042,0.04],[0.287,0.095,0.093],[2.572,0.039,0.04],[null,null,null],[null,null,null],[7.327,0.093,0.097],[0.244,0.043,0.038],[null,null,null],[null,null,null],[null,null,null],[null,null,null],[2.939,0.295,0.294],[null,null,null],[null,null,null],[null,null,null],[null,null,null],[4.716,3.91,3.955],[null,null,null],[0.154,0.083,0.106],[14.426,0.07,0.071],[null,null,null],[null,null,null],[null,null,null],[2.276,0.258,0.272],[null,null,null],[null,null,null],[null,null,null],[null,null,null],[1.832,1.64,1.602],[null,null,null],[null,null,null],[null,null,null],[14.811,0.494,0.497],[null,null,null],[null,null,null],[1.941,0.255,0.255],[5.457,0.172,0.283],[0.476,0.269,0.256],[14.239,0.179,0.178],[3.992,0.112,0.112],[1.031,0.116,0.116],[1.365,0.089,0.088]],"source":"heavyai/results/c6a.4xlarge.json"} ,{"system":"Infobright","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Only 90% of data successfully loaded. Some queries run for days.","tags":["C++","column-oriented","MySQL compatible"],"load_time":2317,"data_size":13760341294,"result":[[0.01,0,0],[2.39,2.4,2.44],[0,0,0],[7.21,6.04,6.91],[16.09,16.86,15.69],[48.8,42.37,48.63],[0,0,0],[3.48,2.42,2.42],[23.56,24.78,22.21],[32.87,31.71,34.48],[14.8,14.83,14.11],[16.7,16.53,17.37],[1752.91,1999.88,1961.4],[1193.43,1167,1220.47],[2184.81,2316.12,2224.14],[32.58,30.69,31.58],[300.17,16221.33,16168.44],[122.4,120.49,124.67],[78927.44,79250.44,78504.89],[3.38,1.22,1.21],[289.73,302.3,285.83],[415.82,389.23,403.28],[573.82,590.81,575.06],[300.13,293.96,285.64],[41.42,37.48,39.64],[75.2,75.37,72.07],[39.22,41.52,40.11],[449.56,445.03,448.68],[null,null,null],[450.87,488.3,453.83],[58.69,59.29,58.07],[84.47,78.92,79.38],[517.97,520.29,504.96],[182468.89,182468.89,182468.89],[182468.89,182468.89,182468.89],[68.43,66.93,67.68],[8.3,3.62,3.61],[1.04,0.62,0.62],[0.22,0.18,0.18],[567.78,566.52,563.02],[0.29,0.14,0.11],[0.17,0.08,0.08],[1.37,1.34,1.32]],"source":"infobright/results/c6a.4xlarge.json"} ,{"system":"MariaDB ColumnStore","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Previous name: InfiniDB.","tags":["C++","column-oriented","MySQL compatible"],"load_time":2507.8,"data_size":19712857022,"result":[[0.151,0.158,0.148],[0.1,0.101,0.106],[1.221,1.233,1.226],[0.739,0.736,0.741],[2.025,2.046,2.004],[3.725,4.801,3.755],[0.871,0.749,0.736],[0.118,0.108,0.103],[2.108,2.029,2.029],[4.225,4.271,4.288],[1.711,1.402,1.407],[1.526,1.435,1.42],[5.339,4.172,3.61],[4.692,4.729,4.96],[4.013,3.86,3.918],[3.236,2.68,2.629],[null,null,null],[null,null,null],[null,null,null],[0.137,0.109,0.147],[12.331,6.069,8.619],[2.162,2.178,2.192],[16.849,30.463,26.639],[92,90.208,92.814],[3.042,1.763,1.791],[1.779,1.772,1.749],[1.793,1.821,1.888],[13.036,10.747,9.59],[null,null,null],[78.224,79.141,77.806],[2.837,2.654,2.675],[5.833,4.552,3.678],[null,null,null],[null,null,null],[null,null,null],[3.626,3.546,3.709],[1.719,1.787,1.876],[1.345,0.906,0.91],[0.117,0.091,0.093],[1.217,1.133,1.133],[0.114,0.063,0.062],[0.1,0.062,0.061],[null,null,null]],"source":"mariadb-columnstore/results/c6a.4xlarge.json"} -,{"system":"MariaDB","date":"2022-07-12","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"For some reason many queries run in almost exactly one hour. Most likely a coincidence.","tags":["C++","row-oriented","MySQL compatible"],"load_time":8625,"data_size":107252505530,"result":[[7910.23,10229.8,70.543],[3595.79,3595.59,3595.21],[3596.12,3595.34,3595.49],[3597.22,3595.7,3595.09],[60.569,3596.08,3595.82],[3596.57,3597.76,3598.47],[3595.94,3595.36,3596.2],[3595.95,3595.38,3595.45],[109.147,84.264,83.707],[105.386,3763.75,90.706],[4519.77,4552.84,4552.7],[4628.21,4629.55,4628.16],[3773.52,3774.44,3779.05],[12198,12192.4,12192.6],[3779.67,3774.97,3777.86],[4277.99,4302.56,4275.29],[4697.19,4367.3,4372.73],[4401.2,4340.43,4310.13],[5335.93,5373.04,5361.51],[3596.91,3598.61,3598.71],[3598.76,3598.56,3598.68],[3598.76,3598.63,3599.39],[64.281,63.709,63.665],[3598.83,3598.2,3598.19],[3599.11,3598.1,3598.17],[3599.04,3598.46,3598.13],[3598.76,3598.71,3598.22],[3599.02,3599.76,3598.18],[104.124,3716.36,3725.92],[3933.26,3854.55,3784.74],[3392.13,3402.26,62.153],[3717.3,3735.06,3723.07],[6126.66,6135.63,6193.45],[4029.78,3998.89,3952.7],[3955.34,4029.28,3941.84],[4881.07,4782.53,4900.72],[24.455,23.862,25.173],[24.195,23.529,23.473],[20.719,21.629,21.582],[24.577,25.877,25.953],[20.686,20.787,21.199],[20.036,17.973,22.759],[null,null,null]],"source":"mariadb/results/c6a.4xlarge.json"} +,{"system":"MariaDB","date":"2022-07-12","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"For some reason many queries run in almost exactly one hour. Most likely a coincidence. It is installed from Debian packages, which might explain the abysmal performance.","tags":["C++","row-oriented","MySQL compatible"],"load_time":8625,"data_size":107252505530,"result":[[7910.23,10229.8,3610.54],[3595.79,3595.59,3595.21],[3596.12,3595.34,3595.49],[3597.22,3595.7,3595.09],[3600.57,3596.08,3595.82],[3596.57,3597.76,3598.47],[3595.94,3595.36,3596.2],[3595.95,3595.38,3595.45],[3649.15,3624.26,3623.71],[3645.39,3763.75,3630.71],[4519.77,4552.84,4552.7],[4628.21,4629.55,4628.16],[3773.52,3774.44,3779.05],[12198,12192.4,12192.6],[3779.67,3774.97,3777.86],[4277.99,4302.56,4275.29],[4697.19,4367.3,4372.73],[4401.2,4340.43,4310.13],[5335.93,5373.04,5361.51],[3596.91,3598.61,3598.71],[3598.76,3598.56,3598.68],[3598.76,3598.63,3599.39],[3604.28,3603.71,3603.66],[3598.83,3598.2,3598.19],[3599.11,3598.1,3598.17],[3599.04,3598.46,3598.13],[3598.76,3598.71,3598.22],[3599.02,3599.76,3598.18],[3644.12,3716.36,3725.92],[3933.26,3854.55,3784.74],[3392.13,3402.26,3602.15],[3717.3,3735.06,3723.07],[6126.66,6135.63,6193.45],[4029.78,3998.89,3952.7],[3955.34,4029.28,3941.84],[4881.07,4782.53,4900.72],[24.455,23.862,25.173],[24.195,23.529,23.473],[20.719,21.629,21.582],[24.577,25.877,25.953],[20.686,20.787,21.199],[20.036,17.973,22.759],[null,null,null]],"source":"mariadb/results/c6a.4xlarge.json"} ,{"system":"MonetDB","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","column-oriented"],"load_time":939,"data_size":49696606499,"result":[[0.000218,0.000157,0.000155],[0.101903,0.019908,0.018439],[0.282431,0.035987,0.034938],[2.868,0.029387,0.029207],[4.675,4.515,4.511],[6.584,4.269,4.65],[0.528827,0.063135,0.065742],[0.506863,0.020966,0.021687],[8.343,4.457,4.408],[7.224,6.548,7.576],[0.267003,0.233353,0.230444],[0.347206,0.28358,0.266085],[5.389,3.099,3.074],[7.653,7.759,8.596],[3.276,3.326,3.292],[5.31,3.465,3.578],[9.341,9.143,9.536],[9.584,9.604,9.419],[19.539,19.783,19.611],[0.004509,0.000702,0.000643],[20.801,1.57,1.603],[2.752,0.418221,0.395884],[14.717,0.800894,0.395477],[14.429,1.804,1.869],[1.386,0.159602,0.156426],[0.189736,0.167664,0.168781],[0.164681,0.176666,0.17126],[3.005,3.113,3.882],[null,null,null],[2.751,2.846,2.676],[7.937,2.579,2.447],[5.12,3.492,3.467],[22.862,22.567,23.211],[33.437,18.889,19.043],[18.898,19.583,19.047],[14.774,12.984,13.803],[3.865,0.322143,0.323117],[0.192149,0.177791,0.175984],[0.194173,0.159398,0.165201],[0.680778,0.592252,0.560738],[0.106465,0.10638,0.102692],[0.154871,0.153752,0.155782],[0.11459,0.09639,0.095594]],"source":"monetdb/results/c6a.4xlarge.json"} ,{"system":"MySQL (MyISAM)","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","row-oriented","MySQL compatible"],"load_time":2512,"data_size":121588958061,"result":[[0,0,0],[283.32,276.83,274.52],[276.93,278.29,283.27],[28.83,23.63,21.55],[46.41,40.81,40.93],[467.04,467.39,469.08],[31.02,25.89,24.2],[277.89,275.3,277.3],[329.34,325.8,325.35],[342.86,338.43,336.95],[282.03,279.87,281.22],[277.74,282.68,282],[335.66,334.83,336.44],[305.24,310.39,307.3],[337.41,338.52,342.94],[308.66,307.34,306.27],[738.38,748.44,740.75],[738.75,734.01,738.25],[867.01,872.92,868.84],[25.65,20.61,18.46],[312.39,313.67,306.66],[301.66,305.12,308.01],[298.12,298.44,312.4],[311.34,309.9,311.85],[281.87,278.5,275],[277.46,277.46,277.46],[280.75,278.04,281.76],[263.9,417.39,406.88],[707.21,711.96,705],[668.1,668.33,665.96],[330.31,333.36,331.94],[506.57,506.18,500.53],[2604.49,2681.96,2703.12],[830.65,832.88,831.14],[831.98,830.46,833.41],[608.49,608.51,613.68],[4.56,4.13,4.16],[3.8,3.8,3.7],[1.65,1.45,1.46],[6.33,5.14,6.15],[1.6,1.41,1.41],[1.56,1.42,1.39],[7.04,1.17,1.13]],"source":"mysql-myisam/results/c6a.4xlarge.json"} ,{"system":"MySQL","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","row-oriented","MySQL compatible"],"load_time":9472,"data_size":171953585825,"result":[[339.77,339.88,339.77],[364.91,371.86,367.55],[366.2,368.91,389.66],[364.39,377.53,571.45],[377.69,390.02,384.86],[569.48,576.51,574.68],[367.4,368.23,370.41],[371.29,384.02,613.22],[478.85,683.22,495.68],[489.9,635.96,662.43],[386.07,396.49,640.15],[389.13,412.55,444.12],[447.97,455.54,448.06],[423.22,845.44,813.6],[452.48,460.07,453.98],[577.54,623.21,586.49],[852.07,856.36,862.66],[838.09,848.92,851.12],[1006.37,1011.16,1023.17],[369.76,375.61,415.28],[412.45,419.9,456.62],[411.65,432.88,482.2],[412.73,420.73,429.5],[551.16,577.62,545.45],[382.89,394.76,386.37],[380.9,391.4,385.05],[385.3,394.67,460.32],[388.95,394.7,387.21],[800.33,807.9,807.11],[706.03,745.27,718.9],[450.9,489.59,530.97],[625.5,651.93,647.32],[2721.13,2792.12,2819.26],[945.9,954.94,957.54],[945.42,953.78,965.16],[684.36,716.29,708.75],[10.01,3.79,3.77],[7.48,3.32,3.27],[5.09,0.98,0.96],[8.7,4.77,4.68],[4.82,0.76,0.74],[4.46,0.77,0.75],[7.04,1.17,1.13]],"source":"mysql/results/c6a.4xlarge.json"} From bcd9453f332743d5fa0afc937210e6ea01d522cf Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Jul 2022 12:15:20 +0200 Subject: [PATCH 483/627] Set meilieserach client version too --- docker/test/integration/runner/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index a124d95b360..84c04dd03ec 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -92,7 +92,7 @@ RUN python3 -m pip install \ requests-kerberos \ pyhdfs \ azure-storage-blob \ - meilisearch + meilisearch==0.18.3 COPY modprobe.sh /usr/local/bin/modprobe COPY dockerd-entrypoint.sh /usr/local/bin/ From 948f15eff317a0ba0462003ad3fc94422a891c44 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 12 Jul 2022 14:01:53 +0300 Subject: [PATCH 484/627] Update run.sh --- docker/test/stress/run.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index b2c6036ba31..fcf99b34064 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -352,7 +352,6 @@ else mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.clean.log # Error messages (we should ignore some errors) - # FIXME https://github.com/ClickHouse/ClickHouse/issues/38629 ("pp.proj, errno: 21") # FIXME https://github.com/ClickHouse/ClickHouse/issues/38643 ("Unknown index: idx.") echo "Check for Error messages in server log:" zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ @@ -376,7 +375,6 @@ else -e "and a merge is impossible: we didn't find" \ -e "found in queue and some source parts for it was lost" \ -e "is lost forever." \ - -e "pp.proj, errno: 21" \ -e "Unknown index: idx." \ /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ From ffcc80c9e14d3c83a1da19baba893df2b615cb84 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Jul 2022 13:26:08 +0200 Subject: [PATCH 485/627] Update cluster.py --- tests/integration/helpers/cluster.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5983c886680..7e4513960f9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2943,7 +2943,9 @@ class ClickHouseInstance: ignore_error=False, query_id=None, ): - logging.debug("Executing query %s on %s", sql, self.name) + # Do not print too big queries, because somethimes they are enormously big. + if len(sql) < 1000: + logging.debug("Executing query %s on %s", sql, self.name) return self.client.query( sql, stdin=stdin, From f35370eac874fb3fcf726cbc821d020599aead4a Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 12 Jul 2022 19:50:58 +0800 Subject: [PATCH 486/627] rewrite testcase --- .../queries/0_stateless/02169_map_functions.reference | 10 ---------- tests/queries/0_stateless/02169_map_functions.sql | 11 ----------- .../0_stateless/02363_mapupdate_improve.reference | 10 ++++++++++ tests/queries/0_stateless/02363_mapupdate_improve.sql | 11 +++++++++++ 4 files changed, 21 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/02363_mapupdate_improve.reference create mode 100644 tests/queries/0_stateless/02363_mapupdate_improve.sql diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 16fda0bb840..160aebbc852 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -31,13 +31,3 @@ {1:2,2:3} {'x':'y','x':'y'} {'x':'y','x':'y'} -{'fruit':'apple','season':'autumn'} -{'fruit':'apple','season':'autumn'} -{'fruit':'apple','season':'autumn'} -{'fruit':'apple','season':'autumn'} -{'fruit':'apple','season':'autumn'} -{'season':'autumn','fruit':'apple'} -{'season':'autumn','fruit':'apple'} -{'season':'autumn','fruit':'apple'} -{'season':'autumn','fruit':'apple'} -{'season':'autumn','fruit':'apple'} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 9e93412739f..4cccaa56722 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -37,14 +37,3 @@ SELECT mapUpdate(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } DROP TABLE table_map; - -DROP TABLE IF EXISTS map_test; -CREATE TABLE map_test(`tags` Map(String, String)) ENGINE = MergeTree PRIMARY KEY tags ORDER BY tags SETTINGS index_granularity = 8192; -INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); -INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); -INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); -INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); -INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); -SELECT mapUpdate(mapFilter((k, v) -> (k in ('fruit')), tags), map('season', 'autumn')) FROM map_test; -SELECT mapUpdate(map('season','autumn'), mapFilter((k, v) -> (k in ('fruit')), tags)) FROM map_test; -DROP TABLE map_test; diff --git a/tests/queries/0_stateless/02363_mapupdate_improve.reference b/tests/queries/0_stateless/02363_mapupdate_improve.reference new file mode 100644 index 00000000000..04e2b943929 --- /dev/null +++ b/tests/queries/0_stateless/02363_mapupdate_improve.reference @@ -0,0 +1,10 @@ +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} diff --git a/tests/queries/0_stateless/02363_mapupdate_improve.sql b/tests/queries/0_stateless/02363_mapupdate_improve.sql new file mode 100644 index 00000000000..6b7723cc9b4 --- /dev/null +++ b/tests/queries/0_stateless/02363_mapupdate_improve.sql @@ -0,0 +1,11 @@ +-- Tags: no-backward-compatibility-check +DROP TABLE IF EXISTS map_test; +CREATE TABLE map_test(`tags` Map(String, String)) ENGINE = MergeTree PRIMARY KEY tags ORDER BY tags SETTINGS index_granularity = 8192; +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +SELECT mapUpdate(mapFilter((k, v) -> (k in ('fruit')), tags), map('season', 'autumn')) FROM map_test; +SELECT mapUpdate(map('season','autumn'), mapFilter((k, v) -> (k in ('fruit')), tags)) FROM map_test; +DROP TABLE map_test; From adfc7a5a78161027e8525ddcdbe15a9eeb5b3ca1 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Jul 2022 14:05:59 +0200 Subject: [PATCH 487/627] Update cluster.py --- tests/integration/helpers/cluster.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7e4513960f9..1f46bac6491 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2943,9 +2943,12 @@ class ClickHouseInstance: ignore_error=False, query_id=None, ): - # Do not print too big queries, because somethimes they are enormously big. - if len(sql) < 1000: - logging.debug("Executing query %s on %s", sql, self.name) + sql_for_log = '' + if len(sql) > 1000: + sql_for_log = sql[:1000] + else: + sql_for_log = sql + logging.debug("Executing query %s on %s", sql_for_log, self.name) return self.client.query( sql, stdin=stdin, From 81da0bb9f558b55f6d7b3d08d0990b30eef5af08 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jul 2022 13:28:10 +0000 Subject: [PATCH 488/627] keep LowCardinality type in array() and map() functions --- src/Functions/array/array.cpp | 1 + src/Functions/map.cpp | 1 + .../02354_array_lowcardinality.reference | 12 ++++++++++++ .../0_stateless/02354_array_lowcardinality.sql | 14 ++++++++++++++ 4 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/02354_array_lowcardinality.reference create mode 100644 tests/queries/0_stateless/02354_array_lowcardinality.sql diff --git a/src/Functions/array/array.cpp b/src/Functions/array/array.cpp index b0a7daac522..f220117407c 100644 --- a/src/Functions/array/array.cpp +++ b/src/Functions/array/array.cpp @@ -23,6 +23,7 @@ public: /// array(..., Nothing, ...) -> Array(..., Nothing, ...) bool useDefaultImplementationForNothing() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } bool isVariadic() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 8c891fdec81..866d3ae3866 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -68,6 +68,7 @@ public: /// map(..., Nothing) -> Map(..., Nothing) bool useDefaultImplementationForNothing() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { diff --git a/tests/queries/0_stateless/02354_array_lowcardinality.reference b/tests/queries/0_stateless/02354_array_lowcardinality.reference new file mode 100644 index 00000000000..d364bbdb5f4 --- /dev/null +++ b/tests/queries/0_stateless/02354_array_lowcardinality.reference @@ -0,0 +1,12 @@ +Array(LowCardinality(String)) +Array(LowCardinality(String)) +Array(LowCardinality(String)) +Array(LowCardinality(String)) +Array(String) +Array(String) +Array(String) +Array(String) +Map(LowCardinality(String), LowCardinality(String)) +Map(LowCardinality(String), LowCardinality(String)) +Map(LowCardinality(String), LowCardinality(String)) +Map(LowCardinality(String), LowCardinality(String)) diff --git a/tests/queries/0_stateless/02354_array_lowcardinality.sql b/tests/queries/0_stateless/02354_array_lowcardinality.sql new file mode 100644 index 00000000000..1aa26b29050 --- /dev/null +++ b/tests/queries/0_stateless/02354_array_lowcardinality.sql @@ -0,0 +1,14 @@ +SELECT toTypeName([toLowCardinality('1'), toLowCardinality('2')]); +SELECT toTypeName([materialize(toLowCardinality('1')), toLowCardinality('2')]); +SELECT toTypeName([toLowCardinality('1'), materialize(toLowCardinality('2'))]); +SELECT toTypeName([materialize(toLowCardinality('1')), materialize(toLowCardinality('2'))]); + +SELECT toTypeName([toLowCardinality('1'), '2']); +SELECT toTypeName([materialize(toLowCardinality('1')), '2']); +SELECT toTypeName([toLowCardinality('1'), materialize('2')]); +SELECT toTypeName([materialize(toLowCardinality('1')), materialize('2')]); + +SELECT toTypeName(map(toLowCardinality('1'), toLowCardinality('2'))); +SELECT toTypeName(map(materialize(toLowCardinality('1')), toLowCardinality('2'))); +SELECT toTypeName(map(toLowCardinality('1'), materialize(toLowCardinality('2')))); +SELECT toTypeName(map(materialize(toLowCardinality('1')), materialize(toLowCardinality('2')))); From 04f9e15012c7b2d668cbb60d9d4a059afffcceda Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jul 2022 13:46:49 +0000 Subject: [PATCH 489/627] fix test --- .../02210_toColumnTypeName_toLowCardinality_const.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference b/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference index 2ac2f690f1b..1e3d3a50562 100644 --- a/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference +++ b/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference @@ -1 +1 @@ -Const(UInt8) +Const(ColumnLowCardinality) From f629fd7e573b4dccd6a8e9fe51b093ded1408934 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Jul 2022 15:50:35 +0200 Subject: [PATCH 490/627] Update docker_compose_meili.yml --- docker/test/integration/runner/compose/docker_compose_meili.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/compose/docker_compose_meili.yml b/docker/test/integration/runner/compose/docker_compose_meili.yml index 14fed95d567..dc58643d26f 100644 --- a/docker/test/integration/runner/compose/docker_compose_meili.yml +++ b/docker/test/integration/runner/compose/docker_compose_meili.yml @@ -7,7 +7,7 @@ services: - ${MEILI_EXTERNAL_PORT}:${MEILI_INTERNAL_PORT} meili_secure: - image: getmeili/meilisearch:latest + image: getmeili/meilisearch:v0.27.0 restart: always ports: - ${MEILI_SECURE_EXTERNAL_PORT}:${MEILI_SECURE_INTERNAL_PORT} From 7c437c3f9b7e977e98358b185a9a20d8afb11d3c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 12 Jul 2022 09:56:09 -0400 Subject: [PATCH 491/627] some optimizations --- utils/self-extracting-executable/compressor.cpp | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/utils/self-extracting-executable/compressor.cpp b/utils/self-extracting-executable/compressor.cpp index d9da3aafe85..f6bbc33aa16 100644 --- a/utils/self-extracting-executable/compressor.cpp +++ b/utils/self-extracting-executable/compressor.cpp @@ -448,7 +448,16 @@ int main(int argc, char* argv[]) if (p != nullptr) { if (p[0] != 0) - level = atoi(p); + { + char * end = nullptr; + level = strtol(p, &end, 10); + if (*end != 0) + { + std::cerr << "Error: level [" << p << "] is not valid" << std::endl; + usage(stderr, argv[0]); + return 1; + } + } ++start_of_files; } @@ -482,7 +491,7 @@ int main(int argc, char* argv[]) } ++start_of_files; - if (decompressor != nullptr && decompressor[0] != 0) + if (decompressor != nullptr) { if (copy_decompressor_file(decompressor, output_fd)) return 1; From 75c3926cbb8d1377a39c5296eeac3555de1d875b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Jul 2022 13:57:36 +0000 Subject: [PATCH 492/627] Fix insert into MV with enabled extremes. --- src/Processors/Transforms/buildPushingToViewsChain.cpp | 1 + .../0_stateless/01549_low_cardinality_mv_fuzz.reference | 0 .../queries/0_stateless/01549_low_cardinality_mv_fuzz.sql | 7 +++++++ 3 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/01549_low_cardinality_mv_fuzz.reference create mode 100644 tests/queries/0_stateless/01549_low_cardinality_mv_fuzz.sql diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index b68ec547280..a8890f5bccb 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -430,6 +430,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions()); auto pipeline = select.buildQueryPipeline(); pipeline.resize(1); + pipeline.dropTotalsAndExtremes(); /// Squashing is needed here because the materialized view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY diff --git a/tests/queries/0_stateless/01549_low_cardinality_mv_fuzz.reference b/tests/queries/0_stateless/01549_low_cardinality_mv_fuzz.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01549_low_cardinality_mv_fuzz.sql b/tests/queries/0_stateless/01549_low_cardinality_mv_fuzz.sql new file mode 100644 index 00000000000..32a6fccc825 --- /dev/null +++ b/tests/queries/0_stateless/01549_low_cardinality_mv_fuzz.sql @@ -0,0 +1,7 @@ +set extremes = '1'; +DROP TABLE IF EXISTS HASH_TEST_INSERT; +DROP TABLE IF EXISTS HASH_MV; +CREATE TABLE HASH_TEST_INSERT (`STR_VAL` String) ENGINE = Null; +CREATE MATERIALIZED VIEW HASH_MV (`HASH_VAL` UInt64, `STR_VAL` LowCardinality(String)) ENGINE = ReplacingMergeTree ORDER BY HASH_VAL AS SELECT xxHash64(STR_VAL) AS HASH_VAL, toLowCardinality(STR_VAL) AS STR_VAL FROM HASH_TEST_INSERT; +INSERT INTO HASH_TEST_INSERT VALUES ('a'); + From b9d9ca5194e4824ffb4a7bfd5f38b4fc953d8f44 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 7 Jul 2022 16:28:06 +0800 Subject: [PATCH 493/627] style fix --- src/Processors/QueryPlan/JoinStep.cpp | 11 +++++------ src/Processors/QueryPlan/JoinStep.h | 1 + 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index d481c745798..04d579a4864 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -35,11 +35,10 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (join->pipelineType() == JoinPipelineType::YShaped) return QueryPipelineBuilder::joinPipelinesYShaped( - std::move(pipelines[0]), std::move(pipelines[1]), - join, output_stream->header, - max_block_size, &processors); + std::move(pipelines[0]), std::move(pipelines[1]), join, output_stream->header, max_block_size, &processors); - return QueryPipelineBuilder::joinPipelinesRightLeft(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, max_streams, keep_left_read_in_order, &processors); + return QueryPipelineBuilder::joinPipelinesRightLeft( + std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, max_streams, keep_left_read_in_order, &processors); } void JoinStep::describePipeline(FormatSettings & settings) const @@ -50,7 +49,8 @@ void JoinStep::describePipeline(FormatSettings & settings) const void JoinStep::updateLeftStream(const DataStream & left_stream_) { input_streams = {left_stream_, input_streams.at(1)}; - output_stream = DataStream{ + output_stream = DataStream + { .header = JoiningTransform::transformHeader(left_stream_.header, join), }; } @@ -108,5 +108,4 @@ void FilledJoinStep::updateOutputStream() input_streams.front(), JoiningTransform::transformHeader(input_streams.front().header, join), getDataStreamTraits()); } - } diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 17a0cc2ae63..1ea0e9b366d 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -1,4 +1,5 @@ #pragma once + #include #include From d3709c6c269a82b070e2ac9e8dcf6d23628bc6ae Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 7 Jul 2022 16:28:21 +0800 Subject: [PATCH 494/627] Avoid redundant join block transformation. --- src/Processors/QueryPlan/JoinStep.cpp | 2 +- src/Processors/Transforms/JoiningTransform.cpp | 5 +++-- src/Processors/Transforms/JoiningTransform.h | 3 ++- src/QueryPipeline/QueryPipelineBuilder.cpp | 6 ++++-- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 04d579a4864..4e5dafb197b 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -98,7 +98,7 @@ void FilledJoinStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu { bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; auto counter = on_totals ? nullptr : finish_counter; - return std::make_shared(header, join, max_block_size, on_totals, default_totals, counter); + return std::make_shared(header, output_stream->header, join, max_block_size, on_totals, default_totals, counter); }); } diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 4f923810715..64343946ff4 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -23,13 +23,14 @@ Block JoiningTransform::transformHeader(Block header, const JoinPtr & join) } JoiningTransform::JoiningTransform( - Block input_header, + const Block & input_header, + const Block & output_header, JoinPtr join_, size_t max_block_size_, bool on_totals_, bool default_totals_, FinishCounterPtr finish_counter_) - : IProcessor({input_header}, {transformHeader(input_header, join_)}) + : IProcessor({input_header}, {output_header}) , join(std::move(join_)) , on_totals(on_totals_) , default_totals(default_totals_) diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 3383849b41d..0595d035657 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -39,7 +39,8 @@ public: using FinishCounterPtr = std::shared_ptr; JoiningTransform( - Block input_header, + const Block & input_header, + const Block & output_header, JoinPtr join_, size_t max_block_size_, bool on_totals_ = false, diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 5f83037e8b0..700ef947882 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -448,9 +448,11 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe auto lit = left->pipe.output_ports.begin(); auto rit = right->pipe.output_ports.begin(); + auto output_header = JoiningTransform::transformHeader(left->getHeader(), join); for (size_t i = 0; i < num_streams; ++i) { - auto joining = std::make_shared(left->getHeader(), join, max_block_size, false, default_totals, finish_counter); + auto joining = std::make_shared( + left->getHeader(), output_header, join, max_block_size, false, default_totals, finish_counter); connect(**lit, joining->getInputs().front()); connect(**rit, joining->getInputs().back()); *lit = &joining->getOutputs().front(); @@ -466,7 +468,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (left->hasTotals()) { - auto joining = std::make_shared(left->getHeader(), join, max_block_size, true, default_totals); + auto joining = std::make_shared(left->getHeader(), output_header, join, max_block_size, true, default_totals); connect(*left->pipe.totals_port, joining->getInputs().front()); connect(**rit, joining->getInputs().back()); left->pipe.totals_port = &joining->getOutputs().front(); From 982e1a73d377f385e7fb7c1458a8c6ba53c37132 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 7 Jul 2022 23:00:40 +0800 Subject: [PATCH 495/627] Better --- src/Processors/QueryPlan/JoinStep.cpp | 9 ++++++++- src/QueryPipeline/QueryPipelineBuilder.cpp | 2 +- src/QueryPipeline/QueryPipelineBuilder.h | 1 + 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 4e5dafb197b..d7ccf003937 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -38,7 +38,14 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines std::move(pipelines[0]), std::move(pipelines[1]), join, output_stream->header, max_block_size, &processors); return QueryPipelineBuilder::joinPipelinesRightLeft( - std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, max_streams, keep_left_read_in_order, &processors); + std::move(pipelines[0]), + std::move(pipelines[1]), + join, + output_stream->header, + max_block_size, + max_streams, + keep_left_read_in_order, + &processors); } void JoinStep::describePipeline(FormatSettings & settings) const diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 700ef947882..88a52defa1e 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -361,6 +361,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe std::unique_ptr left, std::unique_ptr right, JoinPtr join, + const Block & output_header, size_t max_block_size, size_t max_streams, bool keep_left_read_in_order, @@ -448,7 +449,6 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe auto lit = left->pipe.output_ports.begin(); auto rit = right->pipe.output_ports.begin(); - auto output_header = JoiningTransform::transformHeader(left->getHeader(), join); for (size_t i = 0; i < num_streams; ++i) { auto joining = std::make_shared( diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 18e9d718445..2d9b8028627 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -114,6 +114,7 @@ public: std::unique_ptr left, std::unique_ptr right, JoinPtr join, + const Block & output_header, size_t max_block_size, size_t max_streams, bool keep_left_read_in_order, From 582300e08c1c67203bea13636e7bf1fafaf3e428 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Jul 2022 16:22:47 +0200 Subject: [PATCH 496/627] Update 01701_clear_projection_and_part_remove.sql Fix typo --- .../0_stateless/01701_clear_projection_and_part_remove.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01701_clear_projection_and_part_remove.sql b/tests/queries/0_stateless/01701_clear_projection_and_part_remove.sql index e6f2142cac5..e6cc4cbdb10 100644 --- a/tests/queries/0_stateless/01701_clear_projection_and_part_remove.sql +++ b/tests/queries/0_stateless/01701_clear_projection_and_part_remove.sql @@ -11,7 +11,7 @@ alter table tp_1 detach partition '0'; alter table tp_1 clear projection pp; -- Remove projection from table metadata alter table tp_1 drop projection pp; --- Now, we don't load projection pp for attached part, bit it is written on disk +-- Now, we don't load projection pp for attached part, but it is written on disk alter table tp_1 attach partition '0'; -- Make this part obsolete optimize table tp_1 final; From f46d641440151e18f76667b6581f1a9962d5556b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Jul 2022 14:02:09 +0200 Subject: [PATCH 497/627] Fix style check and black check t:wq# Date: Tue Jul 12 14:02:09 2022 +0200 --- src/Storages/StorageMongoDB.cpp | 12 +-- src/Storages/StorageMongoDB.h | 4 +- src/TableFunctions/TableFunctionMongoDB.cpp | 24 ++--- src/TableFunctions/TableFunctionMongoDB.h | 10 ++- .../integration/test_storage_mongodb/test.py | 15 ++-- .../test_table_function_mongodb/test.py | 90 +++++++++++++++---- 6 files changed, 101 insertions(+), 54 deletions(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 1e8cb41f1b5..a33f17191eb 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -163,18 +163,10 @@ Pipe StorageMongoDB::read( return Pipe(std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size)); } - SinkToStoragePtr StorageMongoDB::write( - const ASTPtr & /*query*/, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr /*context*/) + SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) { connectIfNotConnected(); - - return std::make_shared( - collection_name, - database_name, - metadata_snapshot, - connection); + return std::make_shared(collection_name, database_name, metadata_snapshot, connection); } StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, ContextPtr context) diff --git a/src/Storages/StorageMongoDB.h b/src/Storages/StorageMongoDB.h index cc047811d85..0e00b80432b 100644 --- a/src/Storages/StorageMongoDB.h +++ b/src/Storages/StorageMongoDB.h @@ -40,8 +40,8 @@ public: unsigned num_streams) override; SinkToStoragePtr write( - const ASTPtr & query, - const StorageMetadataPtr & /*metadata_snapshot*/, + const ASTPtr & query, + const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; static StorageMongoDBConfiguration getConfiguration(ASTs engine_args, ContextPtr context); diff --git a/src/TableFunctions/TableFunctionMongoDB.cpp b/src/TableFunctions/TableFunctionMongoDB.cpp index d6be0d8b1da..5e96b85e64c 100644 --- a/src/TableFunctions/TableFunctionMongoDB.cpp +++ b/src/TableFunctions/TableFunctionMongoDB.cpp @@ -30,14 +30,14 @@ StoragePtr TableFunctionMongoDB::executeImpl(const ASTPtr & /*ast_function*/, { auto columns = getActualTableStructure(context); auto storage = std::make_shared( - StorageID(configuration_->database, table_name), - configuration_->host, - configuration_->port, - configuration_->database, - configuration_->table, - configuration_->username, - configuration_->password, - configuration_->options, + StorageID(configuration->database, table_name), + configuration->host, + configuration->port, + configuration->database, + configuration->table, + configuration->username, + configuration->password, + configuration->options, columns, ConstraintsDescription(), String{}); @@ -47,7 +47,7 @@ StoragePtr TableFunctionMongoDB::executeImpl(const ASTPtr & /*ast_function*/, ColumnsDescription TableFunctionMongoDB::getActualTableStructure(ContextPtr context) const { - return parseColumnsListFromString(structure_, context); + return parseColumnsListFromString(structure, context); } void TableFunctionMongoDB::parseArguments(const ASTPtr & ast_function, ContextPtr context) @@ -78,13 +78,13 @@ void TableFunctionMongoDB::parseArguments(const ASTPtr & ast_function, ContextPt auto arg_name = function_args[0]->as()->name(); if (arg_name == "structure") - structure_ = checkAndGetLiteralArgument(function_args[1], "structure"); + structure = checkAndGetLiteralArgument(function_args[1], "structure"); else if (arg_name == "options") main_arguments.push_back(function_args[1]); } else if (i == 5) { - structure_ = checkAndGetLiteralArgument(args[i], "structure"); + structure = checkAndGetLiteralArgument(args[i], "structure"); } else if (i == 6) { @@ -92,7 +92,7 @@ void TableFunctionMongoDB::parseArguments(const ASTPtr & ast_function, ContextPt } } - configuration_ = StorageMongoDB::getConfiguration(main_arguments, context); + configuration = StorageMongoDB::getConfiguration(main_arguments, context); } diff --git a/src/TableFunctions/TableFunctionMongoDB.h b/src/TableFunctions/TableFunctionMongoDB.h index cd06a0c2d51..40e4802e9e6 100644 --- a/src/TableFunctions/TableFunctionMongoDB.h +++ b/src/TableFunctions/TableFunctionMongoDB.h @@ -1,4 +1,4 @@ -#include "config_core.h" +#pragma once #include #include @@ -11,6 +11,7 @@ class TableFunctionMongoDB : public ITableFunction { public: static constexpr auto name = "mongodb"; + std::string getName() const override { return name; } private: @@ -20,10 +21,11 @@ private: const char * getStorageTypeName() const override { return "MongoDB"; } -ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - std::optional configuration_; - String structure_; + + std::optional configuration; + String structure; }; } diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index d1a0c624c70..d8ca207d0a6 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -46,15 +46,15 @@ def test_simple_select(started_cluster): db = mongo_connection["test"] db.add_user("root", "clickhouse") simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) node = started_cluster.instances["node"] node.query( "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" ) - for i in range(0, 100): - node.query( - "INSERT INTO simple_mongo_table(key, data) VALUES ({}, '{}')".format(i, hex(i * i)) - ) assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" assert ( @@ -138,15 +138,14 @@ def test_secure_connection(started_cluster): db.add_user("root", "clickhouse") simple_mongo_table = db["simple_table"] data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) node = started_cluster.instances["node"] node.query( "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'ssl=true')" ) - for i in range(0, 100): - node.query( - "INSERT INTO simple_mongo_table(key, data) VALUES ({}, '{}')".format(i, hex(i * i)) - ) assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" assert ( diff --git a/tests/integration/test_table_function_mongodb/test.py b/tests/integration/test_table_function_mongodb/test.py index 44aa07ddd56..e0ad71b0079 100644 --- a/tests/integration/test_table_function_mongodb/test.py +++ b/tests/integration/test_table_function_mongodb/test.py @@ -49,20 +49,33 @@ def test_simple_select(started_cluster): node = started_cluster.instances["node"] for i in range(0, 100): node.query( - "INSERT INTO FUNCTION mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String') (key, data) VALUES ({}, '{}')".format(i, hex(i * i)) + "INSERT INTO FUNCTION mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String') (key, data) VALUES ({}, '{}')".format( + i, hex(i * i) + ) ) - assert node.query("SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')") == "100\n" assert ( - node.query("SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')") + node.query( + "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" + ) + == "100\n" + ) + assert ( + node.query( + "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" + ) == str(sum(range(0, 100))) + "\n" ) assert ( - node.query("SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'key UInt64, data String')") + node.query( + "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'key UInt64, data String')" + ) == str(sum(range(0, 100))) + "\n" ) assert ( - node.query("SELECT data from mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String') where key = 42") + node.query( + "SELECT data from mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String') where key = 42" + ) == hex(42 * 42) + "\n" ) simple_mongo_table.drop() @@ -81,14 +94,23 @@ def test_complex_data_type(started_cluster): node = started_cluster.instances["node"] - assert node.query("SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)')") == "100\n" assert ( - node.query("SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)')") + node.query( + "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)')" + ) + == "100\n" + ) + assert ( + node.query( + "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)')" + ) == str(sum(range(0, 100))) + "\n" ) assert ( - node.query("SELECT data from mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)') where key = 42") + node.query( + "SELECT data from mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)') where key = 42" + ) == hex(42 * 42) + "\n" ) incomplete_mongo_table.drop() @@ -108,7 +130,9 @@ def test_incorrect_data_type(started_cluster): node = started_cluster.instances["node"] with pytest.raises(QueryRuntimeException): - node.query("SELECT aaaa FROM mongodb('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse', structure='key UInt64, data String')") + node.query( + "SELECT aaaa FROM mongodb('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse', structure='key UInt64, data String')" + ) strange_mongo_table.drop() @@ -126,18 +150,29 @@ def test_secure_connection(started_cluster): node = started_cluster.instances["node"] - assert node.query("SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true')") == "100\n" assert ( - node.query("SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true')") + node.query( + "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true')" + ) + == "100\n" + ) + assert ( + node.query( + "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true')" + ) == str(sum(range(0, 100))) + "\n" ) assert ( - node.query("SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'key UInt64, data String', 'ssl=true')") + node.query( + "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'key UInt64, data String', 'ssl=true')" + ) == str(sum(range(0, 100))) + "\n" ) assert ( - node.query("SELECT data from mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true') where key = 42") + node.query( + "SELECT data from mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true') where key = 42" + ) == hex(42 * 42) + "\n" ) simple_mongo_table.drop() @@ -155,7 +190,12 @@ def test_predefined_connection_configuration(started_cluster): simple_mongo_table.insert_many(data) node = started_cluster.instances["node"] - assert node.query("SELECT count() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')") == "100\n" + assert ( + node.query( + "SELECT count() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" + ) + == "100\n" + ) simple_mongo_table.drop() @@ -170,7 +210,12 @@ def test_no_credentials(started_cluster): simple_mongo_table.insert_many(data) node = started_cluster.instances["node"] - assert node.query("SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', '', '', structure='key UInt64, data String')") == "100\n" + assert ( + node.query( + "SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', '', '', structure='key UInt64, data String')" + ) + == "100\n" + ) simple_mongo_table.drop() @@ -197,9 +242,16 @@ def test_auth_source(started_cluster): node = started_cluster.instances["node"] - node.query_and_get_error("SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')") + node.query_and_get_error( + "SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" + ) - assert node.query("SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='authSource=admin')") == "100\n" + assert ( + node.query( + "SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='authSource=admin')" + ) + == "100\n" + ) simple_mongo_table.drop() @@ -217,6 +269,8 @@ def test_missing_columns(started_cluster): simple_mongo_table.insert_many(data) node = started_cluster.instances["node"] - result = node.query("SELECT count() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data Nullable(String)') WHERE isNull(data)") + result = node.query( + "SELECT count() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data Nullable(String)') WHERE isNull(data)" + ) assert result == "10\n" simple_mongo_table.drop() From 351d14f8a3cb105146f55d2b200a2219bf203c28 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Jul 2022 14:30:02 +0000 Subject: [PATCH 498/627] Enable backward_compatibility for 01710_projection_fetch_long, disable s3 for 01643_replicated_merge_tree_fsync_smoke. --- .../0_stateless/01643_replicated_merge_tree_fsync_smoke.sql | 3 ++- tests/queries/0_stateless/01710_projection_fetch_long.sql | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql b/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql index dadd7eaba6c..ee0617e42a3 100644 --- a/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql +++ b/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-s3-storage -- no-parallel -- for flaky check and to avoid "Removing leftovers from table" (for other tables) +-- no-s3-storage -- hangs now, need investigation -- Temporarily skip warning 'table was created by another server at the same moment, will retry' set send_logs_level='error'; diff --git a/tests/queries/0_stateless/01710_projection_fetch_long.sql b/tests/queries/0_stateless/01710_projection_fetch_long.sql index 29effc53745..fd12b84c817 100644 --- a/tests/queries/0_stateless/01710_projection_fetch_long.sql +++ b/tests/queries/0_stateless/01710_projection_fetch_long.sql @@ -1,4 +1,4 @@ --- Tags: long, no-s3-storage, no-backward-compatibility-check +-- Tags: long, no-s3-storage drop table if exists tp_1; drop table if exists tp_2; From 869da0d923c217a37086a92d522da9a5c1b926f4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Jul 2022 17:19:49 +0200 Subject: [PATCH 499/627] Update StorageMongoDB.cpp --- src/Storages/StorageMongoDB.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index a33f17191eb..11e920fdc75 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -163,7 +163,7 @@ Pipe StorageMongoDB::read( return Pipe(std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size)); } - SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) + SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */) { connectIfNotConnected(); return std::make_shared(collection_name, database_name, metadata_snapshot, connection); From 5468a2fde9acb6c2d263b9947987d1a753f964e5 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Jul 2022 17:20:20 +0200 Subject: [PATCH 500/627] Update StorageMongoDB.cpp --- src/Storages/StorageMongoDB.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 11e920fdc75..16e4bea5de2 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -104,7 +104,7 @@ public: { } - String getName() const override { return "StorageMySQLSink"; } + String getName() const override { return "StorageMongoDBSink"; } void consume(Chunk chunk) override { From 41715237ae79ef904d20d1a3d3cd43d4702917ce Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Tue, 12 Jul 2022 15:23:36 +0000 Subject: [PATCH 501/627] Separate compress timescaledb test. Fix chmod to home - previous broke ssh. Non interactive install. --- benchmark/timescaledb-compressed/benchmark.sh | 48 ++++++++ benchmark/timescaledb-compressed/create.sql | 108 ++++++++++++++++++ benchmark/timescaledb-compressed/queries.sql | 43 +++++++ .../results/c6a.4xlarge.json} | 0 benchmark/timescaledb-compressed/run.sh | 13 +++ benchmark/timescaledb/benchmark.sh | 28 ++--- benchmark/timescaledb/run.sh | 2 +- 7 files changed, 224 insertions(+), 18 deletions(-) create mode 100755 benchmark/timescaledb-compressed/benchmark.sh create mode 100644 benchmark/timescaledb-compressed/create.sql create mode 100644 benchmark/timescaledb-compressed/queries.sql rename benchmark/{timescaledb/results/c6a.4xlarge.compression.json => timescaledb-compressed/results/c6a.4xlarge.json} (100%) create mode 100755 benchmark/timescaledb-compressed/run.sh diff --git a/benchmark/timescaledb-compressed/benchmark.sh b/benchmark/timescaledb-compressed/benchmark.sh new file mode 100755 index 00000000000..68600be3486 --- /dev/null +++ b/benchmark/timescaledb-compressed/benchmark.sh @@ -0,0 +1,48 @@ +#!/bin/bash + +# Install +export DEBIAN_FRONTEND=noninteractive +sudo apt-get update +sudo apt-get install -y gnupg postgresql-common apt-transport-https lsb-release wget +sudo /usr/share/postgresql-common/pgdg/apt.postgresql.org.sh +sudo bash -c 'echo "deb https://packagecloud.io/timescale/timescaledb/ubuntu/ $(lsb_release -c -s) main" > /etc/apt/sources.list.d/timescaledb.list' +wget --quiet -O - https://packagecloud.io/timescale/timescaledb/gpgkey | sudo apt-key add - +sudo apt-get update +sudo apt-get install -y timescaledb-2-postgresql-14 +sudo bash -c "echo \"shared_preload_libraries = 'timescaledb'\" >> /etc/postgresql/14/main/postgresql.conf" +sudo systemctl restart postgresql + +sudo -u postgres psql -c "CREATE DATABASE test" +sudo -u postgres psql test -c "CREATE EXTENSION IF NOT EXISTS timescaledb" + +# Import the data + +wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' +gzip -d hits.tsv.gz +sudo chmod og+rX ~ +chmod 777 hits.tsv + +sudo -u postgres psql test < create.sql +sudo -u postgres psql test -c "SELECT create_hypertable('hits', 'eventtime')" +sudo -u postgres psql test -c "CREATE INDEX ix_counterid ON hits (counterid)" +sudo -u postgres psql test -c "ALTER TABLE hits SET (timescaledb.compress, timescaledb.compress_orderby = 'counterid, eventdate, userid, eventtime')" +sudo -u postgres psql test -c "SELECT add_compression_policy('hits', INTERVAL '1s')" + +sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" + +# 1619875.288 ms (26:59.875) + +# See https://github.com/timescale/timescaledb/issues/4473#issuecomment-1167095245 +# https://docs.timescale.com/timescaledb/latest/how-to-guides/compression/manually-compress-chunks/#compress-chunks-manually +# TimescaleDB benchmark wihout compression is available in timescaledb directory + +time sudo -u postgres psql test -c "SELECT compress_chunk(i, if_not_compressed => true) FROM show_chunks('hits') i" + +# 49m45.120s + +./run.sh 2>&1 | tee log.txt + +sudo du -bcs /var/lib/postgresql/14/main/ + +cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | + awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/timescaledb-compressed/create.sql b/benchmark/timescaledb-compressed/create.sql new file mode 100644 index 00000000000..41c961c00fc --- /dev/null +++ b/benchmark/timescaledb-compressed/create.sql @@ -0,0 +1,108 @@ +CREATE TABLE hits +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL +); diff --git a/benchmark/timescaledb-compressed/queries.sql b/benchmark/timescaledb-compressed/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/benchmark/timescaledb-compressed/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +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; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +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; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/timescaledb/results/c6a.4xlarge.compression.json b/benchmark/timescaledb-compressed/results/c6a.4xlarge.json similarity index 100% rename from benchmark/timescaledb/results/c6a.4xlarge.compression.json rename to benchmark/timescaledb-compressed/results/c6a.4xlarge.json diff --git a/benchmark/timescaledb-compressed/run.sh b/benchmark/timescaledb-compressed/run.sh new file mode 100755 index 00000000000..198ab546163 --- /dev/null +++ b/benchmark/timescaledb-compressed/run.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +TRIES=3 + +cat queries.sql | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches + + echo "$query"; + for i in $(seq 1 $TRIES); do + psql test -t -c '\timing' -c "$query" | grep 'Time' + done; +done; diff --git a/benchmark/timescaledb/benchmark.sh b/benchmark/timescaledb/benchmark.sh index 2330deaf1b4..e43509ba915 100755 --- a/benchmark/timescaledb/benchmark.sh +++ b/benchmark/timescaledb/benchmark.sh @@ -2,6 +2,7 @@ # Install +export DEBIAN_FRONTEND=noninteractive sudo apt-get update sudo apt-get install -y gnupg postgresql-common apt-transport-https lsb-release wget sudo /usr/share/postgresql-common/pgdg/apt.postgresql.org.sh @@ -12,33 +13,26 @@ sudo apt-get install -y timescaledb-2-postgresql-14 sudo bash -c "echo \"shared_preload_libraries = 'timescaledb'\" >> /etc/postgresql/14/main/postgresql.conf" sudo systemctl restart postgresql -sudo -u postgres psql -c "CREATE DATABASE test" -sudo -u postgres psql test -c "CREATE EXTENSION IF NOT EXISTS timescaledb" +sudo -u postgres psql -c "CREATE DATABASE uncompressed" +sudo -u postgres psql uncompressed -c "CREATE EXTENSION IF NOT EXISTS timescaledb" # Import the data wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' gzip -d hits.tsv.gz -chmod 777 ~ hits.tsv +sudo chmod og+rX ~ +chmod 777 hits.tsv -sudo -u postgres psql test < create.sql -sudo -u postgres psql test -c "SELECT create_hypertable('hits', 'eventtime')" -sudo -u postgres psql test -c "CREATE INDEX ix_counterid ON hits (counterid)" -sudo -u postgres psql test -c "ALTER TABLE hits SET (timescaledb.compress, timescaledb.compress_orderby = 'counterid, eventdate, userid, eventtime')" -sudo -u postgres psql test -c "SELECT add_compression_policy('hits', INTERVAL '1s')" +sudo -u postgres psql uncompressed < create.sql +sudo -u postgres psql uncompressed -c "SELECT create_hypertable('hits', 'eventtime')" +sudo -u postgres psql uncompressed -c "CREATE INDEX ix_counterid ON hits (counterid)" +sudo -u postgres psql uncompressed -c "ALTER TABLE hits SET (timescaledb.compress, timescaledb.compress_orderby = 'counterid, eventdate, userid, eventtime')" +sudo -u postgres psql uncompressed -c "SELECT add_compression_policy('hits', INTERVAL '1s')" -sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" +sudo -u postgres psql uncompressed -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" # 1619875.288 ms (26:59.875) -# See https://github.com/timescale/timescaledb/issues/4473#issuecomment-1167095245 -# https://docs.timescale.com/timescaledb/latest/how-to-guides/compression/manually-compress-chunks/#compress-chunks-manually -# Omit this step to proceed without compression. - -time sudo -u postgres psql test -c "SELECT compress_chunk(i, if_not_compressed => true) FROM show_chunks('hits') i" - -# 49m45.120s - ./run.sh 2>&1 | tee log.txt sudo du -bcs /var/lib/postgresql/14/main/ diff --git a/benchmark/timescaledb/run.sh b/benchmark/timescaledb/run.sh index ad95151ab55..84edae1f3d3 100755 --- a/benchmark/timescaledb/run.sh +++ b/benchmark/timescaledb/run.sh @@ -8,6 +8,6 @@ cat queries.sql | while read query; do echo "$query"; for i in $(seq 1 $TRIES); do - sudo -u postgres psql test -t -c '\timing' -c "$query" | grep 'Time' + sudo -u postgres psql uncompressed -t -c '\timing' -c "$query" | grep 'Time' done; done; From 0c7388aea8b7e8b92e3a2046f891e58e525b68ba Mon Sep 17 00:00:00 2001 From: Timur Solodovnikov Date: Tue, 12 Jul 2022 09:14:05 -0700 Subject: [PATCH 502/627] Handling SIGTERM signals from k8s (#39130) --- docker/server/entrypoint.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 8dff08feea3..e0a730e5547 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -155,7 +155,7 @@ if [[ $# -lt 1 ]] || [[ "$1" == "--"* ]]; then # so the container can't be finished by ctrl+c CLICKHOUSE_WATCHDOG_ENABLE=${CLICKHOUSE_WATCHDOG_ENABLE:-0} export CLICKHOUSE_WATCHDOG_ENABLE - /usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" "$@" + exec /usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" "$@" fi # Otherwise, we assume the user want to run his own process, for example a `bash` shell to explore this image From 4d48d6e713e1b1eec042f4daab1ab97e8020db30 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Jul 2022 18:27:22 +0200 Subject: [PATCH 503/627] STUPID BUG --- .../DiskObjectStorageTransaction.cpp | 35 +++++++++++++++---- .../DiskObjectStorageTransaction.h | 2 ++ 2 files changed, 31 insertions(+), 6 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 1df5b8549a4..6e807747478 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -57,6 +57,8 @@ struct PureMetadataObjectStorageOperation final : public IDiskObjectStorageOpera void finalize() override { } + + std::string getInfoForLog() const override { return fmt::format("PureMetadataObjectStorageOperation"); } }; struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation @@ -79,6 +81,11 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation , if_exists(if_exists_) {} + std::string getInfoForLog() const override + { + return fmt::format("RemoveObjectStorageOperation (path: {}, if exists: {})", path, if_exists); + } + void execute(MetadataTransactionPtr tx) override { if (!metadata_storage.exists(path)) @@ -158,6 +165,11 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp , file_names_remove_metadata_only(file_names_remove_metadata_only_) {} + std::string getInfoForLog() const override + { + return fmt::format("RemoveRecursiveObjectStorageOperation (path: {})", path); + } + void removeMetadataRecursive(MetadataTransactionPtr tx, const std::string & path_to_remove) { checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. @@ -248,6 +260,11 @@ struct ReplaceFileObjectStorageOperation final : public IDiskObjectStorageOperat , path_to(path_to_) {} + std::string getInfoForLog() const override + { + return fmt::format("ReplaceFileObjectStorageOperation (path_from: {}, path_to: {})", path_from, path_to); + } + void execute(MetadataTransactionPtr tx) override { if (metadata_storage.exists(path_to)) @@ -284,6 +301,11 @@ struct WriteFileObjectStorageOperation final : public IDiskObjectStorageOperatio , object(object_) {} + std::string getInfoForLog() const override + { + return fmt::format("WriteFileObjectStorageOperation"); + } + void setOnExecute(std::function && on_execute_) { on_execute = on_execute_; @@ -325,6 +347,11 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation , to_path(to_path_) {} + std::string getInfoForLog() const override + { + return fmt::format("CopyFileObjectStorageOperation (path_from: {}, path_to: {})", from_path, to_path); + } + void execute(MetadataTransactionPtr tx) override { tx->createEmptyMetadataFile(to_path); @@ -368,7 +395,6 @@ void DiskObjectStorageTransaction::createDirectory(const std::string & path) void DiskObjectStorageTransaction::createDirectories(const std::string & path) { - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "CREATE DIRECTORIES TRANSACTION FOR PATH {}", path); operations_to_execute.emplace_back( std::make_unique(object_storage, metadata_storage, [path](MetadataTransactionPtr tx) { @@ -495,6 +521,7 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile String blob_name; std::optional object_attributes; + blob_name = object_storage.generateBlobNameForPath(path); if (metadata_helper) { auto revision = metadata_helper->revision_counter + 1; @@ -504,10 +531,6 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile }; blob_name = "r" + revisionToString(revision) + "-file-" + blob_name; } - else - { - blob_name = object_storage.generateBlobNameForPath(path); - } auto object = StoredObject::create(object_storage, fs::path(metadata_storage.getObjectStorageRootPath()) / blob_name); auto write_operation = std::make_unique(object_storage, metadata_storage, object); @@ -615,7 +638,7 @@ void DiskObjectStorageTransaction::commit() } catch (Exception & ex) { - ex.addMessage(fmt::format("While executing operation #{}", i)); + ex.addMessage(fmt::format("While executing operation #{} ({})", i, operations_to_execute[i]->getInfoForLog())); for (int64_t j = i; j >= 0; --j) { diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h index 1379375c584..ceed79a23b7 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h @@ -30,6 +30,8 @@ public: /// like removal of blobs. Such implementation can lead to garbage. virtual void finalize() = 0; virtual ~IDiskObjectStorageOperation() = default; + + virtual std::string getInfoForLog() const = 0; }; using DiskObjectStorageOperation = std::unique_ptr; From c38f72653796efc4fa21863c5b09467a46db1a31 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Jul 2022 18:28:20 +0200 Subject: [PATCH 504/627] Remove redundant code --- src/Disks/ObjectStorages/StoredObject.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Disks/ObjectStorages/StoredObject.cpp b/src/Disks/ObjectStorages/StoredObject.cpp index cb5bfb980a8..1341f468bf2 100644 --- a/src/Disks/ObjectStorages/StoredObject.cpp +++ b/src/Disks/ObjectStorages/StoredObject.cpp @@ -49,13 +49,6 @@ StoredObject StoredObject::create( } }; - /// if (metadata_storage->exists(object_path)) - /// { - /// object_size = metadata_storage->getFileSize(object_path); - /// path_key_for_cache_creator = - /// [path_key = path_key_for_cache_creator(object_path)](const String &) { return path_key; }; - /// } - return StoredObject(object_path, object_size, std::move(path_key_for_cache_creator)); } From 0fb35b773fa37864d91f06710efe7c41e0c23592 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Jul 2022 18:37:19 +0200 Subject: [PATCH 505/627] Update StorageMongoDB.cpp --- src/Storages/StorageMongoDB.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 16e4bea5de2..3ada9b214f5 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -163,7 +163,7 @@ Pipe StorageMongoDB::read( return Pipe(std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size)); } - SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */) +SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */) { connectIfNotConnected(); return std::make_shared(collection_name, database_name, metadata_snapshot, connection); From 4f407a7506df0da8b2ad4a921a7805d16a85cc09 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 19:49:49 +0200 Subject: [PATCH 506/627] Rename results for MariaDB to .shame because they do not look presentable --- .../mariadb/results/{c6a.4xlarge.json => c6a.4xlarge.json.shame} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename benchmark/mariadb/results/{c6a.4xlarge.json => c6a.4xlarge.json.shame} (100%) diff --git a/benchmark/mariadb/results/c6a.4xlarge.json b/benchmark/mariadb/results/c6a.4xlarge.json.shame similarity index 100% rename from benchmark/mariadb/results/c6a.4xlarge.json rename to benchmark/mariadb/results/c6a.4xlarge.json.shame From 358876b680013d8e0e0191f3ab674254b1098c12 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 19:50:31 +0200 Subject: [PATCH 507/627] Generate HTML --- benchmark/index.html | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/benchmark/index.html b/benchmark/index.html index eb63373bb61..ce4e6044353 100644 --- a/benchmark/index.html +++ b/benchmark/index.html @@ -348,7 +348,6 @@ const data = [ ,{"system":"HeavyAI","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Previous names: OmniSci, mapD. Many queries cannot run due to errors and limitations.","tags":["C++","column-oriented"],"load_time":572.633,"data_size":50887437386,"result":[[6.525,0.022,0.029],[0.301,0.042,0.04],[0.287,0.095,0.093],[2.572,0.039,0.04],[null,null,null],[null,null,null],[7.327,0.093,0.097],[0.244,0.043,0.038],[null,null,null],[null,null,null],[null,null,null],[null,null,null],[2.939,0.295,0.294],[null,null,null],[null,null,null],[null,null,null],[null,null,null],[4.716,3.91,3.955],[null,null,null],[0.154,0.083,0.106],[14.426,0.07,0.071],[null,null,null],[null,null,null],[null,null,null],[2.276,0.258,0.272],[null,null,null],[null,null,null],[null,null,null],[null,null,null],[1.832,1.64,1.602],[null,null,null],[null,null,null],[null,null,null],[14.811,0.494,0.497],[null,null,null],[null,null,null],[1.941,0.255,0.255],[5.457,0.172,0.283],[0.476,0.269,0.256],[14.239,0.179,0.178],[3.992,0.112,0.112],[1.031,0.116,0.116],[1.365,0.089,0.088]],"source":"heavyai/results/c6a.4xlarge.json"} ,{"system":"Infobright","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Only 90% of data successfully loaded. Some queries run for days.","tags":["C++","column-oriented","MySQL compatible"],"load_time":2317,"data_size":13760341294,"result":[[0.01,0,0],[2.39,2.4,2.44],[0,0,0],[7.21,6.04,6.91],[16.09,16.86,15.69],[48.8,42.37,48.63],[0,0,0],[3.48,2.42,2.42],[23.56,24.78,22.21],[32.87,31.71,34.48],[14.8,14.83,14.11],[16.7,16.53,17.37],[1752.91,1999.88,1961.4],[1193.43,1167,1220.47],[2184.81,2316.12,2224.14],[32.58,30.69,31.58],[300.17,16221.33,16168.44],[122.4,120.49,124.67],[78927.44,79250.44,78504.89],[3.38,1.22,1.21],[289.73,302.3,285.83],[415.82,389.23,403.28],[573.82,590.81,575.06],[300.13,293.96,285.64],[41.42,37.48,39.64],[75.2,75.37,72.07],[39.22,41.52,40.11],[449.56,445.03,448.68],[null,null,null],[450.87,488.3,453.83],[58.69,59.29,58.07],[84.47,78.92,79.38],[517.97,520.29,504.96],[182468.89,182468.89,182468.89],[182468.89,182468.89,182468.89],[68.43,66.93,67.68],[8.3,3.62,3.61],[1.04,0.62,0.62],[0.22,0.18,0.18],[567.78,566.52,563.02],[0.29,0.14,0.11],[0.17,0.08,0.08],[1.37,1.34,1.32]],"source":"infobright/results/c6a.4xlarge.json"} ,{"system":"MariaDB ColumnStore","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Previous name: InfiniDB.","tags":["C++","column-oriented","MySQL compatible"],"load_time":2507.8,"data_size":19712857022,"result":[[0.151,0.158,0.148],[0.1,0.101,0.106],[1.221,1.233,1.226],[0.739,0.736,0.741],[2.025,2.046,2.004],[3.725,4.801,3.755],[0.871,0.749,0.736],[0.118,0.108,0.103],[2.108,2.029,2.029],[4.225,4.271,4.288],[1.711,1.402,1.407],[1.526,1.435,1.42],[5.339,4.172,3.61],[4.692,4.729,4.96],[4.013,3.86,3.918],[3.236,2.68,2.629],[null,null,null],[null,null,null],[null,null,null],[0.137,0.109,0.147],[12.331,6.069,8.619],[2.162,2.178,2.192],[16.849,30.463,26.639],[92,90.208,92.814],[3.042,1.763,1.791],[1.779,1.772,1.749],[1.793,1.821,1.888],[13.036,10.747,9.59],[null,null,null],[78.224,79.141,77.806],[2.837,2.654,2.675],[5.833,4.552,3.678],[null,null,null],[null,null,null],[null,null,null],[3.626,3.546,3.709],[1.719,1.787,1.876],[1.345,0.906,0.91],[0.117,0.091,0.093],[1.217,1.133,1.133],[0.114,0.063,0.062],[0.1,0.062,0.061],[null,null,null]],"source":"mariadb-columnstore/results/c6a.4xlarge.json"} -,{"system":"MariaDB","date":"2022-07-12","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"For some reason many queries run in almost exactly one hour. Most likely a coincidence. It is installed from Debian packages, which might explain the abysmal performance.","tags":["C++","row-oriented","MySQL compatible"],"load_time":8625,"data_size":107252505530,"result":[[7910.23,10229.8,3610.54],[3595.79,3595.59,3595.21],[3596.12,3595.34,3595.49],[3597.22,3595.7,3595.09],[3600.57,3596.08,3595.82],[3596.57,3597.76,3598.47],[3595.94,3595.36,3596.2],[3595.95,3595.38,3595.45],[3649.15,3624.26,3623.71],[3645.39,3763.75,3630.71],[4519.77,4552.84,4552.7],[4628.21,4629.55,4628.16],[3773.52,3774.44,3779.05],[12198,12192.4,12192.6],[3779.67,3774.97,3777.86],[4277.99,4302.56,4275.29],[4697.19,4367.3,4372.73],[4401.2,4340.43,4310.13],[5335.93,5373.04,5361.51],[3596.91,3598.61,3598.71],[3598.76,3598.56,3598.68],[3598.76,3598.63,3599.39],[3604.28,3603.71,3603.66],[3598.83,3598.2,3598.19],[3599.11,3598.1,3598.17],[3599.04,3598.46,3598.13],[3598.76,3598.71,3598.22],[3599.02,3599.76,3598.18],[3644.12,3716.36,3725.92],[3933.26,3854.55,3784.74],[3392.13,3402.26,3602.15],[3717.3,3735.06,3723.07],[6126.66,6135.63,6193.45],[4029.78,3998.89,3952.7],[3955.34,4029.28,3941.84],[4881.07,4782.53,4900.72],[24.455,23.862,25.173],[24.195,23.529,23.473],[20.719,21.629,21.582],[24.577,25.877,25.953],[20.686,20.787,21.199],[20.036,17.973,22.759],[null,null,null]],"source":"mariadb/results/c6a.4xlarge.json"} ,{"system":"MonetDB","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","column-oriented"],"load_time":939,"data_size":49696606499,"result":[[0.000218,0.000157,0.000155],[0.101903,0.019908,0.018439],[0.282431,0.035987,0.034938],[2.868,0.029387,0.029207],[4.675,4.515,4.511],[6.584,4.269,4.65],[0.528827,0.063135,0.065742],[0.506863,0.020966,0.021687],[8.343,4.457,4.408],[7.224,6.548,7.576],[0.267003,0.233353,0.230444],[0.347206,0.28358,0.266085],[5.389,3.099,3.074],[7.653,7.759,8.596],[3.276,3.326,3.292],[5.31,3.465,3.578],[9.341,9.143,9.536],[9.584,9.604,9.419],[19.539,19.783,19.611],[0.004509,0.000702,0.000643],[20.801,1.57,1.603],[2.752,0.418221,0.395884],[14.717,0.800894,0.395477],[14.429,1.804,1.869],[1.386,0.159602,0.156426],[0.189736,0.167664,0.168781],[0.164681,0.176666,0.17126],[3.005,3.113,3.882],[null,null,null],[2.751,2.846,2.676],[7.937,2.579,2.447],[5.12,3.492,3.467],[22.862,22.567,23.211],[33.437,18.889,19.043],[18.898,19.583,19.047],[14.774,12.984,13.803],[3.865,0.322143,0.323117],[0.192149,0.177791,0.175984],[0.194173,0.159398,0.165201],[0.680778,0.592252,0.560738],[0.106465,0.10638,0.102692],[0.154871,0.153752,0.155782],[0.11459,0.09639,0.095594]],"source":"monetdb/results/c6a.4xlarge.json"} ,{"system":"MySQL (MyISAM)","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","row-oriented","MySQL compatible"],"load_time":2512,"data_size":121588958061,"result":[[0,0,0],[283.32,276.83,274.52],[276.93,278.29,283.27],[28.83,23.63,21.55],[46.41,40.81,40.93],[467.04,467.39,469.08],[31.02,25.89,24.2],[277.89,275.3,277.3],[329.34,325.8,325.35],[342.86,338.43,336.95],[282.03,279.87,281.22],[277.74,282.68,282],[335.66,334.83,336.44],[305.24,310.39,307.3],[337.41,338.52,342.94],[308.66,307.34,306.27],[738.38,748.44,740.75],[738.75,734.01,738.25],[867.01,872.92,868.84],[25.65,20.61,18.46],[312.39,313.67,306.66],[301.66,305.12,308.01],[298.12,298.44,312.4],[311.34,309.9,311.85],[281.87,278.5,275],[277.46,277.46,277.46],[280.75,278.04,281.76],[263.9,417.39,406.88],[707.21,711.96,705],[668.1,668.33,665.96],[330.31,333.36,331.94],[506.57,506.18,500.53],[2604.49,2681.96,2703.12],[830.65,832.88,831.14],[831.98,830.46,833.41],[608.49,608.51,613.68],[4.56,4.13,4.16],[3.8,3.8,3.7],[1.65,1.45,1.46],[6.33,5.14,6.15],[1.6,1.41,1.41],[1.56,1.42,1.39],[7.04,1.17,1.13]],"source":"mysql-myisam/results/c6a.4xlarge.json"} ,{"system":"MySQL","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","row-oriented","MySQL compatible"],"load_time":9472,"data_size":171953585825,"result":[[339.77,339.88,339.77],[364.91,371.86,367.55],[366.2,368.91,389.66],[364.39,377.53,571.45],[377.69,390.02,384.86],[569.48,576.51,574.68],[367.4,368.23,370.41],[371.29,384.02,613.22],[478.85,683.22,495.68],[489.9,635.96,662.43],[386.07,396.49,640.15],[389.13,412.55,444.12],[447.97,455.54,448.06],[423.22,845.44,813.6],[452.48,460.07,453.98],[577.54,623.21,586.49],[852.07,856.36,862.66],[838.09,848.92,851.12],[1006.37,1011.16,1023.17],[369.76,375.61,415.28],[412.45,419.9,456.62],[411.65,432.88,482.2],[412.73,420.73,429.5],[551.16,577.62,545.45],[382.89,394.76,386.37],[380.9,391.4,385.05],[385.3,394.67,460.32],[388.95,394.7,387.21],[800.33,807.9,807.11],[706.03,745.27,718.9],[450.9,489.59,530.97],[625.5,651.93,647.32],[2721.13,2792.12,2819.26],[945.9,954.94,957.54],[945.42,953.78,965.16],[684.36,716.29,708.75],[10.01,3.79,3.77],[7.48,3.32,3.27],[5.09,0.98,0.96],[8.7,4.77,4.68],[4.82,0.76,0.74],[4.46,0.77,0.75],[7.04,1.17,1.13]],"source":"mysql/results/c6a.4xlarge.json"} @@ -367,7 +366,7 @@ const data = [ ,{"system":"Snowflake","date":"2022-07-01","machine":"XL","cluster_size":16,"comment":"","tags":["managed","column-oriented"],"load_time":2524,"data_size":12300000000,"result":[[0.071,0.053,0.057],[0.998,0.61,0.24],[0.42,1.138,1.051],[0.653,0.264,0.178],[0.352,0.312,0.349],[1.126,0.431,0.42],[0.067,0.057,0.054],[0.225,0.217,0.2],[0.617,0.366,0.371],[1.006,0.541,0.498],[0.463,0.425,0.293],[0.431,0.36,0.339],[0.392,0.371,0.386],[0.588,0.581,0.59],[0.634,0.414,0.4],[0.368,0.41,0.388],[0.594,0.639,0.663],[0.616,0.581,0.569],[1.092,0.933,0.901],[0.493,0.213,0.16],[0.886,0.48,0.442],[0.448,0.337,0.399],[0.84,0.572,0.505],[2.251,1.23,0.959],[0.295,0.253,0.241],[0.214,0.239,0.278],[0.261,0.232,0.314],[0.422,0.429,0.403],[0.892,0.934,0.883],[1.041,1.017,1.009],[0.715,0.442,0.363],[0.845,0.413,0.461],[1.101,1.085,1.102],[1.294,1.272,1.339],[1.839,1.327,1.241],[0.439,0.399,0.393],[0.199,0.211,0.19],[0.157,0.143,0.14],[0.145,0.157,0.141],[0.331,0.291,0.333],[0.173,0.214,0.138],[0.189,0.15,0.159],[0.135,0.149,0.138]],"source":"snowflake/results/xl.json"} ,{"system":"Snowflake","date":"2022-07-01","machine":"XS","cluster_size":1,"comment":"","tags":["managed","column-oriented"],"load_time":2524,"data_size":12300000000,"result":[[0.169,0.055,0.056],[1.184,0.582,0.386],[1.35,0.56,0.568],[1.27,0.554,0.538],[2.516,2.564,2.506],[2.935,2.649,2.67],[0.052,0.05,0.064],[0.383,0.387,0.397],[3.249,2.993,3.014],[3.589,3.627,3.887],[1.243,0.986,0.966],[1.325,1.08,1.073],[2.038,2.046,2.035],[3.738,3.626,3.718],[2.318,2.159,2.176],[2.733,2.637,2.668],[5.607,5.683,5.667],[3.978,3.923,3.879],[10.085,9.871,9.844],[0.45,0.375,0.469],[5.474,3.103,3.06],[2.012,1.982,1.971],[3.365,2.471,2.501],[11.96,10.619,9.518],[1.074,1.059,1.026],[0.856,0.846,0.879],[1.1,1.085,1.083],[3.057,3.228,3.117],[9.406,9.019,9.158],[6.196,6.243,6.911],[2.906,2.343,2.017],[2.954,2.666,2.565],[9.459,9.565,9.557],[9.555,9.529,9.368],[9.409,9.185,9.294],[2.796,2.88,2.685],[0.299,0.249,0.262],[0.156,0.145,0.18],[0.147,0.146,0.16],[0.371,0.357,0.356],[0.166,0.133,0.155],[0.218,0.14,0.135],[0.14,0.152,0.158]],"source":"snowflake/results/xs.json"} ,{"system":"SQLite","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","embedded","row-oriented"],"load_time":2608,"data_size":75776589824,"result":[[752.739,2.003,1.2],[304.302,291.521,286.965],[293.964,287.619,287.219],[758.302,5.879,5.65],[836.393,48.593,48.452],[362.605,344.884,356.245],[763.993,11.602,10.795],[296.348,286.879,287.557],[365.816,360.339,354.126],[374.403,365.196,362.261],[302.989,293.888,298.432],[303.64,291.729,295.347],[316.824,298.18,301.006],[320.665,301.15,305.227],[313.593,301.021,301.626],[794.881,47,47.225],[355.346,344.615,342.442],[316.499,305.971,305.007],[398.177,380.383,385.571],[751.82,5.082,4.913],[295.522,286.573,287.368],[298.58,287.182,288.303],[296.474,288.747,288.638],[296.579,287.127,287.361],[304.709,286.865,287.56],[300.391,290.633,288.587],[294.605,286.91,287.799],[305.986,312.111,305.626],[null,null,null],[411.225,397.614,394.253],[307.711,295.181,300.266],[312.472,299.079,298.19],[386.674,378.347,376.963],[409.742,409.554,420.273],[468.73,453.709,458.149],[366.015,347.446,346.728],[2.911,0.781,0.757],[1.599,0.609,0.587],[1.288,0.256,0.238],[2.469,1.582,1.52],[1.274,0.303,0.283],[1.322,0.317,0.314],[1.498,0.602,0.613]],"source":"sqlite/results/c6a.4xlarge.json"} -,{"system":"TimescaleDB (compression)","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","PostgreSQL compatible","column-oriented","time-series"],"load_time":4605,"data_size":20333747165,"result":[[2.28686,1.63642,1.64263],[32.6848,1.63476,1.40052],[60.8633,3.70484,3.59342],[36.4029,2.87091,2.80739],[110.391,38.9688,38.0549],[147.379,66.2513,65.6379],[33.0294,2.92031,2.84375],[33.0221,1.2984,1.19227],[115.694,47.4651,47.0125],[156.347,51.577,51.2694],[68.3301,4.75521,4.68007],[77.4356,5.55128,5.56577],[49.7741,11.2911,11.3265],[81.1014,14.9111,14.9541],[82.9569,14.6156,14.6331],[62.0338,26.399,26.3351],[103.259,36.4122,36.6076],[92.8828,26.2395,25.8991],[144.281,63.5102,63.7661],[7.00679,0.573073,0.536283],[75.0203,7.86344,7.90495],[81.2825,9.15868,9.01775],[104.084,13.9528,13.8435],[132.531,81.522,82.1561],[80.6965,3.28231,3.16574],[39.7693,2.51443,2.43849],[80.4245,3.26941,3.13916],[104.015,13.7044,13.5313],[307.26,253.127,252.147],[42.8549,22.4187,22.0325],[137.601,14.9592,14.6804],[136.767,22.8007,22.131],[263.005,168.551,163.355],[156.919,92.6308,91.702],[160.842,96.0512,97.1773],[62.8357,28.0336,28.7397],[1.75869,0.561604,0.541215],[0.46607,0.191863,0.19021],[0.303671,0.137579,0.136615],[2.32031,1.49223,1.52369],[0.563764,0.14192,0.138234],[0.372428,0.122989,0.123709],[0.448574,0.159092,0.154687]],"source":"timescaledb/results/c6a.4xlarge.compression.json"} +,{"system":"TimescaleDB (compression)","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","PostgreSQL compatible","column-oriented","time-series"],"load_time":4605,"data_size":20333747165,"result":[[2.28686,1.63642,1.64263],[32.6848,1.63476,1.40052],[60.8633,3.70484,3.59342],[36.4029,2.87091,2.80739],[110.391,38.9688,38.0549],[147.379,66.2513,65.6379],[33.0294,2.92031,2.84375],[33.0221,1.2984,1.19227],[115.694,47.4651,47.0125],[156.347,51.577,51.2694],[68.3301,4.75521,4.68007],[77.4356,5.55128,5.56577],[49.7741,11.2911,11.3265],[81.1014,14.9111,14.9541],[82.9569,14.6156,14.6331],[62.0338,26.399,26.3351],[103.259,36.4122,36.6076],[92.8828,26.2395,25.8991],[144.281,63.5102,63.7661],[7.00679,0.573073,0.536283],[75.0203,7.86344,7.90495],[81.2825,9.15868,9.01775],[104.084,13.9528,13.8435],[132.531,81.522,82.1561],[80.6965,3.28231,3.16574],[39.7693,2.51443,2.43849],[80.4245,3.26941,3.13916],[104.015,13.7044,13.5313],[307.26,253.127,252.147],[42.8549,22.4187,22.0325],[137.601,14.9592,14.6804],[136.767,22.8007,22.131],[263.005,168.551,163.355],[156.919,92.6308,91.702],[160.842,96.0512,97.1773],[62.8357,28.0336,28.7397],[1.75869,0.561604,0.541215],[0.46607,0.191863,0.19021],[0.303671,0.137579,0.136615],[2.32031,1.49223,1.52369],[0.563764,0.14192,0.138234],[0.372428,0.122989,0.123709],[0.448574,0.159092,0.154687]],"source":"timescaledb-compressed/results/c6a.4xlarge.json"} ,{"system":"TimescaleDB","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","PostgreSQL compatible","row-oriented","time-series"],"load_time":1620,"data_size":72882392030,"result":[[437.7,215.793,176.42],[327.026,259.568,244.578],[262.978,263.09,263.083],[262.807,263.046,266.847],[337.497,334.964,330.852],[355.689,356.801,362.894],[262.762,263.012,262.968],[263.055,263.016,263.028],[319.928,319.388,320.704],[323.584,322.224,322.488],[265.979,265.465,265.375],[266.019,265.543,265.462],[277.018,276.3,276.595],[280.352,279.251,279.572],[279.915,279.896,279.674],[296.377,298.506,297.659],[314.448,314.605,312.57],[302.668,302.672,303.039],[325.81,324.061,324.376],[262.447,262.698,262.704],[267.581,267.467,267.482],[268.085,267.466,267.696],[263.391,263.097,263.126],[38.291,0.435,0.335],[0.127,0.005,0.005],[263.138,263.1,263.092],[0.889,0.341,0.339],[267.586,267.498,267.491],[289.086,290.012,290.093],[263.22,263.071,263.109],[274.78,273.995,273.998],[282.217,281.39,281.47],[429.273,426.588,439.431],[448.808,418.724,418.207],[455.196,422.75,423.142],[299.263,296.937,297.261],[18.693,1.552,1.481],[18.125,0.945,0.937],[18.528,1.062,0.902],[18.268,1.779,1.77],[19.615,1.965,1.966],[18.97,1.435,1.43],[18.33,1.153,0.952]],"source":"timescaledb/results/c6a.4xlarge.json"} ,{"system":"Vertica","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["column-oriented"],"load_time":12239,"data_size":22597367016,"result":[[0.031357,0.020759,0.019373],[0.156612,0.033543,0.032149],[0.404881,0.124383,0.124252],[0.220711,0.074036,0.070379],[2.19663,2.18441,2.1763],[11.5901,11.4847,11.4573],[0.160286,0.063093,0.064844],[0.116135,0.047536,0.046322],[2.36592,2.15174,2.15675],[4.42049,4.37322,4.38596],[1.24771,1.17476,1.21011],[1.30116,1.24332,1.24521],[14.0523,13.9236,13.9943],[39.8961,39.7911,39.771],[14.6776,14.5524,14.5899],[2.82581,2.71968,2.72123],[48.4932,48.4037,48.4298],[36.8767,36.7432,36.9178],[108.743,108.81,108.671],[0.02734,0.018601,0.018892],[8.98822,4.2958,4.38323],[10.0097,4.31009,4.26843],[14.2354,5.48257,5.47171],[3.29109,0.405847,0.337797],[0.088318,0.079666,0.091852],[1.05864,1.26979,1.81786],[0.087436,0.092269,0.087506],[15.0424,15.0873,15.2561],[22.1566,22.1208,21.9509],[2.59797,2.46053,2.46406],[3.63938,3.68086,3.68988],[6.58816,4.84546,4.8853],[20.7209,20.4806,20.5823],[77.3779,78.1301,77.4366],[77.2494,77.646,77.4976],[6.79571,6.81133,6.67973],[0.887148,0.782822,0.753053],[0.369373,0.339676,0.338677],[0.099038,0.089598,0.099019],[2.15693,2.01152,2.00868],[0.096018,0.052062,0.053813],[0.058647,0.046352,0.045415],[0.106915,0.077122,0.076134]],"source":"vertica/results/c6a.4xlarge.json"} ]; // end of data From 90064d5f869db91ccc0ff132c395324c52344f51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 19:51:47 +0200 Subject: [PATCH 508/627] Add LICENSE --- benchmark/LICENSE | 360 ++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 360 insertions(+) create mode 100644 benchmark/LICENSE diff --git a/benchmark/LICENSE b/benchmark/LICENSE new file mode 100644 index 00000000000..0f55d62d28d --- /dev/null +++ b/benchmark/LICENSE @@ -0,0 +1,360 @@ +Creative Commons Attribution-NonCommercial-ShareAlike 4.0 International +Public License + +By exercising the Licensed Rights (defined below), You accept and agree +to be bound by the terms and conditions of this Creative Commons +Attribution-NonCommercial-ShareAlike 4.0 International Public License +("Public License"). To the extent this Public License may be +interpreted as a contract, You are granted the Licensed Rights in +consideration of Your acceptance of these terms and conditions, and the +Licensor grants You such rights in consideration of benefits the +Licensor receives from making the Licensed Material available under +these terms and conditions. + + +Section 1 -- Definitions. + + a. Adapted Material means material subject to Copyright and Similar + Rights that is derived from or based upon the Licensed Material + and in which the Licensed Material is translated, altered, + arranged, transformed, or otherwise modified in a manner requiring + permission under the Copyright and Similar Rights held by the + Licensor. For purposes of this Public License, where the Licensed + Material is a musical work, performance, or sound recording, + Adapted Material is always produced where the Licensed Material is + synched in timed relation with a moving image. + + b. Adapter's License means the license You apply to Your Copyright + and Similar Rights in Your contributions to Adapted Material in + accordance with the terms and conditions of this Public License. + + c. BY-NC-SA Compatible License means a license listed at + creativecommons.org/compatiblelicenses, approved by Creative + Commons as essentially the equivalent of this Public License. + + d. Copyright and Similar Rights means copyright and/or similar rights + closely related to copyright including, without limitation, + performance, broadcast, sound recording, and Sui Generis Database + Rights, without regard to how the rights are labeled or + categorized. For purposes of this Public License, the rights + specified in Section 2(b)(1)-(2) are not Copyright and Similar + Rights. + + e. Effective Technological Measures means those measures that, in the + absence of proper authority, may not be circumvented under laws + fulfilling obligations under Article 11 of the WIPO Copyright + Treaty adopted on December 20, 1996, and/or similar international + agreements. + + f. Exceptions and Limitations means fair use, fair dealing, and/or + any other exception or limitation to Copyright and Similar Rights + that applies to Your use of the Licensed Material. + + g. License Elements means the license attributes listed in the name + of a Creative Commons Public License. The License Elements of this + Public License are Attribution, NonCommercial, and ShareAlike. + + h. Licensed Material means the artistic or literary work, database, + or other material to which the Licensor applied this Public + License. + + i. Licensed Rights means the rights granted to You subject to the + terms and conditions of this Public License, which are limited to + all Copyright and Similar Rights that apply to Your use of the + Licensed Material and that the Licensor has authority to license. + + j. Licensor means the individual(s) or entity(ies) granting rights + under this Public License. + + k. NonCommercial means not primarily intended for or directed towards + commercial advantage or monetary compensation. For purposes of + this Public License, the exchange of the Licensed Material for + other material subject to Copyright and Similar Rights by digital + file-sharing or similar means is NonCommercial provided there is + no payment of monetary compensation in connection with the + exchange. + + l. Share means to provide material to the public by any means or + process that requires permission under the Licensed Rights, such + as reproduction, public display, public performance, distribution, + dissemination, communication, or importation, and to make material + available to the public including in ways that members of the + public may access the material from a place and at a time + individually chosen by them. + + m. Sui Generis Database Rights means rights other than copyright + resulting from Directive 96/9/EC of the European Parliament and of + the Council of 11 March 1996 on the legal protection of databases, + as amended and/or succeeded, as well as other essentially + equivalent rights anywhere in the world. + + n. You means the individual or entity exercising the Licensed Rights + under this Public License. Your has a corresponding meaning. + + +Section 2 -- Scope. + + a. License grant. + + 1. Subject to the terms and conditions of this Public License, + the Licensor hereby grants You a worldwide, royalty-free, + non-sublicensable, non-exclusive, irrevocable license to + exercise the Licensed Rights in the Licensed Material to: + + a. reproduce and Share the Licensed Material, in whole or + in part, for NonCommercial purposes only; and + + b. produce, reproduce, and Share Adapted Material for + NonCommercial purposes only. + + 2. Exceptions and Limitations. For the avoidance of doubt, where + Exceptions and Limitations apply to Your use, this Public + License does not apply, and You do not need to comply with + its terms and conditions. + + 3. Term. The term of this Public License is specified in Section + 6(a). + + 4. Media and formats; technical modifications allowed. The + Licensor authorizes You to exercise the Licensed Rights in + all media and formats whether now known or hereafter created, + and to make technical modifications necessary to do so. The + Licensor waives and/or agrees not to assert any right or + authority to forbid You from making technical modifications + necessary to exercise the Licensed Rights, including + technical modifications necessary to circumvent Effective + Technological Measures. For purposes of this Public License, + simply making modifications authorized by this Section 2(a) + (4) never produces Adapted Material. + + 5. Downstream recipients. + + a. Offer from the Licensor -- Licensed Material. Every + recipient of the Licensed Material automatically + receives an offer from the Licensor to exercise the + Licensed Rights under the terms and conditions of this + Public License. + + b. Additional offer from the Licensor -- Adapted Material. + Every recipient of Adapted Material from You + automatically receives an offer from the Licensor to + exercise the Licensed Rights in the Adapted Material + under the conditions of the Adapter's License You apply. + + c. No downstream restrictions. You may not offer or impose + any additional or different terms or conditions on, or + apply any Effective Technological Measures to, the + Licensed Material if doing so restricts exercise of the + Licensed Rights by any recipient of the Licensed + Material. + + 6. No endorsement. Nothing in this Public License constitutes or + may be construed as permission to assert or imply that You + are, or that Your use of the Licensed Material is, connected + with, or sponsored, endorsed, or granted official status by, + the Licensor or others designated to receive attribution as + provided in Section 3(a)(1)(A)(i). + + b. Other rights. + + 1. Moral rights, such as the right of integrity, are not + licensed under this Public License, nor are publicity, + privacy, and/or other similar personality rights; however, to + the extent possible, the Licensor waives and/or agrees not to + assert any such rights held by the Licensor to the limited + extent necessary to allow You to exercise the Licensed + Rights, but not otherwise. + + 2. Patent and trademark rights are not licensed under this + Public License. + + 3. To the extent possible, the Licensor waives any right to + collect royalties from You for the exercise of the Licensed + Rights, whether directly or through a collecting society + under any voluntary or waivable statutory or compulsory + licensing scheme. In all other cases the Licensor expressly + reserves any right to collect such royalties, including when + the Licensed Material is used other than for NonCommercial + purposes. + + +Section 3 -- License Conditions. + +Your exercise of the Licensed Rights is expressly made subject to the +following conditions. + + a. Attribution. + + 1. If You Share the Licensed Material (including in modified + form), You must: + + a. retain the following if it is supplied by the Licensor + with the Licensed Material: + + i. identification of the creator(s) of the Licensed + Material and any others designated to receive + attribution, in any reasonable manner requested by + the Licensor (including by pseudonym if + designated); + + ii. a copyright notice; + + iii. a notice that refers to this Public License; + + iv. a notice that refers to the disclaimer of + warranties; + + v. a URI or hyperlink to the Licensed Material to the + extent reasonably practicable; + + b. indicate if You modified the Licensed Material and + retain an indication of any previous modifications; and + + c. indicate the Licensed Material is licensed under this + Public License, and include the text of, or the URI or + hyperlink to, this Public License. + + 2. You may satisfy the conditions in Section 3(a)(1) in any + reasonable manner based on the medium, means, and context in + which You Share the Licensed Material. For example, it may be + reasonable to satisfy the conditions by providing a URI or + hyperlink to a resource that includes the required + information. + 3. If requested by the Licensor, You must remove any of the + information required by Section 3(a)(1)(A) to the extent + reasonably practicable. + + b. ShareAlike. + + In addition to the conditions in Section 3(a), if You Share + Adapted Material You produce, the following conditions also apply. + + 1. The Adapter's License You apply must be a Creative Commons + license with the same License Elements, this version or + later, or a BY-NC-SA Compatible License. + + 2. You must include the text of, or the URI or hyperlink to, the + Adapter's License You apply. You may satisfy this condition + in any reasonable manner based on the medium, means, and + context in which You Share Adapted Material. + + 3. You may not offer or impose any additional or different terms + or conditions on, or apply any Effective Technological + Measures to, Adapted Material that restrict exercise of the + rights granted under the Adapter's License You apply. + + +Section 4 -- Sui Generis Database Rights. + +Where the Licensed Rights include Sui Generis Database Rights that +apply to Your use of the Licensed Material: + + a. for the avoidance of doubt, Section 2(a)(1) grants You the right + to extract, reuse, reproduce, and Share all or a substantial + portion of the contents of the database for NonCommercial purposes + only; + + b. if You include all or a substantial portion of the database + contents in a database in which You have Sui Generis Database + Rights, then the database in which You have Sui Generis Database + Rights (but not its individual contents) is Adapted Material, + including for purposes of Section 3(b); and + + c. You must comply with the conditions in Section 3(a) if You Share + all or a substantial portion of the contents of the database. + +For the avoidance of doubt, this Section 4 supplements and does not +replace Your obligations under this Public License where the Licensed +Rights include other Copyright and Similar Rights. + + +Section 5 -- Disclaimer of Warranties and Limitation of Liability. + + a. UNLESS OTHERWISE SEPARATELY UNDERTAKEN BY THE LICENSOR, TO THE + EXTENT POSSIBLE, THE LICENSOR OFFERS THE LICENSED MATERIAL AS-IS + AND AS-AVAILABLE, AND MAKES NO REPRESENTATIONS OR WARRANTIES OF + ANY KIND CONCERNING THE LICENSED MATERIAL, WHETHER EXPRESS, + IMPLIED, STATUTORY, OR OTHER. THIS INCLUDES, WITHOUT LIMITATION, + WARRANTIES OF TITLE, MERCHANTABILITY, FITNESS FOR A PARTICULAR + PURPOSE, NON-INFRINGEMENT, ABSENCE OF LATENT OR OTHER DEFECTS, + ACCURACY, OR THE PRESENCE OR ABSENCE OF ERRORS, WHETHER OR NOT + KNOWN OR DISCOVERABLE. WHERE DISCLAIMERS OF WARRANTIES ARE NOT + ALLOWED IN FULL OR IN PART, THIS DISCLAIMER MAY NOT APPLY TO YOU. + + b. TO THE EXTENT POSSIBLE, IN NO EVENT WILL THE LICENSOR BE LIABLE + TO YOU ON ANY LEGAL THEORY (INCLUDING, WITHOUT LIMITATION, + NEGLIGENCE) OR OTHERWISE FOR ANY DIRECT, SPECIAL, INDIRECT, + INCIDENTAL, CONSEQUENTIAL, PUNITIVE, EXEMPLARY, OR OTHER LOSSES, + COSTS, EXPENSES, OR DAMAGES ARISING OUT OF THIS PUBLIC LICENSE OR + USE OF THE LICENSED MATERIAL, EVEN IF THE LICENSOR HAS BEEN + ADVISED OF THE POSSIBILITY OF SUCH LOSSES, COSTS, EXPENSES, OR + DAMAGES. WHERE A LIMITATION OF LIABILITY IS NOT ALLOWED IN FULL OR + IN PART, THIS LIMITATION MAY NOT APPLY TO YOU. + + c. The disclaimer of warranties and limitation of liability provided + above shall be interpreted in a manner that, to the extent + possible, most closely approximates an absolute disclaimer and + waiver of all liability. + + +Section 6 -- Term and Termination. + + a. This Public License applies for the term of the Copyright and + Similar Rights licensed here. However, if You fail to comply with + this Public License, then Your rights under this Public License + terminate automatically. + + b. Where Your right to use the Licensed Material has terminated under + Section 6(a), it reinstates: + + 1. automatically as of the date the violation is cured, provided + it is cured within 30 days of Your discovery of the + violation; or + + 2. upon express reinstatement by the Licensor. + + For the avoidance of doubt, this Section 6(b) does not affect any + right the Licensor may have to seek remedies for Your violations + of this Public License. + + c. For the avoidance of doubt, the Licensor may also offer the + Licensed Material under separate terms or conditions or stop + distributing the Licensed Material at any time; however, doing so + will not terminate this Public License. + + d. Sections 1, 5, 6, 7, and 8 survive termination of this Public + License. + + +Section 7 -- Other Terms and Conditions. + + a. The Licensor shall not be bound by any additional or different + terms or conditions communicated by You unless expressly agreed. + + b. Any arrangements, understandings, or agreements regarding the + Licensed Material not stated herein are separate from and + independent of the terms and conditions of this Public License. + + +Section 8 -- Interpretation. + + a. For the avoidance of doubt, this Public License does not, and + shall not be interpreted to, reduce, limit, restrict, or impose + conditions on any use of the Licensed Material that could lawfully + be made without permission under this Public License. + + b. To the extent possible, if any provision of this Public License is + deemed unenforceable, it shall be automatically reformed to the + minimum extent necessary to make it enforceable. If the provision + cannot be reformed, it shall be severed from this Public License + without affecting the enforceability of the remaining terms and + conditions. + + c. No term or condition of this Public License will be waived and no + failure to comply consented to unless expressly agreed to by the + Licensor. + + d. Nothing in this Public License constitutes or may be interpreted + as a limitation upon, or waiver of, any privileges and immunities + that apply to the Licensor or You, including from the legal + processes of any jurisdiction or authority. From 93bf22f58534b94d52130346d96c1e79e428bc2a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Jul 2022 17:59:31 +0000 Subject: [PATCH 509/627] Fix data race in CompletedPipelineExecutor. --- src/Processors/Executors/CompletedPipelineExecutor.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index 8876b3a0fb4..9e5ea3916bc 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -72,9 +72,11 @@ void CompletedPipelineExecutor::execute() data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element); data->executor->setReadProgressCallback(pipeline.getReadProgressCallback()); - auto func = [&, thread_group = CurrentThread::getGroup()]() + /// Avoid passing this to labmda, copy ptr to data instead. + /// Destructor of unique_ptr copy raw ptr into local variable first, only then calls object destructor. + auto func = [data_ptr = data.get(), num_threads = pipeline.getNumThreads(), thread_group = CurrentThread::getGroup()]() { - threadFunction(*data, thread_group, pipeline.getNumThreads()); + threadFunction(*data_ptr, thread_group, num_threads); }; data->thread = ThreadFromGlobalPool(std::move(func)); From df18d6a416753cd5155501976fcd0255d2a48429 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Jul 2022 18:12:59 +0000 Subject: [PATCH 510/627] Fix style --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1f46bac6491..219bc830a07 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2943,7 +2943,7 @@ class ClickHouseInstance: ignore_error=False, query_id=None, ): - sql_for_log = '' + sql_for_log = "" if len(sql) > 1000: sql_for_log = sql[:1000] else: From 18c81ff68b51dac615acc90026ac4e446e35e7e8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 20:20:58 +0200 Subject: [PATCH 511/627] Move benchmark into separate repository --- benchmark/.gitignore | 1 - benchmark/LICENSE | 360 ------- benchmark/README.md | 412 +------- benchmark/athena/README.md | 21 - benchmark/athena/create_partitioned.sql | 112 -- benchmark/athena/create_single.sql | 112 -- benchmark/athena/queries.sql | 43 - benchmark/athena/results/partitioned.json | 58 -- benchmark/athena/results/single.json | 58 -- benchmark/athena/run.sh | 9 - benchmark/aurora-mysql/README.md | 67 -- benchmark/aurora-mysql/create.sql | 109 -- benchmark/aurora-mysql/queries.sql | 43 - benchmark/aurora-mysql/results/16acu.json | 58 -- benchmark/aurora-mysql/run.sh | 9 - benchmark/aurora-postgresql/README.md | 59 -- benchmark/aurora-postgresql/create.sql | 109 -- benchmark/aurora-postgresql/queries.sql | 43 - .../aurora-postgresql/results/16acu.json | 58 -- benchmark/aurora-postgresql/run.sh | 10 - benchmark/bigquery/.gitignore | 1 - benchmark/bigquery/README.md | 38 - benchmark/bigquery/create.sql | 108 -- benchmark/bigquery/queries.sql | 43 - benchmark/bigquery/run.sh | 10 - benchmark/brytlytdb/README.md | 1 - benchmark/citus/benchmark.sh | 27 - benchmark/citus/create.sql | 110 -- benchmark/citus/queries.sql | 43 - benchmark/citus/results/c6a.4xlarge.json | 58 -- benchmark/citus/run.sh | 13 - benchmark/clickhouse-local/benchmark.sh | 14 - benchmark/clickhouse-local/create.sql | 109 -- benchmark/clickhouse-local/queries.sql | 43 - .../results/c6a.4xlarge.partitioned.json | 58 -- .../results/c6a.4xlarge.single.json | 58 -- benchmark/clickhouse-local/run.sh | 20 - benchmark/clickhouse/benchmark.sh | 22 - benchmark/clickhouse/create.sql | 110 -- benchmark/clickhouse/queries.sql | 43 - benchmark/clickhouse/results/c6a.4xlarge.json | 58 -- benchmark/clickhouse/results/c6a.metal.json | 58 -- benchmark/clickhouse/run.sh | 20 - benchmark/cratedb/benchmark.sh | 39 - benchmark/cratedb/create.sql | 109 -- benchmark/cratedb/queries.sql | 43 - benchmark/cratedb/results/c6a.4xlarge.json | 58 -- benchmark/cratedb/run.sh | 13 - benchmark/databend/README.md | 7 - benchmark/databend/benchmark.sh | 105 -- benchmark/databend/create.sql | 109 -- benchmark/databend/queries.sql | 43 - benchmark/databend/results/c6a.4xlarge.json | 58 -- benchmark/databend/run.sh | 20 - benchmark/druid/benchmark.sh | 43 - benchmark/druid/check.json | 1 - benchmark/druid/ingest.json | 573 ----------- benchmark/druid/queries.sql | 43 - benchmark/druid/results/c6a.4xlarge.json | 58 -- benchmark/druid/run.sh | 23 - benchmark/duckdb/README.md | 2 - benchmark/duckdb/benchmark.sh | 24 - benchmark/duckdb/create.sql | 109 -- benchmark/duckdb/load.py | 18 - benchmark/duckdb/queries.sql | 43 - benchmark/duckdb/query.py | 20 - benchmark/duckdb/results/c6a.4xlarge.json | 58 -- benchmark/duckdb/run.sh | 10 - benchmark/elasticsearch/README.md | 1 - benchmark/elasticsearch/benchmark.sh | 30 - benchmark/exasol/README.md | 3 - benchmark/generate-results.sh | 23 - benchmark/greenplum/benchmark.sh | 73 -- benchmark/greenplum/create.sql | 116 --- benchmark/greenplum/log.txt | 215 ---- benchmark/greenplum/queries.sql | 43 - benchmark/greenplum/results/c6a.4xlarge.json | 58 -- benchmark/greenplum/run.sh | 15 - benchmark/hardware.sh | 248 ++++- benchmark/hardware/benchmark-chyt.sh | 22 - benchmark/hardware/benchmark-new.sh | 29 - benchmark/hardware/benchmark-yql.sh | 19 - benchmark/hardware/benchmark_cloud.sh | 43 - benchmark/hardware/hardware.sh | 247 ----- benchmark/hardware/queries.sql | 43 - benchmark/heavyai/benchmark.sh | 50 - benchmark/heavyai/create.sql | 108 -- benchmark/heavyai/queries.sql | 43 - benchmark/heavyai/results/c6a.4xlarge.json | 58 -- benchmark/heavyai/run.sh | 13 - benchmark/index.html | 956 ------------------ benchmark/infobright/benchmark.sh | 38 - benchmark/infobright/create.sql | 108 -- benchmark/infobright/queries.sql | 43 - benchmark/infobright/results/c6a.4xlarge.json | 58 -- benchmark/infobright/run.sh | 12 - benchmark/locustdb/README.md | 24 - benchmark/locustdb/benchmark.sh | 46 - benchmark/mariadb-columnstore/README.md | 9 - benchmark/mariadb-columnstore/benchmark.sh | 36 - benchmark/mariadb-columnstore/create.sql | 108 -- benchmark/mariadb-columnstore/queries.sql | 43 - .../results/c6a.4xlarge.json | 58 -- benchmark/mariadb-columnstore/run.sh | 12 - benchmark/mariadb/benchmark.sh | 29 - benchmark/mariadb/create.sql | 109 -- benchmark/mariadb/queries.sql | 43 - .../mariadb/results/c6a.4xlarge.json.shame | 58 -- benchmark/mariadb/run.sh | 12 - benchmark/monetdb/benchmark.sh | 40 - benchmark/monetdb/create.sql | 108 -- benchmark/monetdb/queries.sql | 43 - benchmark/monetdb/query.expect | 10 - benchmark/monetdb/results/c6a.4xlarge.json | 58 -- benchmark/monetdb/run.sh | 12 - benchmark/mysql-myisam/benchmark.sh | 29 - benchmark/mysql-myisam/create.sql | 110 -- benchmark/mysql-myisam/queries.sql | 43 - .../mysql-myisam/results/c6a.4xlarge.json | 58 -- benchmark/mysql-myisam/run.sh | 12 - benchmark/mysql/benchmark.sh | 29 - benchmark/mysql/create.sql | 110 -- benchmark/mysql/queries.sql | 43 - benchmark/mysql/results/c6a.4xlarge.json | 58 -- benchmark/mysql/run.sh | 12 - benchmark/pinot/benchmark.sh | 44 - benchmark/pinot/local.yaml | 38 - benchmark/pinot/offline_table.json | 17 - benchmark/pinot/queries.sql | 43 - benchmark/pinot/results/c6a.4xlarge.json | 58 -- benchmark/pinot/run.sh | 15 - benchmark/pinot/schema.json | 437 -------- benchmark/pinot/splitted.yaml | 37 - benchmark/postgresql/benchmark.sh | 23 - benchmark/postgresql/create.sql | 109 -- benchmark/postgresql/queries.sql | 43 - benchmark/postgresql/results/c6a.4xlarge.json | 58 -- benchmark/postgresql/run.sh | 13 - benchmark/questdb/benchmark.sh | 29 - benchmark/questdb/create.sql | 108 -- benchmark/questdb/queries.sql | 43 - benchmark/questdb/results/c6a.4xlarge.json | 58 -- benchmark/questdb/run.sh | 19 - benchmark/redshift-serverless/README.md | 55 - benchmark/redshift-serverless/create.sql | 109 -- benchmark/redshift-serverless/queries.sql | 43 - .../results/serverless.json | 58 -- benchmark/redshift-serverless/run.sh | 10 - benchmark/redshift/README.md | 56 - benchmark/redshift/create.sql | 109 -- benchmark/redshift/queries.sql | 43 - benchmark/redshift/results/4x.ra3.xplus.json | 58 -- benchmark/redshift/run.sh | 10 - benchmark/singlestore/benchmark.sh | 44 - benchmark/singlestore/create.sql | 109 -- benchmark/singlestore/queries.sql | 43 - .../singlestore/results/c6a.4xlarge.json | 58 -- benchmark/singlestore/run.sh | 12 - benchmark/snowflake/NOTES.md | 67 -- benchmark/snowflake/README.md | 56 - benchmark/snowflake/create.sql | 109 -- benchmark/snowflake/queries.sql | 43 - benchmark/snowflake/results/2xl.json | 58 -- benchmark/snowflake/results/3xl.json | 58 -- benchmark/snowflake/results/4xl.json | 58 -- benchmark/snowflake/results/l.json | 58 -- benchmark/snowflake/results/m.json | 58 -- benchmark/snowflake/results/s.json | 58 -- benchmark/snowflake/results/xl.json | 58 -- benchmark/snowflake/results/xs.json | 58 -- benchmark/snowflake/run.sh | 3 - benchmark/sqlite/benchmark.sh | 20 - benchmark/sqlite/create.sql | 109 -- benchmark/sqlite/queries.sql | 43 - benchmark/sqlite/results/c6a.4xlarge.json | 58 -- benchmark/sqlite/run.sh | 13 - benchmark/starrocks/README.md | 3 - benchmark/starrocks/benchmark.sh | 3 - benchmark/timescaledb-compressed/benchmark.sh | 48 - benchmark/timescaledb-compressed/create.sql | 108 -- benchmark/timescaledb-compressed/queries.sql | 43 - .../results/c6a.4xlarge.json | 58 -- benchmark/timescaledb-compressed/run.sh | 13 - benchmark/timescaledb/benchmark.sh | 41 - benchmark/timescaledb/create.sql | 108 -- benchmark/timescaledb/queries.sql | 43 - .../timescaledb/results/c6a.4xlarge.json | 58 -- benchmark/timescaledb/run.sh | 13 - benchmark/trino/README.md | 1 - benchmark/trino/benchmark.sh | 10 - benchmark/trino/create_partitioned.sql | 112 -- benchmark/trino/create_single.sql | 107 -- benchmark/trino/queries.sql | 43 - benchmark/vertica/.gitignore | 1 - benchmark/vertica/README.md | 5 - benchmark/vertica/benchmark.sh | 26 - benchmark/vertica/create.sql | 109 -- benchmark/vertica/queries.sql | 43 - benchmark/vertica/run.sh | 13 - 199 files changed, 248 insertions(+), 12277 deletions(-) delete mode 100644 benchmark/.gitignore delete mode 100644 benchmark/LICENSE delete mode 100644 benchmark/athena/README.md delete mode 100644 benchmark/athena/create_partitioned.sql delete mode 100644 benchmark/athena/create_single.sql delete mode 100644 benchmark/athena/queries.sql delete mode 100644 benchmark/athena/results/partitioned.json delete mode 100644 benchmark/athena/results/single.json delete mode 100755 benchmark/athena/run.sh delete mode 100644 benchmark/aurora-mysql/README.md delete mode 100644 benchmark/aurora-mysql/create.sql delete mode 100644 benchmark/aurora-mysql/queries.sql delete mode 100644 benchmark/aurora-mysql/results/16acu.json delete mode 100755 benchmark/aurora-mysql/run.sh delete mode 100644 benchmark/aurora-postgresql/README.md delete mode 100644 benchmark/aurora-postgresql/create.sql delete mode 100644 benchmark/aurora-postgresql/queries.sql delete mode 100644 benchmark/aurora-postgresql/results/16acu.json delete mode 100755 benchmark/aurora-postgresql/run.sh delete mode 100644 benchmark/bigquery/.gitignore delete mode 100644 benchmark/bigquery/README.md delete mode 100644 benchmark/bigquery/create.sql delete mode 100644 benchmark/bigquery/queries.sql delete mode 100755 benchmark/bigquery/run.sh delete mode 100644 benchmark/brytlytdb/README.md delete mode 100755 benchmark/citus/benchmark.sh delete mode 100644 benchmark/citus/create.sql delete mode 100644 benchmark/citus/queries.sql delete mode 100644 benchmark/citus/results/c6a.4xlarge.json delete mode 100755 benchmark/citus/run.sh delete mode 100755 benchmark/clickhouse-local/benchmark.sh delete mode 100644 benchmark/clickhouse-local/create.sql delete mode 100644 benchmark/clickhouse-local/queries.sql delete mode 100644 benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json delete mode 100644 benchmark/clickhouse-local/results/c6a.4xlarge.single.json delete mode 100755 benchmark/clickhouse-local/run.sh delete mode 100755 benchmark/clickhouse/benchmark.sh delete mode 100644 benchmark/clickhouse/create.sql delete mode 100644 benchmark/clickhouse/queries.sql delete mode 100644 benchmark/clickhouse/results/c6a.4xlarge.json delete mode 100644 benchmark/clickhouse/results/c6a.metal.json delete mode 100755 benchmark/clickhouse/run.sh delete mode 100755 benchmark/cratedb/benchmark.sh delete mode 100644 benchmark/cratedb/create.sql delete mode 100644 benchmark/cratedb/queries.sql delete mode 100644 benchmark/cratedb/results/c6a.4xlarge.json delete mode 100755 benchmark/cratedb/run.sh delete mode 100644 benchmark/databend/README.md delete mode 100755 benchmark/databend/benchmark.sh delete mode 100644 benchmark/databend/create.sql delete mode 100644 benchmark/databend/queries.sql delete mode 100644 benchmark/databend/results/c6a.4xlarge.json delete mode 100755 benchmark/databend/run.sh delete mode 100755 benchmark/druid/benchmark.sh delete mode 100644 benchmark/druid/check.json delete mode 100644 benchmark/druid/ingest.json delete mode 100644 benchmark/druid/queries.sql delete mode 100644 benchmark/druid/results/c6a.4xlarge.json delete mode 100755 benchmark/druid/run.sh delete mode 100644 benchmark/duckdb/README.md delete mode 100755 benchmark/duckdb/benchmark.sh delete mode 100644 benchmark/duckdb/create.sql delete mode 100755 benchmark/duckdb/load.py delete mode 100644 benchmark/duckdb/queries.sql delete mode 100755 benchmark/duckdb/query.py delete mode 100644 benchmark/duckdb/results/c6a.4xlarge.json delete mode 100755 benchmark/duckdb/run.sh delete mode 100644 benchmark/elasticsearch/README.md delete mode 100755 benchmark/elasticsearch/benchmark.sh delete mode 100644 benchmark/exasol/README.md delete mode 100755 benchmark/generate-results.sh delete mode 100755 benchmark/greenplum/benchmark.sh delete mode 100644 benchmark/greenplum/create.sql delete mode 100644 benchmark/greenplum/log.txt delete mode 100644 benchmark/greenplum/queries.sql delete mode 100644 benchmark/greenplum/results/c6a.4xlarge.json delete mode 100755 benchmark/greenplum/run.sh mode change 120000 => 100755 benchmark/hardware.sh delete mode 100755 benchmark/hardware/benchmark-chyt.sh delete mode 100755 benchmark/hardware/benchmark-new.sh delete mode 100755 benchmark/hardware/benchmark-yql.sh delete mode 100755 benchmark/hardware/benchmark_cloud.sh delete mode 100755 benchmark/hardware/hardware.sh delete mode 100644 benchmark/hardware/queries.sql delete mode 100755 benchmark/heavyai/benchmark.sh delete mode 100644 benchmark/heavyai/create.sql delete mode 100644 benchmark/heavyai/queries.sql delete mode 100644 benchmark/heavyai/results/c6a.4xlarge.json delete mode 100755 benchmark/heavyai/run.sh delete mode 100644 benchmark/index.html delete mode 100755 benchmark/infobright/benchmark.sh delete mode 100644 benchmark/infobright/create.sql delete mode 100644 benchmark/infobright/queries.sql delete mode 100644 benchmark/infobright/results/c6a.4xlarge.json delete mode 100755 benchmark/infobright/run.sh delete mode 100644 benchmark/locustdb/README.md delete mode 100755 benchmark/locustdb/benchmark.sh delete mode 100644 benchmark/mariadb-columnstore/README.md delete mode 100755 benchmark/mariadb-columnstore/benchmark.sh delete mode 100644 benchmark/mariadb-columnstore/create.sql delete mode 100644 benchmark/mariadb-columnstore/queries.sql delete mode 100644 benchmark/mariadb-columnstore/results/c6a.4xlarge.json delete mode 100755 benchmark/mariadb-columnstore/run.sh delete mode 100755 benchmark/mariadb/benchmark.sh delete mode 100644 benchmark/mariadb/create.sql delete mode 100644 benchmark/mariadb/queries.sql delete mode 100644 benchmark/mariadb/results/c6a.4xlarge.json.shame delete mode 100755 benchmark/mariadb/run.sh delete mode 100755 benchmark/monetdb/benchmark.sh delete mode 100644 benchmark/monetdb/create.sql delete mode 100644 benchmark/monetdb/queries.sql delete mode 100755 benchmark/monetdb/query.expect delete mode 100644 benchmark/monetdb/results/c6a.4xlarge.json delete mode 100755 benchmark/monetdb/run.sh delete mode 100755 benchmark/mysql-myisam/benchmark.sh delete mode 100644 benchmark/mysql-myisam/create.sql delete mode 100644 benchmark/mysql-myisam/queries.sql delete mode 100644 benchmark/mysql-myisam/results/c6a.4xlarge.json delete mode 100755 benchmark/mysql-myisam/run.sh delete mode 100755 benchmark/mysql/benchmark.sh delete mode 100644 benchmark/mysql/create.sql delete mode 100644 benchmark/mysql/queries.sql delete mode 100644 benchmark/mysql/results/c6a.4xlarge.json delete mode 100755 benchmark/mysql/run.sh delete mode 100755 benchmark/pinot/benchmark.sh delete mode 100644 benchmark/pinot/local.yaml delete mode 100644 benchmark/pinot/offline_table.json delete mode 100644 benchmark/pinot/queries.sql delete mode 100644 benchmark/pinot/results/c6a.4xlarge.json delete mode 100755 benchmark/pinot/run.sh delete mode 100644 benchmark/pinot/schema.json delete mode 100644 benchmark/pinot/splitted.yaml delete mode 100755 benchmark/postgresql/benchmark.sh delete mode 100644 benchmark/postgresql/create.sql delete mode 100644 benchmark/postgresql/queries.sql delete mode 100644 benchmark/postgresql/results/c6a.4xlarge.json delete mode 100755 benchmark/postgresql/run.sh delete mode 100755 benchmark/questdb/benchmark.sh delete mode 100644 benchmark/questdb/create.sql delete mode 100644 benchmark/questdb/queries.sql delete mode 100644 benchmark/questdb/results/c6a.4xlarge.json delete mode 100755 benchmark/questdb/run.sh delete mode 100644 benchmark/redshift-serverless/README.md delete mode 100644 benchmark/redshift-serverless/create.sql delete mode 100644 benchmark/redshift-serverless/queries.sql delete mode 100644 benchmark/redshift-serverless/results/serverless.json delete mode 100755 benchmark/redshift-serverless/run.sh delete mode 100644 benchmark/redshift/README.md delete mode 100644 benchmark/redshift/create.sql delete mode 100644 benchmark/redshift/queries.sql delete mode 100644 benchmark/redshift/results/4x.ra3.xplus.json delete mode 100755 benchmark/redshift/run.sh delete mode 100755 benchmark/singlestore/benchmark.sh delete mode 100644 benchmark/singlestore/create.sql delete mode 100644 benchmark/singlestore/queries.sql delete mode 100644 benchmark/singlestore/results/c6a.4xlarge.json delete mode 100755 benchmark/singlestore/run.sh delete mode 100644 benchmark/snowflake/NOTES.md delete mode 100644 benchmark/snowflake/README.md delete mode 100644 benchmark/snowflake/create.sql delete mode 100644 benchmark/snowflake/queries.sql delete mode 100644 benchmark/snowflake/results/2xl.json delete mode 100644 benchmark/snowflake/results/3xl.json delete mode 100644 benchmark/snowflake/results/4xl.json delete mode 100644 benchmark/snowflake/results/l.json delete mode 100644 benchmark/snowflake/results/m.json delete mode 100644 benchmark/snowflake/results/s.json delete mode 100644 benchmark/snowflake/results/xl.json delete mode 100644 benchmark/snowflake/results/xs.json delete mode 100755 benchmark/snowflake/run.sh delete mode 100755 benchmark/sqlite/benchmark.sh delete mode 100644 benchmark/sqlite/create.sql delete mode 100644 benchmark/sqlite/queries.sql delete mode 100644 benchmark/sqlite/results/c6a.4xlarge.json delete mode 100755 benchmark/sqlite/run.sh delete mode 100644 benchmark/starrocks/README.md delete mode 100755 benchmark/starrocks/benchmark.sh delete mode 100755 benchmark/timescaledb-compressed/benchmark.sh delete mode 100644 benchmark/timescaledb-compressed/create.sql delete mode 100644 benchmark/timescaledb-compressed/queries.sql delete mode 100644 benchmark/timescaledb-compressed/results/c6a.4xlarge.json delete mode 100755 benchmark/timescaledb-compressed/run.sh delete mode 100755 benchmark/timescaledb/benchmark.sh delete mode 100644 benchmark/timescaledb/create.sql delete mode 100644 benchmark/timescaledb/queries.sql delete mode 100644 benchmark/timescaledb/results/c6a.4xlarge.json delete mode 100755 benchmark/timescaledb/run.sh delete mode 100644 benchmark/trino/README.md delete mode 100755 benchmark/trino/benchmark.sh delete mode 100644 benchmark/trino/create_partitioned.sql delete mode 100644 benchmark/trino/create_single.sql delete mode 100644 benchmark/trino/queries.sql delete mode 100644 benchmark/vertica/.gitignore delete mode 100644 benchmark/vertica/README.md delete mode 100755 benchmark/vertica/benchmark.sh delete mode 100644 benchmark/vertica/create.sql delete mode 100644 benchmark/vertica/queries.sql delete mode 100755 benchmark/vertica/run.sh diff --git a/benchmark/.gitignore b/benchmark/.gitignore deleted file mode 100644 index 751553b3acb..00000000000 --- a/benchmark/.gitignore +++ /dev/null @@ -1 +0,0 @@ -*.bak diff --git a/benchmark/LICENSE b/benchmark/LICENSE deleted file mode 100644 index 0f55d62d28d..00000000000 --- a/benchmark/LICENSE +++ /dev/null @@ -1,360 +0,0 @@ -Creative Commons Attribution-NonCommercial-ShareAlike 4.0 International -Public License - -By exercising the Licensed Rights (defined below), You accept and agree -to be bound by the terms and conditions of this Creative Commons -Attribution-NonCommercial-ShareAlike 4.0 International Public License -("Public License"). To the extent this Public License may be -interpreted as a contract, You are granted the Licensed Rights in -consideration of Your acceptance of these terms and conditions, and the -Licensor grants You such rights in consideration of benefits the -Licensor receives from making the Licensed Material available under -these terms and conditions. - - -Section 1 -- Definitions. - - a. Adapted Material means material subject to Copyright and Similar - Rights that is derived from or based upon the Licensed Material - and in which the Licensed Material is translated, altered, - arranged, transformed, or otherwise modified in a manner requiring - permission under the Copyright and Similar Rights held by the - Licensor. For purposes of this Public License, where the Licensed - Material is a musical work, performance, or sound recording, - Adapted Material is always produced where the Licensed Material is - synched in timed relation with a moving image. - - b. Adapter's License means the license You apply to Your Copyright - and Similar Rights in Your contributions to Adapted Material in - accordance with the terms and conditions of this Public License. - - c. BY-NC-SA Compatible License means a license listed at - creativecommons.org/compatiblelicenses, approved by Creative - Commons as essentially the equivalent of this Public License. - - d. Copyright and Similar Rights means copyright and/or similar rights - closely related to copyright including, without limitation, - performance, broadcast, sound recording, and Sui Generis Database - Rights, without regard to how the rights are labeled or - categorized. For purposes of this Public License, the rights - specified in Section 2(b)(1)-(2) are not Copyright and Similar - Rights. - - e. Effective Technological Measures means those measures that, in the - absence of proper authority, may not be circumvented under laws - fulfilling obligations under Article 11 of the WIPO Copyright - Treaty adopted on December 20, 1996, and/or similar international - agreements. - - f. Exceptions and Limitations means fair use, fair dealing, and/or - any other exception or limitation to Copyright and Similar Rights - that applies to Your use of the Licensed Material. - - g. License Elements means the license attributes listed in the name - of a Creative Commons Public License. The License Elements of this - Public License are Attribution, NonCommercial, and ShareAlike. - - h. Licensed Material means the artistic or literary work, database, - or other material to which the Licensor applied this Public - License. - - i. Licensed Rights means the rights granted to You subject to the - terms and conditions of this Public License, which are limited to - all Copyright and Similar Rights that apply to Your use of the - Licensed Material and that the Licensor has authority to license. - - j. Licensor means the individual(s) or entity(ies) granting rights - under this Public License. - - k. NonCommercial means not primarily intended for or directed towards - commercial advantage or monetary compensation. For purposes of - this Public License, the exchange of the Licensed Material for - other material subject to Copyright and Similar Rights by digital - file-sharing or similar means is NonCommercial provided there is - no payment of monetary compensation in connection with the - exchange. - - l. Share means to provide material to the public by any means or - process that requires permission under the Licensed Rights, such - as reproduction, public display, public performance, distribution, - dissemination, communication, or importation, and to make material - available to the public including in ways that members of the - public may access the material from a place and at a time - individually chosen by them. - - m. Sui Generis Database Rights means rights other than copyright - resulting from Directive 96/9/EC of the European Parliament and of - the Council of 11 March 1996 on the legal protection of databases, - as amended and/or succeeded, as well as other essentially - equivalent rights anywhere in the world. - - n. You means the individual or entity exercising the Licensed Rights - under this Public License. Your has a corresponding meaning. - - -Section 2 -- Scope. - - a. License grant. - - 1. Subject to the terms and conditions of this Public License, - the Licensor hereby grants You a worldwide, royalty-free, - non-sublicensable, non-exclusive, irrevocable license to - exercise the Licensed Rights in the Licensed Material to: - - a. reproduce and Share the Licensed Material, in whole or - in part, for NonCommercial purposes only; and - - b. produce, reproduce, and Share Adapted Material for - NonCommercial purposes only. - - 2. Exceptions and Limitations. For the avoidance of doubt, where - Exceptions and Limitations apply to Your use, this Public - License does not apply, and You do not need to comply with - its terms and conditions. - - 3. Term. The term of this Public License is specified in Section - 6(a). - - 4. Media and formats; technical modifications allowed. The - Licensor authorizes You to exercise the Licensed Rights in - all media and formats whether now known or hereafter created, - and to make technical modifications necessary to do so. The - Licensor waives and/or agrees not to assert any right or - authority to forbid You from making technical modifications - necessary to exercise the Licensed Rights, including - technical modifications necessary to circumvent Effective - Technological Measures. For purposes of this Public License, - simply making modifications authorized by this Section 2(a) - (4) never produces Adapted Material. - - 5. Downstream recipients. - - a. Offer from the Licensor -- Licensed Material. Every - recipient of the Licensed Material automatically - receives an offer from the Licensor to exercise the - Licensed Rights under the terms and conditions of this - Public License. - - b. Additional offer from the Licensor -- Adapted Material. - Every recipient of Adapted Material from You - automatically receives an offer from the Licensor to - exercise the Licensed Rights in the Adapted Material - under the conditions of the Adapter's License You apply. - - c. No downstream restrictions. You may not offer or impose - any additional or different terms or conditions on, or - apply any Effective Technological Measures to, the - Licensed Material if doing so restricts exercise of the - Licensed Rights by any recipient of the Licensed - Material. - - 6. No endorsement. Nothing in this Public License constitutes or - may be construed as permission to assert or imply that You - are, or that Your use of the Licensed Material is, connected - with, or sponsored, endorsed, or granted official status by, - the Licensor or others designated to receive attribution as - provided in Section 3(a)(1)(A)(i). - - b. Other rights. - - 1. Moral rights, such as the right of integrity, are not - licensed under this Public License, nor are publicity, - privacy, and/or other similar personality rights; however, to - the extent possible, the Licensor waives and/or agrees not to - assert any such rights held by the Licensor to the limited - extent necessary to allow You to exercise the Licensed - Rights, but not otherwise. - - 2. Patent and trademark rights are not licensed under this - Public License. - - 3. To the extent possible, the Licensor waives any right to - collect royalties from You for the exercise of the Licensed - Rights, whether directly or through a collecting society - under any voluntary or waivable statutory or compulsory - licensing scheme. In all other cases the Licensor expressly - reserves any right to collect such royalties, including when - the Licensed Material is used other than for NonCommercial - purposes. - - -Section 3 -- License Conditions. - -Your exercise of the Licensed Rights is expressly made subject to the -following conditions. - - a. Attribution. - - 1. If You Share the Licensed Material (including in modified - form), You must: - - a. retain the following if it is supplied by the Licensor - with the Licensed Material: - - i. identification of the creator(s) of the Licensed - Material and any others designated to receive - attribution, in any reasonable manner requested by - the Licensor (including by pseudonym if - designated); - - ii. a copyright notice; - - iii. a notice that refers to this Public License; - - iv. a notice that refers to the disclaimer of - warranties; - - v. a URI or hyperlink to the Licensed Material to the - extent reasonably practicable; - - b. indicate if You modified the Licensed Material and - retain an indication of any previous modifications; and - - c. indicate the Licensed Material is licensed under this - Public License, and include the text of, or the URI or - hyperlink to, this Public License. - - 2. You may satisfy the conditions in Section 3(a)(1) in any - reasonable manner based on the medium, means, and context in - which You Share the Licensed Material. For example, it may be - reasonable to satisfy the conditions by providing a URI or - hyperlink to a resource that includes the required - information. - 3. If requested by the Licensor, You must remove any of the - information required by Section 3(a)(1)(A) to the extent - reasonably practicable. - - b. ShareAlike. - - In addition to the conditions in Section 3(a), if You Share - Adapted Material You produce, the following conditions also apply. - - 1. The Adapter's License You apply must be a Creative Commons - license with the same License Elements, this version or - later, or a BY-NC-SA Compatible License. - - 2. You must include the text of, or the URI or hyperlink to, the - Adapter's License You apply. You may satisfy this condition - in any reasonable manner based on the medium, means, and - context in which You Share Adapted Material. - - 3. You may not offer or impose any additional or different terms - or conditions on, or apply any Effective Technological - Measures to, Adapted Material that restrict exercise of the - rights granted under the Adapter's License You apply. - - -Section 4 -- Sui Generis Database Rights. - -Where the Licensed Rights include Sui Generis Database Rights that -apply to Your use of the Licensed Material: - - a. for the avoidance of doubt, Section 2(a)(1) grants You the right - to extract, reuse, reproduce, and Share all or a substantial - portion of the contents of the database for NonCommercial purposes - only; - - b. if You include all or a substantial portion of the database - contents in a database in which You have Sui Generis Database - Rights, then the database in which You have Sui Generis Database - Rights (but not its individual contents) is Adapted Material, - including for purposes of Section 3(b); and - - c. You must comply with the conditions in Section 3(a) if You Share - all or a substantial portion of the contents of the database. - -For the avoidance of doubt, this Section 4 supplements and does not -replace Your obligations under this Public License where the Licensed -Rights include other Copyright and Similar Rights. - - -Section 5 -- Disclaimer of Warranties and Limitation of Liability. - - a. UNLESS OTHERWISE SEPARATELY UNDERTAKEN BY THE LICENSOR, TO THE - EXTENT POSSIBLE, THE LICENSOR OFFERS THE LICENSED MATERIAL AS-IS - AND AS-AVAILABLE, AND MAKES NO REPRESENTATIONS OR WARRANTIES OF - ANY KIND CONCERNING THE LICENSED MATERIAL, WHETHER EXPRESS, - IMPLIED, STATUTORY, OR OTHER. THIS INCLUDES, WITHOUT LIMITATION, - WARRANTIES OF TITLE, MERCHANTABILITY, FITNESS FOR A PARTICULAR - PURPOSE, NON-INFRINGEMENT, ABSENCE OF LATENT OR OTHER DEFECTS, - ACCURACY, OR THE PRESENCE OR ABSENCE OF ERRORS, WHETHER OR NOT - KNOWN OR DISCOVERABLE. WHERE DISCLAIMERS OF WARRANTIES ARE NOT - ALLOWED IN FULL OR IN PART, THIS DISCLAIMER MAY NOT APPLY TO YOU. - - b. TO THE EXTENT POSSIBLE, IN NO EVENT WILL THE LICENSOR BE LIABLE - TO YOU ON ANY LEGAL THEORY (INCLUDING, WITHOUT LIMITATION, - NEGLIGENCE) OR OTHERWISE FOR ANY DIRECT, SPECIAL, INDIRECT, - INCIDENTAL, CONSEQUENTIAL, PUNITIVE, EXEMPLARY, OR OTHER LOSSES, - COSTS, EXPENSES, OR DAMAGES ARISING OUT OF THIS PUBLIC LICENSE OR - USE OF THE LICENSED MATERIAL, EVEN IF THE LICENSOR HAS BEEN - ADVISED OF THE POSSIBILITY OF SUCH LOSSES, COSTS, EXPENSES, OR - DAMAGES. WHERE A LIMITATION OF LIABILITY IS NOT ALLOWED IN FULL OR - IN PART, THIS LIMITATION MAY NOT APPLY TO YOU. - - c. The disclaimer of warranties and limitation of liability provided - above shall be interpreted in a manner that, to the extent - possible, most closely approximates an absolute disclaimer and - waiver of all liability. - - -Section 6 -- Term and Termination. - - a. This Public License applies for the term of the Copyright and - Similar Rights licensed here. However, if You fail to comply with - this Public License, then Your rights under this Public License - terminate automatically. - - b. Where Your right to use the Licensed Material has terminated under - Section 6(a), it reinstates: - - 1. automatically as of the date the violation is cured, provided - it is cured within 30 days of Your discovery of the - violation; or - - 2. upon express reinstatement by the Licensor. - - For the avoidance of doubt, this Section 6(b) does not affect any - right the Licensor may have to seek remedies for Your violations - of this Public License. - - c. For the avoidance of doubt, the Licensor may also offer the - Licensed Material under separate terms or conditions or stop - distributing the Licensed Material at any time; however, doing so - will not terminate this Public License. - - d. Sections 1, 5, 6, 7, and 8 survive termination of this Public - License. - - -Section 7 -- Other Terms and Conditions. - - a. The Licensor shall not be bound by any additional or different - terms or conditions communicated by You unless expressly agreed. - - b. Any arrangements, understandings, or agreements regarding the - Licensed Material not stated herein are separate from and - independent of the terms and conditions of this Public License. - - -Section 8 -- Interpretation. - - a. For the avoidance of doubt, this Public License does not, and - shall not be interpreted to, reduce, limit, restrict, or impose - conditions on any use of the Licensed Material that could lawfully - be made without permission under this Public License. - - b. To the extent possible, if any provision of this Public License is - deemed unenforceable, it shall be automatically reformed to the - minimum extent necessary to make it enforceable. If the provision - cannot be reformed, it shall be severed from this Public License - without affecting the enforceability of the remaining terms and - conditions. - - c. No term or condition of this Public License will be waived and no - failure to comply consented to unless expressly agreed to by the - Licensor. - - d. Nothing in this Public License constitutes or may be interpreted - as a limitation upon, or waiver of, any privileges and immunities - that apply to the Licensor or You, including from the legal - processes of any jurisdiction or authority. diff --git a/benchmark/README.md b/benchmark/README.md index fa178537628..ac43d967d5f 100644 --- a/benchmark/README.md +++ b/benchmark/README.md @@ -1,411 +1 @@ -# ClickBench: a Benchmark For Analytical Databases - -## Overview - -This benchmark represents typical workload in the following areas: click stream and traffic analysis, web analytics, machine-generated data, structured logs, and events data. It covers the typical queries in ad-hoc analytics and real-time dashboards. - -The dataset from this benchmark was obtained from the actual traffic recording of one of the world's largest web analytics platforms. It is anonymized while keeping all the essential distributions of the data. The set of queries was improvised to reflect the realistic workloads, while the queries are not directly from production. - -## Goals - -The main goals of this benchmark are: - -### Reproducibility - -You can quickly reproduce every test in as little as 20 minutes (those some systems may take several hours) in a semi-automated way. The test setup is documented and uses inexpensive cloud VMs. The test process is documented in the form of a shell script, covering the installation of every system, loading of the data, running the workload, and collecting the result numbers. The dataset is published and made available for download in multiple formats. - -### Compatibility - -The tables and queries use mostly standard SQL and require minimum or no adaptation for most SQL DBMS. The dataset has been filtered to avoid difficulties with parsing and loading. - -### Diversity - -The benchmark process is easy enough to cover a wide range of systems. It includes: modern and historical self-managed OLAP DBMS; traditional OLTP DBMS are included for comparison baseline; managed database-as-a-service offerings are included, as well as serverless cloud-native databases; some NoSQL, document, and specialized time-series databases are included as well for a reference, even if they should not be comparable on the same workload. - -### Realism - -The dataset is derived from accurate production data. The realistic data distributions allow to correctly account for compression, indices, codecs, custom data structures, etc. which is not possible with most of the random dataset generators. The workload consists of 43 queries and can test the efficiency of full scan and filtered scan, as well as index lookups, and main relational operations. It can test various aspects of hardware as well: some queries require high storage throughput; some queries benefit from a large number of CPU cores and some benefit from single-core speed; some queries benefit from high main memory bandwidth. - -## Limitations - -The limitations of this benchmark allow keeping it easy to reproduce and to include more systems in the comparison. The benchmark represents only a subset of all possible workloads and scenarios. While it aims to be as fair as possible, the focus on a specific subset of workloads may give an advantage to the systems specialized in that workloads. - -The following limitations should be acknowledged: - -1. The dataset is represented by one flat table. This is not representative of classical data warehouses which are using a normalized star or snowflake data model. The systems for classical data warehouses may get an unfair disadvantage on this benchmark. - -2. The table consists of exactly 99 997 497 records. This is rather small by modern standards but allows to perform tests in a reasonable time. - -3. While this benchmark allows testing distributed systems, and it includes multi-node and serverless cloud-native setups, most of the results so far have been obtained on a single node setup. - -4. The benchmark runs queries one after another and does not test a workload with concurrent requests; neither does it test for system capacity. Every query is run only a few times and this allows some variability in the results. - -6. Many setups and systems are different enough to make direct comparison tricky. It is not possible to test the efficiency of storage used for in-memory databases, or the time of data loading for stateless query engines. The goal of the benchmark is to give the numbers for comparison and let you derive the conclusions on your own. - -TLDR: *All Benchmarks Are ~~Bastards~~ Liars*. - -## Rules and Contribution - -### How To Add a New Result - -To introduce a new system, simply copy-paste one of the directories and edit the files accordingly: -- `benchmark.sh`: this is the main script to run the benchmark on a fresh VM; Ubuntu 22.04 or newer should be used by default, or any other system if specified in the comments. The script is not necessarily can be run in a fully automated manner - it is recommended always to copy-paste the commands one by one and observe the results. For managed databases, if the setup requires clicking in the UI, write a `README.md` instead. -- `README.md`: contains comments and observations if needed. For managed databases, it can describe the setup procedure instead of a shell script. -- `create.sql`: a CREATE TABLE statement. If it's a NoSQL system, another file like `wtf.json` can be presented. -- `queries.sql`: contains 43 queries to run; -- `run.sh`: a loop of running the queries; every query is run three times; if it's a database with local on-disk storage, the first query should be run after dropping the page cache; -- `results`: put the .json files with the results for every hardware configuration there. - -To introduce a new result for the existing system on different hardware configurations, add a new file to `results`. - -To introduce a new result for an existing system with a different usage scenario, either copy the whole directory and name it differently (e.g. `timescaledb`, `timescaledb-compression`) or add a new file to the `results` directory. - -### Installation And Fine-Tuning - -The systems can be installed or used in any reasonable way: from a binary distribution, from a Docker container, from the package manager, or compiled - whatever is more natural and simple or gives better results. - -It's better to use the default settings and avoid fine-tuning. Configuration changes can be applied if it is considered strictly necessary and documented. - -Fine-tuning and optimization for the benchmark are not recommended but allowed. In this case, add the results on vanilla configuration and fine-tuned configuration separately. - -### Data Loading - -The dataset is available in `CSV`, `TSV`, `JSONlines` and `Parquet` formats by the following links: - -- https://datasets.clickhouse.com/hits_compatible/hits.csv.gz -- https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz -- https://datasets.clickhouse.com/hits_compatible/hits.json.gz -- https://datasets.clickhouse.com/hits_compatible/hits.parquet - -The format of the source data can be selected up to convenience. - -Additional sources for stateless table engines are provided: -- https://datasets.clickhouse.com/hits_compatible/athena/hits.parquet (the same parquet file in its own subdirectory) -- https://datasets.clickhouse.com/hits_compatible/athena_partitioned/hits_{0..99}.parquet (100 files) - -To correctly compare the insertion time, the dataset should be downloaded and decompressed before loading (if it's using external compression; the parquet file includes internal compression and can be loaded as is). The dataset should be loaded as a single file in the most straightforward way. Splitting the dataset for parallel loading is not recommended, as it will make comparisons more difficult. Splitting the dataset is possible if the system cannot eat it as a whole due to its limitations. - -You should not wait for cool down after data loading or running OPTIMIZE / VACUUM before the main benchmark queries unless it is strictly required for the system. - -The used storage size can be measured without accounting for temporary data if there is temporary data that will be removed in the background. The built-in introspection capabilities can be used to measure the storage size, or it can be measured by checking the used space in the filesystem. - -### Indexing - -The benchmark table has one index - the primary key. The primary key is not necessary to be unique. The index of the primary key can be made clustered (ordered, partitioned, sharded). - -Manual creation of other indices is not recommended, although if the system creates indexes automatically, it is considered ok. - -### Preaggregation - -The creation of pre-aggregated tables or indices, projections, or materialized views is not recommended for the purpose of this benchmark. Although you can add results on fine-tuned setup for reference, they will be out of competition. - -If a system is of a "multidimensional OLAP" kind, so always or implicitly doing aggregations, it can be added for comparison. - -### Caching - -If the system contains a cache for query results, it should be disabled. - -It is okay if the system performs caching for source data (buffer pools and similar). If the cache or buffer pools can be flushed, it should be flushed before the first run of every query. - -If the system contains a cache for intermediate data, it should be disabled if this cache is located near the end of the query execution pipeline, thus similar to a query result cache. - -### Incomplete Results - -Many systems cannot run the full benchmark suite successfully due to OOMs, crashes, or unsupported queries. The partial results should be included nevertheless. Put `null` for the missing numbers. - -### If The Results Cannot Be Published - -Some vendors don't allow publishing the benchmark results due to the infamous [DeWitt Clause](https://cube.dev/blog/dewitt-clause-or-can-you-benchmark-a-database). Most of them are still allowed to use the system for benchmarks. In this case, please submit the full information about installation and reproduction, but without the `results` directory. A `.gitignore` file can be added to prevent accidental publishing. - -We allow both open-source and proprietary systems in our benchmark, as well as managed services, even if registration, credit card, or salesperson call is required - you still can submit the testing description if you don't violate the TOS. - -Please let us know if some results were published by mistake by opening an issue on GitHub. - -### If a Mistake Or Misrepresentation Is Found - -It is easy to accidentally misrepresent some systems. While acting in a good faith, the authors admit their lack of deep knowledge of most systems. Please send a pull request to correct the mistakes. - -### Results Usage And Scoreboards - -The results can be used for comparison of various systems, but always take them with a grain of salt due to the vast amount of caveats and hidden details. Always reference the original benchmark and this text. - -We allow but do not recommend creating scoreboards from this benchmark or tell that one system is better (faster, cheaper, etc) than another. - -There is a web page to navigate across benchmark results and present a summary report. It allows to filter out some systems, setups, or queries. E.g. if you found some subset of the 43 queries irrelevant, you can simply exclude them from the calculation and share the report without these queries. - -You can select the summary metric from one of the following: "Cold Run", "Hot Run", "Load Time", and "Data Size". If you select the "Load Time" or "Data Size", the entries will be simply ordered from best to worst, and additionally, the ratio to the best non-zero result will be shown (how times one system is worse than the best system in this metric). Load time can be zero for stateless query engines like `clickhouse-local` or `Amazon Athena`. - -If you select "Cold Run" or "Hot Run", the aggregation across the queries is performed in the following way: - -1. The first run for every query is selected for Cold Run. For Hot Run, the minimum from 2nd and 3rd run time is selected, if both runs are successful, or null if some were unsuccessful. - -By default, the "Hot Run" metric is selected, because it's not always possible to obtain a cold runtime for managed services, while for on-premise a quite slow EBS volume is used by default which makes the comparison slightly less interesting. - -2. For every query, find a system that demonstrated the best (fastest) query time and take it as a baseline. - -This gives us a point of comparison. Alternatively, we can take a benchmark entry like "ClickHouse on c6a.metal" as a baseline and divide all queries time by the times from a baseline. But it would be quite arbitrary and asymmetric. Instead, we take the best result for every query separately. - -3. For every query, if the result is present, calculate the ratio to the baseline, but add constant 10ms to the nominator and denominator, so the formula will be: `(10ms + query_time) / (10ms + baseline_query_time)`. This formula gives a value >= 1, which is equal to 1 for the best benchmark entry on this query. - -We are interested in relative query run times, not absolute. The benchmark has a broad set of queries, and there can be queries that typically run in 100ms (e.g. for interactive dashboards) and some queries that typically run in a minute (e.g. complex ad-hoc queries). And we want to treat these queries equally important in the benchmark, that's why we need relative values. - -The constant shift is needed to make the formula well-defined when query time approaches zero. E.g. some systems can get query results in 0 ms using table metadata lookup, and another in 10 ms by range scan. But this should not be treated as the infinite advantage of one system over the other. With the constant shift, we will treat it as only two times an advantage. - -4. For every query, if the result is not present, substitute it with a "penalty" calculated as follows: take the maximum query runtime for this benchmark entry across other queries that have a result, but if it is less than 300 seconds, put it 300 seconds. Then multiply the value by 2. Then calculate the ratio as explained above. - -For example, one system crashed while trying to run a query which can highlight the maturity, or lack of maturity, of a system. Or does not run a query due to limitations. If this system shows run times like 1..1000 sec. on other queries, we will substitute 2000 sec. instead of this missing result. - -5. Take the geometric mean of the ratios across the queries. It will be the summary rating. - -Why geometric mean? The ratios can only be naturally averaged in this way. Imagine there are two queries and two systems. The first system ran the first query in 1s and the second query in 20s. The second system ran the first query in 2s and the second query in 10s. So, the first system is two times faster on the first query and two times slower on the second query and vice-versa. The final score should be identical for these systems. - - -## History and Motivation - -The benchmark has been created in October 2013 for evaluating various DBMS to use for a web analytics system. It has been made by taking a 1/50th of one week of production pageviews (a.k.a. "hits") data and taking the first one billion, one hundred million, and ten million records from it. It has been run on a 3-node cluster of Xeon E2650v2 with 128 GiB RAM, 8x6TB HDD in md-RAID-6, and 10 Gbit network in a private datacenter in Finland. - -The following systems were tested in 2013: ClickHouse, MonetDB, InfiniDB, Infobright, LucidDB, Vertica, Hive and MySQL. To ensure fairness, the benchmark has been conducted by a person without ClickHouse experience. ClickHouse has been selected for production usage by the results of this benchmark. - -The benchmark continued to be occasionally used privately until 2016 when the results has been published with the ClickHouse release in open-source. While the results were made public, the datasets were not, as they contain customer data. - -We needed to publish the dataset to facilitate open-source development and testing, but it was not possible to do it as is. In 2019, the `clickhouse-obfuscator` tool has been introduced to anonymize the data, and the dataset has been published. Read more about the challenge of data obfuscation [here](https://habr.com/en/company/yandex/blog/485096/). - -More systems were included in the benchmark over time: Greenplum, MemSQL (now SingleStore), OmniSci (now HeavyAI), DuckDB, PostgreSQL, and TimescaleDB. - -In [2021](https://clickhouse.com/blog/introducing-click-house-inc/) the original cluster for benchmark stopped being used, and we were unable to add new results without rerunning the old results on different hardware. Rerunning the old results appeared to be difficult: due to the natural churn of the software, the old step-by-step instructions become stale. - -The original benchmark dataset included many details that were natural for ClickHouse and web analytics data but hard for other systems: unsigned integers (not supported by standard SQL), strings with zero bytes, fixed-length string data types, etc. Only ClickHouse is being able to load the dataset as is, while most other databases require non-trivial adjustments to the data and queries. - -The idea of the new benchmark is: -- normalize the dataset to a "common denominator", so it can be loaded to most of the systems without a hassle. -- normalize the queries to use only standard SQL - they will not use any advantages of ClickHouse but will be runnable on every system. -- ideally make it automated. At least make it simple - runnable by a short shell script that can be run by copy-pasting a few commands in the terminal, in the worst case. -- run everything on widely available cloud VMs and allow to record the results from various types of instances. - -The benchmark is created and used by the ClickHouse team. It can be surprising, but we [did not perform](https://clickhouse.com/blog/clickhouse-over-the-years-with-benchmarks/) any specific optimizations in ClickHouse for the queries in the benchmark, which allowed us to keep some reasonable sense of fairness with respect to other systems. - -Now the new benchmark is easy to use and the results for any system can be reproduced in around 20 minutes. - -We also introduced the [Hardware Benchmark](https://clickhouse.com/benchmark/hardware/) for testing servers and VMs. - -## Systems Included - -- [x] ClickHouse -- [ ] ClickHouse operating like "Athena" -- [x] clickhouse-local without data loading -- [x] MySQL InnoDB -- [x] MySQL MyISAM -- [ ] MariaDB -- [x] MariaDB ColumnStore -- [x] MemSQL/SingleStore -- [x] PostgreSQL -- [x] Greenplum -- [x] TimescaleDB -- [x] Citus -- [x] Vertica (without publishing) -- [x] QuestDB -- [x] DuckDB -- [x] MonetDB -- [x] mapD/Omnisci/HeavyAI -- [x] Databend -- [ ] Doris/PALO -- [x] Druid -- [x] Pinot -- [x] CrateDB -- [ ] Spark SQL -- [ ] Starrocks -- [ ] ShitholeDB -- [ ] Hive -- [ ] Impala -- [ ] Hyper -- [x] SQLite -- [x] Redshift -- [x] Redshift Serverless -- [ ] Presto/Trino -- [x] Amazon Athena -- [x] Bigquery (without publishing) -- [x] Snowflake -- [ ] Rockset -- [ ] CockroachDB -- [ ] CockroachDB Serverless -- [ ] Databricks -- [ ] Planetscale (without publishing) -- [ ] TiDB (TiFlash) -- [x] Amazon RDS Aurora for MySQL -- [x] Amazon RDS Aurora for Postgres -- [ ] InfluxDB -- [ ] VictoriaMetrics -- [ ] TDEngine -- [ ] MongoDB -- [ ] Cassandra -- [ ] ScyllaDB -- [ ] Elasticsearch -- [ ] Apache Ignite -- [ ] Infobright -- [ ] Actian Vector -- [ ] Manticore Search -- [x] Vertica (without publishing) -- [ ] Azure Synapse -- [ ] Starburst Galaxy -- [ ] MS SQL Server with Column Store Index (without publishing) -- [ ] Dremio (without publishing) -- [ ] Exasol -- [ ] LocustDB -- [ ] EventQL -- [ ] Apache Drill -- [ ] Apache Kudu -- [ ] Apache Kylin - -By default, all tests are run on c6a.4xlarge VM in AWS with 500 GB gp2. - -Please help us add more systems and run the benchmarks on more types of VMs. - -## Similar Projects - -Many alternative benchmarks are applicable to OLAP DBMS with their own advantages and disadvantages. - -### Brown University Mgbench - -https://github.com/crottyan/mgbench - -A new analytical benchmark for machine-generated log data. By Andrew Crottyan from Brown University. - -Advantages: -- somewhat realistic dataset; -- a diverse set of queries; -- good coverage of systems; -- easy to reproduce; - -Disadvantages: -- very small dataset size; -- favors in-memory databases; -- mostly abandoned. - -### UC Berkeley AMPLab Big Data Benchmark - -https://amplab.cs.berkeley.edu/benchmark/ - -Poor coverage of queries that are too simple. The benchmark is abandoned. - -### Mark Litwinschik's NYC Taxi - -https://tech.marksblogg.com/benchmarks.html - -Advantages: -- real-world dataset; -- good coverage of systems; many unusual entries; -- contains a story for every benchmark entry; - -Disadvantages: -- unreasonably small set of queries: 4 mostly trivial queries don't represent any realistic workload and are subjects for over-optimization; -- compares different systems on different hardware; -- many results are outdated; -- no automated or easy way to reproduce the results; -- while many results are performed independently of corporations or academia, some benchmark entries may have been sponsored; -- the dataset is not readily available for downloads: originally 1.1 billion records are used, while it's more than 4 billion records in 2022. - -### Database-like ops Benchmark from h2o.ai - -https://h2oai.github.io/db-benchmark/ - -A benchmark for data-frame libraries and embedded databases. Good coverage of data-frame libraries and a few full-featured DBMS as well. - -### A benchmark for querying large JSON datasets - -https://colab.research.google.com/github/dcmoura/spyql/blob/master/notebooks/json_benchmark.ipynb - -A good benchmark for command-line tools for processing semistructured data. Can be used to test DBMS as well. - -### Star Schema Benchmark - -Pat O'Neil, Betty O'Neil, Xuedong Chen -https://www.cs.umb.edu/~poneil/StarSchemaB.PDF - -It is a simplified version of TPC-H. - -Advantages: -- well-specified; -- popular in academia; - -Disadvantages: -- represents a classic data warehouse schema; -- database generator produces random distributions that are not realistic and the benchmark does not allow to capture the difference in various optimizations that matter on real-world data; -- many research systems in academia targeting for this benchmark which makes many aspects of it exhausted; - -### TPC-H - -A benchmark suite from Transaction Processing Council - one of the oldest organizations specializing in DBMS benchmarks. - -Advantages: -- well-specified; - -Disadvantages: -- requires official certification; -- represents a classic data warehouse schema; -- database generator produces random distributions that are not realistic and the benchmark does not allow to capture the difference in various optimizations that matter on real-world data; -- many systems are targeting this benchmark which makes many aspects of it exhausted; - -### TPC-DS - -More advanced than TPC-H, focused on complex ad-hoc queries. This also requires official certification. - -Advantages: -- an extensive collection of complex queries. - -Disadvantages: -- requires official certification; -- official results have only sparse coverage of systems; -- biased towards complex queries over many tables. - -### Ontime - -Introduced by Vadim Tkachenko from Percona [in 2009](https://www.percona.com/blog/2009/10/02/analyzing-air-traffic-performance-with-infobright-and-monetdb/). - -Based on the US Bureau of Transportation Statistics open data. - -Advantages: -- real-world dataset; - -Disadvantages: -- not widely used; -- the set of queries is not standardized; -- the table contains too much redundancy; - -### TSBS - -Time Series Benchmark Suite. https://github.com/timescale/tsbs -Originally from InfluxDB, and supported by TimescaleDB. - -Advantages: -- a benchmark for time-series scenarios; - -Disadvantages: -- not applicable for scenarios with data analytics. - -### Fair Database Benchmarks - -https://github.com/db-benchmarks/db-benchmarks - -A benchmark suite inspired by ClickHouse benchmarks. -Used mostly to compare search engines: Elasticsearch and Manticore. - -### STAC - -https://www.stacresearch.com/ - -Disadvantages: -- requires a paid membership. - -### More... - -Please let me know if you know more well-defined, realistic, and reproducible benchmarks for analytical workloads. - -In addition, I collect every benchmark that includes ClickHouse [here](https://github.com/ClickHouse/ClickHouse/issues/22398). - -## Additional Outcomes - -This benchmark can be used to collect the snippets for installation and data loading across a wide variety of DBMS. The usability and quality of the documentation can be compared. It has been used to improve the quality of the participants as demonstrated in [duckdb#3969](https://github.com/duckdb/duckdb/issues/3969), [timescaledb#4473](https://github.com/timescale/timescaledb/issues/4473), [mariadb-corporation#16](https://github.com/mariadb-corporation/mariadb-community-columnstore-docker/issues/16), [MonetDB#7309](https://github.com/duckdb/duckdb/issues/3969), [questdb#2272](https://github.com/questdb/questdb/issues/2272), [crate#12654](https://github.com/crate/crate/issues/12654), [LocustDB#152](https://github.com/cswinter/LocustDB/issues/152), etc; - -### References and Citation - -Alexey Milovidov, 2022. +Benchmark is located in a separate repository: https://github.com/ClickHouse/ClickBench diff --git a/benchmark/athena/README.md b/benchmark/athena/README.md deleted file mode 100644 index ff94029b429..00000000000 --- a/benchmark/athena/README.md +++ /dev/null @@ -1,21 +0,0 @@ -Data Sources -> AWS Data Catalog -> Create Table -> Use S3 bucket data - -Note: Athena does not support files. Only directories. - -Go to query editor and run `create.sql`. - -``` -sudo apt-get install -y jq -export OUTPUT='s3://athena-experiments-milovidov/' - -./run1.sh | tee ids.txt -``` - -Wait a few minutes. Then: - -``` -cat ids.txt | xargs -I{} aws --output json athena get-query-execution --query-execution-id {} | tee log.txt - -cat log.txt | grep -P 'TotalExecutionTimeInMillis|FAILED' | grep -oP '\d+|FAILED' | - awk '{ if ($1 == "ERROR") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : ($1 / 1000); if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' -``` diff --git a/benchmark/athena/create_partitioned.sql b/benchmark/athena/create_partitioned.sql deleted file mode 100644 index b0f9e5d0be5..00000000000 --- a/benchmark/athena/create_partitioned.sql +++ /dev/null @@ -1,112 +0,0 @@ -CREATE EXTERNAL TABLE IF NOT EXISTS `test`.`hits` ( -`watchid` bigint, -`javaenable` smallint, -`title` string, -`goodevent` smallint, -`eventtime` timestamp, -`eventdate` date, -`counterid` int, -`clientip` int, -`regionid` int, -`userid` bigint, -`counterclass` smallint, -`os` smallint, -`useragent` smallint, -`url` string, -`referer` string, -`isrefresh` smallint, -`referercategoryid` smallint, -`refererregionid` int, -`urlcategoryid` smallint, -`urlregionid` int, -`resolutionwidth` smallint, -`resolutionheight` smallint, -`resolutiondepth` smallint, -`flashmajor` smallint, -`flashminor` smallint, -`flashminor2` string, -`netmajor` smallint, -`netminor` smallint, -`useragentmajor` smallint, -`useragentminor` string, -`cookieenable` smallint, -`javascriptenable` smallint, -`ismobile` smallint, -`mobilephone` smallint, -`mobilephonemodel` string, -`params` string, -`ipnetworkid` int, -`traficsourceid` smallint, -`searchengineid` smallint, -`searchphrase` string, -`advengineid` smallint, -`isartifical` smallint, -`windowclientwidth` smallint, -`windowclientheight` smallint, -`clienttimezone` smallint, -`clienteventtime` timestamp, -`silverlightversion1` smallint, -`silverlightversion2` smallint, -`silverlightversion3` int, -`silverlightversion4` smallint, -`pagecharset` string, -`codeversion` int, -`islink` smallint, -`isdownload` smallint, -`isnotbounce` smallint, -`funiqid` bigint, -`originalurl` string, -`hid` int, -`isoldcounter` smallint, -`isevent` smallint, -`isparameter` smallint, -`dontcounthits` smallint, -`withhash` smallint, -`hitcolor` string, -`localeventtime` timestamp, -`age` smallint, -`sex` smallint, -`income` smallint, -`interests` smallint, -`robotness` smallint, -`remoteip` int, -`windowname` int, -`openername` int, -`historylength` smallint, -`browserlanguage` string, -`browsercountry` string, -`socialnetwork` string, -`socialaction` string, -`httperror` smallint, -`sendtiming` int, -`dnstiming` int, -`connecttiming` int, -`responsestarttiming` int, -`responseendtiming` int, -`fetchtiming` int, -`socialsourcenetworkid` smallint, -`socialsourcepage` string, -`paramprice` bigint, -`paramorderid` string, -`paramcurrency` string, -`paramcurrencyid` smallint, -`openstatservicename` string, -`openstatcampaignid` string, -`openstatadid` string, -`openstatsourceid` string, -`utmsource` string, -`utmmedium` string, -`utmcampaign` string, -`utmcontent` string, -`utmterm` string, -`fromtag` string, -`hasgclid` smallint, -`refererhash` bigint, -`urlhash` bigint, -`clid` int -) -ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' -WITH SERDEPROPERTIES ( -'serialization.format' = '1' -) LOCATION 's3://clickhouse-public-datasets/hits_compatible/athena_partitioned' -TBLPROPERTIES ('has_encrypted_data'='false'); diff --git a/benchmark/athena/create_single.sql b/benchmark/athena/create_single.sql deleted file mode 100644 index ef8e12a9915..00000000000 --- a/benchmark/athena/create_single.sql +++ /dev/null @@ -1,112 +0,0 @@ -CREATE EXTERNAL TABLE IF NOT EXISTS `test`.`hits` ( -`watchid` bigint, -`javaenable` smallint, -`title` string, -`goodevent` smallint, -`eventtime` timestamp, -`eventdate` date, -`counterid` int, -`clientip` int, -`regionid` int, -`userid` bigint, -`counterclass` smallint, -`os` smallint, -`useragent` smallint, -`url` string, -`referer` string, -`isrefresh` smallint, -`referercategoryid` smallint, -`refererregionid` int, -`urlcategoryid` smallint, -`urlregionid` int, -`resolutionwidth` smallint, -`resolutionheight` smallint, -`resolutiondepth` smallint, -`flashmajor` smallint, -`flashminor` smallint, -`flashminor2` string, -`netmajor` smallint, -`netminor` smallint, -`useragentmajor` smallint, -`useragentminor` string, -`cookieenable` smallint, -`javascriptenable` smallint, -`ismobile` smallint, -`mobilephone` smallint, -`mobilephonemodel` string, -`params` string, -`ipnetworkid` int, -`traficsourceid` smallint, -`searchengineid` smallint, -`searchphrase` string, -`advengineid` smallint, -`isartifical` smallint, -`windowclientwidth` smallint, -`windowclientheight` smallint, -`clienttimezone` smallint, -`clienteventtime` timestamp, -`silverlightversion1` smallint, -`silverlightversion2` smallint, -`silverlightversion3` int, -`silverlightversion4` smallint, -`pagecharset` string, -`codeversion` int, -`islink` smallint, -`isdownload` smallint, -`isnotbounce` smallint, -`funiqid` bigint, -`originalurl` string, -`hid` int, -`isoldcounter` smallint, -`isevent` smallint, -`isparameter` smallint, -`dontcounthits` smallint, -`withhash` smallint, -`hitcolor` string, -`localeventtime` timestamp, -`age` smallint, -`sex` smallint, -`income` smallint, -`interests` smallint, -`robotness` smallint, -`remoteip` int, -`windowname` int, -`openername` int, -`historylength` smallint, -`browserlanguage` string, -`browsercountry` string, -`socialnetwork` string, -`socialaction` string, -`httperror` smallint, -`sendtiming` int, -`dnstiming` int, -`connecttiming` int, -`responsestarttiming` int, -`responseendtiming` int, -`fetchtiming` int, -`socialsourcenetworkid` smallint, -`socialsourcepage` string, -`paramprice` bigint, -`paramorderid` string, -`paramcurrency` string, -`paramcurrencyid` smallint, -`openstatservicename` string, -`openstatcampaignid` string, -`openstatadid` string, -`openstatsourceid` string, -`utmsource` string, -`utmmedium` string, -`utmcampaign` string, -`utmcontent` string, -`utmterm` string, -`fromtag` string, -`hasgclid` smallint, -`refererhash` bigint, -`urlhash` bigint, -`clid` int -) -ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' -WITH SERDEPROPERTIES ( -'serialization.format' = '1' -) LOCATION 's3://clickhouse-public-datasets/hits_compatible/athena' -TBLPROPERTIES ('has_encrypted_data'='false'); diff --git a/benchmark/athena/queries.sql b/benchmark/athena/queries.sql deleted file mode 100644 index ef8c727ead6..00000000000 --- a/benchmark/athena/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, extract(minute FROM EventTime), SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC OFFSET 1000 LIMIT 10; -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 >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN (SearchEngineID = 0 AND AdvEngineID = 0) THEN Referer ELSE '' END, URL ORDER BY PageViews DESC OFFSET 1000 LIMIT 10; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC OFFSET 100 LIMIT 10; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC OFFSET 10000 LIMIT 10; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-14' AND EventDate <= DATE '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) OFFSET 1000 LIMIT 10; diff --git a/benchmark/athena/results/partitioned.json b/benchmark/athena/results/partitioned.json deleted file mode 100644 index 8a67c09bb47..00000000000 --- a/benchmark/athena/results/partitioned.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Athena (partitioned)", - "date": "2022-07-01", - "machine": "serverless", - "cluster_size": "serverless", - "comment": "", - - "tags": ["stateless", "managed", "Java", "column-oriented"], - - "load_time": 0, - "data_size": 13800000000, - - "result": [ -[2.777,3.275,2.925], -[1.503,3.136,4.003], -[4.544,3.833,3.64], -[3.9,2.514,3.522], -[3.46,2.186,3.244], -[3.624,2.742,3.185], -[2.21,1.984,3.123], -[3.207,2.403,2.685], -[2.936,2.014,3.869], -[8.333,7.102,4.434], -[7.401,4.697,3.155], -[4.214,3.065,4.748], -[6.207,4.213,2.576], -[3.428,3.085,3.401], -[2.92,3.3,3.278], -[2.205,2.558,2.419], -[4.641,3.888,2.155], -[3.219,2.822,3.292], -[3.23,3.579,4.31], -[2.288,3.543,3.95], -[3.032,2.859,2.807], -[3.926,3.247,2.928], -[4.477,4.048,4.392], -[7.407,6.375,6.123], -[2.611,2.872,2.827], -[2.566,2.567,3.6], -[3.673,3.733,2.925], -[2.426,3.218,2.78], -[5.125,3.778,4.25], -[4.565,4.03,4.066], -[3.628,3.219,2.953], -[6.207,5.973,3.158], -[4.339,5.601,4.234], -[2.618,3.107,3.433], -[4.661,2.79,2.846], -[2.373,1.629,2.734], -[2.721,2.15,1.962], -[3.207,2.154,2.186], -[2.453,2.477,3.217], -[2.691,4.732,3.584], -[2.589,2.613,3.231], -[1.926,3.617,1.82], -[1.506,2.404,2.343] -] -} diff --git a/benchmark/athena/results/single.json b/benchmark/athena/results/single.json deleted file mode 100644 index d2b84187f11..00000000000 --- a/benchmark/athena/results/single.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Athena (single)", - "date": "2022-07-01", - "machine": "serverless", - "cluster_size": "serverless", - "comment": "", - - "tags": ["stateless", "managed", "Java", "column-oriented"], - - "load_time": 0, - "data_size": 13800000000, - - "result": [ -[2.268,1.327,2.137], -[3.427,2.248,3.605], -[3.254,2.548,2.316], -[3.025,2.314,3.003], -[2.264,2.876,4.213], -[3.044,2.745,2.698], -[2.732,2.199,2.659], -[2.022,3.692,3.072], -[2.746,2.477,2.785], -[3.53,2.782,4.031], -[2.709,2.047,2.853], -[2.318,1.969,3.4], -[2.635,1.935,2.707], -[3.049,3.38,3.071], -[3.661,2.387,2.476], -[2.479,2.591,2.21], -[3.093,3.698,4.351], -[3.479,3.236,2.274], -[4.36,2.97,3.457], -[2.525,2.384,3.328], -[3.34,3.174,3.409], -[3.163,2.971,3.034], -[2.999,3.539,2.906], -[6.454,7.597,7.858], -[2.754,1.951,2.645], -[2.852,3.018,2.718], -[2.513,2.678,2.417], -[3.293,2.521,2.771], -[4.392,3.863,3.981], -[3.658,4.246,4.027], -[3.028,3.87,2.337], -[2.923,3.635,3.591], -[3.142,4.105,3.15], -[3.66,3.187,4.745], -[2.652,2.695,2.742], -[2.262,2.776,1.815], -[1.881,2.212,2.053], -[1.934,2.551,1.524], -[2.069,2.26,1.805], -[2.626,2.902,2.793], -[1.791,2.082,2.481], -[3.757,2.6,1.946], -[2.608,1.994,3.967] -] -} diff --git a/benchmark/athena/run.sh b/benchmark/athena/run.sh deleted file mode 100755 index f1ce446f0a8..00000000000 --- a/benchmark/athena/run.sh +++ /dev/null @@ -1,9 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - for i in $(seq 1 $TRIES); do - aws athena --output json start-query-execution --query-execution-context 'Database=test' --result-configuration "OutputLocation=${OUTPUT}" --query-string "${query}" | jq '.QueryExecutionId' - done -done diff --git a/benchmark/aurora-mysql/README.md b/benchmark/aurora-mysql/README.md deleted file mode 100644 index 48faddfa2b3..00000000000 --- a/benchmark/aurora-mysql/README.md +++ /dev/null @@ -1,67 +0,0 @@ -Select Aurora. -Select Aurora for MySQL. -Select the latest version 3.02.0 (compatible with MySQL 8.0.23) -Select Production template. - -Database: database-1 -User name: admin -Master password: vci43A32#1 - -Select serverless. -16 minimum and maximum ACU (32 GB RAM). -Don't create an Aurora replica. -Public access: yes. -Turn off DevOps Guru. - -Creation took around 15 seconds. -But creation of endpoints took longer. - -Find the writer instance endpoint. -Example: database-1.cluster-cnkeohbxcwr1.eu-central-1.rds.amazonaws.com - -``` -sudo apt-get update -sudo apt-get install -y mysql-client -``` - -Find "Security", click on the group in "VPC security groups". -Edit "Inbound rules". Add "Custom TCP", port 3306, from 0.0.0.0/0. - -``` -export HOST="database-1.cluster-cnkeohbxcwr1.eu-central-1.rds.amazonaws.com" -export PASSWORD="..." - -mysql -h "${HOST}" -u admin --password="${PASSWORD}" -e "CREATE DATABASE test" -``` - -Load the data - -``` -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -mysql -h "${HOST}" -u admin --password="${PASSWORD}" test < create.sql - -time mysql --local-infile=1 -h "${HOST}" -u admin --password="${PASSWORD}" test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" -``` - -> 128m7.318s - -Go to "Monitoring", find "[Billed] Volume Bytes Used". - -> 83.46 GiB - -``` -./run.sh 2>&1 | tee log.txt - -cat log.txt | - grep -P 'rows? in set|Empty set|^ERROR' | - sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | - awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' -``` - -You will get -> ERROR 1114 (HY000) at line 1: The table '/rdsdbdata/tmp/#sqlaff_e5_0' is full - -to some queries. diff --git a/benchmark/aurora-mysql/create.sql b/benchmark/aurora-mysql/create.sql deleted file mode 100644 index 1850bffedce..00000000000 --- a/benchmark/aurora-mysql/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/aurora-mysql/queries.sql b/benchmark/aurora-mysql/queries.sql deleted file mode 100644 index 3f04f648222..00000000000 --- a/benchmark/aurora-mysql/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, extract(minute FROM EventTime), SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/aurora-mysql/results/16acu.json b/benchmark/aurora-mysql/results/16acu.json deleted file mode 100644 index ef00d213842..00000000000 --- a/benchmark/aurora-mysql/results/16acu.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Aurora for MySQL", - "date": "2022-07-01", - "machine": "16acu", - "cluster_size": 1, - "comment": "Some queries cannot run due to ERROR 1114 (HY000) at line 1: The table '/rdsdbdata/tmp/#sqlaff_e5_0' is full", - - "tags": ["managed", "C++", "MySQL compatible", "row-oriented"], - - "load_time": 7687.318, - "data_size": 89614492631, - - "result": [ -[740.42,739.91,746.65], -[828.2,835.67,832.87], -[830.08,830.98,832.38], -[829.88,832.83,830.87], -[845.99,842.4,843.21], -[869.51,870.69,869.75], -[823.77,829.08,825.54], -[827.74,832.87,829.25], -[916.26,909.46,929.17], -[946.49,939.27,932.32], -[852.37,857.69,854.74], -[857.99,864.05,825.14], -[null,null,null], -[863.37,860.2,865.62], -[null,null,null], -[891.84,895.28,893.68], -[null,null,null], -[null,null,null], -[1420.12,1419.34,1445.08], -[28.94,0.21,0.21], -[917.64,917.56,916.92], -[923.47,921.7,923.82], -[919.95,918.37,920.17], -[1002.19,1002.07,1001.2], -[902.23,902.65,901.8], -[901.17,900.02,898.3], -[900.04,898.89,903.35], -[901.78,902.71,901.28], -[null,null,null], -[1153.29,1154,1156.46], -[862.57,863.35,859.69], -[923.14,921.1,923.92], -[1370.78,1401.72,1401.44], -[1454.67,1455.55,1458.79], -[1463.31,1466.75,1461.83], -[941.03,944.07,937.23], -[7.42,2.80,2.77], -[2.57,2.52,2.59], -[1.50,1.52,1.59], -[3.62,3.57,3.61], -[0.95,0.94,0.94], -[0.90,0.92,0.91], -[1.69,1.72,1.69] -] -} diff --git a/benchmark/aurora-mysql/run.sh b/benchmark/aurora-mysql/run.sh deleted file mode 100755 index c6e2bedd27b..00000000000 --- a/benchmark/aurora-mysql/run.sh +++ /dev/null @@ -1,9 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - for i in $(seq 1 $TRIES); do - mysql -h "${HOST}" -u admin --password="${PASSWORD}" test -vvv -e "${query}" - done; -done; diff --git a/benchmark/aurora-postgresql/README.md b/benchmark/aurora-postgresql/README.md deleted file mode 100644 index 906258f65c5..00000000000 --- a/benchmark/aurora-postgresql/README.md +++ /dev/null @@ -1,59 +0,0 @@ -Select Aurora. -Select Aurora for PostgreSQL. -Select the latest version PostgreSQL 14.3. -Select Production template. - -Database: database-2 -User name: postgres -Master password: vci43A32#1 - -Select serverless. -16 minimum and maximum ACU (32 GB RAM). -Don't create an Aurora replica. -Public access: yes. -Turn off DevOps Guru. - -Creation took around 15 seconds. -But creation of endpoints took longer (around 5..10 minutes). - -Find the writer instance endpoint. -Example: database-1.cluster-cnkeohbxcwr1.eu-central-1.rds.amazonaws.com - -``` -sudo apt-get update -sudo apt-get install -y postgresql-client -``` - -Find "Security", click on the group in "VPC security groups". -Edit "Inbound rules". Add "Custom TCP", port 5432, from 0.0.0.0/0. - -``` -export HOST="database-2-instance-1.cnkeohbxcwr1.eu-central-1.rds.amazonaws.com" -echo "*:*:*:*:..." > .pgpass -chmod 400 .pgpass -``` - -Load the data - -``` -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -psql -U postgres -h "${HOST}" -t -c 'CREATE DATABASE test' -psql -U postgres -h "${HOST}" test -t < create.sql -psql -U postgres -h "${HOST}" test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" -``` - -> COPY 99997497 -> Time: 2126515.516 ms (35:26.516) - -Go to "Monitoring", find "[Billed] Volume Bytes Used". - -> 48.6 GiB - -``` -./run.sh 2>&1 | tee log.txt - -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' -``` diff --git a/benchmark/aurora-postgresql/create.sql b/benchmark/aurora-postgresql/create.sql deleted file mode 100644 index 1850bffedce..00000000000 --- a/benchmark/aurora-postgresql/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/aurora-postgresql/queries.sql b/benchmark/aurora-postgresql/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/aurora-postgresql/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/aurora-postgresql/results/16acu.json b/benchmark/aurora-postgresql/results/16acu.json deleted file mode 100644 index 194329d7f4f..00000000000 --- a/benchmark/aurora-postgresql/results/16acu.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Aurora for PostgreSQL", - "date": "2022-07-01", - "machine": "16acu", - "cluster_size": 1, - "comment": "", - - "tags": ["managed", "C", "PostgreSQL compatible", "row-oriented"], - - "load_time": 2127, - "data_size": 52183852646, - - "result": [ -[12.8361,5.71812,5.8241], -[61.2565,62.1402,63.7173], -[68.0578,68.1218,67.609], -[7.83207,5.90193,6.0461], -[48.7194,48.0233,48.2198], -[289.492,304.639,282.436], -[6.30572,6.31857,6.21598], -[53.644,53.8931,53.5307], -[131.526,131.45,131.102], -[137.724,136.921,137.758], -[57.2079,56.2775,56.2152], -[56.5349,56.2048,55.9569], -[82.3897,82.8866,83.534], -[97.0569,97.1392,96.4731], -[85.6557,86.7783,86.2804], -[49.4325,42.4309,42.5743], -[111.537,114.59,111.807], -[88.4322,89.3756,87.7899], -[160.781,163.866,161.394], -[1025.04,2.10165,2.10065], -[106.741,56.2731,56.1535], -[59.2681,59.5272,59.536], -[58.6083,57.6054,57.3935], -[54.8271,55.1397,56.3487], -[54.718,52.469,53.271], -[53.5387,53.1926,52.4008], -[52.0042,51.9581,52.2453], -[60.1317,59.9695,59.2187], -[244.608,242.954,243.815], -[91.8674,92.4165,91.5884], -[63.7122,64.277,64.2783], -[69.2596,68.9535,69.4508], -[234.222,241.138,240.316], -[488.169,462.257,460.466], -[472.929,471.809,476.635], -[103.664,116.131,103.467], -[16.8124,3.34058,3.37782], -[0.852414,0.832073,0.859857], -[0.305464,0.31166,0.306694], -[4.55625,4.54098,4.58501], -[0.299746,0.297532,0.30334], -[0.275732,0.279817,0.27766], -[0.332107,0.324387,0.320099] -] -} diff --git a/benchmark/aurora-postgresql/run.sh b/benchmark/aurora-postgresql/run.sh deleted file mode 100755 index f2c694359f1..00000000000 --- a/benchmark/aurora-postgresql/run.sh +++ /dev/null @@ -1,10 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - echo "$query"; - for i in $(seq 1 $TRIES); do - psql -U postgres -h "${HOST}" test -t -c '\timing' -c "$query" | grep 'Time' - done; -done; diff --git a/benchmark/bigquery/.gitignore b/benchmark/bigquery/.gitignore deleted file mode 100644 index 1a06816d838..00000000000 --- a/benchmark/bigquery/.gitignore +++ /dev/null @@ -1 +0,0 @@ -results diff --git a/benchmark/bigquery/README.md b/benchmark/bigquery/README.md deleted file mode 100644 index 3563fd7a87b..00000000000 --- a/benchmark/bigquery/README.md +++ /dev/null @@ -1,38 +0,0 @@ -BigQuery has "DeWitt" clause that restricts from disclosing the benchmark results. -Nevertheless, it does not prevent from doing the benchmarks. - -It's very difficult to find, how to create a database. -Databases are named "datasets". You need to press on `⋮` near project. - -Create dataset `test`. -Go to the query editor and paste the contents of `create.sql`. -It will take two seconds to create a table. - -Download Google Cloud CLI: -``` -curl -O https://dl.google.com/dl/cloudsdk/channels/rapid/downloads/google-cloud-cli-392.0.0-linux-x86_64.tar.gz -tar -xf google-cloud-cli-392.0.0-linux-x86_64.tar.gz -./google-cloud-sdk/install.sh -source .bashrc -./google-cloud-sdk/bin/gcloud init -``` - -Load the data: -``` -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' -gzip -d hits.csv.gz - -time bq load --source_format CSV --allow_quoted_newlines=1 test.hits hits.csv -``` - -Run the benchmark: - -``` -./run.sh 2>&1 | tee log.txt - -cat log.txt | - grep -P '^real|^Error' | - sed -r -e 's/^Error.*$/null/; s/^real\s*([0-9.]+)m([0-9.]+)s$/\1 \2/' | - awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | - awk '{ if ($1 == "null") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' -``` diff --git a/benchmark/bigquery/create.sql b/benchmark/bigquery/create.sql deleted file mode 100644 index 9012df89a77..00000000000 --- a/benchmark/bigquery/create.sql +++ /dev/null @@ -1,108 +0,0 @@ -CREATE TABLE test.hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title String NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL String NOT NULL, - Referer String NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 String NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor String NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel String NOT NULL, - Params String NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase String NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset String NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL String NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor String NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage String NOT NULL, - BrowserCountry String NOT NULL, - SocialNetwork String NOT NULL, - SocialAction String NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage String NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID String NOT NULL, - ParamCurrency String NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName String NOT NULL, - OpenstatCampaignID String NOT NULL, - OpenstatAdID String NOT NULL, - OpenstatSourceID String NOT NULL, - UTMSource String NOT NULL, - UTMMedium String NOT NULL, - UTMCampaign String NOT NULL, - UTMContent String NOT NULL, - UTMTerm String NOT NULL, - FromTag String NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -); diff --git a/benchmark/bigquery/queries.sql b/benchmark/bigquery/queries.sql deleted file mode 100644 index 3dc8f405aaa..00000000000 --- a/benchmark/bigquery/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM test.hits; -SELECT COUNT(*) FROM test.hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM test.hits; -SELECT AVG(UserID) FROM test.hits; -SELECT COUNT(DISTINCT UserID) FROM test.hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM test.hits; -SELECT MIN(EventDate), MAX(EventDate) FROM test.hits; -SELECT AdvEngineID, COUNT(*) FROM test.hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM test.hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM test.hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM test.hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM test.hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM test.hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM test.hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM test.hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM test.hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM test.hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM test.hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM test.hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM test.hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM test.hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM test.hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM test.hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM test.hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM test.hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM test.hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM test.hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM test.hits WHERE URL <> '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM test.hits 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 test.hits; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM test.hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM test.hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM test.hits GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC(EventTime, MINUTE) AS M, COUNT(*) AS PageViews FROM test.hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY M ORDER BY M LIMIT 10 OFFSET 1000; diff --git a/benchmark/bigquery/run.sh b/benchmark/bigquery/run.sh deleted file mode 100755 index 1a48f9a1c9b..00000000000 --- a/benchmark/bigquery/run.sh +++ /dev/null @@ -1,10 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - echo "$query"; - for i in $(seq 1 $TRIES); do - time bq query --use_legacy_sql=false --use_cache=false <<< "$query" - done -done diff --git a/benchmark/brytlytdb/README.md b/benchmark/brytlytdb/README.md deleted file mode 100644 index f0b4e044940..00000000000 --- a/benchmark/brytlytdb/README.md +++ /dev/null @@ -1 +0,0 @@ -An attempt to use their service resulted in a failure. It showed "Error: cannot create connection" shortly after registration and advised to ask for support. I emailed to support, and they assured that the problem will be resolved soon. diff --git a/benchmark/citus/benchmark.sh b/benchmark/citus/benchmark.sh deleted file mode 100755 index c7b71b367c4..00000000000 --- a/benchmark/citus/benchmark.sh +++ /dev/null @@ -1,27 +0,0 @@ -#!/bin/bash - -sudo apt-get update -sudo apt-get install -y docker.io -sudo apt-get install -y postgresql-client - -sudo docker run -d --name citus -p 5432:5432 -e POSTGRES_PASSWORD=mypass citusdata/citus:11.0 - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -echo "*:*:*:*:mypass" > .pgpass -chmod 400 .pgpass - -psql -U postgres -h localhost -d postgres --no-password -t -c 'CREATE DATABASE test' -psql -U postgres -h localhost -d postgres --no-password test -t < create.sql -psql -U postgres -h localhost -d postgres --no-password test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" - -# COPY 99997497 -# Time: 1579203.482 ms (26:19.203) - -./run.sh 2>&1 | tee log.txt - -sudo docker exec -it citus du -bcs /var/lib/postgresql/data - -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/citus/create.sql b/benchmark/citus/create.sql deleted file mode 100644 index ab013c71fd8..00000000000 --- a/benchmark/citus/create.sql +++ /dev/null @@ -1,110 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -) -USING COLUMNAR; diff --git a/benchmark/citus/queries.sql b/benchmark/citus/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/citus/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/citus/results/c6a.4xlarge.json b/benchmark/citus/results/c6a.4xlarge.json deleted file mode 100644 index 23d170aa869..00000000000 --- a/benchmark/citus/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Citus", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C", "PostgreSQL compatible", "column-oriented"], - - "load_time": 1579, - "data_size": 18980918899, - - "result": [ -[7.58503,6.70447,6.52499], -[6.33941,5.06063,5.00238], -[11.7488,9.86417,9.93223], -[12.6306,9.36305,9.17061], -[40.6101,39.0803,38.1187], -[117.654,113.912,113.441], -[10.3404,8.08936,7.70732], -[6.31542,4.72821,4.72989], -[82.5425,77.2124,76.9219], -[91.1776,83.4492,82.4727], -[14.5474,10.0815,10.3873], -[15.4899,11.2922,11.1877], -[19.9794,15.5002,17.4492], -[76.9216,72.5172,72.7915], -[21.5446,17.5691,18.561], -[56.9438,54.6387,53.5745], -[75.0977,69.7842,70.0259], -[31.3299,27.0267,26.3216], -[129.417,122.956,121.182], -[3.73386,2.14148,2.12737], -[34.6021,27.9727,28.6878], -[37.152,29.6193,29.2966], -[52.2157,37.8589,37.6994], -[181.955,149.08,148.471], -[15.4687,11.3138,10.3856], -[10.2779,8.46868,8.8324], -[14.4687,10.4076,11.4263], -[47.009,40.2969,39.6888], -[749.946,742.979,744.461], -[69.4383,67.5636,67.2128], -[27.0317,21.4008,20.9524], -[36.6675,25.6347,26.4408], -[140.424,130.546,129.738], -[106.959,92.033,90.1609], -[110.98,94.4787,96.2656], -[64.4474,60.1853,60.6816], -[6.17549,6.25376,5.87004], -[1.99153,1.81776,1.80596], -[1.00141,0.800271,0.801975], -[7.91778,7.70928,8.33299], -[0.929845,0.642076,0.638478], -[0.866536,0.683567,0.680218], -[0.937823,0.784747,0.765929] -] -} diff --git a/benchmark/citus/run.sh b/benchmark/citus/run.sh deleted file mode 100755 index 7adee7c460d..00000000000 --- a/benchmark/citus/run.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - echo "$query"; - for i in $(seq 1 $TRIES); do - psql -U postgres -h localhost -d postgres --no-password -t -c '\timing' -c "$query" | grep 'Time' - done; -done; diff --git a/benchmark/clickhouse-local/benchmark.sh b/benchmark/clickhouse-local/benchmark.sh deleted file mode 100755 index 9b660cc2cef..00000000000 --- a/benchmark/clickhouse-local/benchmark.sh +++ /dev/null @@ -1,14 +0,0 @@ -#!/bin/bash - -# Install - -curl https://clickhouse.com/ | sh - -# wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.parquet' -seq 0 99 | xargs -P100 -I{} bash -c 'wget --continue https://datasets.clickhouse.com/hits_compatible/athena_partitioned/hits_{}.parquet' - -# Run the queries - -./run.sh - -du -b hits.parquet diff --git a/benchmark/clickhouse-local/create.sql b/benchmark/clickhouse-local/create.sql deleted file mode 100644 index 180a5806e46..00000000000 --- a/benchmark/clickhouse-local/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -) -ENGINE = File(Parquet, 'hits_*.parquet'); diff --git a/benchmark/clickhouse-local/queries.sql b/benchmark/clickhouse-local/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/clickhouse-local/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json b/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json deleted file mode 100644 index c44114d12c1..00000000000 --- a/benchmark/clickhouse-local/results/c6a.4xlarge.partitioned.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "clickhouse-local (partitioned)", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], - - "load_time": 0, - "data_size": 14737666736, - - "result": [ -[0.850, 0.102, 0.114], -[1.342, 0.090, 0.099], -[2.547, 0.179, 0.189], -[1.681, 0.245, 0.252], -[2.704, 1.680, 1.648], -[2.194, 1.460, 1.489], -[0.832, 0.113, 0.102], -[1.371, 0.106, 0.101], -[2.240, 0.790, 0.825], -[4.548, 1.021, 1.026], -[3.094, 0.552, 0.552], -[3.088, 0.623, 0.630], -[2.017, 1.170, 1.165], -[4.319, 1.677, 1.708], -[2.157, 1.496, 1.500], -[1.629, 1.138, 1.139], -[5.026, 3.267, 3.241], -[4.142, 2.303, 2.319], -[8.295, 5.569, 5.629], -[1.331, 0.255, 0.252], -[10.712, 3.668, 3.786], -[13.053, 4.185, 4.202], -[24.170, 7.935, 8.008], -[55.965, 23.933, 23.071], -[4.417, 0.947, 0.974], -[1.793, 0.698, 0.690], -[4.376, 0.955, 0.956], -[11.731, 4.385, 4.321], -[11.403, 8.549, 8.288], -[2.764, 2.754, 2.735], -[5.096, 1.262, 1.273], -[9.515, 1.682, 1.688], -[10.325, 6.745, 6.608], -[11.686, 6.261, 6.242], -[11.769, 6.301, 6.364], -[1.675, 1.490, 1.495], -[14.937, 3.631, 3.604], -[14.187, 3.609, 3.631], -[14.842, 3.769, 3.741], -[22.222, 6.355, 6.263], -[7.212, 0.836, 0.838], -[7.863, 0.716, 0.718], -[5.120, 0.587, 0.574] -] -} diff --git a/benchmark/clickhouse-local/results/c6a.4xlarge.single.json b/benchmark/clickhouse-local/results/c6a.4xlarge.single.json deleted file mode 100644 index 8d19a6d0ce4..00000000000 --- a/benchmark/clickhouse-local/results/c6a.4xlarge.single.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "clickhouse-local (single)", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C++", "column-oriented", "embedded", "stateless", "ClickHouse derivative"], - - "load_time": 0, - "data_size": 14779976446, - - "result": [ -[1.176, 0.251, 0.249], -[1.037, 0.134, 0.111], -[1.609, 0.403, 0.369], -[1.616, 0.372, 0.370], -[3.008, 2.338, 2.266], -[9.061, 7.537, 7.535], -[1.206, 0.191, 0.187], -[0.882, 0.144, 0.135], -[4.610, 3.406, 3.256], -[6.712, 4.479, 4.469], -[4.081, 2.413, 2.394], -[3.894, 2.719, 2.691], -[7.651, 6.436, 6.243], -[10.765, 8.043, 7.894], -[9.860, 8.945, 8.235], -[7.159, 5.815, 5.814], -[20.916, 18.159, 18.013], -[20.952, 17.862, 17.850], -[37.585, 32.649, 32.487], -[1.767, 0.401, 0.393], -[23.713, 15.687, 15.755], -[28.700, 19.241, 19.198], -[50.740, 33.161, 33.011], -[152.485, 117.417, 118.178], -[7.606, 4.491, 5.326], -[4.331, 4.214, 3.587], -[6.743, 4.486, 5.357], -[22.910, 15.043, 15.183], -[43.342, 37.167, 36.842], -[11.807, 4.490, 4.546], -[9.557, 6.349, 6.263], -[13.964, 8.493, 8.464], -[38.110, 33.642, 33.996], -[41.266, 35.080, 27.073], -[34.056, 26.814, 26.902], -[8.855, 7.548, 7.475], -[22.596, 12.615, 12.669], -[23.217, 13.956, 13.831], -[22.528, 21.601, 13.207], -[37.890, 23.115, 22.955], -[6.490, 1.548, 1.522], -[6.413, 1.474, 1.416], -[3.796, 1.339, 1.316] -] -} diff --git a/benchmark/clickhouse-local/run.sh b/benchmark/clickhouse-local/run.sh deleted file mode 100755 index 6c6980ec68a..00000000000 --- a/benchmark/clickhouse-local/run.sh +++ /dev/null @@ -1,20 +0,0 @@ -#!/bin/bash - -TRIES=3 -QUERY_NUM=1 -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - - echo -n "[" - for i in $(seq 1 $TRIES); do - RES=$(./clickhouse local --time --format Null --multiquery --query="$(cat create.sql); $query" 2>&1 | tail -n1) - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - - echo "${QUERY_NUM},${i},${RES}" >> result.csv - done - echo "]," - - QUERY_NUM=$((QUERY_NUM + 1)) -done diff --git a/benchmark/clickhouse/benchmark.sh b/benchmark/clickhouse/benchmark.sh deleted file mode 100755 index 2c06df90164..00000000000 --- a/benchmark/clickhouse/benchmark.sh +++ /dev/null @@ -1,22 +0,0 @@ -#!/bin/bash - -# Install - -curl https://clickhouse.com/ | sh -sudo DEBIAN_FRONTEND=noninteractive ./clickhouse install -sudo clickhouse start - -# Load the data - -clickhouse-client < create.sql - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -clickhouse-client --time --query "INSERT INTO hits FORMAT TSV" < hits.tsv - -# Run the queries - -./run.sh - -clickhouse-client --query "SELECT total_bytes FROM system.tables WHERE name = 'hits' AND database = 'default'" diff --git a/benchmark/clickhouse/create.sql b/benchmark/clickhouse/create.sql deleted file mode 100644 index 0297e914a19..00000000000 --- a/benchmark/clickhouse/create.sql +++ /dev/null @@ -1,110 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -) -ENGINE = MergeTree; diff --git a/benchmark/clickhouse/queries.sql b/benchmark/clickhouse/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/clickhouse/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/clickhouse/results/c6a.4xlarge.json b/benchmark/clickhouse/results/c6a.4xlarge.json deleted file mode 100644 index 4bc9508ea6e..00000000000 --- a/benchmark/clickhouse/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "ClickHouse", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C++", "column-oriented", "ClickHouse derivative"], - - "load_time": 475.529, - "data_size": 14345515782, - - "result": [ -[0.027, 0.001, 0.001], -[0.035, 0.015, 0.021], -[0.083, 0.034, 0.033], -[0.171, 0.044, 0.045], -[1.552, 1.495, 1.574], -[1.270, 1.075, 1.063], -[0.045, 0.026, 0.025], -[0.032, 0.016, 0.015], -[0.717, 0.615, 0.607], -[0.843, 0.821, 0.747], -[0.293, 0.219, 0.216], -[0.312, 0.226, 0.235], -[0.804, 0.694, 0.702], -[1.476, 1.047, 1.029], -[1.013, 0.898, 0.911], -[1.043, 0.964, 1.453], -[3.632, 2.715, 2.711], -[1.867, 1.750, 1.714], -[5.187, 4.797, 4.953], -[0.112, 0.068, 0.041], -[8.637, 1.761, 1.212], -[9.902, 0.902, 0.869], -[18.831, 2.067, 1.829], -[41.903, 4.476, 3.486], -[1.801, 0.254, 0.238], -[0.627, 0.214, 0.207], -[2.181, 0.241, 0.246], -[8.868, 0.748, 0.733], -[9.674, 6.891, 5.770], -[2.620, 2.355, 2.368], -[1.395, 0.533, 0.525], -[4.454, 0.730, 0.712], -[5.453, 4.990, 5.922], -[9.955, 3.968, 4.096], -[9.987, 4.035, 4.476], -[1.695, 1.236, 1.241], -[0.142, 0.079, 0.103], -[0.066, 0.033, 0.040], -[0.065, 0.030, 0.033], -[0.246, 0.207, 0.192], -[0.044, 0.019, 0.020], -[0.030, 0.023, 0.012], -[0.030, 0.018, 0.013] -] -} diff --git a/benchmark/clickhouse/results/c6a.metal.json b/benchmark/clickhouse/results/c6a.metal.json deleted file mode 100644 index 92727d881b8..00000000000 --- a/benchmark/clickhouse/results/c6a.metal.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "ClickHouse", - "date": "2022-07-01", - "machine": "c6a.metal, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C++", "column-oriented", "ClickHouse derivative"], - - "load_time": 136.869, - "data_size": 14571706777, - - "result": [ -[0.011, 0.001, 0.001], -[0.040, 0.015, 0.013], -[0.045, 0.021, 0.023], -[0.090, 0.023, 0.023], -[1.922, 1.565, 1.576], -[0.961, 0.737, 0.739], -[0.040, 0.023, 0.018], -[0.032, 0.028, 0.028], -[0.321, 0.287, 0.275], -[0.632, 0.284, 0.287], -[0.166, 0.124, 0.118], -[0.235, 0.100, 0.102], -[1.006, 0.182, 0.159], -[1.637, 0.216, 0.213], -[0.871, 0.174, 0.177], -[0.258, 0.148, 0.148], -[1.804, 0.370, 0.358], -[1.235, 0.275, 0.278], -[3.143, 0.854, 0.815], -[0.071, 0.024, 0.016], -[8.816, 0.215, 0.155], -[10.239, 0.203, 0.173], -[19.179, 0.388, 0.357], -[43.152, 0.824, 0.823], -[1.821, 0.059, 0.052], -[0.992, 0.045, 0.051], -[2.539, 0.063, 0.058], -[9.258, 0.300, 0.278], -[7.923, 0.961, 0.936], -[0.445, 0.431, 0.428], -[1.367, 0.131, 0.113], -[4.819, 0.205, 0.175], -[3.808, 0.739, 0.726], -[8.935, 0.607, 0.600], -[8.988, 0.634, 0.615], -[0.242, 0.220, 0.226], -[0.075, 0.058, 0.056], -[0.038, 0.028, 0.026], -[0.043, 0.028, 0.021], -[0.172, 0.127, 0.119], -[0.028, 0.018, 0.017], -[0.027, 0.019, 0.014], -[0.018, 0.026, 0.015] -] -} diff --git a/benchmark/clickhouse/run.sh b/benchmark/clickhouse/run.sh deleted file mode 100755 index 86bc4c03f72..00000000000 --- a/benchmark/clickhouse/run.sh +++ /dev/null @@ -1,20 +0,0 @@ -#!/bin/bash - -TRIES=3 -QUERY_NUM=1 -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - - echo -n "[" - for i in $(seq 1 $TRIES); do - RES=$(clickhouse-client --time --format=Null --query="$query" 2>&1 ||:) - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - - echo "${QUERY_NUM},${i},${RES}" >> result.csv - done - echo "]," - - QUERY_NUM=$((QUERY_NUM + 1)) -done diff --git a/benchmark/cratedb/benchmark.sh b/benchmark/cratedb/benchmark.sh deleted file mode 100755 index b45826ee861..00000000000 --- a/benchmark/cratedb/benchmark.sh +++ /dev/null @@ -1,39 +0,0 @@ -#!/bin/bash - -bash -c "$(curl -L https://try.crate.io/)" > crate.log 2>&1 & - -sudo apt-get update -sudo apt-get install -y postgresql-client - -psql -U crate -h localhost --no-password -t -c 'SELECT 1' - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -psql -U crate -h localhost --no-password -t < create.sql - -psql -U crate -h localhost --no-password -t -c '\timing' -c " - COPY hits - FROM 'file://$(pwd)/hits.tsv' - WITH - ( - "delimiter"=e'\t', - "format"='csv', - "header"=false, - "empty_string_as_null"=false - ) - RETURN SUMMARY;" - -# One record did not load: -# 99997496 -# {"Missing closing quote for value\n at [Source: UNKNOWN; line: 1, column: 1069]":{"count":1,"line_numbers":[93557187]}} -# Time: 10687056.069 ms (02:58:07.056) - -./run.sh 2>&1 | tee log.txt - -# For some queries it gives "Data too large". - -du -bcs crate-* - -cat log.txt | grep -oP 'Time: \d+\.\d+ ms|ERROR' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if ($1 == "ERROR") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : ($1 / 1000); if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' diff --git a/benchmark/cratedb/create.sql b/benchmark/cratedb/create.sql deleted file mode 100644 index 3bf1815ceba..00000000000 --- a/benchmark/cratedb/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate TIMESTAMP NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor TEXT NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/cratedb/queries.sql b/benchmark/cratedb/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/cratedb/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/cratedb/results/c6a.4xlarge.json b/benchmark/cratedb/results/c6a.4xlarge.json deleted file mode 100644 index 0e469b6ca27..00000000000 --- a/benchmark/cratedb/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "CrateDB", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "For some queries it gives \"Data too large\".", - - "tags": ["Java", "column-oriented"], - - "load_time": 10687, - "data_size": 109636633416, - - "result": [ -[0.008162,0.005118,0.002553], -[0.350014,0.39977,0.133775], -[2.58426,2.47192,2.59779], -[2.12939,0.532981,0.507246], -[null,null,null], -[null,null,null], -[1.18488,1.06603,1.07219], -[0.209264,0.073284,0.067912], -[null,null,null], -[null,null,null], -[1.68892,1.2866,1.47428], -[1.62976,1.43073,1.26904], -[12.7517,13.0334,13.2685], -[18.8587,null,18.6951], -[11.2982,11.2108,11.577], -[20.2964,20.4035,19.1076], -[null,null,null], -[null,null,null], -[null,null,null], -[0.202044,0.010009,0.005566], -[9.22964,4.54606,0.774149], -[1.41673,1.09885,0.789775], -[12.3933,8.06911,1.69671], -[1.45018,0.969528,0.979718], -[0.357589,0.14887,0.153326], -[0.189282,0.133963,0.130279], -[0.153222,0.140756,0.139861], -[27.5195,19.6862,20.1825], -[72.7575,68.2,67.1238], -[144.533,146.579,152.144], -[8.76866,9.00563,8.46917], -[17.6652,16.6755,16.0558], -[null,null,null], -[null,null,null], -[null,null,null], -[42.2967,44.9621,44.4386], -[0.786911,0.4904,0.508416], -[0.602075,0.226261,0.182399], -[0.131407,0.058958,0.054518], -[0.954736,1.1361,1.14233], -[0.23764,0.139109,0.134472], -[0.110253,0.057695,0.056073], -[0.124285,0.150479,0.066226] -] -} diff --git a/benchmark/cratedb/run.sh b/benchmark/cratedb/run.sh deleted file mode 100755 index 477f463289b..00000000000 --- a/benchmark/cratedb/run.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - echo "$query"; - for i in $(seq 1 $TRIES); do - psql -U crate -h localhost --no-password -t -c '\timing' -c "$query" | grep 'Time' - done; -done; diff --git a/benchmark/databend/README.md b/benchmark/databend/README.md deleted file mode 100644 index fd7877efb27..00000000000 --- a/benchmark/databend/README.md +++ /dev/null @@ -1,7 +0,0 @@ -It is written in Rust and is blazing. - -Update from @BohuTANG: - -> Thanks for the benchmark! -> Databend is a cloud warehouse designed for object storage(like Amazon S3), not the local file system. The FS model is only used for testing for some cases, we didn't do any optimization, and we know it has some performance issues. -> I believe that ClickHouse is also being designed for the cloud, and looking forward to the S3 benchmark results :) diff --git a/benchmark/databend/benchmark.sh b/benchmark/databend/benchmark.sh deleted file mode 100755 index 47ff4ff564c..00000000000 --- a/benchmark/databend/benchmark.sh +++ /dev/null @@ -1,105 +0,0 @@ -#!/bin/bash - -mkdir databend && cd databend -curl -LJO 'https://github.com/datafuselabs/databend/releases/download/v0.7.113-nightly/databend-v0.7.113-nightly-x86_64-unknown-linux-musl.tar.gz' -tar xzvf 'databend-v0.7.113-nightly-x86_64-unknown-linux-musl.tar.gz' - -echo 'dir = "metadata/_logs" -admin_api_address = "127.0.0.1:8101" -grpc_api_address = "127.0.0.1:9101" - -[raft_config] -id = 1 -single = true -raft_dir = "metadata/datas"' > databend-meta.toml - -./bin/databend-meta -c ./databend-meta.toml > meta.log 2>&1 & -curl -I 'http://127.0.0.1:8101/v1/health' - -echo '[log] -level = "INFO" -dir = "benddata/_logs" - -[query] -# For admin RESET API. -admin_api_address = "127.0.0.1:8001" - -# Metrics. -metric_api_address = "127.0.0.1:7071" - -# Cluster flight RPC. -flight_api_address = "127.0.0.1:9091" - -# Query MySQL Handler. -mysql_handler_host = "127.0.0.1" -mysql_handler_port = 3307 - -# Query ClickHouse Handler. -clickhouse_handler_host = "127.0.0.1" -clickhouse_handler_port = 9001 - -# Query ClickHouse HTTP Handler. -clickhouse_http_handler_host = "127.0.0.1" -clickhouse_http_handler_port = 8125 - -# Query HTTP Handler. -http_handler_host = "127.0.0.1" -http_handler_port = 8081 - -tenant_id = "tenant1" -cluster_id = "cluster1" - -[meta] -# databend-meta grpc api address. -address = "127.0.0.1:9101" -username = "root" -password = "root" - -[storage] -# fs|s3 -type = "fs" - -[storage.fs] -data_path = "benddata/datas"' > databend-query.toml - -./bin/databend-query -c ./databend-query.toml > query.log 2>&1 & - -curl https://clickhouse.com/ | sh -sudo ./clickhouse install - -# Load the data - -curl 'http://default@localhost:8124/' --data-binary @create.sql - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' -gzip -d hits.csv.gz - -# Note: if I run -# clickhouse-client --time --query "INSERT INTO hits FORMAT TSV" < hits.tsv -# it panics: -# ERROR common_tracing::panic_hook: panicked at 'called `Result::unwrap()` on an `Err` value: SendError - -# Note: if I run -# curl -XPUT 'http://root:@127.0.0.1:8000/v1/streaming_load' -H 'insert_sql: insert into hits format CSV' -H 'skip_header: 0' -H 'field_delimiter: ,' -H 'record_delimiter: \n' -F 'upload=@"./hits.csv"' -# curl: (55) Send failure: Broken pipe - -# This is not entirely correct, but starts to work: -# curl -XPUT 'http://root:@127.0.0.1:8000/v1/streaming_load' -H 'insert_sql: insert into hits format TSV' -H 'skip_header: 0' -H 'field_delimiter: \t' -H 'record_delimiter: \n' -F 'upload=@"./hits.tsv"' -# and fails after 7 minutes 38 seconds without loading any data: -# Code: 4000, displayText = invalid data (Expected to have terminated string literal.) (while in processor thread 5). -# the diagnostics is terrible. - -head -n 90000000 hits.tsv > hits90m.tsv -time curl -XPUT 'http://root:@127.0.0.1:8000/v1/streaming_load' -H 'insert_sql: insert into hits format TSV' -H 'skip_header: 0' -H 'field_delimiter: \t' -H 'record_delimiter: \n' -F 'upload=@"./hits90m.tsv"' - -# {"id":"08f59e6c-2924-483e-bb96-cbcb458588f5","state":"SUCCESS","stats":{"rows":90000000,"bytes":73152552024},"error":null} -# real 7m15.312s - -du -bcs _data -# 38714978944 - -# It does not support ClickHouse protocol well (it hangs on some queries if they are too long). - -./run.sh 2>&1 | tee log.txt - -# Note: divide every number by 0.9 as only 90% of the data was loaded successfully. diff --git a/benchmark/databend/create.sql b/benchmark/databend/create.sql deleted file mode 100644 index b446288b409..00000000000 --- a/benchmark/databend/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -) -CLUSTER BY (CounterID, EventDate, UserID, EventTime, WatchID); diff --git a/benchmark/databend/queries.sql b/benchmark/databend/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/databend/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/databend/results/c6a.4xlarge.json b/benchmark/databend/results/c6a.4xlarge.json deleted file mode 100644 index a26bad5b49f..00000000000 --- a/benchmark/databend/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Databend", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "Only 90% of data successfully loaded.", - - "tags": ["Rust", "column-oriented", "ClickHouse derivative"], - - "load_time": 484, - "data_size": 43016643271, - - "result": [ -[0.010087, 0.002961, 0.003271], -[0.127964, 0.080012, 0.075741], -[0.162388, 0.143967, 0.144762], -[0.252904, 0.217471, 0.217369], -[34.281026, 34.844158, 34.526942], -[25.290307, 25.793068, 25.620563], -[0.112484, 0.093867, 0.090891], -[0.086604, 0.07796, 0.076448], -[20.723203, 20.7483, 20.354869], -[20.81994, 20.72446, 20.696573], -[1.964378, 1.93559, 1.893824], -[1.846866, 1.789111, 1.763664], -[4.468158, 4.407959, 4.438036], -[19.947276, 19.8859, 19.853514], -[5.478573, 5.474461, 5.460604], -[5.509521, 5.513413, 5.363123], -[15.430359, 15.5406, 15.461211], -[14.905998, 15.029721, 15.019642], -[31.069663, 30.811763, 30.737336], -[0.281067, 0.220021, 0.217741], -[8.89374, 4.12692, 4.131689], -[10.38448, 4.603694, 4.571757], -[19.980572, 8.836322, 8.892694], -[59.786474, 52.452881, 39.941988], -[2.804019, 0.994794, 0.958224], -[0.765299, 0.730434, 0.723964], -[2.784648, 0.94665, 0.936684], -[8.905027, 5.418438, 5.386109], -[12.187652, 12.230066, 12.164123], -[3.35748, 3.395991, 3.319434], -[4.309389, 3.854977, 3.772506], -[9.958201, 7.027432, 6.888253], -[50.200569, 50.535126, 50.283066], -[24.469412, 21.222713, 21.010188], -[26.115852, 23.93507, 24.835342], -[7.511517, 7.296179, 7.324549], -[2.156784, 1.298258, 1.278441], -[2.155447, 1.314499, 1.331237], -[2.007053, 1.181676, 1.155612], -[null, null, null], -[0.485363, 0.420291, 0.416819], -[0.372131, 0.322068, 0.323578], -[null, null, null] -] -} diff --git a/benchmark/databend/run.sh b/benchmark/databend/run.sh deleted file mode 100755 index 5a87f07b2ec..00000000000 --- a/benchmark/databend/run.sh +++ /dev/null @@ -1,20 +0,0 @@ -#!/bin/bash - -TRIES=3 -QUERY_NUM=1 -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - - echo -n "[" - for i in $(seq 1 $TRIES); do - RES=$(curl -w 'Time: %{time_total}\n' http://default@localhost:8124/ -d "${query}" 2>&1 | grep -P '^Time: ' | sed 's/Time: //') - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - - echo "${QUERY_NUM},${i},${RES}" >> result.csv - done - echo "]," - - QUERY_NUM=$((QUERY_NUM + 1)) -done diff --git a/benchmark/druid/benchmark.sh b/benchmark/druid/benchmark.sh deleted file mode 100755 index 5c237a85d03..00000000000 --- a/benchmark/druid/benchmark.sh +++ /dev/null @@ -1,43 +0,0 @@ -#!/bin/bash - -sudo apt-get update -sudo apt install -y openjdk-8-jdk python2 -sudo update-alternatives --config java - -# Install - -VERSION=0.23.0 - -wget -O"apache-druid-${VERSION}-bin.tar.gz" "https://dlcdn.apache.org/druid/${VERSION}/apache-druid-${VERSION}-bin.tar.gz" -tar xf apache-druid-${VERSION}-bin.tar.gz -./apache-druid-${VERSION}/bin/verify-java - -# Have to increase indexer memory limit -sed -i 's MaxDirectMemorySize=1g MaxDirectMemorySize=5g g' apache-druid-$VERSION/conf/druid/single-server/medium/middleManager/runtime.properties - -# Disable cache to test query performance -sed -i 's druid.historical.cache.useCache=true druid.historical.cache.useCache=false g' apache-druid-$VERSION/conf/druid/single-server/medium/historical/runtime.properties -sed -i 's druid.historical.cache.populateCache=true druid.historical.cache.populateCache=false g' apache-druid-$VERSION/conf/druid/single-server/medium/historical/runtime.properties -sed -i 's druid.processing.buffer.sizeBytes=500MiB druid.processing.buffer.sizeBytes=1000MiB g' apache-druid-$VERSION/conf/druid/single-server/medium/historical/runtime.properties - -echo "druid.query.groupBy.maxMergingDictionarySize=5000000000" >> apache-druid-$VERSION/conf/druid/single-server/medium/historical/runtime.properties -# Druid launcher does not start Druid as a daemon. Run it in background -./apache-druid-${VERSION}/bin/start-single-server-medium & - -# Load the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -./apache-druid-${VERSION}/bin/post-index-task --file ingest.json --url http://localhost:8081 - -# The command above will fail due to timeout but still continue to run in background. -# The loading time should be checked from the logs. - -# Run the queries -./run.sh - -# stop Druid services -kill %1 - -du -bcs ./apache-druid-${VERSION}/var diff --git a/benchmark/druid/check.json b/benchmark/druid/check.json deleted file mode 100644 index a61bc7bacd9..00000000000 --- a/benchmark/druid/check.json +++ /dev/null @@ -1 +0,0 @@ -{"query": "SELECT COUNT(*) from hits"} diff --git a/benchmark/druid/ingest.json b/benchmark/druid/ingest.json deleted file mode 100644 index 8d4c741934c..00000000000 --- a/benchmark/druid/ingest.json +++ /dev/null @@ -1,573 +0,0 @@ -{ - "type": "index_parallel", - "spec": { - "ioConfig": { - "type": "index_parallel", - "inputSource": { - "type": "local", - "baseDir": "../", - "filter": "hits.tsv" - }, - "inputFormat": { - "type": "tsv", - "findColumnsFromHeader": false, - "columns": [ - "WatchID", - "JavaEnable", - "Title", - "GoodEvent", - "EventTime", - "EventDate", - "CounterID", - "ClientIP", - "RegionID", - "UserID", - "CounterClass", - "OS", - "UserAgent", - "URL", - "Referer", - "IsRefresh", - "RefererCategoryID", - "RefererRegionID", - "URLCategoryID", - "URLRegionID", - "ResolutionWidth", - "ResolutionHeight", - "ResolutionDepth", - "FlashMajor", - "FlashMinor", - "FlashMinor2", - "NetMajor", - "NetMinor", - "UserAgentMajor", - "UserAgentMinor", - "CookieEnable", - "JavascriptEnable", - "IsMobile", - "MobilePhone", - "MobilePhoneModel", - "Params", - "IPNetworkID", - "TraficSourceID", - "SearchEngineID", - "SearchPhrase", - "AdvEngineID", - "IsArtifical", - "WindowClientWidth", - "WindowClientHeight", - "ClientTimeZone", - "ClientEventTime", - "SilverlightVersion1", - "SilverlightVersion2", - "SilverlightVersion3", - "SilverlightVersion4", - "PageCharset", - "CodeVersion", - "IsLink", - "IsDownload", - "IsNotBounce", - "FUniqID", - "OriginalURL", - "HID", - "IsOldCounter", - "IsEvent", - "IsParameter", - "DontCountHits", - "WithHash", - "HitColor", - "LocalEventTime", - "Age", - "Sex", - "Income", - "Interests", - "Robotness", - "RemoteIP", - "WindowName", - "OpenerName", - "HistoryLength", - "BrowserLanguage", - "BrowserCountry", - "SocialNetwork", - "SocialAction", - "HTTPError", - "SendTiming", - "DNSTiming", - "ConnectTiming", - "ResponseStartTiming", - "ResponseEndTiming", - "FetchTiming", - "SocialSourceNetworkID", - "SocialSourcePage", - "ParamPrice", - "ParamOrderID", - "ParamCurrency", - "ParamCurrencyID", - "OpenstatServiceName", - "OpenstatCampaignID", - "OpenstatAdID", - "OpenstatSourceID", - "UTMSource", - "UTMMedium", - "UTMCampaign", - "UTMContent", - "UTMTerm", - "FromTag", - "HasGCLID", - "RefererHash", - "URLHash", - "CLID" - ] - } - }, - "tuningConfig": { - "type": "index_parallel", - "partitionsSpec": { - "type": "hashed", - "partitionDimensions": [ - "CounterID", - "EventDate", - "UserID", - "EventTime", - "WatchID" - ] - }, - "forceGuaranteedRollup": true, - "logParseExceptions": true, - "maxParseExceptions": 1, - "maxNumConcurrentSubTasks": 10 - }, - "dataSchema": { - "dataSource": "hits", - "timestampSpec": { - "column": "EventTime", - "format": "yyyy-MM-dd HH:mm:ss" - }, - "dimensionsSpec": { - "dimensions": [ - { - "name": "WatchID", - "type": "long" - }, - { - "name": "JavaEnable", - "type": "long" - }, - { - "name": "Title", - "type": "string" - }, - { - "name": "GoodEvent", - "type": "long" - }, - { - "name": "EventDate", - "type": "string" - }, - { - "name": "CounterID", - "type": "long" - }, - { - "name": "ClientIP", - "type": "long" - }, - { - "name": "RegionID", - "type": "long" - }, - { - "name": "UserID", - "type": "long" - }, - { - "name": "CounterClass", - "type": "long" - }, - { - "name": "OS", - "type": "long" - }, - { - "name": "UserAgent", - "type": "long" - }, - { - "name": "URL", - "type": "string" - }, - { - "name": "Referer", - "type": "string" - }, - { - "name": "IsRefresh", - "type": "long" - }, - { - "name": "RefererCategoryID", - "type": "long" - }, - { - "name": "RefererRegionID", - "type": "long" - }, - { - "name": "URLCategoryID", - "type": "long" - }, - { - "name": "URLRegionID", - "type": "long" - }, - { - "name": "ResolutionWidth", - "type": "long" - }, - { - "name": "ResolutionHeight", - "type": "long" - }, - { - "name": "ResolutionDepth", - "type": "long" - }, - { - "name": "FlashMajor", - "type": "long" - }, - { - "name": "FlashMinor", - "type": "long" - }, - { - "name": "FlashMinor2", - "type": "string" - }, - { - "name": "NetMajor", - "type": "long" - }, - { - "name": "NetMinor", - "type": "long" - }, - { - "name": "UserAgentMajor", - "type": "long" - }, - { - "name": "UserAgentMinor", - "type": "string" - }, - { - "name": "CookieEnable", - "type": "long" - }, - { - "name": "JavascriptEnable", - "type": "long" - }, - { - "name": "IsMobile", - "type": "long" - }, - { - "name": "MobilePhone", - "type": "long" - }, - { - "name": "MobilePhoneModel", - "type": "string" - }, - { - "name": "Params", - "type": "string" - }, - { - "name": "IPNetworkID", - "type": "long" - }, - { - "name": "TraficSourceID", - "type": "long" - }, - { - "name": "SearchEngineID", - "type": "long" - }, - { - "name": "SearchPhrase", - "type": "string" - }, - { - "name": "AdvEngineID", - "type": "long" - }, - { - "name": "IsArtifical", - "type": "long" - }, - { - "name": "WindowClientWidth", - "type": "long" - }, - { - "name": "WindowClientHeight", - "type": "long" - }, - { - "name": "ClientTimeZone", - "type": "long" - }, - { - "name": "ClientEventTime", - "type": "string" - }, - { - "name": "SilverlightVersion1", - "type": "long" - }, - { - "name": "SilverlightVersion2", - "type": "long" - }, - { - "name": "SilverlightVersion3", - "type": "long" - }, - { - "name": "SilverlightVersion4", - "type": "long" - }, - { - "name": "PageCharset", - "type": "string" - }, - { - "name": "CodeVersion", - "type": "long" - }, - { - "name": "IsLink", - "type": "long" - }, - { - "name": "IsDownload", - "type": "long" - }, - { - "name": "IsNotBounce", - "type": "long" - }, - { - "name": "FUniqID", - "type": "long" - }, - { - "name": "OriginalURL", - "type": "string" - }, - { - "name": "HID", - "type": "long" - }, - { - "name": "IsOldCounter", - "type": "long" - }, - { - "name": "IsEvent", - "type": "long" - }, - { - "name": "IsParameter", - "type": "long" - }, - { - "name": "DontCountHits", - "type": "long" - }, - { - "name": "WithHash", - "type": "long" - }, - { - "name": "HitColor", - "type": "string" - }, - { - "name": "LocalEventTime", - "type": "string" - }, - { - "name": "Age", - "type": "long" - }, - { - "name": "Sex", - "type": "long" - }, - { - "name": "Income", - "type": "long" - }, - { - "name": "Interests", - "type": "long" - }, - { - "name": "Robotness", - "type": "long" - }, - { - "name": "RemoteIP", - "type": "long" - }, - { - "name": "WindowName", - "type": "long" - }, - { - "name": "OpenerName", - "type": "long" - }, - { - "name": "HistoryLength", - "type": "long" - }, - { - "name": "BrowserLanguage", - "type": "string" - }, - { - "name": "BrowserCountry", - "type": "string" - }, - { - "name": "SocialNetwork", - "type": "string" - }, - { - "name": "SocialAction", - "type": "string" - }, - { - "name": "HTTPError", - "type": "long" - }, - { - "name": "SendTiming", - "type": "long" - }, - { - "name": "DNSTiming", - "type": "long" - }, - { - "name": "ConnectTiming", - "type": "long" - }, - { - "name": "ResponseStartTiming", - "type": "long" - }, - { - "name": "ResponseEndTiming", - "type": "long" - }, - { - "name": "FetchTiming", - "type": "long" - }, - { - "name": "SocialSourceNetworkID", - "type": "long" - }, - { - "name": "SocialSourcePage", - "type": "string" - }, - { - "name": "ParamPrice", - "type": "long" - }, - { - "name": "ParamOrderID", - "type": "string" - }, - { - "name": "ParamCurrency", - "type": "string" - }, - { - "name": "ParamCurrencyID", - "type": "long" - }, - { - "name": "OpenstatServiceName", - "type": "string" - }, - { - "name": "OpenstatCampaignID", - "type": "string" - }, - { - "name": "OpenstatAdID", - "type": "string" - }, - { - "name": "OpenstatSourceID", - "type": "string" - }, - { - "name": "UTMSource", - "type": "string" - }, - { - "name": "UTMMedium", - "type": "string" - }, - { - "name": "UTMCampaign", - "type": "string" - }, - { - "name": "UTMContent", - "type": "string" - }, - { - "name": "UTMTerm", - "type": "string" - }, - { - "name": "FromTag", - "type": "string" - }, - { - "name": "HasGCLID", - "type": "long" - }, - { - "name": "RefererHash", - "type": "long" - }, - { - "name": "URLHash", - "type": "long" - }, - { - "name": "CLID", - "type": "long" - } - ] - }, - "granularitySpec": { - "queryGranularity": "none", - "rollup": false, - "segmentGranularity": "day" - } - } - } -} diff --git a/benchmark/druid/queries.sql b/benchmark/druid/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/druid/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/druid/results/c6a.4xlarge.json b/benchmark/druid/results/c6a.4xlarge.json deleted file mode 100644 index 4759a3a05b6..00000000000 --- a/benchmark/druid/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Druid", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "Druid is killed and restarted after every query. Otherwise some queries make Druid degraded and results are incorrect. For example after Q13 even SELECT 1 works for 7 seconds", - - "tags": ["Java", "column-oriented"], - - "load_time": 19620, - "data_size": 45188608472, - - "result": [ -[0.032365, 0.016716, 0.016921], -[0.400766, 0.275591, 0.271057], -[0.382350, 0.152084, 0.151652], -[1.379492, 0.137201, 0.128902], -[3.278731, 2.527105, 2.515128], -[8.576382, 6.546540, 6.503001], -[null, null, null], -[0.563852, 0.273795, 0.275086], -[11.509993, 10.636571, 10.597993], -[13.357647, 12.421210, 12.337247], -[1.636875, 0.821300, 0.900056], -[1.692544, 0.512066, 0.440511], -[2.453274, 1.769806, 1.807207], -[8.503408, 7.261406, 7.334872], -[61.056041, 59.251083, 59.500549], -[8.620670, 8.236657, 8.225380], -[164.840762, null, null], -[24.165797, 22.308466, null], -[null, null, null], -[null, null, null], -[25.973369, 25.597864, 25.602509], -[null, null, null], -[null, null, null], -[7.805347, 6.629776, 6.947366], -[0.257845, 0.020327, 0.016976], -[null, null, null], -[null, null, null], -[32.948154, 31.046770, 29.221959], -[null, null, null], -[7.230865, 7.033713, 6.972421], -[20.546250, 19.237428, 19.258469], -[54.065945, 52.451318, 52.466653], -[null, null, null], -[17.499267, null, null], -[null, null, null], -[60.478315, 60.054940, 60.458946], -[1.698088, 1.490317, 1.461969], -[1.409572, 0.939003, 0.907252], -[0.866729, 0.329539, 0.287435], -[null, null, null], -[0.932473, 0.420781, 0.359095], -[0.723142, 0.325300, 0.296865], -[0.603483, 0.150892, 0.140716] - ] -} diff --git a/benchmark/druid/run.sh b/benchmark/druid/run.sh deleted file mode 100755 index faa88431bc0..00000000000 --- a/benchmark/druid/run.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/bin/bash - -TRIES=3 -cat queries.sql | while read query; do - sync - for i in $(seq 1 100); do - CHECK=$(curl -o /dev/null -w '%{http_code}' -s -XPOST -H'Content-Type: application/json' http://localhost:8888/druid/v2/sql/ -d @check.json }) - [[ "$CHECK" == "200" ]] && break - sleep 1 - done - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - echo -n "[" - for i in $(seq 1 $TRIES); do - echo "{\"query\":\"$query\", \"context\": {\"timeout\": 1000000} }"| sed -e 's EventTime __time g' | tr -d ';' > query.json - curl -w '%{http_code} %{time_total}\n' -s -XPOST -H'Content-Type: application/json' http://localhost:8888/druid/v2/sql/ -d @query.json | awk '{ if ($1!="200") { printf "null" } }' - [[ "$i" != $TRIES ]] && echo -n ", " - done - echo "]," - - # Ugly hack to measure independently queries. Otherwise some queries make Druid degraded and results are incorrect. For example after Q13 even SELECT 1 works for 7 seconds - pkill -f historical - sleep 3 -done diff --git a/benchmark/duckdb/README.md b/benchmark/duckdb/README.md deleted file mode 100644 index d2d7b22c81b..00000000000 --- a/benchmark/duckdb/README.md +++ /dev/null @@ -1,2 +0,0 @@ -DuckDB cannot load parquet file due to OOM. -The only option is to load a CSV file, but sometimes it also fails with OOM. diff --git a/benchmark/duckdb/benchmark.sh b/benchmark/duckdb/benchmark.sh deleted file mode 100755 index 392f084c560..00000000000 --- a/benchmark/duckdb/benchmark.sh +++ /dev/null @@ -1,24 +0,0 @@ -#!/bin/bash - -# Install - -sudo apt-get update -sudo apt-get install -y python3-pip -pip install duckdb psutil - -# Load the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' -gzip -d hits.csv.gz - -./load.py -# 4216.5390389899985 seconds - -# Run the queries - -./run.sh 2>&1 | tee log.txt - -wc -c my-db.duckdb - -cat log.txt | grep -P '^\d|Killed|Segmentation' | sed -r -e 's/^.*(Killed|Segmentation).*$/null\nnull\nnull/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/duckdb/create.sql b/benchmark/duckdb/create.sql deleted file mode 100644 index 744d595ecac..00000000000 --- a/benchmark/duckdb/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT, - Referer TEXT, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT, - Params TEXT, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT, - BrowserCountry TEXT, - SocialNetwork TEXT, - SocialAction TEXT, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT, - ParamCurrency TEXT, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT, - OpenstatCampaignID TEXT, - OpenstatAdID TEXT, - OpenstatSourceID TEXT, - UTMSource TEXT, - UTMMedium TEXT, - UTMCampaign TEXT, - UTMContent TEXT, - UTMTerm TEXT, - FromTag TEXT, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/duckdb/load.py b/benchmark/duckdb/load.py deleted file mode 100755 index d4265d15fc3..00000000000 --- a/benchmark/duckdb/load.py +++ /dev/null @@ -1,18 +0,0 @@ -#!/usr/bin/env python3 - -import duckdb -import timeit -import psutil - -con = duckdb.connect(database="my-db.duckdb", read_only=False) -# See https://github.com/duckdb/duckdb/issues/3969 -con.execute("PRAGMA memory_limit='{}b'".format(psutil.virtual_memory().total / 4)) -con.execute("PRAGMA threads={}".format(psutil.cpu_count(logical=False))) - -print("Will load the data") - -start = timeit.default_timer() -con.execute(open("create.sql").read()) -con.execute("INSERT INTO hits SELECT * FROM read_csv_auto('hits.csv')") -end = timeit.default_timer() -print(end - start) diff --git a/benchmark/duckdb/queries.sql b/benchmark/duckdb/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/duckdb/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/duckdb/query.py b/benchmark/duckdb/query.py deleted file mode 100755 index 43739be56f3..00000000000 --- a/benchmark/duckdb/query.py +++ /dev/null @@ -1,20 +0,0 @@ -#!/usr/bin/env python3 - -import duckdb -import timeit -import psutil -import sys - -query = sys.stdin.read() -print(query) - -con = duckdb.connect(database="my-db.duckdb", read_only=False) -# See https://github.com/duckdb/duckdb/issues/3969 -con.execute("PRAGMA memory_limit='{}b'".format(psutil.virtual_memory().total / 4)) -con.execute("PRAGMA threads={}".format(psutil.cpu_count(logical=False))) - -for try_num in range(3): - start = timeit.default_timer() - con.execute(query) - end = timeit.default_timer() - print(end - start) diff --git a/benchmark/duckdb/results/c6a.4xlarge.json b/benchmark/duckdb/results/c6a.4xlarge.json deleted file mode 100644 index 40b9d88bc71..00000000000 --- a/benchmark/duckdb/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "DuckDB", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "Many queries triggered OOM", - - "tags": ["C", "column-oriented", "embedded"], - - "load_time": 4217, - "data_size": 27241492480, - - "result": [ -[0.005694353996659629,0.003944558004150167,0.003837226002360694], -[0.16991353100456763,0.03919722700084094,0.03835860399703961], -[0.44898432699847035,0.04947217500011902,0.04852217998995911], -[0.07586832098604646,0.07051395199960098,0.07007493599667214], -[9.554053236002801,8.153356187991449,8.73448242500308], -[7.66042533799191,6.931124911992811,7.103380946995458], -[0.030703739990713075,0.027668555994750932,0.027583695002249442], -[0.1778664360026596,0.03942437999648973,0.03882004099432379], -[8.53439180701389,8.869582625004114,9.020313234999776], -[10.40215514000738,11.125320470004226,8.941559945000336], -[1.1747649609897053,1.04221136700653,1.004799570000614], -[1.2380354650085792,1.1211603130068397,2.4278587239969056], -[3.1751541379926493,0.9360461989999749,0.8868292279948946], -[6.855684430003748,7.300301584007684,5.712960822012974], -[3.70588762400439,1.0249276379909134,0.9473389159975341], -[2.1037107890006155,1.6215517020027619,1.5671920729946578], -[null,null,null], -[null,null,null], -[null,null,null], -[0.0002772739971987903,0.00016792300448287278,0.0001574420020915568], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[2.9310110910009826,0.19020285899750888,0.1736805049877148], -[2.939304119994631,0.18754731099761557,0.18073286200524308], -[2.8706370779982535,0.18822155400994234,0.17905898999015335], -[null,null,null], -[null,null,null], -[0.884408778991201,0.714329167996766,0.7135983259940986], -[5.3762675570033025,0.8803737630078103,0.8728962720051641], -[7.249190265996731,2.9648747390019707,2.866687831003219], -[null,null,null], -[null,null,null], -[null,null,null], -[4.515183198003797,4.030519469000865,4.014251719010645], -[0.11604027298744768,0.040539135996368714,0.04280066800129134], -[0.0457908230018802,0.021069509006338194,0.019683108999743126], -[0.0680370800109813,0.011889394998434,0.01056639499438461], -[0.22029169600864407,0.08547276000899728,0.09095505000732373], -[0.03759863799496088,0.008373684002435766,0.007633563989656977], -[0.025631797994719818,0.008081699008471332,0.007858585988287814], -[0.034359957004198804,0.025543516996549442,0.02533275399764534] -] -} diff --git a/benchmark/duckdb/run.sh b/benchmark/duckdb/run.sh deleted file mode 100755 index 68cea914794..00000000000 --- a/benchmark/duckdb/run.sh +++ /dev/null @@ -1,10 +0,0 @@ -#!/bin/bash - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - - # We need to restart and reinitialize DuckDB after every query, - # because it often fails with Segmentation fault (core dumped) - ./query.py <<< "${query}" -done diff --git a/benchmark/elasticsearch/README.md b/benchmark/elasticsearch/README.md deleted file mode 100644 index 82b0b2ff959..00000000000 --- a/benchmark/elasticsearch/README.md +++ /dev/null @@ -1 +0,0 @@ -Incomplete. diff --git a/benchmark/elasticsearch/benchmark.sh b/benchmark/elasticsearch/benchmark.sh deleted file mode 100755 index 740df544b3d..00000000000 --- a/benchmark/elasticsearch/benchmark.sh +++ /dev/null @@ -1,30 +0,0 @@ -#!/bin/bash - -wget -qO - https://artifacts.elastic.co/GPG-KEY-elasticsearch | sudo gpg --dearmor -o /usr/share/keyrings/elasticsearch-keyring.gpg -sudo apt-get update && sudo apt-get install -y apt-transport-https -echo "deb [signed-by=/usr/share/keyrings/elasticsearch-keyring.gpg] https://artifacts.elastic.co/packages/8.x/apt stable main" | sudo tee /etc/apt/sources.list.d/elastic-8.x.list -sudo apt-get update && sudo apt-get install -y elasticsearch - -sudo systemctl start elasticsearch.service -sudo /usr/share/elasticsearch/bin/elasticsearch-reset-password -u elastic - -# Example: -# User: elastic -# Password: C0Qq9kNYMUunKTXMDOUZ - -export PASSWORD='...' - -curl -k -XGET 'https://localhost:9200' -u "elastic:${PASSWORD}" - -# This will create an index. -curl -k -XPUT -u "elastic:${PASSWORD}" 'https://localhost:9200/hits' - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.json.gz' -gzip -d hits.json.gz - -# Prevent 'curl' from OOM. - -split -l 1000000000 hits.json hits_ -for table in hits_*; do mv ${table} ${table}.json; done - -time for table in hits_*; do curl -k -H "Transfer-Encoding: chunked" -XPOST -u "elastic:${PASSWORD}" 'https://localhost:9200/_bulk' -T ${table}; done diff --git a/benchmark/exasol/README.md b/benchmark/exasol/README.md deleted file mode 100644 index abe9c5dfd67..00000000000 --- a/benchmark/exasol/README.md +++ /dev/null @@ -1,3 +0,0 @@ -EXASOL does not allow using the community edition for more than 10 GiB data, therefore testing is not possible. I advise you not to trust the unfounded claims of performance. - -https://github.com/exasol/docker-db diff --git a/benchmark/generate-results.sh b/benchmark/generate-results.sh deleted file mode 100755 index 7651fd6d17e..00000000000 --- a/benchmark/generate-results.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/bin/bash -e - -# This script will substitute the benchmark results into the HTML page. -# Note: editing HTML with sed may look strange, but at least we avoid using node.js and npm, and that's good. - -( - sed '/^const data = \[$/q' index.html - - FIRST=1 - ls -1 */results/*.json | while read file - do - [ "${FIRST}" = "0" ] && echo -n ',' - jq --compact-output ". += {\"source\": \"${file}\"}" "${file}" - FIRST=0 - done - - echo ']; // end of data' - sed '0,/^\]; \/\/ end of data$/d' index.html - -) > index.html.new - -mv index.html index.html.bak -mv index.html.new index.html diff --git a/benchmark/greenplum/benchmark.sh b/benchmark/greenplum/benchmark.sh deleted file mode 100755 index a1802ee223b..00000000000 --- a/benchmark/greenplum/benchmark.sh +++ /dev/null @@ -1,73 +0,0 @@ -#!/bin/bash - -# NOTE: it requires Ubuntu 18.04 -# Greenplum does not install on any newer system. - -echo "This script must be run from gpadmin user. Press enter to continue." -read -sudo apt update -sudo apt install -y software-properties-common -sudo add-apt-repository ppa:greenplum/db -sudo apt update -sudo apt install greenplum-db-6 -sudo rm -rf /gpmaster /gpdata* -ssh-keygen -t rsa -b 4096 -cat /home/gpadmin/.ssh/id_rsa.pub >> /home/gpadmin/.ssh/authorized_keys -mod 600 ~/.ssh/authorized_keys -sudo echo "# kernel.shmall = _PHYS_PAGES / 2 # See Shared Memory Pages -kernel.shmall = 197951838 -# kernel.shmmax = kernel.shmall * PAGE_SIZE -kernel.shmmax = 810810728448 -kernel.shmmni = 4096 -vm.overcommit_memory = 2 # See Segment Host Memory -vm.overcommit_ratio = 95 # See Segment Host Memory - -net.ipv4.ip_local_port_range = 10000 65535 # See Port Settings -kernel.sem = 500 2048000 200 4096 -kernel.sysrq = 1 -kernel.core_uses_pid = 1 -kernel.msgmnb = 65536 -kernel.msgmax = 65536 -kernel.msgmni = 2048 -net.ipv4.tcp_syncookies = 1 -net.ipv4.conf.default.accept_source_route = 0 -net.ipv4.tcp_max_syn_backlog = 4096 -net.ipv4.conf.all.arp_filter = 1 -net.core.netdev_max_backlog = 10000 -net.core.rmem_max = 2097152 -net.core.wmem_max = 2097152 -vm.swappiness = 10 -vm.zone_reclaim_mode = 0 -vm.dirty_expire_centisecs = 500 -vm.dirty_writeback_centisecs = 100 -vm.dirty_background_ratio = 0 # See System Memory -vm.dirty_ratio = 0 -vm.dirty_background_bytes = 1610612736 -vm.dirty_bytes = 4294967296" |sudo tee -a /etc/sysctl.conf -sudo sysctl -p - -echo "* soft nofile 524288 -* hard nofile 524288 -* soft nproc 131072 -* hard nproc 131072" |sudo tee -a /etc/security/limits.conf -echo "RemoveIPC=no" |sudo tee -a /etc/systemd/logind.conf -echo "Now you need to reboot the machine. Press Enter if you already rebooted, or reboot now and run the script once again" -read -source /opt/greenplum-db-*.0/greenplum_path.sh -cp $GPHOME/docs/cli_help/gpconfigs/gpinitsystem_singlenode . -echo localhost > ./hostlist_singlenode -sed -i "s/MASTER_HOSTNAME=[a-z_]*/MASTER_HOSTNAME=$(hostname)/" gpinitsystem_singlenode -sed -i "s@declare -a DATA_DIRECTORY=(/gpdata1 /gpdata2)@declare -a DATA_DIRECTORY=(/gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14)@" gpinitsystem_singlenode -sudo mkdir /gpmaster /gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14 -sudo chmod 777 /gpmaster /gpdata1 /gpdata2 /gpdata3 /gpdata4 /gpdata5 /gpdata6 /gpdata7 /gpdata8 /gpdata9 /gpdata10 /gpdata11 /gpdata12 /gpdata13 /gpdata14 -gpinitsystem -ac gpinitsystem_singlenode -export MASTER_DATA_DIRECTORY=/gpmaster/gpsne-1/ -#wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -#gzip -d hits.tsv.gz -chmod 777 ~ hits.tsv -psql -d postgres -f create.sql -nohup gpfdist & -time psql -d postgres -t -c '\timing' -c "insert into hits select * from hits_ext;" -du -sh /gpdata* -./run.sh 2>&1 | tee log.txt -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' |awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/greenplum/create.sql b/benchmark/greenplum/create.sql deleted file mode 100644 index ddaf587d0f9..00000000000 --- a/benchmark/greenplum/create.sql +++ /dev/null @@ -1,116 +0,0 @@ -drop table if exists hits; -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -) -with (appendoptimized=true,orientation=column,compresstype=zstd) -DISTRIBUTED RANDOMLY; -CREATE INDEX hits_idx on hits using btree (CounterID, EventDate, UserID, EventTime, WatchID); -drop external table if exists hits_ext; -CREATE EXTERNAL TABLE hits_ext (like hits) -LOCATION ('gpfdist://localhost:8080/hits.tsv') -FORMAT 'TEXT'; diff --git a/benchmark/greenplum/log.txt b/benchmark/greenplum/log.txt deleted file mode 100644 index 91eeeb8447e..00000000000 --- a/benchmark/greenplum/log.txt +++ /dev/null @@ -1,215 +0,0 @@ -3 -SELECT COUNT(*) FROM hits; -Time: 2200.088 ms -Time: 1458.828 ms -Time: 1135.728 ms -3 -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -Time: 1286.610 ms -Time: 1078.970 ms -Time: 1018.564 ms -3 -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -Time: 2258.198 ms -Time: 2421.929 ms -Time: 2596.925 ms -3 -SELECT AVG(UserID) FROM hits; -Time: 1893.709 ms -Time: 1924.597 ms -Time: 1738.663 ms -3 -SELECT COUNT(DISTINCT UserID) FROM hits; -Time: 9544.705 ms -Time: 9719.561 ms -Time: 10035.660 ms -3 -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -Time: 4559.571 ms -Time: 4101.119 ms -Time: 4704.585 ms -3 -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -Time: 1651.671 ms -Time: 1897.668 ms -Time: 1970.511 ms -3 -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -Time: 1025.366 ms -Time: 999.323 ms -Time: 1005.235 ms -3 -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -Time: 7897.922 ms -Time: 7757.179 ms -Time: 8012.193 ms -3 -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -Time: 11377.889 ms -Time: 11600.291 ms -Time: 11255.236 ms -3 -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -Time: 2119.548 ms -Time: 2050.025 ms -Time: 1955.304 ms -3 -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -Time: 2019.971 ms -Time: 1979.324 ms -Time: 1982.835 ms -3 -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -Time: 2886.663 ms -Time: 2928.279 ms -Time: 2968.219 ms -3 -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -Time: 4062.722 ms -Time: 4017.708 ms -Time: 4030.113 ms -3 -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -Time: 3463.033 ms -Time: 3247.791 ms -Time: 3064.752 ms -3 -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -Time: 8437.169 ms -Time: 8186.413 ms -Time: 7992.017 ms -3 -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -Time: 9021.759 ms -Time: 8915.718 ms -Time: 8927.481 ms -3 -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -Time: 8669.821 ms -Time: 8393.315 ms -Time: 8624.970 ms -3 -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -Time: 15132.099 ms -Time: 14950.178 ms -Time: 15234.422 ms -3 -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -Time: 1008.775 ms -Time: 934.589 ms -Time: 975.342 ms -3 -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -Time: 11058.183 ms -Time: 3258.070 ms -Time: 3176.875 ms -3 -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -Time: 12263.435 ms -Time: 3518.802 ms -Time: 3937.050 ms -3 -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -Time: 24152.214 ms -Time: 4863.278 ms -Time: 4887.644 ms -3 -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -Time: 67151.698 ms -Time: 20400.002 ms -Time: 20261.041 ms -3 -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -Time: 2579.895 ms -Time: 1661.144 ms -Time: 1904.318 ms -3 -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -Time: 1728.308 ms -Time: 1895.359 ms -Time: 1251.501 ms -3 -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -Time: 2441.544 ms -Time: 1465.950 ms -Time: 1882.380 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; -Time: 11117.452 ms -Time: 6095.514 ms -Time: 6022.919 ms -3 -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www.)?([^/]+)/.*$', '1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; -Time: 83040.984 ms -Time: 82978.396 ms -Time: 82867.763 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; -Time: 82544.095 ms -Time: 82542.673 ms -Time: 83983.646 ms -3 -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -Time: 3096.198 ms -Time: 3222.251 ms -Time: 3194.314 ms -3 -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -Time: 5775.759 ms -Time: 4059.735 ms -Time: 4003.655 ms -3 -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -Time: 43682.613 ms -Time: 40437.523 ms -Time: 40107.640 ms -3 -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -Time: 18097.819 ms -Time: 13274.130 ms -Time: 12889.385 ms -3 -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -Time: 17318.672 ms -Time: 13541.070 ms -Time: 13592.715 ms -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; -Time: 9724.053 ms -Time: 9900.294 ms -Time: 10017.686 ms -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 IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -Time: 294.344 ms -Time: 169.606 ms -Time: 173.804 ms -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 IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -Time: 162.524 ms -Time: 117.489 ms -Time: 115.532 ms -3 -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -Time: 145.205 ms -Time: 98.342 ms -Time: 97.275 ms -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 IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -Time: 334.809 ms -Time: 275.365 ms -Time: 265.053 ms -3 -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -Time: 154.522 ms -Time: 107.654 ms -Time: 105.290 ms -3 -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -Time: 158.957 ms -Time: 117.284 ms -Time: 119.068 ms -3 -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; -Time: 193.756 ms -Time: 144.787 ms -Time: 145.485 ms diff --git a/benchmark/greenplum/queries.sql b/benchmark/greenplum/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/greenplum/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/greenplum/results/c6a.4xlarge.json b/benchmark/greenplum/results/c6a.4xlarge.json deleted file mode 100644 index 68052fdd12b..00000000000 --- a/benchmark/greenplum/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Greenplum", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C", "column-oriented", "PostgreSQL compatible"], - - "load_time": 1080, - "data_size": 42000000000, - - "result": [ -[2.20009,1.45883,1.13573], -[1.28661,1.07897,1.01856], -[2.2582,2.42193,2.59693], -[1.89371,1.9246,1.73866], -[9.54471,9.71956,10.0357], -[4.55957,4.10112,4.70458], -[1.65167,1.89767,1.97051], -[1.02537,0.999323,1.00524], -[7.89792,7.75718,8.01219], -[11.3779,11.6003,11.2552], -[2.11955,2.05003,1.9553], -[2.01997,1.97932,1.98284], -[2.88666,2.92828,2.96822], -[4.06272,4.01771,4.03011], -[3.46303,3.24779,3.06475], -[8.43717,8.18641,7.99202], -[9.02176,8.91572,8.92748], -[8.66982,8.39332,8.62497], -[15.1321,14.9502,15.2344], -[1.00877,0.934589,0.975342], -[11.0582,3.25807,3.17687], -[12.2634,3.5188,3.93705], -[24.1522,4.86328,4.88764], -[67.1517,20.4,20.261], -[2.5799,1.66114,1.90432], -[1.72831,1.89536,1.2515], -[2.44154,1.46595,1.88238], -[11.1175,6.09551,6.02292], -[83.041,82.9784,82.8678], -[82.5441,82.5427,83.9836], -[3.0962,3.22225,3.19431], -[5.77576,4.05973,4.00366], -[43.6826,40.4375,40.1076], -[18.0978,13.2741,12.8894], -[17.3187,13.5411,13.5927], -[9.72405,9.90029,10.0177], -[0.294344,0.169606,0.173804], -[0.162524,0.117489,0.115532], -[0.145205,0.098342,0.097275], -[0.334809,0.275365,0.265053], -[0.154522,0.107654,0.10529], -[0.158957,0.117284,0.119068], -[0.193756,0.144787,0.145485] -] -} diff --git a/benchmark/greenplum/run.sh b/benchmark/greenplum/run.sh deleted file mode 100755 index 1eb622de977..00000000000 --- a/benchmark/greenplum/run.sh +++ /dev/null @@ -1,15 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - echo "$query"; - for i in $(seq 1 $TRIES); do - echo '\timing' > /tmp/query_temp.sql - echo "$query" >> /tmp/query_temp.sql - psql -d postgres -t -f /tmp/query_temp.sql | grep 'Time' - done; -done; diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh deleted file mode 120000 index 8b5c8838fb3..00000000000 --- a/benchmark/hardware.sh +++ /dev/null @@ -1 +0,0 @@ -hardware/hardware.sh \ No newline at end of file diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh new file mode 100755 index 00000000000..635f3a9eb61 --- /dev/null +++ b/benchmark/hardware.sh @@ -0,0 +1,247 @@ +#!/bin/bash -e + +QUERIES_FILE="queries.sql" +TRIES=3 + +mkdir -p clickhouse-benchmark +pushd clickhouse-benchmark + +# Download the binary +if [[ ! -x clickhouse ]]; then + curl https://clickhouse.com/ | sh +fi + +if [[ ! -f $QUERIES_FILE ]]; then + wget "https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/hardware/$QUERIES_FILE" +fi + +uptime + +echo "Starting clickhouse-server" + +./clickhouse server >/dev/null 2>&1 & +PID=$! + +function finish { + kill $PID + wait +} +trap finish EXIT + +echo "Waiting for clickhouse-server to start" + +for i in {1..30}; do + sleep 1 + ./clickhouse client --query "SELECT 'Ok.'" 2>/dev/null && break || echo -n '.' + if [[ $i == 30 ]]; then exit 1; fi +done + +if [[ $(./clickhouse client --query "EXISTS hits") == '1' && $(./clickhouse client --query "SELECT count() FROM hits") == '100000000' ]]; then + echo "Dataset already downloaded" +else + echo "Will download the dataset" + if [ "`uname`" = "Darwin" ] + then + ./clickhouse client --receive_timeout 1000 --max_insert_threads $(sysctl -n hw.ncpu) --progress --query " + CREATE OR REPLACE TABLE hits ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) + AS SELECT * FROM url('https://datasets.clickhouse.com/hits/native/hits_100m_obfuscated_{0..255}.native.zst')" + else + ./clickhouse client --receive_timeout 1000 --max_insert_threads $(nproc || 4) --progress --query " + CREATE OR REPLACE TABLE hits ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) + AS SELECT * FROM url('https://datasets.clickhouse.com/hits/native/hits_100m_obfuscated_{0..255}.native.zst')" + fi + ./clickhouse client --query "SELECT 'The dataset size is: ', count() FROM hits" +fi + +if [[ $(./clickhouse client --query "SELECT count() FROM system.parts WHERE table = 'hits' AND database = 'default' AND active") == '1' ]]; then + echo "Dataset already prepared" +else + echo "Will prepare the dataset" + ./clickhouse client --receive_timeout 1000 --query "OPTIMIZE TABLE hits FINAL" +fi + +echo +echo "Will perform benchmark. Results:" +echo + +>result.csv +QUERY_NUM=1 + +cat "$QUERIES_FILE" | sed "s/{table}/hits/g" | while read query; do + sync + if [ "`uname`" = "Darwin" ] + then + sudo purge > /dev/null + else + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + fi + + echo -n "[" + for i in $(seq 1 $TRIES); do + RES=$(./clickhouse client --time --format=Null --query="$query" 2>&1 ||:) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + + echo "${QUERY_NUM},${i},${RES}" >> result.csv + done + echo "]," + + QUERY_NUM=$((QUERY_NUM + 1)) +done + + +echo +echo "Benchmark complete. System info:" +echo + +touch {cpu_model,cpu,df,memory,memory_total,blk,mdstat,instance}.txt + +if [ "`uname`" = "Darwin" ] +then + echo '----Version, build id-----------' + ./clickhouse local --query "SELECT format('Version: {}', version())" + ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw + ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" + echo '----CPU-------------------------' + sysctl hw.model | tee cpu_model.txt + sysctl -a | grep -E 'hw.activecpu|hw.memsize|hw.byteorder|cachesize' | tee cpu.txt + echo '----Disk Free and Total--------' + df -h . | tee df.txt + echo '----Memory Free and Total-------' + vm_stat | tee memory.txt + echo '----Physical Memory Amount------' + ls -l /var/vm | tee memory_total.txt + echo '--------------------------------' +else + echo '----Version, build id-----------' + ./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())" + ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw + ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" + echo '----CPU-------------------------' + cat /proc/cpuinfo | grep -i -F 'model name' | uniq | tee cpu_model.txt + lscpu | tee cpu.txt + echo '----Block Devices---------------' + lsblk | tee blk.txt + echo '----Disk Free and Total--------' + df -h . | tee df.txt + echo '----Memory Free and Total-------' + free -h | tee memory.txt + echo '----Physical Memory Amount------' + cat /proc/meminfo | grep MemTotal | tee memory_total.txt + echo '----RAID Info-------------------' + cat /proc/mdstat| tee mdstat.txt + echo '--------------------------------' +fi +echo + +echo "Instance type from IMDS (if available):" +curl -s --connect-timeout 1 'http://169.254.169.254/latest/meta-data/instance-type' | tee instance.txt +echo + +echo "Uploading the results (if possible)" + +UUID=$(./clickhouse local --query "SELECT generateUUIDv4()") + +./clickhouse local --query " + SELECT + '${UUID}' AS run_id, + version() AS version, + now() AS test_time, + (SELECT value FROM system.settings WHERE name = 'max_threads') AS threads, + filesystemCapacity() AS fs_capacity, + filesystemAvailable() AS fs_available, + file('cpu_model.txt') AS cpu_model, + file('cpu.txt') AS cpu, + file('df.txt') AS df, + file('memory.txt') AS memory, + file('memory_total.txt') AS memory_total, + file('blk.txt') AS blk, + file('mdstat.txt') AS mdstat, + file('instance.txt') AS instance +" | tee meta.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query " + INSERT INTO benchmark_runs + (run_id, version, test_time, threads, fs_capacity, fs_available, cpu_model, cpu, df, memory, memory_total, blk, mdstat, instance) + FORMAT TSV" || echo "Cannot upload results." + +./clickhouse local --query " + SELECT + '${UUID}' AS run_id, + c1 AS query_num, + c2 AS try_num, + c3 AS time + FROM file('result.csv') +" | tee results.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query " + INSERT INTO benchmark_results + (run_id, query_num, try_num, time) + FORMAT TSV" || echo "Cannot upload results. Please send the output to feedback@clickhouse.com" + +< 1 + ) + GROUP BY run_id + ORDER BY k ASC +) AS t +INNER JOIN benchmark_runs USING (run_id) + +//// diff --git a/benchmark/hardware/benchmark-chyt.sh b/benchmark/hardware/benchmark-chyt.sh deleted file mode 100755 index 778ce4f86ce..00000000000 --- a/benchmark/hardware/benchmark-chyt.sh +++ /dev/null @@ -1,22 +0,0 @@ -#!/usr/bin/env bash - -QUERIES_FILE="queries.sql" -TABLE=$1 -TRIES=3 - -cat "$QUERIES_FILE" | sed "s|{table}|\"${TABLE}\"|g" | while read query; do - - echo -n "[" - for i in $(seq 1 $TRIES); do - while true; do - RES=$(command time -f %e -o /dev/stdout curl -sS -G --data-urlencode "query=$query" --data "default_format=Null&max_memory_usage=100000000000&max_memory_usage_for_all_queries=100000000000&max_concurrent_queries_for_user=100&database=*$YT_CLIQUE_ID" --location-trusted -H "Authorization: OAuth $YT_TOKEN" "$YT_PROXY.yt.yandex.net/query" 2>/dev/null); - if [[ $? == 0 ]]; then - [[ $RES =~ 'fail|Exception' ]] || break; - fi - done - - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - done - echo "]," -done diff --git a/benchmark/hardware/benchmark-new.sh b/benchmark/hardware/benchmark-new.sh deleted file mode 100755 index 0c4cad6e5e3..00000000000 --- a/benchmark/hardware/benchmark-new.sh +++ /dev/null @@ -1,29 +0,0 @@ -#!/usr/bin/env bash - -QUERIES_FILE="queries.sql" -TABLE=$1 -TRIES=3 - -if [ -x ./clickhouse ] -then - CLICKHOUSE_CLIENT="./clickhouse client" -elif command -v clickhouse-client >/dev/null 2>&1 -then - CLICKHOUSE_CLIENT="clickhouse-client" -else - echo "clickhouse-client is not found" - exit 1 -fi - -cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - - echo -n "[" - for i in $(seq 1 $TRIES); do - RES=$(${CLICKHOUSE_CLIENT} --time --format=Null --max_memory_usage=100G --query="$query" 2>&1) - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - done - echo "]," -done diff --git a/benchmark/hardware/benchmark-yql.sh b/benchmark/hardware/benchmark-yql.sh deleted file mode 100755 index 7d30d39e7d3..00000000000 --- a/benchmark/hardware/benchmark-yql.sh +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/env bash - -QUERIES_FILE="queries.sql" -TABLE=$1 -TRIES=3 - -cat "$QUERIES_FILE" | sed "s|{table}|\"${TABLE}\"|g" | while read query; do - - echo -n "[" - for i in $(seq 1 $TRIES); do - while true; do - RES=$(command time -f %e -o time ./yql --clickhouse --syntax-version 1 -f empty <<< "USE chyt.hume; PRAGMA max_memory_usage = 100000000000; PRAGMA max_memory_usage_for_all_queries = 100000000000; $query" >/dev/null 2>&1 && cat time) && break; - done - - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - done - echo "]," -done diff --git a/benchmark/hardware/benchmark_cloud.sh b/benchmark/hardware/benchmark_cloud.sh deleted file mode 100755 index 01376e4009e..00000000000 --- a/benchmark/hardware/benchmark_cloud.sh +++ /dev/null @@ -1,43 +0,0 @@ -#!/usr/bin/env bash - -QUERIES_FILE="queries.sql" -TABLE=$1 -TRIES=3 - -PARAMS="--host ... --secure --password ..." - -if [ -x ./clickhouse ] -then - CLICKHOUSE_CLIENT="./clickhouse client" -elif command -v clickhouse-client >/dev/null 2>&1 -then - CLICKHOUSE_CLIENT="clickhouse-client" -else - echo "clickhouse-client is not found" - exit 1 -fi - -QUERY_ID_PREFIX="benchmark_$RANDOM" -QUERY_NUM=1 - -cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query - do - for i in $(seq 1 $TRIES) - do - QUERY_ID="${QUERY_ID_PREFIX}_${QUERY_NUM}_${i}" - ${CLICKHOUSE_CLIENT} ${PARAMS} --query_id "${QUERY_ID}" --format=Null --max_memory_usage=100G --query="$query" - echo -n '.' - done - QUERY_NUM=$((QUERY_NUM + 1)) - echo -done - -sleep 10 - -${CLICKHOUSE_CLIENT} ${PARAMS} --query " - WITH extractGroups(query_id, '(\d+)_(\d+)\$') AS num_run, num_run[1]::UInt8 AS num, num_run[2]::UInt8 AS run - SELECT groupArrayInsertAt(query_duration_ms / 1000, (run - 1)::UInt8)::String || ',' - FROM clusterAllReplicas(default, system.query_log) - WHERE event_date >= yesterday() AND type = 2 AND query_id LIKE '${QUERY_ID_PREFIX}%' - GROUP BY num ORDER BY num FORMAT TSV -" diff --git a/benchmark/hardware/hardware.sh b/benchmark/hardware/hardware.sh deleted file mode 100755 index 635f3a9eb61..00000000000 --- a/benchmark/hardware/hardware.sh +++ /dev/null @@ -1,247 +0,0 @@ -#!/bin/bash -e - -QUERIES_FILE="queries.sql" -TRIES=3 - -mkdir -p clickhouse-benchmark -pushd clickhouse-benchmark - -# Download the binary -if [[ ! -x clickhouse ]]; then - curl https://clickhouse.com/ | sh -fi - -if [[ ! -f $QUERIES_FILE ]]; then - wget "https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/hardware/$QUERIES_FILE" -fi - -uptime - -echo "Starting clickhouse-server" - -./clickhouse server >/dev/null 2>&1 & -PID=$! - -function finish { - kill $PID - wait -} -trap finish EXIT - -echo "Waiting for clickhouse-server to start" - -for i in {1..30}; do - sleep 1 - ./clickhouse client --query "SELECT 'Ok.'" 2>/dev/null && break || echo -n '.' - if [[ $i == 30 ]]; then exit 1; fi -done - -if [[ $(./clickhouse client --query "EXISTS hits") == '1' && $(./clickhouse client --query "SELECT count() FROM hits") == '100000000' ]]; then - echo "Dataset already downloaded" -else - echo "Will download the dataset" - if [ "`uname`" = "Darwin" ] - then - ./clickhouse client --receive_timeout 1000 --max_insert_threads $(sysctl -n hw.ncpu) --progress --query " - CREATE OR REPLACE TABLE hits ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) - AS SELECT * FROM url('https://datasets.clickhouse.com/hits/native/hits_100m_obfuscated_{0..255}.native.zst')" - else - ./clickhouse client --receive_timeout 1000 --max_insert_threads $(nproc || 4) --progress --query " - CREATE OR REPLACE TABLE hits ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) - AS SELECT * FROM url('https://datasets.clickhouse.com/hits/native/hits_100m_obfuscated_{0..255}.native.zst')" - fi - ./clickhouse client --query "SELECT 'The dataset size is: ', count() FROM hits" -fi - -if [[ $(./clickhouse client --query "SELECT count() FROM system.parts WHERE table = 'hits' AND database = 'default' AND active") == '1' ]]; then - echo "Dataset already prepared" -else - echo "Will prepare the dataset" - ./clickhouse client --receive_timeout 1000 --query "OPTIMIZE TABLE hits FINAL" -fi - -echo -echo "Will perform benchmark. Results:" -echo - ->result.csv -QUERY_NUM=1 - -cat "$QUERIES_FILE" | sed "s/{table}/hits/g" | while read query; do - sync - if [ "`uname`" = "Darwin" ] - then - sudo purge > /dev/null - else - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - fi - - echo -n "[" - for i in $(seq 1 $TRIES); do - RES=$(./clickhouse client --time --format=Null --query="$query" 2>&1 ||:) - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - - echo "${QUERY_NUM},${i},${RES}" >> result.csv - done - echo "]," - - QUERY_NUM=$((QUERY_NUM + 1)) -done - - -echo -echo "Benchmark complete. System info:" -echo - -touch {cpu_model,cpu,df,memory,memory_total,blk,mdstat,instance}.txt - -if [ "`uname`" = "Darwin" ] -then - echo '----Version, build id-----------' - ./clickhouse local --query "SELECT format('Version: {}', version())" - ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw - ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" - echo '----CPU-------------------------' - sysctl hw.model | tee cpu_model.txt - sysctl -a | grep -E 'hw.activecpu|hw.memsize|hw.byteorder|cachesize' | tee cpu.txt - echo '----Disk Free and Total--------' - df -h . | tee df.txt - echo '----Memory Free and Total-------' - vm_stat | tee memory.txt - echo '----Physical Memory Amount------' - ls -l /var/vm | tee memory_total.txt - echo '--------------------------------' -else - echo '----Version, build id-----------' - ./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())" - ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw - ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" - echo '----CPU-------------------------' - cat /proc/cpuinfo | grep -i -F 'model name' | uniq | tee cpu_model.txt - lscpu | tee cpu.txt - echo '----Block Devices---------------' - lsblk | tee blk.txt - echo '----Disk Free and Total--------' - df -h . | tee df.txt - echo '----Memory Free and Total-------' - free -h | tee memory.txt - echo '----Physical Memory Amount------' - cat /proc/meminfo | grep MemTotal | tee memory_total.txt - echo '----RAID Info-------------------' - cat /proc/mdstat| tee mdstat.txt - echo '--------------------------------' -fi -echo - -echo "Instance type from IMDS (if available):" -curl -s --connect-timeout 1 'http://169.254.169.254/latest/meta-data/instance-type' | tee instance.txt -echo - -echo "Uploading the results (if possible)" - -UUID=$(./clickhouse local --query "SELECT generateUUIDv4()") - -./clickhouse local --query " - SELECT - '${UUID}' AS run_id, - version() AS version, - now() AS test_time, - (SELECT value FROM system.settings WHERE name = 'max_threads') AS threads, - filesystemCapacity() AS fs_capacity, - filesystemAvailable() AS fs_available, - file('cpu_model.txt') AS cpu_model, - file('cpu.txt') AS cpu, - file('df.txt') AS df, - file('memory.txt') AS memory, - file('memory_total.txt') AS memory_total, - file('blk.txt') AS blk, - file('mdstat.txt') AS mdstat, - file('instance.txt') AS instance -" | tee meta.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query " - INSERT INTO benchmark_runs - (run_id, version, test_time, threads, fs_capacity, fs_available, cpu_model, cpu, df, memory, memory_total, blk, mdstat, instance) - FORMAT TSV" || echo "Cannot upload results." - -./clickhouse local --query " - SELECT - '${UUID}' AS run_id, - c1 AS query_num, - c2 AS try_num, - c3 AS time - FROM file('result.csv') -" | tee results.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query " - INSERT INTO benchmark_results - (run_id, query_num, try_num, time) - FORMAT TSV" || echo "Cannot upload results. Please send the output to feedback@clickhouse.com" - -< 1 - ) - GROUP BY run_id - ORDER BY k ASC -) AS t -INNER JOIN benchmark_runs USING (run_id) - -//// diff --git a/benchmark/hardware/queries.sql b/benchmark/hardware/queries.sql deleted file mode 100644 index 89c4616c642..00000000000 --- a/benchmark/hardware/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -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 uniq(UserID) FROM {table} ; -SELECT uniq(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, uniq(UserID) AS u FROM {table} GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, sum(AdvEngineID), count() AS c, avg(ResolutionWidth), uniq(UserID) FROM {table} GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, uniq(UserID) AS u FROM {table} WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, uniq(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, uniq(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, toMinute(EventTime) AS m, SearchPhrase, count() FROM {table} GROUP BY UserID, m, SearchPhrase ORDER BY count() DESC LIMIT 10; -SELECT UserID FROM {table} WHERE UserID = 12345678901234567890; -SELECT count() FROM {table} WHERE URL LIKE '%metrika%'; -SELECT SearchPhrase, any(URL), count() AS c FROM {table} WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, any(URL), any(Title), count() AS c, uniq(UserID) FROM {table} WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM {table} WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -SELECT 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 c > 100000 ORDER BY l DESC LIMIT 25; -SELECT domainWithoutWWW(Referer) AS key, avg(length(Referer)) AS l, count() AS c, any(Referer) FROM {table} WHERE Referer != '' GROUP BY key HAVING c > 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 AS x, x - 1, x - 2, x - 3, count() AS c FROM {table} GROUP BY x, x - 1, x - 2, x - 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 NOT DontCountHits AND NOT Refresh AND notEmpty(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 NOT DontCountHits AND NOT Refresh AND notEmpty(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 NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -SELECT TraficSourceID, SearchEngineID, AdvEngineID, ((SearchEngineID = 0 AND AdvEngineID = 0) ? Referer : '') AS Src, URL AS Dst, count() AS PageViews FROM {table} 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 {table} 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 WindowClientWidth, WindowClientHeight, count() AS PageViews FROM {table} 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 toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM {table} 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; diff --git a/benchmark/heavyai/benchmark.sh b/benchmark/heavyai/benchmark.sh deleted file mode 100755 index 366d19bac34..00000000000 --- a/benchmark/heavyai/benchmark.sh +++ /dev/null @@ -1,50 +0,0 @@ -#!/bin/bash - -# Install - -sudo apt update -sudo apt install default-jre-headless -sudo apt install apt-transport-https -sudo useradd -U -m heavyai -sudo curl https://releases.heavy.ai/GPG-KEY-heavyai | sudo apt-key add - -echo "deb https://releases.heavy.ai/os/apt/ stable cpu" | sudo tee /etc/apt/sources.list.d/heavyai.list -sudo apt update -sudo apt install heavyai - -export HEAVYAI_USER=heavyai -export HEAVYAI_GROUP=heavyai -export HEAVYAI_STORAGE=/var/lib/heavyai -export HEAVYAI_PATH=/opt/heavyai -export HEAVYAI_LOG=/var/lib/heavyai/data/mapd_log - -cd $HEAVYAI_PATH/systemd -./install_heavy_systemd.sh - -# Press Enter multiple times. - -sudo systemctl start heavydb -sudo systemctl enable heavydb - -# Load the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' -gzip -d hits.csv.gz -chmod 777 ~ hits.csv - -sudo bash -c "echo 'allowed-import-paths = [\"/home/ubuntu/\"]' > /var/lib/heavyai/heavy.conf_" -sudo bash -c "cat /var/lib/heavyai/heavy.conf >> /var/lib/heavyai/heavy.conf_" -sudo bash -c "mv /var/lib/heavyai/heavy.conf_ /var/lib/heavyai/heavy.conf && chown heavyai /var/lib/heavyai/heavy.conf" -sudo systemctl restart heavydb - -/opt/heavyai/bin/heavysql -t -p HyperInteractive < create.sql -time /opt/heavyai/bin/heavysql -t -p HyperInteractive <<< "COPY hits FROM '$(pwd)/hits.csv' WITH (HEADER = 'false');" - -# Loaded: 99997497 recs, Rejected: 0 recs in 572.633000 secs - -./run.sh 2>&1 | tee log.txt - -du -bcs /var/lib/heavyai/ - -cat log.txt | grep -P 'Total time|null' | sed -r -e 's/^.*Total time: ([0-9]+) ms$/\1/' | - awk '{ if ($1 == "null") { print } else { print $1 / 1000 } }' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/heavyai/create.sql b/benchmark/heavyai/create.sql deleted file mode 100644 index 41c961c00fc..00000000000 --- a/benchmark/heavyai/create.sql +++ /dev/null @@ -1,108 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -); diff --git a/benchmark/heavyai/queries.sql b/benchmark/heavyai/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/heavyai/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/heavyai/results/c6a.4xlarge.json b/benchmark/heavyai/results/c6a.4xlarge.json deleted file mode 100644 index 61307fec311..00000000000 --- a/benchmark/heavyai/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "HeavyAI", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "Previous names: OmniSci, mapD. Many queries cannot run due to errors and limitations.", - - "tags": ["C++", "column-oriented"], - - "load_time": 572.633, - "data_size": 50887437386, - - "result": [ -[6.525,0.022,0.029], -[0.301,0.042,0.04], -[0.287,0.095,0.093], -[2.572,0.039,0.04], -[null,null,null], -[null,null,null], -[7.327,0.093,0.097], -[0.244,0.043,0.038], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[2.939,0.295,0.294], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[4.716,3.91,3.955], -[null,null,null], -[0.154,0.083,0.106], -[14.426,0.07,0.071], -[null,null,null], -[null,null,null], -[null,null,null], -[2.276,0.258,0.272], -[null,null,null], -[null,null,null], -[null,null,null], -[null,null,null], -[1.832,1.64,1.602], -[null,null,null], -[null,null,null], -[null,null,null], -[14.811,0.494,0.497], -[null,null,null], -[null,null,null], -[1.941,0.255,0.255], -[5.457,0.172,0.283], -[0.476,0.269,0.256], -[14.239,0.179,0.178], -[3.992,0.112,0.112], -[1.031,0.116,0.116], -[1.365,0.089,0.088] -] -} diff --git a/benchmark/heavyai/run.sh b/benchmark/heavyai/run.sh deleted file mode 100755 index 97387bee42b..00000000000 --- a/benchmark/heavyai/run.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - echo "$query"; - for i in $(seq 1 $TRIES); do - /opt/heavyai/bin/heavysql -t -p HyperInteractive <<< "${query}" | grep 'Total time' || echo 'null' - done; -done; diff --git a/benchmark/index.html b/benchmark/index.html deleted file mode 100644 index ce4e6044353..00000000000 --- a/benchmark/index.html +++ /dev/null @@ -1,956 +0,0 @@ - - - - - ClickBench — a Benchmark For Analytical DBMS - - - - - - - - - - - -
- 🌚🌞 -

ClickBench — a Benchmark For Analytical DBMS

- Methodology | Reproduce and Validate the Results | Add a System | Report Mistake | Hardware Benchmark -
- - - - - - - - - - - - - - - - - - - - - - -
System: - All -
Type: - All -
Machine: - All -
Cluster size: - All -
Metric: - Cold Run - Hot Run - Load Time - Storage Size -
- - - - - - - - - - -
- System & Machine - - Relative time (lower is better) -
- -
Nothing selected
- -
-

Detailed Comparison

-
- - - - - - - - -
- - - - diff --git a/benchmark/infobright/benchmark.sh b/benchmark/infobright/benchmark.sh deleted file mode 100755 index 5d80afed863..00000000000 --- a/benchmark/infobright/benchmark.sh +++ /dev/null @@ -1,38 +0,0 @@ -#!/bin/bash - -# Install - -sudo apt-get update -sudo apt-get install -y docker.io - -mkdir infobright -sudo docker run --name mysql_ib -e MYSQL_ROOT_PASSWORD=mypass -v $(pwd)/infobright:/mnt/mysql_data -p 5029:5029 -p 5555 -d flolas/infobright - -sudo docker run -it --rm --network host mysql:5 mysql --host 127.0.0.1 --port 5029 --user=root --password=mypass -e "CREATE DATABASE test" -sudo docker run -it --rm --network host mysql:5 mysql --host 127.0.0.1 --port 5029 --user=root --password=mypass --database=test -e "$(cat create.sql)" - -# Load the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -# ERROR 2 (HY000) at line 1: Wrong data or column definition. Row: 93557187, field: 100. -head -n 90000000 hits.tsv > hits90m.tsv - -time sudo docker run -it --rm --volume $(pwd):/workdir --network host mysql:5 mysql --host 127.0.0.1 --port 5029 --user=root --password=mypass --database=test -e " - LOAD DATA LOCAL INFILE '/workdir/hits90m.tsv' INTO TABLE test.hits - FIELDS TERMINATED BY '\\t' ENCLOSED BY '' ESCAPED BY '\\\\' LINES TERMINATED BY '\\n' STARTING BY ''" - -# 38m37.466s - -sudo docker exec mysql_ib du -bcs /mnt/mysql_data/ /usr/local/infobright-4.0.7-x86_64/cache - -# 13 760 341 294 - -./run.sh 2>&1 | log - -cat log.txt | - grep -P 'rows? in set|Empty set|^ERROR' | - sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) days? )?(([0-9.]+) hours? )?(([0-9.]+) min )?([0-9.]+) sec\).*?$/\2,\4,\6,\7/' | - awk -F, '{ if ($1 == "null") { print } else { print $1 * 86400 + $2 * 3600 + $3 * 60 + $4 } }' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/infobright/create.sql b/benchmark/infobright/create.sql deleted file mode 100644 index 4d23eaac635..00000000000 --- a/benchmark/infobright/create.sql +++ /dev/null @@ -1,108 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT, - Referer TEXT, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT, - Params TEXT, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT, - BrowserCountry TEXT, - SocialNetwork TEXT, - SocialAction TEXT, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT, - ParamCurrency TEXT, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT, - OpenstatCampaignID TEXT, - OpenstatAdID TEXT, - OpenstatSourceID TEXT, - UTMSource TEXT, - UTMMedium TEXT, - UTMCampaign TEXT, - UTMContent TEXT, - UTMTerm TEXT, - FromTag TEXT, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -); diff --git a/benchmark/infobright/queries.sql b/benchmark/infobright/queries.sql deleted file mode 100644 index ea2bde47802..00000000000 --- a/benchmark/infobright/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/infobright/results/c6a.4xlarge.json b/benchmark/infobright/results/c6a.4xlarge.json deleted file mode 100644 index c9157d0b101..00000000000 --- a/benchmark/infobright/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Infobright", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "Only 90% of data successfully loaded. Some queries run for days.", - - "tags": ["C++", "column-oriented", "MySQL compatible"], - - "load_time": 2317, - "data_size": 13760341294, - - "result": [ -[0.01, 0, 0], -[2.39, 2.4, 2.44], -[0, 0, 0], -[7.21, 6.04, 6.91], -[16.09, 16.86, 15.69], -[48.8, 42.37, 48.63], -[0, 0, 0], -[3.48, 2.42, 2.42], -[23.56, 24.78, 22.21], -[32.87, 31.71, 34.48], -[14.8, 14.83, 14.11], -[16.7, 16.53, 17.37], -[1752.91, 1999.88, 1961.4], -[1193.43, 1167, 1220.47], -[2184.81, 2316.12, 2224.14], -[32.58, 30.69, 31.58], -[300.17, 16221.33, 16168.44], -[122.4, 120.49, 124.67], -[78927.44, 79250.44, 78504.89], -[3.38, 1.22, 1.21], -[289.73, 302.3, 285.83], -[415.82, 389.23, 403.28], -[573.82, 590.81, 575.06], -[300.13, 293.96, 285.64], -[41.42, 37.48, 39.64], -[75.2, 75.37, 72.07], -[39.22, 41.52, 40.11], -[449.56, 445.03, 448.68], -[null, null, null], -[450.87, 488.3, 453.83], -[58.69, 59.29, 58.07], -[84.47, 78.92, 79.38], -[517.97, 520.29, 504.96], -[182468.89, 182468.89, 182468.89], -[182468.89, 182468.89, 182468.89], -[68.43, 66.93, 67.68], -[8.3, 3.62, 3.61], -[1.04, 0.62, 0.62], -[0.22, 0.18, 0.18], -[567.78, 566.52, 563.02], -[0.29, 0.14, 0.11], -[0.17, 0.08, 0.08], -[1.37, 1.34, 1.32] -] -} diff --git a/benchmark/infobright/run.sh b/benchmark/infobright/run.sh deleted file mode 100755 index 36e5fa167fb..00000000000 --- a/benchmark/infobright/run.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - for i in $(seq 1 $TRIES); do - sudo docker run --rm --network host mysql:5 mysql --host 127.0.0.1 --port 5029 --user=root --password=mypass --database=test -vvv -e "${query}" - done; -done; diff --git a/benchmark/locustdb/README.md b/benchmark/locustdb/README.md deleted file mode 100644 index fb40147cf72..00000000000 --- a/benchmark/locustdb/README.md +++ /dev/null @@ -1,24 +0,0 @@ -This system does not work at all: - -``` -locustdb> SELECT * FROM default LIMIT 1 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -note: run with `RUST_BACKTRACE=1` environment variable to display a backtrace -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -``` - -It is memory-safe and blazing fast. diff --git a/benchmark/locustdb/benchmark.sh b/benchmark/locustdb/benchmark.sh deleted file mode 100755 index 89cb950d90f..00000000000 --- a/benchmark/locustdb/benchmark.sh +++ /dev/null @@ -1,46 +0,0 @@ -#!/bin/bash - -# https://rustup.rs/ -curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -source $HOME/.cargo/env - -sudo apt-get update -sudo apt-get install -y git - -git clone https://github.com/cswinter/LocustDB.git -cd LocustDB - -sudo apt-get install -y g++ capnproto libclang-14-dev - -cargo build --features "enable_rocksdb" --features "enable_lz4" --release - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' -gzip -d hits.csv.gz - -target/release/repl --load hits.csv --db-path db - -# Loaded data in 920s. -# Table `default` (99997496 rows, 15.0GiB) - -# SELECT * FROM default LIMIT 1 - -# And it immediately panicked and hung: - -#locustdb> SELECT * FROM default LIMIT 1 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#note: run with `RUST_BACKTRACE=1` environment variable to display a backtrace -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 -#thread '' panicked at 'index out of bounds: the len is 65536 but the index is 65536', src/stringpack.rs:91:15 diff --git a/benchmark/mariadb-columnstore/README.md b/benchmark/mariadb-columnstore/README.md deleted file mode 100644 index 6be1ce43e2b..00000000000 --- a/benchmark/mariadb-columnstore/README.md +++ /dev/null @@ -1,9 +0,0 @@ -MariaDB ColumnStore failed after 5 minutes of data loading: - -``` -ubuntu@ip-172-31-4-179:~$ time mysql --password="${PASSWORD}" --host 127.0.0.1 test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" -ERROR 1030 (HY000) at line 1: Got error -1 "Internal error < 0 (Not system error)" from storage engine ColumnStore -``` - -They don't have an issue tracker on GitHub, only JIRA. -JIRA requires login, but does not support SSO. diff --git a/benchmark/mariadb-columnstore/benchmark.sh b/benchmark/mariadb-columnstore/benchmark.sh deleted file mode 100755 index 450120af282..00000000000 --- a/benchmark/mariadb-columnstore/benchmark.sh +++ /dev/null @@ -1,36 +0,0 @@ -#!/bin/bash - -# Install - -sudo apt-get update -sudo apt-get install -y docker.io -sudo docker run -d -p 3306:3306 -e ANALYTICS_ONLY=1 --name mcs_container mariadb/columnstore - -export PASSWORD="tsFgm457%3cj" -sudo docker exec mcs_container mariadb -e "GRANT ALL PRIVILEGES ON *.* TO 'ubuntu'@'%' IDENTIFIED BY '${PASSWORD}';" - -sudo apt-get install -y mariadb-client - -mysql --password="${PASSWORD}" --host 127.0.0.1 -e "CREATE DATABASE test" -mysql --password="${PASSWORD}" --host 127.0.0.1 test < create.sql - -# Load the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -time mysql --password="${PASSWORD}" --host 127.0.0.1 test -e " - LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits - FIELDS TERMINATED BY '\\t' ENCLOSED BY '' ESCAPED BY '\\\\' LINES TERMINATED BY '\\n' STARTING BY ''" - -# 41m47.856s - -./run.sh 2>&1 | tee log.txt - -sudo docker exec mcs_container du -bcs /var/lib/columnstore - -cat log.txt | - grep -P 'rows? in set|Empty set|^ERROR' | - sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | - awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/mariadb-columnstore/create.sql b/benchmark/mariadb-columnstore/create.sql deleted file mode 100644 index 565596f04c9..00000000000 --- a/benchmark/mariadb-columnstore/create.sql +++ /dev/null @@ -1,108 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT, - Referer TEXT, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT, - Params TEXT, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT, - BrowserCountry TEXT, - SocialNetwork TEXT, - SocialAction TEXT, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT, - ParamCurrency TEXT, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT, - OpenstatCampaignID TEXT, - OpenstatAdID TEXT, - OpenstatSourceID TEXT, - UTMSource TEXT, - UTMMedium TEXT, - UTMCampaign TEXT, - UTMContent TEXT, - UTMTerm TEXT, - FromTag TEXT, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -) ENGINE=Columnstore; diff --git a/benchmark/mariadb-columnstore/queries.sql b/benchmark/mariadb-columnstore/queries.sql deleted file mode 100644 index ea2bde47802..00000000000 --- a/benchmark/mariadb-columnstore/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/mariadb-columnstore/results/c6a.4xlarge.json b/benchmark/mariadb-columnstore/results/c6a.4xlarge.json deleted file mode 100644 index e318b64da41..00000000000 --- a/benchmark/mariadb-columnstore/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "MariaDB ColumnStore", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "Previous name: InfiniDB.", - - "tags": ["C++", "column-oriented", "MySQL compatible"], - - "load_time": 2507.8, - "data_size": 19712857022, - - "result": [ -[0.151,0.158,0.148], -[0.100,0.101,0.106], -[1.221,1.233,1.226], -[0.739,0.736,0.741], -[2.025,2.046,2.004], -[3.725,4.801,3.755], -[0.871,0.749,0.736], -[0.118,0.108,0.103], -[2.108,2.029,2.029], -[4.225,4.271,4.288], -[1.711,1.402,1.407], -[1.526,1.435,1.420], -[5.339,4.172,3.610], -[4.692,4.729,4.960], -[4.013,3.860,3.918], -[3.236,2.680,2.629], -[null,null,null], -[null,null,null], -[null,null,null], -[0.137,0.109,0.147], -[12.331,6.069,8.619], -[2.162,2.178,2.192], -[16.849,30.463,26.639], -[92,90.208,92.814], -[3.042,1.763,1.791], -[1.779,1.772,1.749], -[1.793,1.821,1.888], -[13.036,10.747,9.590], -[null,null,null], -[78.224,79.141,77.806], -[2.837,2.654,2.675], -[5.833,4.552,3.678], -[null,null,null], -[null,null,null], -[null,null,null], -[3.626,3.546,3.709], -[1.719,1.787,1.876], -[1.345,0.906,0.910], -[0.117,0.091,0.093], -[1.217,1.133,1.133], -[0.114,0.063,0.062], -[0.100,0.062,0.061], -[null,null,null] -] -} diff --git a/benchmark/mariadb-columnstore/run.sh b/benchmark/mariadb-columnstore/run.sh deleted file mode 100755 index b917a815d2c..00000000000 --- a/benchmark/mariadb-columnstore/run.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - for i in $(seq 1 $TRIES); do - mysql --password="${PASSWORD}" --host 127.0.0.1 -vvv test -e "${query}" - done; -done; diff --git a/benchmark/mariadb/benchmark.sh b/benchmark/mariadb/benchmark.sh deleted file mode 100755 index 33f69f45c39..00000000000 --- a/benchmark/mariadb/benchmark.sh +++ /dev/null @@ -1,29 +0,0 @@ -#!/bin/bash - -# Install - -sudo apt-get update -sudo apt-get install -y mariadb-server -sudo bash -c "echo -e '[mysql]\nlocal-infile=1\n\n[mysqld]\nlocal-infile=1\n' > /etc/mysql/conf.d/local_infile.cnf" -sudo service mariadb restart - -# Load the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -sudo mariadb -e "CREATE DATABASE test" -sudo mariadb test < create.sql -time sudo mariadb test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" - -# 2:23:45 elapsed - -./run.sh 2>&1 | tee log.txt - -sudo du -bcs /var/lib/mysql - -cat log.txt | - grep -P 'rows? in set|Empty set|^ERROR' | - sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) days? )?(([0-9.]+) hours? )?(([0-9.]+) min )?([0-9.]+) sec\).*?$/\2,\4,\6,\7/' | - awk -F, '{ if ($1 == "null") { print } else { print $1 * 86400 + $2 * 3600 + $3 * 60 + $4 } }' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/mariadb/create.sql b/benchmark/mariadb/create.sql deleted file mode 100644 index 1850bffedce..00000000000 --- a/benchmark/mariadb/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/mariadb/queries.sql b/benchmark/mariadb/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/mariadb/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/mariadb/results/c6a.4xlarge.json.shame b/benchmark/mariadb/results/c6a.4xlarge.json.shame deleted file mode 100644 index e402032e20c..00000000000 --- a/benchmark/mariadb/results/c6a.4xlarge.json.shame +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "MariaDB", - "date": "2022-07-12", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "For some reason many queries run in almost exactly one hour. Most likely a coincidence. It is installed from Debian packages, which might explain the abysmal performance.", - - "tags": ["C++", "row-oriented", "MySQL compatible"], - - "load_time": 8625, - "data_size": 107252505530, - - "result": [ -[7910.23,10229.8,3610.54], -[3595.79,3595.59,3595.21], -[3596.12,3595.34,3595.49], -[3597.22,3595.7,3595.09], -[3600.57,3596.08,3595.82], -[3596.57,3597.76,3598.47], -[3595.94,3595.36,3596.2], -[3595.95,3595.38,3595.45], -[3649.15,3624.26,3623.71], -[3645.39,3763.75,3630.71], -[4519.77,4552.84,4552.7], -[4628.21,4629.55,4628.16], -[3773.52,3774.44,3779.05], -[12198,12192.4,12192.6], -[3779.67,3774.97,3777.86], -[4277.99,4302.56,4275.29], -[4697.19,4367.3,4372.73], -[4401.2,4340.43,4310.13], -[5335.93,5373.04,5361.51], -[3596.91,3598.61,3598.71], -[3598.76,3598.56,3598.68], -[3598.76,3598.63,3599.39], -[3604.28,3603.71,3603.66], -[3598.83,3598.2,3598.19], -[3599.11,3598.1,3598.17], -[3599.04,3598.46,3598.13], -[3598.76,3598.71,3598.22], -[3599.02,3599.76,3598.18], -[3644.12,3716.36,3725.92], -[3933.26,3854.55,3784.74], -[3392.13,3402.26,3602.15], -[3717.3,3735.06,3723.07], -[6126.66,6135.63,6193.45], -[4029.78,3998.89,3952.7], -[3955.34,4029.28,3941.84], -[4881.07,4782.53,4900.72], -[24.455,23.862,25.173], -[24.195,23.529,23.473], -[20.719,21.629,21.582], -[24.577,25.877,25.953], -[20.686,20.787,21.199], -[20.036,17.973,22.759], -[null,null,null] -] -} diff --git a/benchmark/mariadb/run.sh b/benchmark/mariadb/run.sh deleted file mode 100755 index 5f80a539bbe..00000000000 --- a/benchmark/mariadb/run.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - for i in $(seq 1 $TRIES); do - sudo mariadb test -vvv -e "${query}" - done; -done; diff --git a/benchmark/monetdb/benchmark.sh b/benchmark/monetdb/benchmark.sh deleted file mode 100755 index f0afb400802..00000000000 --- a/benchmark/monetdb/benchmark.sh +++ /dev/null @@ -1,40 +0,0 @@ -#!/bin/bash - -# Install - -echo "deb https://dev.monetdb.org/downloads/deb/ $(lsb_release -cs) monetdb" | sudo tee /etc/apt/sources.list.d/monetdb.list - -sudo wget --output-document=/etc/apt/trusted.gpg.d/monetdb.gpg https://www.monetdb.org/downloads/MonetDB-GPG-KEY.gpg -sudo apt-get update -sudo apt-get install -y monetdb5-sql monetdb-client dos2unix - -sudo systemctl enable monetdbd -sudo systemctl start monetdbd - -sudo monetdbd create /var/lib/monetdb -sudo monetdbd start /var/lib/monetdb -sudo usermod -a -G monetdb $USER - -sudo monetdb create test -sudo monetdb release test - -sudo apt-get install -y expect - -./query.expect "$(cat create.sql)" - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz -chmod 777 ~ hits.tsv - -./query.expect "COPY INTO hits FROM '$(pwd)/hits.tsv' USING DELIMITERS '\t'" - -# 99997497 affected rows -# clk: 15:39 min - -./run.sh 2>&1 | tee log.txt - -sudo du -bcs /var/monetdb5/ - -cat log.txt | dos2unix -f | grep -P 'clk|tuple' | - awk '/tuple/ { ok = 1 } /clk/ { if (ok) { if ($3 == "ms") { print $2 / 1000 } else { print $2 } } else { print "null" }; ok = 0 }' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/monetdb/create.sql b/benchmark/monetdb/create.sql deleted file mode 100644 index 41c961c00fc..00000000000 --- a/benchmark/monetdb/create.sql +++ /dev/null @@ -1,108 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -); diff --git a/benchmark/monetdb/queries.sql b/benchmark/monetdb/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/monetdb/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/monetdb/query.expect b/benchmark/monetdb/query.expect deleted file mode 100755 index 8e8e43cea98..00000000000 --- a/benchmark/monetdb/query.expect +++ /dev/null @@ -1,10 +0,0 @@ -#!/usr/bin/expect - -set timeout 3600 -set query [lindex $argv 0] - -spawn mclient -u monetdb -d test; -expect "password:"; send "monetdb\r"; -expect "sql>"; send "\\t clock\r"; -expect "sql>"; send "$query;\r"; -expect "sql>"; send "\4" diff --git a/benchmark/monetdb/results/c6a.4xlarge.json b/benchmark/monetdb/results/c6a.4xlarge.json deleted file mode 100644 index eecb21b37d7..00000000000 --- a/benchmark/monetdb/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "MonetDB", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C", "column-oriented"], - - "load_time": 939, - "data_size": 49696606499, - - "result": [ -[0.000218,0.000157,0.000155], -[0.101903,0.019908,0.018439], -[0.282431,0.035987,0.034938], -[2.868,0.029387,0.029207], -[4.675,4.515,4.511], -[6.584,4.269,4.650], -[0.528827,0.063135,0.065742], -[0.506863,0.020966,0.021687], -[8.343,4.457,4.408], -[7.224,6.548,7.576], -[0.267003,0.233353,0.230444], -[0.347206,0.28358,0.266085], -[5.389,3.099,3.074], -[7.653,7.759,8.596], -[3.276,3.326,3.292], -[5.310,3.465,3.578], -[9.341,9.143,9.536], -[9.584,9.604,9.419], -[19.539,19.783,19.611], -[0.004509,0.000702,0.000643], -[20.801,1.570,1.603], -[2.752,0.418221,0.395884], -[14.717,0.800894,0.395477], -[14.429,1.804,1.869], -[1.386,0.159602,0.156426], -[0.189736,0.167664,0.168781], -[0.164681,0.176666,0.17126], -[3.005,3.113,3.882], -[null,null,null], -[2.751,2.846,2.676], -[7.937,2.579,2.447], -[5.120,3.492,3.467], -[22.862,22.567,23.211], -[33.437,18.889,19.043], -[18.898,19.583,19.047], -[14.774,12.984,13.803], -[3.865,0.322143,0.323117], -[0.192149,0.177791,0.175984], -[0.194173,0.159398,0.165201], -[0.680778,0.592252,0.560738], -[0.106465,0.10638,0.102692], -[0.154871,0.153752,0.155782], -[0.11459,0.09639,0.095594] -] -} diff --git a/benchmark/monetdb/run.sh b/benchmark/monetdb/run.sh deleted file mode 100755 index c1eadeab22c..00000000000 --- a/benchmark/monetdb/run.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - for i in $(seq 1 $TRIES); do - ./query.expect "$query" 2>&1 - done; -done; diff --git a/benchmark/mysql-myisam/benchmark.sh b/benchmark/mysql-myisam/benchmark.sh deleted file mode 100755 index fa948d86132..00000000000 --- a/benchmark/mysql-myisam/benchmark.sh +++ /dev/null @@ -1,29 +0,0 @@ -#!/bin/bash - -# Install - -sudo apt-get update -sudo apt-get install -y mysql-server-8.0 -sudo bash -c "echo -e '[mysql]\nlocal-infile=1\n\n[mysqld]\nlocal-infile=1\n' > /etc/mysql/conf.d/local_infile.cnf" -sudo service mysql restart - -# Load the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -sudo mysql -e "CREATE DATABASE test" -sudo mysql test < create.sql -time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" - -# 41m8.979s - -./run.sh 2>&1 | tee log.txt - -sudo du -bcs /var/lib/mysql - -cat log.txt | - grep -P 'rows? in set|Empty set|^ERROR' | - sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | - awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/mysql-myisam/create.sql b/benchmark/mysql-myisam/create.sql deleted file mode 100644 index 961a0fea486..00000000000 --- a/benchmark/mysql-myisam/create.sql +++ /dev/null @@ -1,110 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -) -ENGINE = MyISAM; diff --git a/benchmark/mysql-myisam/queries.sql b/benchmark/mysql-myisam/queries.sql deleted file mode 100644 index ea2bde47802..00000000000 --- a/benchmark/mysql-myisam/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/mysql-myisam/results/c6a.4xlarge.json b/benchmark/mysql-myisam/results/c6a.4xlarge.json deleted file mode 100644 index b4493f1fb4a..00000000000 --- a/benchmark/mysql-myisam/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "MySQL (MyISAM)", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C++", "row-oriented", "MySQL compatible"], - - "load_time": 2512, - "data_size": 121588958061, - - "result": [ -[0.00,0.00,0.00], -[283.32,276.83,274.52], -[276.93,278.29,283.27], -[28.83,23.63,21.55], -[46.41,40.81,40.93], -[467.04,467.39,469.08], -[31.02,25.89,24.20], -[277.89,275.3,277.3], -[329.34,325.8,325.35], -[342.86,338.43,336.95], -[282.03,279.87,281.22], -[277.74,282.68,282], -[335.66,334.83,336.44], -[305.24,310.39,307.3], -[337.41,338.52,342.94], -[308.66,307.34,306.27], -[738.38,748.44,740.75], -[738.75,734.01,738.25], -[867.01,872.92,868.84], -[25.65,20.61,18.46], -[312.39,313.67,306.66], -[301.66,305.12,308.01], -[298.12,298.44,312.4], -[311.34,309.9,311.85], -[281.87,278.5,275], -[277.46,277.46,277.46], -[280.75,278.04,281.76], -[263.9,417.39,406.88], -[707.21,711.96,705], -[668.1,668.33,665.96], -[330.31,333.36,331.94], -[506.57,506.18,500.53], -[2604.49,2681.96,2703.12], -[830.65,832.88,831.14], -[831.98,830.46,833.41], -[608.49,608.51,613.68], -[4.56,4.13,4.16], -[3.80,3.80,3.70], -[1.65,1.45,1.46], -[6.33,5.14,6.15], -[1.60,1.41,1.41], -[1.56,1.42,1.39], -[7.04,1.17,1.13] -] -} diff --git a/benchmark/mysql-myisam/run.sh b/benchmark/mysql-myisam/run.sh deleted file mode 100755 index 31f31682f32..00000000000 --- a/benchmark/mysql-myisam/run.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - for i in $(seq 1 $TRIES); do - sudo mysql test -vvv -e "${query}" - done; -done; diff --git a/benchmark/mysql/benchmark.sh b/benchmark/mysql/benchmark.sh deleted file mode 100755 index ab44a3ce5d7..00000000000 --- a/benchmark/mysql/benchmark.sh +++ /dev/null @@ -1,29 +0,0 @@ -#!/bin/bash - -# Install - -sudo apt-get update -sudo apt-get install -y mysql-server-8.0 -sudo bash -c "echo -e '[mysql]\nlocal-infile=1\n\n[mysqld]\nlocal-infile=1\n' > /etc/mysql/conf.d/local_infile.cnf" -sudo service mysql restart - -# Load the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -sudo mysql -e "CREATE DATABASE test" -sudo mysql test < create.sql -time sudo mysql test -e "LOAD DATA LOCAL INFILE 'hits.tsv' INTO TABLE hits" - -# 2:37:52 elapsed - -./run.sh 2>&1 | tee log.txt - -sudo du -bcs /var/lib/mysql - -cat log.txt | - grep -P 'rows? in set|Empty set|^ERROR' | - sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | - awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/mysql/create.sql b/benchmark/mysql/create.sql deleted file mode 100644 index 7c1b36b4f4a..00000000000 --- a/benchmark/mysql/create.sql +++ /dev/null @@ -1,110 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -) -ENGINE = InnoDB; diff --git a/benchmark/mysql/queries.sql b/benchmark/mysql/queries.sql deleted file mode 100644 index ea2bde47802..00000000000 --- a/benchmark/mysql/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:00:00') LIMIT 10 OFFSET 1000; diff --git a/benchmark/mysql/results/c6a.4xlarge.json b/benchmark/mysql/results/c6a.4xlarge.json deleted file mode 100644 index 18b8509215f..00000000000 --- a/benchmark/mysql/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "MySQL", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C++", "row-oriented", "MySQL compatible"], - - "load_time": 9472, - "data_size": 171953585825, - - "result": [ -[339.77,339.88,339.77], -[364.91,371.86,367.55], -[366.2,368.91,389.66], -[364.39,377.53,571.45], -[377.69,390.02,384.86], -[569.48,576.51,574.68], -[367.4,368.23,370.41], -[371.29,384.02,613.22], -[478.85,683.22,495.68], -[489.9,635.96,662.43], -[386.07,396.49,640.15], -[389.13,412.55,444.12], -[447.97,455.54,448.06], -[423.22,845.44,813.6], -[452.48,460.07,453.98], -[577.54,623.21,586.49], -[852.07,856.36,862.66], -[838.09,848.92,851.12], -[1006.37,1011.16,1023.17], -[369.76,375.61,415.28], -[412.45,419.9,456.62], -[411.65,432.88,482.2], -[412.73,420.73,429.5], -[551.16,577.62,545.45], -[382.89,394.76,386.37], -[380.9,391.4,385.05], -[385.3,394.67,460.32], -[388.95,394.7,387.21], -[800.33,807.90,807.11], -[706.03,745.27,718.9], -[450.9,489.59,530.97], -[625.5,651.93,647.32], -[2721.13,2792.12,2819.26], -[945.9,954.94,957.54], -[945.42,953.78,965.16], -[684.36,716.29,708.75], -[10.01,3.79,3.77], -[7.48,3.32,3.27], -[5.09,0.98,0.96], -[8.70,4.77,4.68], -[4.82,0.76,0.74], -[4.46,0.77,0.75], -[7.04,1.17,1.13] -] -} diff --git a/benchmark/mysql/run.sh b/benchmark/mysql/run.sh deleted file mode 100755 index 31f31682f32..00000000000 --- a/benchmark/mysql/run.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - for i in $(seq 1 $TRIES); do - sudo mysql test -vvv -e "${query}" - done; -done; diff --git a/benchmark/pinot/benchmark.sh b/benchmark/pinot/benchmark.sh deleted file mode 100755 index fffcfe6702d..00000000000 --- a/benchmark/pinot/benchmark.sh +++ /dev/null @@ -1,44 +0,0 @@ -#!/bin/bash - -sudo apt-get update -sudo apt install openjdk-11-jdk jq -y -sudo update-alternatives --config java - -# Install - -PINOT_VERSION=0.10.0 - -wget https://downloads.apache.org/pinot/apache-pinot-$PINOT_VERSION/apache-pinot-$PINOT_VERSION-bin.tar.gz -tar -zxvf apache-pinot-$PINOT_VERSION-bin.tar.gz - -./apache-pinot-$PINOT_VERSION-bin/bin/pinot-admin.sh QuickStart -type batch & -sleep 30 -./apache-pinot-$PINOT_VERSION-bin/bin/pinot-admin.sh AddTable -tableConfigFile offline_table.json -schemaFile schema.json -exec - -# Load the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -# Pinot was unable to load data as a single file wihout any errors returned. We have to split the data -split -d --additional-suffix .tsv --verbose -n l/100 hits.tsv parts - -# Pinot can't load value '"tatuirovarki_redmond' so we need to fix this row to make it work -sed parts93.tsv -e 's "tatuirovarki_redmond tatuirovarki_redmond g' -i - -# Fix path to local directory -sed splitted.yaml 's PWD_DIR_PLACEHOLDER '$PWD' g' -i -sed local.yaml 's PWD_DIR_PLACEHOLDER '$PWD' g' -i - -# Load data -./apache-pinot-$PINOT_VERSION-bin/bin/pinot-admin.sh LaunchDataIngestionJob -jobSpecFile splitted.yaml - -# After upload it shows 94465149 rows instead of 99997497 in the dataset - -# Run the queries -./run.sh - -# stop Druid services -kill %1 - -du -bcs ./batch diff --git a/benchmark/pinot/local.yaml b/benchmark/pinot/local.yaml deleted file mode 100644 index e2bae09526f..00000000000 --- a/benchmark/pinot/local.yaml +++ /dev/null @@ -1,38 +0,0 @@ -executionFrameworkSpec: - name: 'standalone' - segmentGenerationJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentGenerationJobRunner' - segmentTarPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentTarPushJobRunner' - segmentUriPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentUriPushJobRunner' - segmentMetadataPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentMetadataPushJobRunner' -jobType: SegmentCreationAndTarPush -inputDirURI: 'PWD_DIR_PLACEHOLDER' -includeFileNamePattern: 'glob:PWD_DIR_PLACEHOLDER/hits.tsv' -outputDirURI: 'batch/hits/segments' -overwriteOutput: true -segmentCreationJobParallelism: 10 -pinotFSSpecs: - - scheme: file - className: org.apache.pinot.spi.filesystem.LocalPinotFS -recordReaderSpec: - dataFormat: 'csv' - className: 'org.apache.pinot.plugin.inputformat.csv.CSVRecordReader' - configClassName: 'org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig' - configs: - fileFormat: 'default' - delimiter: ' ' - multiValueDelimiter: '' - header: 'WatchID JavaEnable Title GoodEvent EventTime EventDate CounterID ClientIP RegionID UserID CounterClass OS UserAgent URL Referer IsRefresh RefererCategoryID RefererRegionID URLCategoryID URLRegionID ResolutionWidth ResolutionHeight ResolutionDepth FlashMajor FlashMinor FlashMinor2 NetMajor NetMinor UserAgentMajor UserAgentMinor CookieEnable JavascriptEnable IsMobile MobilePhone MobilePhoneModel Params IPNetworkID TraficSourceID SearchEngineID SearchPhrase AdvEngineID IsArtifical WindowClientWidth WindowClientHeight ClientTimeZone ClientEventTime SilverlightVersion1 SilverlightVersion2 SilverlightVersion3 SilverlightVersion4 PageCharset CodeVersion IsLink IsDownload IsNotBounce FUniqID OriginalURL HID IsOldCounter IsEvent IsParameter DontCountHits WithHash HitColor LocalEventTime Age Sex Income Interests Robotness RemoteIP WindowName OpenerName HistoryLength BrowserLanguage BrowserCountry SocialNetwork SocialAction HTTPError SendTiming DNSTiming ConnectTiming ResponseStartTiming ResponseEndTiming FetchTiming SocialSourceNetworkID SocialSourcePage ParamPrice ParamOrderID ParamCurrency ParamCurrencyID OpenstatServiceName OpenstatCampaignID OpenstatAdID OpenstatSourceID UTMSource UTMMedium UTMCampaign UTMContent UTMTerm FromTag HasGCLID RefererHash URLHash' - - -tableSpec: - tableName: 'hits' - schemaURI: 'http://localhost:9000/tables/hits/schema' - tableConfigURI: 'http://localhost:9000/tables/hits' - -pinotClusterSpecs: - - controllerURI: 'http://localhost:9000' - -pushJobSpec: - pushAttempts: 10 - pushRetryIntervalMillis: 1000 - segmentUriPrefix: 'file://' diff --git a/benchmark/pinot/offline_table.json b/benchmark/pinot/offline_table.json deleted file mode 100644 index da386d53d38..00000000000 --- a/benchmark/pinot/offline_table.json +++ /dev/null @@ -1,17 +0,0 @@ -{ - "tableName": "hits", - "tableType": "OFFLINE", - "segmentsConfig": { - "segmentPushType": "APPEND", - "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy", - "schemaName": "hitsSchema", - "replication": "1" - }, - "tableIndexConfig":{}, - "metadata":{}, - "tenants": - { - "server": "DefaultTenant", - "broker": "DefaultTenant" - } -} diff --git a/benchmark/pinot/queries.sql b/benchmark/pinot/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/pinot/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/pinot/results/c6a.4xlarge.json b/benchmark/pinot/results/c6a.4xlarge.json deleted file mode 100644 index 8be94900f42..00000000000 --- a/benchmark/pinot/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Pinot", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "It successfully loaded only 94465149 out of 99997497 records. Some queries returned NullPointerException. The loading process is painful - splitting to 100 pieces required. It does not correctly report errors on data loading, the results may be incorrect.", - - "tags": ["Java", "column-oriented"], - - "load_time": 2032, - "data_size": null, - - "result": [ -[0.002, 0.001, 0.001], -[0.186, 0.186, 0.185], -[0.251, 0.276, 0.258], -[0.475, 0.281, 0.238], -[3.907, 3.655, 3.633], -[30.471, 14.687, 14.93], -[null, null, null], -[0.135, 0.134, 0.148], -[3.039, 2.902, 2.938], -[3.159, 3.212, 3.225], -[4.217, 4.197, 4.384], -[4.145, 4.124, 4.121], -[2.989, 3.145, 3.18], -[6.402, 6.886, 6.374], -[3.245, 3.35, 3.129], -[5.112, 5.027, 5.141], -[5.509, 5.279, 5.257], -[0.865, 0.856, 0.829], -[null, null, null], -[0.017, 0.015, 0.015], -[54.348, 19.562, 19.128], -[null, null, null], -[76.596, 74.719, 14.228], -[7.441, 5.77, 5.87], -[0.376, 0.327, 0.286], -[7.689, 0.395, 1.281], -[3.434, 0.499, 0.5], -[27.679, 2.378, 2.393], -[null, null, null], -[2.221, 2.227, 2.167], -[4.941, 4.639, 4.565], -[5.641, 5.37, 5.007], -[5.295, 5.006, 5.357], -[5.28, 5.21, 5.105], -[6.231, 6.238, 6.385], -[5.918, 5.933, 5.934], -[0.26, 0.202, 0.21], -[0.364, 0.072, 0.069], -[0.042, 0.034, 0.035], -[1.483, 0.686, 0.651], -[0.113, 0.071, 0.079], -[0.042, 0.051, 0.037], -[null, null, null] -] -} diff --git a/benchmark/pinot/run.sh b/benchmark/pinot/run.sh deleted file mode 100755 index 9434c6ee8a7..00000000000 --- a/benchmark/pinot/run.sh +++ /dev/null @@ -1,15 +0,0 @@ -#!/bin/bash - -TRIES=3 -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - echo -n "[" - for i in $(seq 1 $TRIES); do - echo "{\"sql\":\"$query option(timeoutMs=300000)\"}"| tr -d ';' > query.json - RES=$(curl -s -XPOST -H'Content-Type: application/json' http://localhost:8000/query/sql/ -d @query.json | jq 'if .exceptions == [] then .timeUsedMs/1000 else "-" end' ) - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - done - echo "]," -done diff --git a/benchmark/pinot/schema.json b/benchmark/pinot/schema.json deleted file mode 100644 index cc603fb45a4..00000000000 --- a/benchmark/pinot/schema.json +++ /dev/null @@ -1,437 +0,0 @@ -{ - "metricFieldSpecs": [ -{ - "name": "AdvEngineID", - "dataType": "LONG" - }, - { - "name": "ClientIP", - "dataType": "LONG" - }, - { - "name": "RegionID", - "dataType": "LONG" - }, - { - "name": "IsRefresh", - "dataType": "LONG" - }, - { - "name": "ResolutionWidth", - "dataType": "LONG" - }, - { - "name": "ResolutionHeight", - "dataType": "LONG" - }, - { - "name": "WindowClientWidth", - "dataType": "LONG" - }, - { - "name": "WindowClientHeight", - "dataType": "LONG" - }, - { - "name": "URLHash", - "dataType": "LONG" - }, - { - "name": "JavaEnable", - "dataType": "LONG" - }, - { - "name": "GoodEvent", - "dataType": "LONG" - }, - { - "name": "CounterClass", - "dataType": "LONG" - }, - { - "name": "OS", - "dataType": "LONG" - }, - { - "name": "UserAgent", - "dataType": "LONG" - }, - { - "name": "RefererCategoryID", - "dataType": "LONG" - }, - { - "name": "RefererRegionID", - "dataType": "LONG" - }, - { - "name": "URLCategoryID", - "dataType": "LONG" - }, - { - "name": "URLRegionID", - "dataType": "LONG" - }, - { - "name": "ResolutionDepth", - "dataType": "LONG" - }, - { - "name": "FlashMajor", - "dataType": "LONG" - }, - { - "name": "FlashMinor", - "dataType": "LONG" - }, - { - "name": "NetMajor", - "dataType": "LONG" - }, - { - "name": "NetMinor", - "dataType": "LONG" - }, - { - "name": "UserAgentMajor", - "dataType": "LONG" - }, - { - "name": "CookieEnable", - "dataType": "LONG" - }, - { - "name": "JavascriptEnable", - "dataType": "LONG" - }, - { - "name": "IsMobile", - "dataType": "LONG" - }, - { - "name": "MobilePhone", - "dataType": "LONG" - }, - { - "name": "IPNetworkID", - "dataType": "LONG" - }, - { - "name": "TraficSourceID", - "dataType": "LONG" - }, - { - "name": "SearchEngineID", - "dataType": "LONG" - }, - { - "name": "IsArtifical", - "dataType": "LONG" - }, - { - "name": "ClientTimeZone", - "dataType": "LONG" - }, - { - "name": "SilverlightVersion1", - "dataType": "LONG" - }, - { - "name": "SilverlightVersion2", - "dataType": "LONG" - }, - { - "name": "SilverlightVersion3", - "dataType": "LONG" - }, - { - "name": "SilverlightVersion4", - "dataType": "LONG" - }, - { - "name": "CodeVersion", - "dataType": "LONG" - }, - { - "name": "IsLink", - "dataType": "LONG" - }, - { - "name": "IsDownload", - "dataType": "LONG" - }, - { - "name": "IsNotBounce", - "dataType": "LONG" - }, - { - "name": "FUniqID", - "dataType": "LONG" - }, - { - "name": "HID", - "dataType": "LONG" - }, - { - "name": "IsOldCounter", - "dataType": "LONG" - }, - { - "name": "IsEvent", - "dataType": "LONG" - }, - { - "name": "IsParameter", - "dataType": "LONG" - }, - { - "name": "DontCountHits", - "dataType": "LONG" - }, - { - "name": "WithHash", - "dataType": "LONG" - }, - { - "name": "Age", - "dataType": "LONG" - }, - { - "name": "Sex", - "dataType": "LONG" - }, - { - "name": "Income", - "dataType": "LONG" - }, - { - "name": "Interests", - "dataType": "LONG" - }, - { - "name": "Robotness", - "dataType": "LONG" - }, - { - "name": "RemoteIP", - "dataType": "LONG" - }, - { - "name": "WindowName", - "dataType": "LONG" - }, - { - "name": "OpenerName", - "dataType": "LONG" - }, - { - "name": "HistoryLength", - "dataType": "LONG" - }, - { - "name": "HTTPError", - "dataType": "LONG" - }, - { - "name": "SendTiming", - "dataType": "LONG" - }, - { - "name": "DNSTiming", - "dataType": "LONG" - }, - { - "name": "ConnectTiming", - "dataType": "LONG" - }, - { - "name": "ResponseStartTiming", - "dataType": "LONG" - }, - { - "name": "ResponseEndTiming", - "dataType": "LONG" - }, - { - "name": "FetchTiming", - "dataType": "LONG" - }, - { - "name": "SocialSourceNetworkID", - "dataType": "LONG" - }, - { - "name": "ParamPrice", - "dataType": "LONG" - }, - { - "name": "ParamCurrencyID", - "dataType": "LONG" - }, - { - "name": "HasGCLID", - "dataType": "LONG" - }, - { - "name": "RefererHash", - "dataType": "LONG" - }, - { - "name": "CLID", - "dataType": "LONG" - } - - ], - "dimensionFieldSpecs": [ - { - "name": "CounterID", - "dataType": "LONG" - }, - { - "name": "UserID", - "dataType": "LONG" - }, - { - "name": "URL", - "dataType": "STRING" - }, - { - "name": "OpenstatServiceName", - "dataType": "STRING" - }, - { - "name": "OpenstatCampaignID", - "dataType": "STRING" - }, - { - "name": "OpenstatAdID", - "dataType": "STRING" - }, - { - "name": "UserAgentMinor", - "dataType": "STRING" - }, - { - "name": "OpenstatSourceID", - "dataType": "STRING" - }, - { - "name": "MobilePhoneModel", - "dataType": "STRING" - }, - { - "name": "Params", - "dataType": "STRING" - }, - { - "name": "UTMSource", - "dataType": "STRING" - }, - { - "name": "PageCharset", - "dataType": "STRING" - }, - { - "name": "SearchPhrase", - "dataType": "STRING" - }, - { - "name": "UTMMedium", - "dataType": "STRING" - }, - { - "name": "OriginalURL", - "dataType": "STRING" - }, - { - "name": "UTMCampaign", - "dataType": "STRING" - }, - { - "name": "ClientEventTime", - "dataType": "STRING" - }, - { - "name": "ParamOrderID", - "dataType": "STRING" - }, - { - "name": "ParamCurrency", - "dataType": "STRING" - }, - { - "name": "UTMContent", - "dataType": "STRING" - }, - { - "name": "UTMTerm", - "dataType": "STRING" - }, - { - "name": "FlashMinor2", - "dataType": "STRING" - }, - { - "name": "FromTag", - "dataType": "STRING" - }, - - { - "name": "Referer", - "dataType": "STRING" - }, - { - "name": "Title", - "dataType": "STRING" - }, - { - "name": "HitColor", - "dataType": "STRING" - }, - { - "name": "LocalEventTime", - "dataType": "STRING" - }, - { - "name": "BrowserLanguage", - "dataType": "STRING" - }, - { - "name": "SocialSourcePage", - "dataType": "STRING" - }, - { - "name": "BrowserCountry", - "dataType": "STRING" - }, - { - "name": "SocialNetwork", - "dataType": "STRING" - }, - { - "name": "SocialAction", - "dataType": "STRING" - }, - - { - "name": "WatchID", - "dataType": "LONG" - } - - ], - "dateTimeFieldSpecs": [ - { - "name": "EventTime", - "dataType": "STRING", - "format": "1:SECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd HH:mm:ss", - "granularity": "1:SECONDS" - }, - { - "name": "EventDate", - "dataType": "STRING", - "format": "1:DAYS:SIMPLE_DATE_FORMAT:yyyy-MM-dd", - "granularity": "1:DAYS" - } - ], - "schemaName": "hitsSchema" -} diff --git a/benchmark/pinot/splitted.yaml b/benchmark/pinot/splitted.yaml deleted file mode 100644 index 4f2b1eec62c..00000000000 --- a/benchmark/pinot/splitted.yaml +++ /dev/null @@ -1,37 +0,0 @@ -executionFrameworkSpec: - name: 'standalone' - segmentGenerationJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentGenerationJobRunner' - segmentTarPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentTarPushJobRunner' - segmentUriPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentUriPushJobRunner' - segmentMetadataPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentMetadataPushJobRunner' -jobType: SegmentCreationAndTarPush -inputDirURI: 'PWD_DIR_PLACEHOLDER' -includeFileNamePattern: 'glob:PWD_DIR_PLACEHOLDER/parts??.tsv' -outputDirURI: 'batch/hits/segments' -overwriteOutput: true -segmentCreationJobParallelism: 16 -pinotFSSpecs: - - scheme: file - className: org.apache.pinot.spi.filesystem.LocalPinotFS -recordReaderSpec: - dataFormat: 'csv' - className: 'org.apache.pinot.plugin.inputformat.csv.CSVRecordReader' - configClassName: 'org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig' - configs: - fileFormat: 'default' - delimiter: ' ' - multiValueDelimiter: '' - header: 'WatchID JavaEnable Title GoodEvent EventTime EventDate CounterID ClientIP RegionID UserID CounterClass OS UserAgent URL Referer IsRefresh RefererCategoryID RefererRegionID URLCategoryID URLRegionID ResolutionWidth ResolutionHeight ResolutionDepth FlashMajor FlashMinor FlashMinor2 NetMajor NetMinor UserAgentMajor UserAgentMinor CookieEnable JavascriptEnable IsMobile MobilePhone MobilePhoneModel Params IPNetworkID TraficSourceID SearchEngineID SearchPhrase AdvEngineID IsArtifical WindowClientWidth WindowClientHeight ClientTimeZone ClientEventTime SilverlightVersion1 SilverlightVersion2 SilverlightVersion3 SilverlightVersion4 PageCharset CodeVersion IsLink IsDownload IsNotBounce FUniqID OriginalURL HID IsOldCounter IsEvent IsParameter DontCountHits WithHash HitColor LocalEventTime Age Sex Income Interests Robotness RemoteIP WindowName OpenerName HistoryLength BrowserLanguage BrowserCountry SocialNetwork SocialAction HTTPError SendTiming DNSTiming ConnectTiming ResponseStartTiming ResponseEndTiming FetchTiming SocialSourceNetworkID SocialSourcePage ParamPrice ParamOrderID ParamCurrency ParamCurrencyID OpenstatServiceName OpenstatCampaignID OpenstatAdID OpenstatSourceID UTMSource UTMMedium UTMCampaign UTMContent UTMTerm FromTag HasGCLID RefererHash URLHash' - - -tableSpec: - tableName: 'hits' - schemaURI: 'http://localhost:9000/tables/hits/schema' - tableConfigURI: 'http://localhost:9000/tables/hits' - -pinotClusterSpecs: - - controllerURI: 'http://localhost:9000' - -pushJobSpec: - pushAttempts: 10 - pushRetryIntervalMillis: 1000 diff --git a/benchmark/postgresql/benchmark.sh b/benchmark/postgresql/benchmark.sh deleted file mode 100755 index 13459abfe8a..00000000000 --- a/benchmark/postgresql/benchmark.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/bin/bash - -sudo apt-get update -sudo apt-get install -y postgresql-common -sudo apt-get install -y postgresql-14 - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz -chmod 777 ~ hits.tsv - -sudo -u postgres psql -t -c 'CREATE DATABASE test' -sudo -u postgres psql test -t < create.sql -sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" - -# COPY 99997497 -# Time: 2341543.463 ms (39:01.543) - -./run.sh 2>&1 | tee log.txt - -sudo du -bcs /var/lib/postgresql/14/main/ - -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/postgresql/create.sql b/benchmark/postgresql/create.sql deleted file mode 100644 index 1850bffedce..00000000000 --- a/benchmark/postgresql/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/postgresql/queries.sql b/benchmark/postgresql/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/postgresql/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/postgresql/results/c6a.4xlarge.json b/benchmark/postgresql/results/c6a.4xlarge.json deleted file mode 100644 index 0a0a6731021..00000000000 --- a/benchmark/postgresql/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "PostgreSQL", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C", "row-oriented", "PostgreSQL compatible"], - - "load_time": 2342, - "data_size": 77797067741, - - "result": [ -[439.753,309.785,282.017], -[317.874,254.238,254.941], -[262.883,263.072,263.090], -[32.421,5.310,5.060], -[57.134,42.648,42.334], -[358.423,356.315,358.342], -[31.524,5.350,4.994], -[263.145,263.193,263.165], -[323.659,322.858,321.918], -[327.395,326.170,326.231], -[265.983,265.681,265.912], -[269.984,265.336,265.379], -[284.096,284.560,282.234], -[277.250,279.455,280.035], -[285.660,286.200,283.611], -[66.605,32.023,38.282], -[312.452,304.431,305.391], -[289.209,290.449,287.578], -[331.706,327.485,334.428], -[24.646,2.543,2.263], -[267.561,267.496,267.524], -[267.729,267.690,268.184], -[263.074,263.120,267.040], -[267.602,267.488,267.494], -[263.141,263.859,263.137], -[262.923,263.102,263.113], -[262.885,263.088,263.114], -[267.864,269.127,268.204], -[303.376,306.925,308.664], -[263.221,263.119,263.148], -[270.814,270.575,270.294], -[278.342,275.925,276.224], -[584.599,576.932,591.502], -[462.576,446.962,439.779], -[429.930,417.696,416.704], -[296.875,297.283,295.140], -[3.461,0.842,0.794], -[2.179,0.564,0.558], -[2.258,0.566,0.416], -[2.805,1.311,1.317], -[2.936,0.820,0.615], -[2.197,0.736,0.535], -[1.983,0.320,0.312] -] -} diff --git a/benchmark/postgresql/run.sh b/benchmark/postgresql/run.sh deleted file mode 100755 index ad95151ab55..00000000000 --- a/benchmark/postgresql/run.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - echo "$query"; - for i in $(seq 1 $TRIES); do - sudo -u postgres psql test -t -c '\timing' -c "$query" | grep 'Time' - done; -done; diff --git a/benchmark/questdb/benchmark.sh b/benchmark/questdb/benchmark.sh deleted file mode 100755 index dce8569e927..00000000000 --- a/benchmark/questdb/benchmark.sh +++ /dev/null @@ -1,29 +0,0 @@ -#!/bin/bash - -# Install - -wget https://github.com/questdb/questdb/releases/download/6.4.1/questdb-6.4.1-rt-linux-amd64.tar.gz -tar xf questdb*.tar.gz -questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start - -# Import the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' -gzip -d hits.csv.gz - -curl -G --data-urlencode "query=$(cat create.sql)" 'http://localhost:9000/exec?timings=true' -time curl -F data=@hits.csv 'http://localhost:9000/imp?name=hits' - -# 27m 47.546s - -sed -i 's/query.timeout.sec=60/query.timeout.sec=6000/' .questdb/conf/server.conf -questdb-6.4.1-rt-linux-amd64/bin/questdb.sh stop -questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start - -./run.sh 2>&1 | tee log.txt - -du -bcs .questdb/db/hits - -cat log.txt | grep -P '"timings"|"error"|null' | sed -r -e 's/^.*"error".*$/null/; s/^.*"compiler":([0-9]*),"execute":([0-9]*),.*$/\1 \2/' | - awk '{ print ($1 + $2) / 1000000000 }' | sed -r -e 's/^0$/null/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/questdb/create.sql b/benchmark/questdb/create.sql deleted file mode 100644 index 5c0f777a8b2..00000000000 --- a/benchmark/questdb/create.sql +++ /dev/null @@ -1,108 +0,0 @@ -CREATE TABLE hits -( - WatchID long, - JavaEnable int, - Title string, - GoodEvent int, - EventTime timestamp, - Eventdate date, - CounterID int, - ClientIP int, - RegionID int, - UserID long, - CounterClass int, - OS int, - UserAgent int, - URL string, - Referer string, - IsRefresh int, - RefererCategoryID int, - RefererRegionID int, - URLCategoryID int, - URLRegionID int, - ResolutionWidth int, - ResolutionHeight int, - ResolutionDepth int, - FlashMajor int, - FlashMinor int, - FlashMinor2 string, - NetMajor int, - NetMinor int, - UserAgentMajor int, - UserAgentMinor string, - CookieEnable int, - JavascriptEnable int, - IsMobile int, - MobilePhone int, - MobilePhoneModel string, - Params string, - IPNetworkID int, - TraficSourceID int, - SearchEngineID int, - SearchPhrase string, - AdvEngineID int, - IsArtifical int, - WindowClientWidth int, - WindowClientHeight int, - ClientTimeZone int, - ClientEventTime timestamp, - SilverlightVersion1 int, - SilverlightVersion2 int, - SilverlightVersion3 int, - SilverlightVersion4 int, - PageCharset string, - CodeVersion int, - IsLink int, - IsDownload int, - IsNotBounce int, - FUniqID long, - OriginalURL string, - HID int, - IsOldCounter int, - IsEvent int, - IsParameter int, - DontCountHits int, - WithHash int, - HitColor string, - LocalEventTime timestamp, - Age int, - Sex int, - Income int, - Interests int, - Robotness int, - RemoteIP int, - WindowName int, - OpenerName int, - HistoryLength int, - BrowserLanguage string, - BrowserCountry string, - SocialNetwork string, - SocialAction string, - HTTPError int, - SendTiming int, - DNSTiming int, - ConnectTiming int, - ResponseStartTiming int, - ResponseEndTiming int, - FetchTiming int, - SocialSourceNetworkID int, - SocialSourcePage string, - ParamPrice long, - ParamOrderID string, - ParamCurrency string, - ParamCurrencyID int, - OpenstatServiceName string, - OpenstatCampaignID string, - OpenstatAdID string, - OpenstatSourceID string, - UTMSource string, - UTMMedium string, - UTMCampaign string, - UTMContent string, - UTMTerm string, - FromTag string, - HasGCLID int, - RefererHash long, - URLHash long, - CLID int -); diff --git a/benchmark/questdb/queries.sql b/benchmark/questdb/queries.sql deleted file mode 100644 index 5ddf9fa318c..00000000000 --- a/benchmark/questdb/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT count_distinct(CAST(UserID AS string)) FROM hits; -SELECT count_distinct(SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) AS c FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY c DESC; -SELECT RegionID, count_distinct(CAST(UserID AS string)) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), count_distinct(CAST(UserID AS string)) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, count_distinct(CAST(UserID AS string)) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) AS c FROM hits GROUP BY UserID ORDER BY c DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) AS c FROM hits GROUP BY UserID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) AS c FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, count_distinct(CAST(UserID AS string)) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 1000, 10; -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 EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000, 10; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100, 10; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-01T00:00:00Z' AND EventTime <= '2013-07-31T23:59:59Z' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 100000, 10; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventTime >= '2013-07-14T00:00:00Z' AND EventTime <= '2013-07-15T23:59:59Z' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY M LIMIT 1000, 10; diff --git a/benchmark/questdb/results/c6a.4xlarge.json b/benchmark/questdb/results/c6a.4xlarge.json deleted file mode 100644 index 95b9c382464..00000000000 --- a/benchmark/questdb/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "QuestDB", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "Many queries cannot run. It also crashes and hangs.", - - "tags": ["Java", "time-series"], - - "load_time": 1667.5, - "data_size": 126680518680, - - "result": [ -[0.0155878,0.000283657,0.000328587], -[0.586734,0.0486348,0.0485346], -[6.41881,5.57954,6.25638], -[6.8554,3.11997,3.11813], -[47.8723,54.5425,53.1198], -[null,null,null], -[5.63439,3.70619,3.92043], -[0.699764,0.130462,0.126272], -[55.7179,62.8433,62.0045], -[null,59.6416,64.7753], -[63.7134,null,39.8928], -[46.759,46.8544,null], -[16.7181,10.9064,10.0735], -[79.9885,null,52.5617], -[16.1015,12.1835,13.1469], -[17.0225,11.7342,10.6194], -[24.1167,15.6749,16.8193], -[23.0848,15.3149,17.375], -[42.0965,26.2421,25.7593], -[0.0356335,0.027459,0.0289404], -[70.7679,71.5825,71.4573], -[null,null,null], -[null,null,null], -[77.0625,13.2257,12.7578], -[11.0775,2.90421,2.33398], -[3.31611,3.38837,3.35419], -[3.13233,5.2785,3.07075], -[null,null,null], -[null,null,null], -[186.032,185.627,180.963], -[26.7279,16.4799,18.8758], -[63.8785,32.1097,32.1561], -[64.4635,31.4538,35.654], -[118.897,null,119.015], -[null,null,null], -[19.6853,17.4427,16.7998], -[1.50985,0.877967,0.885536], -[0.805639,0.577352,0.534731], -[0.509284,0.448942,0.467679], -[1.37609,0.966942,0.912858], -[0.567887,0.394619,0.438952], -[0.508977,0.441015,0.40528], -[null,null,null] -] -} diff --git a/benchmark/questdb/run.sh b/benchmark/questdb/run.sh deleted file mode 100755 index a9c27c952c6..00000000000 --- a/benchmark/questdb/run.sh +++ /dev/null @@ -1,19 +0,0 @@ -#!/bin/bash - -TRIES=3 - -questdb-6.4.1-rt-linux-amd64/bin/questdb.sh stop -questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start -sleep 5 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - echo "$query"; - for i in $(seq 1 $TRIES); do - curl -sS --max-time 6000 -G --data-urlencode "query=${query}" 'http://localhost:9000/exec?timings=true' 2>&1 | grep '"timings"' || - (questdb-6.4.1-rt-linux-amd64/bin/questdb.sh stop >/dev/null 2>&1; questdb-6.4.1-rt-linux-amd64/bin/questdb.sh start >/dev/null 2>&1; sleep 5; echo 'null') - echo - done; -done; diff --git a/benchmark/redshift-serverless/README.md b/benchmark/redshift-serverless/README.md deleted file mode 100644 index 59aec7496cd..00000000000 --- a/benchmark/redshift-serverless/README.md +++ /dev/null @@ -1,55 +0,0 @@ -This benchmark is not automated. - -Go to AWS Redshift service. -Try Redshift Serverless. Use the default configuration. -The cluster will take a few minutes to start. -Go to "Query Editor". Establishing a connection takes around 10 seconds. - -Run the CREATE TABLE statement you find in `create.sql`. - -Note: Redshift prefers VARCHAR(MAX) instead of TEXT. - -Then press on the "Load data". -This will generate a statement: - -``` -COPY dev.public.hits FROM 's3://clickhouse-public-datasets/hits_compatible/hits.csv.gz' GZIP -IAM_ROLE 'arn:aws:iam::...:role/service-role/AmazonRedshift-CommandsAccessRole-...' -FORMAT AS CSV DELIMITER ',' QUOTE '"' -REGION AS 'eu-central-1' -``` - -> Elapsed time: 32m 13.7s - -It also have run 2380 "queries" for this task. - -Namespace configuration, -General Information, Storage used: - -30.3 GB - -Change admin user password: -dev, fGH4{dbas7 - -It's very difficult to find how to connect to it: -https://docs.aws.amazon.com/redshift/latest/mgmt/serverless-connecting.html - -We will run the queries from another server with `psql` client. - -``` -sudo apt-get install -y postgresql-client - -echo "*:*:*:*:your_password" > .pgpass -chmod 400 .pgpass - -psql -h default.111111111111.eu-central-1.redshift-serverless.amazonaws.com -U dev -d dev -p 5439 -``` - -Then run the benchmark: -``` -export HOST=... -./run.sh 2>&1 | tee log.txt - -cat log.txt | grep -oP 'Time: \d+\.\d+ ms|ERROR' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if ($1 == "ERROR") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : ($1 / 1000); if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' -``` diff --git a/benchmark/redshift-serverless/create.sql b/benchmark/redshift-serverless/create.sql deleted file mode 100644 index 94a8fb0958a..00000000000 --- a/benchmark/redshift-serverless/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title VARCHAR(MAX) NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL VARCHAR(MAX) NOT NULL, - Referer VARCHAR(MAX) NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 VARCHAR(MAX) NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(MAX) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel VARCHAR(MAX) NOT NULL, - Params VARCHAR(MAX) NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase VARCHAR(MAX) NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset VARCHAR(MAX) NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL VARCHAR(MAX) NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor VARCHAR(MAX) NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage VARCHAR(MAX) NOT NULL, - BrowserCountry VARCHAR(MAX) NOT NULL, - SocialNetwork VARCHAR(MAX) NOT NULL, - SocialAction VARCHAR(MAX) NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage VARCHAR(MAX) NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID VARCHAR(MAX) NOT NULL, - ParamCurrency VARCHAR(MAX) NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName VARCHAR(MAX) NOT NULL, - OpenstatCampaignID VARCHAR(MAX) NOT NULL, - OpenstatAdID VARCHAR(MAX) NOT NULL, - OpenstatSourceID VARCHAR(MAX) NOT NULL, - UTMSource VARCHAR(MAX) NOT NULL, - UTMMedium VARCHAR(MAX) NOT NULL, - UTMCampaign VARCHAR(MAX) NOT NULL, - UTMContent VARCHAR(MAX) NOT NULL, - UTMTerm VARCHAR(MAX) NOT NULL, - FromTag VARCHAR(MAX) NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/redshift-serverless/queries.sql b/benchmark/redshift-serverless/queries.sql deleted file mode 100644 index 3a5a4145464..00000000000 --- a/benchmark/redshift-serverless/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(www\.)?([^/]+)/.*$', '\2') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/redshift-serverless/results/serverless.json b/benchmark/redshift-serverless/results/serverless.json deleted file mode 100644 index 3ba94e542dc..00000000000 --- a/benchmark/redshift-serverless/results/serverless.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Redshift", - "date": "2022-07-01", - "machine": "serverless", - "cluster_size": "serverless", - "comment": "One query did not run due to overflow check and another due to missing regexp function.", - - "tags": ["managed", "column-oriented"], - - "load_time": 1933, - "data_size": 30300000000, - - "result": [ -[0.436955,0.390225,0.387823], -[2.74529,0.280571,0.198074], -[7.24569,0.295612,0.386568], -[null,null,null], -[3.98825,0.488446,0.325645], -[4.24601,0.538891,0.690124], -[7.25492,0.288505,0.207515], -[5.99594,0.212732,0.281278], -[6.17534,0.681868,0.586073], -[6.37779,1.37865,1.31614], -[6.41405,0.549735,0.421345], -[6.64688,0.445678,0.477323], -[4.2294,0.361772,0.520471], -[4.73701,0.788258,0.743465], -[4.49038,0.670446,0.52727], -[6.19886,0.663381,0.583588], -[5.9867,0.679607,0.672772], -[2.76661,0.567555,0.51494], -[6.20219,0.816422,0.760568], -[2.51526,0.053,0.025066], -[2.96003,0.282737,0.226979], -[6.79648,0.57495,0.400798], -[6.37854,0.92746,1.05793], -[25.8462,0.902664,0.905365], -[5.00521,0.247895,0.308836], -[6.38373,0.308781,0.244082], -[4.25427,0.229966,0.247201], -[5.98382,0.398218,0.455249], -[null,null,null], -[6.52367,1.35877,1.30562], -[7.36935,0.536226,0.582304], -[7.05948,0.621982,0.639653], -[4.62901,0.954522,0.908651], -[4.95273,1.03062,1.10289], -[4.71404,1.06378,1.04157], -[4.8201,0.499996,0.575546], -[5.32757,0.566517,0.562058], -[5.37681,0.626458,0.658628], -[5.23137,0.470622,0.540079], -[6.09326,0.561312,0.574978], -[4.86561,0.595546,0.534209], -[4.34256,0.433804,0.414541], -[4.19814,0.288269,0.415328] -] -} diff --git a/benchmark/redshift-serverless/run.sh b/benchmark/redshift-serverless/run.sh deleted file mode 100755 index 07a47eda1ab..00000000000 --- a/benchmark/redshift-serverless/run.sh +++ /dev/null @@ -1,10 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - echo "$query"; - for i in $(seq 1 $TRIES); do - psql -h "${HOST}" -U awsuser -d dev -p 5439 -t -c 'SET enable_result_cache_for_session = off' -c '\timing' -c "$query" | grep 'Time' - done; -done; diff --git a/benchmark/redshift/README.md b/benchmark/redshift/README.md deleted file mode 100644 index e8d56840ae2..00000000000 --- a/benchmark/redshift/README.md +++ /dev/null @@ -1,56 +0,0 @@ -This benchmark is not automated. - -Go to AWS Redshift service. -Create a cluster. Note: this is a classic Redshift, not "serverless". - -Choose the node type and cluster size. -I've selected 4 nodes of ra3.xplus 4vCPU to get 16vCPU in total. - -Set up some password for the admin user. -The cluster will take a few minutes to start. - -We need to perform two modifications: -1. Allow inbound access. Go to VPC and edit the security group. Modify inbound rules. Allow connections from any IPv4 to port 5439. -2. Add IAM role. Just create something by default. - -To create a table, you can go to the Query Editor v2. -Open the "dev" database. -Run the CREATE TABLE statement you find in `create.sql`. - -Note: Redshift prefers VARCHAR(MAX) instead of TEXT. - -Then press on the "Load data". -This will generate a statement: - -``` -COPY dev.public.hits FROM 's3://clickhouse-public-datasets/hits_compatible/hits.csv.gz' GZIP -IAM_ROLE 'arn:aws:iam::...:role/service-role/AmazonRedshift-CommandsAccessRole-...' -FORMAT AS CSV DELIMITER ',' QUOTE '"' -REGION AS 'eu-central-1' -``` - -> Elapsed time: 35m 35.9s - -We will run the queries from another server with `psql` client. - -``` -sudo apt-get install -y postgresql-client - -echo "*:*:*:*:your_password" > .pgpass -chmod 400 .pgpass - -psql -h redshift-cluster-1.chedgchbam32.eu-central-1.redshift.amazonaws.com -U awsuser -d dev -p 5439 -``` - -Then run the benchmark: -``` -export HOST=... -./run.sh 2>&1 | tee log.txt - -cat log.txt | grep -oP 'Time: \d+\.\d+ ms|ERROR' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if ($1 == "ERROR") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : ($1 / 1000); if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' -``` - -`SELECT sum(used * 1048576) FROM stv_node_storage_capacity` - -> 30 794 579 968 diff --git a/benchmark/redshift/create.sql b/benchmark/redshift/create.sql deleted file mode 100644 index 94a8fb0958a..00000000000 --- a/benchmark/redshift/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title VARCHAR(MAX) NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL VARCHAR(MAX) NOT NULL, - Referer VARCHAR(MAX) NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 VARCHAR(MAX) NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(MAX) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel VARCHAR(MAX) NOT NULL, - Params VARCHAR(MAX) NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase VARCHAR(MAX) NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset VARCHAR(MAX) NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL VARCHAR(MAX) NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor VARCHAR(MAX) NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage VARCHAR(MAX) NOT NULL, - BrowserCountry VARCHAR(MAX) NOT NULL, - SocialNetwork VARCHAR(MAX) NOT NULL, - SocialAction VARCHAR(MAX) NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage VARCHAR(MAX) NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID VARCHAR(MAX) NOT NULL, - ParamCurrency VARCHAR(MAX) NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName VARCHAR(MAX) NOT NULL, - OpenstatCampaignID VARCHAR(MAX) NOT NULL, - OpenstatAdID VARCHAR(MAX) NOT NULL, - OpenstatSourceID VARCHAR(MAX) NOT NULL, - UTMSource VARCHAR(MAX) NOT NULL, - UTMMedium VARCHAR(MAX) NOT NULL, - UTMCampaign VARCHAR(MAX) NOT NULL, - UTMContent VARCHAR(MAX) NOT NULL, - UTMTerm VARCHAR(MAX) NOT NULL, - FromTag VARCHAR(MAX) NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/redshift/queries.sql b/benchmark/redshift/queries.sql deleted file mode 100644 index 3a5a4145464..00000000000 --- a/benchmark/redshift/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(www\.)?([^/]+)/.*$', '\2') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/redshift/results/4x.ra3.xplus.json b/benchmark/redshift/results/4x.ra3.xplus.json deleted file mode 100644 index fecc515e475..00000000000 --- a/benchmark/redshift/results/4x.ra3.xplus.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Redshift", - "date": "2022-07-01", - "machine": "ra3.xplus", - "cluster_size": 4, - "comment": "One query did not run due to overflow check and another due to missing regexp function.", - - "tags": ["managed", "column-oriented"], - - "load_time": 2136, - "data_size": 30794579968, - - "result": [ -[0.081437,0.022376,0.022491], -[2.54868,0.024112,0.024072], -[2.62053,0.049948,0.049768], -[null,null,null], -[0.62714,0.447655,0.43262], -[1.14153,1.09479,1.09441], -[0.153399,0.053941,0.048224], -[3.76153,0.032235,0.030949], -[4.43439,0.746808,0.723719], -[6.13424,2.20458,2.20031], -[4.35338,0.220122,0.228684], -[5.15139,0.231856,0.230993], -[5.00006,1.10841,1.12871], -[5.80125,1.86531,1.90209], -[5.16246,1.21239,1.14848], -[0.579428,0.535859,0.546178], -[6.05764,2.17455,2.18286], -[5.86612,2.52634,2.48672], -[7.80075,3.35512,3.41153], -[2.54112,0.036378,0.035944], -[6.47189,2.26909,2.2673], -[6.95344,2.61929,2.65637], -[12.9508,6.85457,6.99], -[25.3022,11.8857,11.8493], -[4.37592,0.452737,0.452867], -[4.17199,0.469457,0.476302], -[6.24746,0.470935,0.482502], -[6.00065,2.08332,2.08059], -[null,null,null], -[22.2567,18.5376,18.3441], -[5.37492,0.868068,0.849486], -[7.29067,1.06155,1.11209], -[7.7832,4.07132,4.00384], -[8.95385,5.15488,5.21863], -[9.2232,5.32052,5.73207], -[4.99205,0.664347,0.618918], -[6.82279,1.89738,1.89398], -[7.09077,2.19008,2.26612], -[5.29731,0.19626,0.204603], -[6.07138,0.276315,0.267161], -[4.03108,0.054134,0.058568], -[4.35647,0.061157,0.053367], -[5.23605,0.037217,0.036335] -] -} diff --git a/benchmark/redshift/run.sh b/benchmark/redshift/run.sh deleted file mode 100755 index 07a47eda1ab..00000000000 --- a/benchmark/redshift/run.sh +++ /dev/null @@ -1,10 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - echo "$query"; - for i in $(seq 1 $TRIES); do - psql -h "${HOST}" -U awsuser -d dev -p 5439 -t -c 'SET enable_result_cache_for_session = off' -c '\timing' -c "$query" | grep 'Time' - done; -done; diff --git a/benchmark/singlestore/benchmark.sh b/benchmark/singlestore/benchmark.sh deleted file mode 100755 index 492a7aa1e86..00000000000 --- a/benchmark/singlestore/benchmark.sh +++ /dev/null @@ -1,44 +0,0 @@ -#!/bin/bash - -# Install - -sudo apt-get update -sudo apt-get install -y docker.io - -export LICENSE_KEY="..." -export ROOT_PASSWORD="..." - -sudo docker run -i --init \ - --name memsql-ciab \ - -e LICENSE_KEY="${LICENSE_KEY}" \ - -e ROOT_PASSWORD="${ROOT_PASSWORD}" \ - -p 3306:3306 -p 8080:8080 \ - memsql/cluster-in-a-box - -sudo docker start memsql-ciab - -sudo docker exec -it memsql-ciab memsql -p"${ROOT_PASSWORD}" - -# Load the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz -sudo docker cp hits.tsv memsql-ciab:/ - -sudo docker exec -it memsql-ciab memsql -p"${ROOT_PASSWORD}" -e "CREATE DATABASE test" -sudo docker exec memsql-ciab memsql -p"${ROOT_PASSWORD}" --database=test -e "USE test; $(cat create.sql)" -time sudo docker exec -it memsql-ciab memsql -vvv -p"${ROOT_PASSWORD}" --database=test -e "LOAD DATA INFILE '/hits.tsv' INTO TABLE test.hits" - -# Query OK, 99997497 rows affected (11 min 30.11 sec) - -./run.sh 2>&1 | tee log.txt - -sudo docker exec memsql-ciab du -bcs /var/lib/memsql - -# 29836263469 bytes - -cat log.txt | - grep -P 'rows? in set|Empty set|^ERROR' | - sed -r -e 's/^ERROR.*$/null/; s/^.*?\((([0-9.]+) min )?([0-9.]+) sec\).*?$/\2 \3/' | - awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/singlestore/create.sql b/benchmark/singlestore/create.sql deleted file mode 100644 index 84dc14ea22f..00000000000 --- a/benchmark/singlestore/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - SORT KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/singlestore/queries.sql b/benchmark/singlestore/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/singlestore/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/singlestore/results/c6a.4xlarge.json b/benchmark/singlestore/results/c6a.4xlarge.json deleted file mode 100644 index d434ddafdb1..00000000000 --- a/benchmark/singlestore/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "SingleStore", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "Previous name: MemSQL. Some queries did not run due to memory limits", - - "tags": ["MySQL compatible", "column-oriented"], - - "load_time": 690, - "data_size": 29836263469, - - "result": [ -[0.09,0.00,0.00], -[0.23,0.04,0.01], -[0.47,0.15,0.15], -[0.63,0.09,0.08], -[1.65,1.23,1.20], -[7.96,2.79,2.63], -[0.10,0.00,0.00], -[0.17,0.02,0.02], -[1.90,1.40,1.39], -[4.79,3.52,3.48], -[0.94,0.22,0.23], -[0.89,0.24,0.23], -[5.82,2.26,2.25], -[6.97,4.62,4.66], -[3.05,2.28,2.31], -[3.92,2.70,2.28], -[5.83,4.48,4.42], -[4.76,4.13,4.12], -[14.03,null,null], -[0.57,0.05,0.04], -[18.18,1.74,1.94], -[20.85,2.17,0.98], -[31.98,3.12,1.22], -[78.96,3.35,108.85], -[2.82,0.39,0.32], -[1.83,0.44,0.35], -[2.81,0.33,0.32], -[18.33,2.57,1.15], -[null,null,null], -[3.56,2.40,2.40], -[3.83,1.11,1.11], -[7.35,1.73,1.70], -[null,null,null], -[null,null,null], -[null,null,null], -[2.53,1.92,1.84], -[0.92,0.23,0.19], -[0.84,0.15,0.08], -[0.70,0.05,0.05], -[3.12,0.38,0.36], -[0.29,0.03,0.03], -[0.22,0.06,0.02], -[0.27,0.11,0.12] -] -} diff --git a/benchmark/singlestore/run.sh b/benchmark/singlestore/run.sh deleted file mode 100755 index 16cc104e712..00000000000 --- a/benchmark/singlestore/run.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - for i in $(seq 1 $TRIES); do - sudo docker exec memsql-ciab memsql -vvv -p"${ROOT_PASSWORD}" --database=test -e "USE test; ${query}" - done; -done; diff --git a/benchmark/snowflake/NOTES.md b/benchmark/snowflake/NOTES.md deleted file mode 100644 index c7159dc0c2b..00000000000 --- a/benchmark/snowflake/NOTES.md +++ /dev/null @@ -1,67 +0,0 @@ -The choice of a warehouse size is unclear. Let's choose X-Large by default. -It is using "credits" for pricing. - -Storage cost: $23 USD per compressed TB per month -One credit is: $2.016/hour -X-Large: 16 credits/hour = $32/hour - -It is very expensive, so let's touch it with a ten-foot pole and run away as quickly as possible. - -Set up SnowSQL. - -``` -curl -O https://sfc-repo.snowflakecomputing.com/snowsql/bootstrap/1.2/linux_x86_64/snowsql-1.2.22-linux_x86_64.bash -bash snowsql-1.2.22-linux_x86_64.bash -source .profile -``` - -``` -snowsql -a HA12345 -u USER -``` - -It does not connect after typing the password. - -``` -250001 (08001): Failed to connect to DB. Verify the account name is correct: HA12345.snowflakecomputing.com:443. 000403: 403: HTTP 403: Forbidden -If the error message is unclear, enable logging using -o log_level=DEBUG and see the log to find out the cause. Contact support for further help. -Goodbye! -``` - -It said "Goodbye!" in active-aggressive tone. - -To know the account name, we have to go to the "classic console" and look at the URL in the browser. - -> https://{this}.eu-central-1.snowflakecomputing.com/console/login?disableDirectLogin=true - -But it does not help. - -It works if I specify the region in the command line. -Although `snowsql --help` saying that it is DEPRECATED. - -``` -snowsql -a nn12345 -u USER --region eu-central-1 --schemaname PUBLIC --dbname TEST --warehouse TEST -``` - -Notes: SnowSQL is using autocomplete using well known Python library. -Autocomplete is not context-aware. - -Upload the data: - -``` -put file:///home/ubuntu/hits.csv @test.public.%hits -``` - -The syntax is strange (all these @%#). -The query hung and did nothing. - -Actually it is not hung. The snowsql is using 100% to parse CSV in Python for hours. - -Let's try a different upload method. - -``` -COPY INTO test.public.hits2 FROM 's3://clickhouse-public-datasets/hits_compatible/hits.csv.gz' FILE_FORMAT = (TYPE = CSV, COMPRESSION = GZIP, FIELD_OPTIONALLY_ENCLOSED_BY = '"') -``` - -For some reason, it has selected X-Small warehouse, will need to change to X-Large. - -42 min 4 sec. diff --git a/benchmark/snowflake/README.md b/benchmark/snowflake/README.md deleted file mode 100644 index c2de02dded7..00000000000 --- a/benchmark/snowflake/README.md +++ /dev/null @@ -1,56 +0,0 @@ -Snowflake recently removed the DeWitt Clause, so we are allowed to make benchmarks. - -> Customer may conduct benchmark tests of the Service (each a “Test”). Other than with respect to Tests involving Previews, which may not be disclosed externally, Customer may externally disclose a Test or otherwise cause the results of a Test to be externally disclosed if it includes as part of the disclosure all information necessary to replicate the Test. - -https://www.snowflake.com/legal/acceptable-use-policy/ - -Account setup took only 3 seconds. - -Data -> Databases -> + Database -Database 'test' created. -Press on "public" schema. - -Create table "standard". -Paste "create.sql". -Press on "create table" again. - -Press on "admin", "warehouses", + Warehouse -The choice of a warehouse size is unclear. Let's choose X-Large by default. -It is using "credits" for pricing. - -Set up SnowSQL. - -``` -curl -O https://sfc-repo.snowflakecomputing.com/snowsql/bootstrap/1.2/linux_x86_64/snowsql-1.2.22-linux_x86_64.bash -bash snowsql-1.2.22-linux_x86_64.bash -source .profile -``` - -Upload the data: - -``` -COPY INTO test.public.hits2 FROM 's3://clickhouse-public-datasets/hits_compatible/hits.csv.gz' FILE_FORMAT = (TYPE = CSV, COMPRESSION = GZIP, FIELD_OPTIONALLY_ENCLOSED_BY = '"') -``` - -42 min 4 sec. - -``` -export SNOWSQL_PWD='...' SNOWSQL_ACCOUNT='...' SNOWSQL_USER='myuser' - -snowsql --region eu-central-1 --schemaname PUBLIC --dbname HITS --warehouse TEST --query "SELECT 1" -``` - -Before the benchmark: -``` -ALTER USER myuser SET USE_CACHED_RESULT = false; -``` - -Run the benchmark: -``` -./run.sh 2>&1 | tee log.txt - -cat log.txt | - grep -P 'Time Elapsed|^\d+ \(\w+\):' | - sed -r -e 's/^[0-9]+ \([0-9A-Za-z]+\):.*$/null/; s/^.*Time Elapsed:\s*([0-9.]+)s$/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' -``` diff --git a/benchmark/snowflake/create.sql b/benchmark/snowflake/create.sql deleted file mode 100644 index 62800d2e1b2..00000000000 --- a/benchmark/snowflake/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits2 -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/snowflake/queries.sql b/benchmark/snowflake/queries.sql deleted file mode 100644 index d188fd36e25..00000000000 --- a/benchmark/snowflake/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM test.public.hits2; -SELECT COUNT(*) FROM test.public.hits2 WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM test.public.hits2; -SELECT AVG(UserID) FROM test.public.hits2; -SELECT COUNT(DISTINCT UserID) FROM test.public.hits2; -SELECT COUNT(DISTINCT SearchPhrase) FROM test.public.hits2; -SELECT MIN(EventDate), MAX(EventDate) FROM test.public.hits2; -SELECT AdvEngineID, COUNT(*) FROM test.public.hits2 WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM test.public.hits2 GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM test.public.hits2 GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM test.public.hits2 WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM test.public.hits2 WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM test.public.hits2 WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM test.public.hits2 WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM test.public.hits2 WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM test.public.hits2 GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM test.public.hits2 GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM test.public.hits2 GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM test.public.hits2 GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM test.public.hits2 WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM test.public.hits2 WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM test.public.hits2 WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM test.public.hits2 WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM test.public.hits2 WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM test.public.hits2 WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM test.public.hits2 WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM test.public.hits2 WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM test.public.hits2 WHERE URL <> '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; -SELECT REGEXP_REPLACE(Referer, '^https?://(www\.)?([^/]+)/.*$', '\2') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM test.public.hits2 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 test.public.hits2; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.public.hits2 WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.public.hits2 WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM test.public.hits2 GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM test.public.hits2 GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM test.public.hits2 GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM test.public.hits2 GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM test.public.hits2 WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/snowflake/results/2xl.json b/benchmark/snowflake/results/2xl.json deleted file mode 100644 index 294eb66e65e..00000000000 --- a/benchmark/snowflake/results/2xl.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Snowflake", - "date": "2022-07-01", - "machine": "2XL", - "cluster_size": 32, - "comment": "", - - "tags": ["managed", "column-oriented"], - - "load_time": 2524, - "data_size": 12300000000, - - "result": [ -[0.177,0.052,0.090], -[0.903,0.324,0.782], -[0.458,2.909,0.275], -[0.881,0.316,0.166], -[0.404,0.257,0.256], -[0.481,0.325,0.339], -[0.056,0.062,0.060], -[0.183,0.324,0.280], -[0.444,0.314,0.320], -[0.408,0.426,0.417], -[0.345,0.241,0.253], -[0.406,0.245,0.235], -[0.521,1.259,0.326], -[0.466,0.493,0.526], -[0.447,0.349,0.362], -[0.327,0.322,0.302], -[0.462,0.508,0.461], -[0.489,0.481,0.455], -[0.731,0.659,0.674], -[0.151,0.156,0.160], -[0.832,0.339,0.312], -[0.289,0.292,0.340], -[0.591,0.484,0.419], -[2.661,0.716,0.696], -[0.190,0.199,0.377], -[0.181,0.182,0.194], -[0.220,0.209,0.195], -[0.368,0.330,0.347], -[0.677,0.645,0.646], -[0.877,0.886,0.871], -[0.415,0.282,0.301], -[1.265,0.404,0.550], -[0.786,0.750,0.757], -[0.905,0.835,0.841], -[0.864,0.865,0.823], -[0.352,0.360,0.364], -[0.201,0.186,0.200], -[0.143,0.137,0.276], -[0.290,0.132,0.146], -[0.310,0.301,0.304], -[0.212,0.197,0.170], -[0.223,0.174,0.177], -[0.172,0.172,0.172] -] -} diff --git a/benchmark/snowflake/results/3xl.json b/benchmark/snowflake/results/3xl.json deleted file mode 100644 index 841d570b249..00000000000 --- a/benchmark/snowflake/results/3xl.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Snowflake", - "date": "2022-07-01", - "machine": "3XL", - "cluster_size": 64, - "comment": "", - - "tags": ["managed", "column-oriented"], - - "load_time": 2524, - "data_size": 12300000000, - - "result": [ -[0.165,0.061,0.049], -[1.356,1.252,1.502], -[1.287,0.470,0.325], -[0.627,0.331,0.181], -[0.265,0.265,0.483], -[0.887,0.450,0.298], -[0.054,0.066,0.062], -[0.182,0.222,0.194], -[0.408,0.438,0.319], -[0.434,0.462,0.411], -[1.357,0.247,0.412], -[0.343,0.250,0.517], -[0.273,0.289,0.600], -[0.404,0.405,0.392], -[0.379,0.304,0.455], -[0.275,0.271,0.264], -[0.418,0.386,0.388], -[0.417,0.434,0.567], -[0.753,0.759,0.506], -[0.291,0.307,0.400], -[0.954,0.423,0.280], -[0.568,0.562,0.248], -[0.568,0.477,0.496], -[1.458,0.492,0.514], -[0.179,0.300,0.184], -[0.165,0.169,0.176], -[0.197,0.186,0.190], -[0.289,0.547,0.397], -[0.513,0.544,0.632], -[0.766,0.754,0.775], -[0.389,0.374,0.383], -[0.484,0.297,0.286], -[0.505,0.505,0.734], -[0.656,0.598,0.621], -[0.634,0.646,0.609], -[0.309,0.298,0.370], -[0.192,0.219,0.212], -[0.840,0.174,0.139], -[0.172,0.163,0.151], -[0.323,0.296,0.347], -[0.200,0.154,0.144], -[0.191,0.121,0.125], -[0.137,0.233,0.148] -] -} diff --git a/benchmark/snowflake/results/4xl.json b/benchmark/snowflake/results/4xl.json deleted file mode 100644 index 48c671277d4..00000000000 --- a/benchmark/snowflake/results/4xl.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Snowflake", - "date": "2022-07-01", - "machine": "4XL", - "cluster_size": 128, - "comment": "", - - "tags": ["managed", "column-oriented"], - - "load_time": 2524, - "data_size": 12300000000, - - "result": [ -[0.164,0.061,0.078], -[2.471,2.436,1.927], -[1.656,0.222,0.639], -[0.336,1.244,0.206], -[0.435,0.414,0.373], -[0.520,0.495,0.326], -[0.052,0.051,0.057], -[0.244,0.515,0.358], -[0.473,0.477,0.659], -[0.706,0.523,0.499], -[0.619,0.361,0.303], -[0.463,0.367,0.290], -[0.385,0.319,0.376], -[0.661,0.436,0.452], -[0.446,0.317,0.336], -[0.504,0.283,0.395], -[0.478,1.395,0.350], -[0.555,1.041,0.993], -[0.565,0.558,1.473], -[0.310,0.684,0.278], -[0.637,1.202,0.249], -[0.467,0.294,0.436], -[0.671,0.478,0.611], -[6.262,0.449,0.425], -[0.476,0.213,0.184], -[0.552,0.241,0.201], -[0.458,0.415,0.402], -[0.339,0.357,0.322], -[0.732,0.549,0.483], -[0.914,0.757,0.743], -[0.718,0.310,0.606], -[0.605,0.363,0.425], -[0.468,0.860,0.784], -[0.868,1.787,0.713], -[0.807,0.691,0.544], -[0.485,0.338,0.672], -[0.263,0.221,0.230], -[0.147,0.145,0.139], -[0.135,0.239,0.136], -[0.322,0.378,0.348], -[0.236,0.138,0.132], -[0.193,0.124,0.139], -[0.146,0.145,0.139] -] -} diff --git a/benchmark/snowflake/results/l.json b/benchmark/snowflake/results/l.json deleted file mode 100644 index 05fd7793646..00000000000 --- a/benchmark/snowflake/results/l.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Snowflake", - "date": "2022-07-01", - "machine": "L", - "cluster_size": 8, - "comment": "", - - "tags": ["managed", "column-oriented"], - - "load_time": 2524, - "data_size": 12300000000, - - "result": [ -[0.067,0.054,0.051], -[1.158,1.409,0.190], -[1.470,0.566,0.715], -[0.892,0.534,0.264], -[0.474,0.499,0.489], -[0.950,0.650,0.699], -[0.069,0.055,0.110], -[0.317,0.307,0.321], -[0.949,0.593,0.654], -[0.713,1.099,0.860], -[0.622,0.370,0.404], -[0.457,0.369,0.414], -[0.610,0.566,0.653], -[0.970,1.006,0.976], -[1.517,0.636,0.603], -[0.532,0.541,0.533], -[1.018,1.001,1.022], -[0.942,0.996,0.940], -[2.246,1.596,1.560], -[0.181,0.184,0.200], -[1.135,0.788,0.609], -[0.669,0.528,0.524], -[1.164,0.827,0.882], -[3.545,2.214,2.107], -[0.559,0.431,0.426], -[0.340,0.296,0.383], -[0.695,0.314,0.368], -[0.628,0.658,0.637], -[1.511,1.385,1,440], -[1.390,1.418,1.322], -[1.107,0.687,0.537], -[1.026,0.737,0.659], -[1.712,1.681,1.728], -[2.141,2.130,2.225], -[2.163,2.157,2.110], -[0.650,0.619,0.627], -[0.204,0.195,0.225], -[0.159,0.152,0.156], -[0.146,0.136,0.150], -[0.359,0.290,0.364], -[0.196,0.129,0.227], -[0.201,0.128,0.143], -[0.176,0.129,0.146] -] -} diff --git a/benchmark/snowflake/results/m.json b/benchmark/snowflake/results/m.json deleted file mode 100644 index fb98b643c9b..00000000000 --- a/benchmark/snowflake/results/m.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Snowflake", - "date": "2022-07-01", - "machine": "M", - "cluster_size": 4, - "comment": "", - - "tags": ["managed", "column-oriented"], - - "load_time": 2524, - "data_size": 12300000000, - - "result": [ -[0.054,0.062,0.064], -[0.698,0.563,0.407], -[0.586,0.412,0.266], -[1.386,0.363,0.301], -[0.814,0.825,0.984], -[1.303,1.024,1.048], -[0.069,0.066,0.062], -[0.334,0.254,0.270], -[1.043,0.952,0.993], -[1.210,1.209,1.171], -[0.667,0.483,0.456], -[0.543,0.495,0.500], -[1.005,0.889,0.888], -[1.644,1.646,1.652], -[1.054,1.044,0.966], -[0.893,0.874,0.907], -[1.737,1.779,1.837], -[1.518,1.539,1.526], -[3.082,2.818,2.842], -[0.309,0.286,0.256], -[1.594,1.017,0.993], -[0.781,0.853,0.735], -[1.461,1.226,1.080], -[5.308,2.974,2.642], -[0.511,0.625,0.467], -[0.405,0.382,0.439], -[0.601,0.535,0.471], -[0.947,1.624,1.192], -[2.631,2.486,2.490], -[1.938,1.960,1.954], -[1.930,0.830,0.835], -[1.359,1.140,1.062], -[3.599,3.623,3.621], -[3.619,3.741,3.663], -[3.725,3.614,3.786], -[1.149,1.126,1.055], -[0.202,0.207,0.196], -[0.152,0.139,0.145], -[0.149,0.144,0.148], -[0.383,0.287,0.294], -[0.203,0.137,0.119], -[0.200,0.312,0.137], -[0.149,0.130,0.214] -] -} diff --git a/benchmark/snowflake/results/s.json b/benchmark/snowflake/results/s.json deleted file mode 100644 index 7a686b95204..00000000000 --- a/benchmark/snowflake/results/s.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Snowflake", - "date": "2022-07-01", - "machine": "S", - "cluster_size": 2, - "comment": "", - - "tags": ["managed", "column-oriented"], - - "load_time": 2524, - "data_size": 12300000000, - - "result": [ -[0.186,0.060,0.062], -[0.980,0.574,0.311], -[0.977,0.554,0.426], -[0.686,0.573,0.404], -[1.386,1.384,1.349], -[1.871,1.697,1.704], -[0.052,0.059,0.227], -[0.309,0.536,0.508], -[1.768,1.631,1.635], -[2.039,2.219,1.908], -[0.807,0.647,0.587], -[0.763,0.690,0.631], -[1.403,1.586,1.404], -[2.593,2.584,2.554], -[1.670,1.538,1.653], -[1.659,1.509,1.514], -[2.875,2.990,2.998], -[2.605,2.549,2.598], -[6.120,5.894,5.766], -[0.320,0.431,0.416], -[2.406,1.703,1.609], -[1.189,1.186,1.163], -[2.104,1.441,1.370], -[7.144,5.174,4.139], -[0.839,0.659,0.641], -[0.527,0.518,0.509], -[0.633,0.621,0.695], -[1.491,1.509,1.514], -[4.848,4.485,4.571], -[3.067,3.106,3.098], -[1.521,1.224,1.236], -[1.839,1.690,1.497], -[5.692,5.751,6.087], -[6.733,6.755,6.712], -[6.722,6.709,6.676], -[1.704,1.686,1.676], -[0.203,0.231,0.218], -[0.151,0.134,0.214], -[0.140,0.156,0.163], -[0.317,0.328,0.319], -[0.166,0.133,0.141], -[0.166,0.120,0.140], -[0.120,0.119,0.126] -] -} diff --git a/benchmark/snowflake/results/xl.json b/benchmark/snowflake/results/xl.json deleted file mode 100644 index 9b417b24a42..00000000000 --- a/benchmark/snowflake/results/xl.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Snowflake", - "date": "2022-07-01", - "machine": "XL", - "cluster_size": 16, - "comment": "", - - "tags": ["managed", "column-oriented"], - - "load_time": 2524, - "data_size": 12300000000, - - "result": [ -[0.071,0.053,0.057], -[0.998,0.610,0.240], -[0.420,1.138,1.051], -[0.653,0.264,0.178], -[0.352,0.312,0.349], -[1.126,0.431,0.420], -[0.067,0.057,0.054], -[0.225,0.217,0.200], -[0.617,0.366,0.371], -[1.006,0.541,0.498], -[0.463,0.425,0.293], -[0.431,0.360,0.339], -[0.392,0.371,0.386], -[0.588,0.581,0.590], -[0.634,0.414,0.400], -[0.368,0.410,0.388], -[0.594,0.639,0.663], -[0.616,0.581,0.569], -[1.092,0.933,0.901], -[0.493,0.213,0.160], -[0.886,0.480,0.442], -[0.448,0.337,0.399], -[0.840,0.572,0.505], -[2.251,1.230,0.959], -[0.295,0.253,0.241], -[0.214,0.239,0.278], -[0.261,0.232,0.314], -[0.422,0.429,0.403], -[0.892,0.934,0.883], -[1.041,1.017,1.009], -[0.715,0.442,0.363], -[0.845,0.413,0.461], -[1.101,1.085,1.102], -[1.294,1.272,1.339], -[1.839,1.327,1.241], -[0.439,0.399,0.393], -[0.199,0.211,0.190], -[0.157,0.143,0.140], -[0.145,0.157,0.141], -[0.331,0.291,0.333], -[0.173,0.214,0.138], -[0.189,0.150,0.159], -[0.135,0.149,0.138] -] -} diff --git a/benchmark/snowflake/results/xs.json b/benchmark/snowflake/results/xs.json deleted file mode 100644 index 32fbfeb0dff..00000000000 --- a/benchmark/snowflake/results/xs.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "Snowflake", - "date": "2022-07-01", - "machine": "XS", - "cluster_size": 1, - "comment": "", - - "tags": ["managed", "column-oriented"], - - "load_time": 2524, - "data_size": 12300000000, - - "result": [ -[0.169,0.055,0.056], -[1.184,0.582,0.386], -[1.350,0.560,0.568], -[1.270,0.554,0.538], -[2.516,2.564,2.506], -[2.935,2.649,2.670], -[0.052,0.050,0.064], -[0.383,0.387,0.397], -[3.249,2.993,3.014], -[3.589,3.627,3.887], -[1.243,0.986,0.966], -[1.325,1.080,1.073], -[2.038,2.046,2.035], -[3.738,3.626,3.718], -[2.318,2.159,2.176], -[2.733,2.637,2.668], -[5.607,5.683,5.667], -[3.978,3.923,3.879], -[10.085,9.871,9.844], -[0.450,0.375,0.469], -[5.474,3.103,3.060], -[2.012,1.982,1.971], -[3.365,2.471,2.501], -[11.960,10.619,9.518], -[1.074,1.059,1.026], -[0.856,0.846,0.879], -[1.100,1.085,1.083], -[3.057,3.228,3.117], -[9.406,9.019,9.158], -[6.196,6.243,6.911], -[2.906,2.343,2.017], -[2.954,2.666,2.565], -[9.459,9.565,9.557], -[9.555,9.529,9.368], -[9.409,9.185,9.294], -[2.796,2.880,2.685], -[0.299,0.249,0.262], -[0.156,0.145,0.180], -[0.147,0.146,0.160], -[0.371,0.357,0.356], -[0.166,0.133,0.155], -[0.218,0.140,0.135], -[0.140,0.152,0.158] -] -} diff --git a/benchmark/snowflake/run.sh b/benchmark/snowflake/run.sh deleted file mode 100755 index 35739a4d7f3..00000000000 --- a/benchmark/snowflake/run.sh +++ /dev/null @@ -1,3 +0,0 @@ -#!/bin/bash - -sed -r -e 's/^(.*)$/\1 \1 \1/' queries.sql | snowsql --region eu-central-1 --schemaname PUBLIC --dbname HITS --warehouse TEST diff --git a/benchmark/sqlite/benchmark.sh b/benchmark/sqlite/benchmark.sh deleted file mode 100755 index a144acf680b..00000000000 --- a/benchmark/sqlite/benchmark.sh +++ /dev/null @@ -1,20 +0,0 @@ -#!/bin/bash - -sudo apt-get update -sudo apt-get install -y sqlite3 - -sqlite3 mydb < create.sql - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.csv.gz' -gzip -d hits.csv.gz - -time sqlite3 mydb '.import --csv hits.csv hits' -wc -c mydb - -./run.sh 2>&1 | tee log.txt - -cat log.txt | - grep -P '^real|^Error' | - sed -r -e 's/^Error.*$/null/; s/^real\s*([0-9.]+)m([0-9.]+)s$/\1 \2/' | - awk '{ if ($2) { print $1 * 60 + $2 } else { print $1 } }' | - awk '{ if ($1 == "null") { skip = 1 } else { if (i % 3 == 0) { printf "[" }; printf skip ? "null" : $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; skip = 0; } }' diff --git a/benchmark/sqlite/create.sql b/benchmark/sqlite/create.sql deleted file mode 100644 index 1850bffedce..00000000000 --- a/benchmark/sqlite/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL, - PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) -); diff --git a/benchmark/sqlite/queries.sql b/benchmark/sqlite/queries.sql deleted file mode 100644 index ed185494351..00000000000 --- a/benchmark/sqlite/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, strftime('%M', EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT strftime('%M', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY strftime('%M', EventTime) ORDER BY strftime('%M', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/sqlite/results/c6a.4xlarge.json b/benchmark/sqlite/results/c6a.4xlarge.json deleted file mode 100644 index e442dcadc80..00000000000 --- a/benchmark/sqlite/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "SQLite", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C", "embedded", "row-oriented"], - - "load_time": 2608, - "data_size": 75776589824, - - "result": [ -[752.739,2.003,1.2], -[304.302,291.521,286.965], -[293.964,287.619,287.219], -[758.302,5.879,5.65], -[836.393,48.593,48.452], -[362.605,344.884,356.245], -[763.993,11.602,10.795], -[296.348,286.879,287.557], -[365.816,360.339,354.126], -[374.403,365.196,362.261], -[302.989,293.888,298.432], -[303.64,291.729,295.347], -[316.824,298.18,301.006], -[320.665,301.15,305.227], -[313.593,301.021,301.626], -[794.881,47,47.225], -[355.346,344.615,342.442], -[316.499,305.971,305.007], -[398.177,380.383,385.571], -[751.82,5.082,4.913], -[295.522,286.573,287.368], -[298.58,287.182,288.303], -[296.474,288.747,288.638], -[296.579,287.127,287.361], -[304.709,286.865,287.56], -[300.391,290.633,288.587], -[294.605,286.91,287.799], -[305.986,312.111,305.626], -[null,null,null], -[411.225,397.614,394.253], -[307.711,295.181,300.266], -[312.472,299.079,298.19], -[386.674,378.347,376.963], -[409.742,409.554,420.273], -[468.73,453.709,458.149], -[366.015,347.446,346.728], -[2.911,0.781,0.757], -[1.599,0.609,0.587], -[1.288,0.256,0.238], -[2.469,1.582,1.52], -[1.274,0.303,0.283], -[1.322,0.317,0.314], -[1.498,0.602,0.613] -] -} diff --git a/benchmark/sqlite/run.sh b/benchmark/sqlite/run.sh deleted file mode 100755 index e3eccc1cc0b..00000000000 --- a/benchmark/sqlite/run.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - echo "$query"; - for i in $(seq 1 $TRIES); do - time sqlite3 mydb <<< "${query}" - done; -done; diff --git a/benchmark/starrocks/README.md b/benchmark/starrocks/README.md deleted file mode 100644 index a04fe13890d..00000000000 --- a/benchmark/starrocks/README.md +++ /dev/null @@ -1,3 +0,0 @@ -It requires providing an email for downloading. -But then the button on the website does not work and it does not download. -I cannot find the packages anywhere. diff --git a/benchmark/starrocks/benchmark.sh b/benchmark/starrocks/benchmark.sh deleted file mode 100755 index 2418d847397..00000000000 --- a/benchmark/starrocks/benchmark.sh +++ /dev/null @@ -1,3 +0,0 @@ -#!/bin/bash - -# Install diff --git a/benchmark/timescaledb-compressed/benchmark.sh b/benchmark/timescaledb-compressed/benchmark.sh deleted file mode 100755 index 68600be3486..00000000000 --- a/benchmark/timescaledb-compressed/benchmark.sh +++ /dev/null @@ -1,48 +0,0 @@ -#!/bin/bash - -# Install -export DEBIAN_FRONTEND=noninteractive -sudo apt-get update -sudo apt-get install -y gnupg postgresql-common apt-transport-https lsb-release wget -sudo /usr/share/postgresql-common/pgdg/apt.postgresql.org.sh -sudo bash -c 'echo "deb https://packagecloud.io/timescale/timescaledb/ubuntu/ $(lsb_release -c -s) main" > /etc/apt/sources.list.d/timescaledb.list' -wget --quiet -O - https://packagecloud.io/timescale/timescaledb/gpgkey | sudo apt-key add - -sudo apt-get update -sudo apt-get install -y timescaledb-2-postgresql-14 -sudo bash -c "echo \"shared_preload_libraries = 'timescaledb'\" >> /etc/postgresql/14/main/postgresql.conf" -sudo systemctl restart postgresql - -sudo -u postgres psql -c "CREATE DATABASE test" -sudo -u postgres psql test -c "CREATE EXTENSION IF NOT EXISTS timescaledb" - -# Import the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz -sudo chmod og+rX ~ -chmod 777 hits.tsv - -sudo -u postgres psql test < create.sql -sudo -u postgres psql test -c "SELECT create_hypertable('hits', 'eventtime')" -sudo -u postgres psql test -c "CREATE INDEX ix_counterid ON hits (counterid)" -sudo -u postgres psql test -c "ALTER TABLE hits SET (timescaledb.compress, timescaledb.compress_orderby = 'counterid, eventdate, userid, eventtime')" -sudo -u postgres psql test -c "SELECT add_compression_policy('hits', INTERVAL '1s')" - -sudo -u postgres psql test -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" - -# 1619875.288 ms (26:59.875) - -# See https://github.com/timescale/timescaledb/issues/4473#issuecomment-1167095245 -# https://docs.timescale.com/timescaledb/latest/how-to-guides/compression/manually-compress-chunks/#compress-chunks-manually -# TimescaleDB benchmark wihout compression is available in timescaledb directory - -time sudo -u postgres psql test -c "SELECT compress_chunk(i, if_not_compressed => true) FROM show_chunks('hits') i" - -# 49m45.120s - -./run.sh 2>&1 | tee log.txt - -sudo du -bcs /var/lib/postgresql/14/main/ - -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/timescaledb-compressed/create.sql b/benchmark/timescaledb-compressed/create.sql deleted file mode 100644 index 41c961c00fc..00000000000 --- a/benchmark/timescaledb-compressed/create.sql +++ /dev/null @@ -1,108 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -); diff --git a/benchmark/timescaledb-compressed/queries.sql b/benchmark/timescaledb-compressed/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/timescaledb-compressed/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/timescaledb-compressed/results/c6a.4xlarge.json b/benchmark/timescaledb-compressed/results/c6a.4xlarge.json deleted file mode 100644 index 457038eb591..00000000000 --- a/benchmark/timescaledb-compressed/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "TimescaleDB (compression)", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C", "PostgreSQL compatible", "column-oriented", "time-series"], - - "load_time": 4605, - "data_size": 20333747165, - - "result": [ -[2.28686,1.63642,1.64263], -[32.6848,1.63476,1.40052], -[60.8633,3.70484,3.59342], -[36.4029,2.87091,2.80739], -[110.391,38.9688,38.0549], -[147.379,66.2513,65.6379], -[33.0294,2.92031,2.84375], -[33.0221,1.2984,1.19227], -[115.694,47.4651,47.0125], -[156.347,51.577,51.2694], -[68.3301,4.75521,4.68007], -[77.4356,5.55128,5.56577], -[49.7741,11.2911,11.3265], -[81.1014,14.9111,14.9541], -[82.9569,14.6156,14.6331], -[62.0338,26.399,26.3351], -[103.259,36.4122,36.6076], -[92.8828,26.2395,25.8991], -[144.281,63.5102,63.7661], -[7.00679,0.573073,0.536283], -[75.0203,7.86344,7.90495], -[81.2825,9.15868,9.01775], -[104.084,13.9528,13.8435], -[132.531,81.522,82.1561], -[80.6965,3.28231,3.16574], -[39.7693,2.51443,2.43849], -[80.4245,3.26941,3.13916], -[104.015,13.7044,13.5313], -[307.26,253.127,252.147], -[42.8549,22.4187,22.0325], -[137.601,14.9592,14.6804], -[136.767,22.8007,22.131], -[263.005,168.551,163.355], -[156.919,92.6308,91.702], -[160.842,96.0512,97.1773], -[62.8357,28.0336,28.7397], -[1.75869,0.561604,0.541215], -[0.46607,0.191863,0.19021], -[0.303671,0.137579,0.136615], -[2.32031,1.49223,1.52369], -[0.563764,0.14192,0.138234], -[0.372428,0.122989,0.123709], -[0.448574,0.159092,0.154687] -] -} diff --git a/benchmark/timescaledb-compressed/run.sh b/benchmark/timescaledb-compressed/run.sh deleted file mode 100755 index 198ab546163..00000000000 --- a/benchmark/timescaledb-compressed/run.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - echo "$query"; - for i in $(seq 1 $TRIES); do - psql test -t -c '\timing' -c "$query" | grep 'Time' - done; -done; diff --git a/benchmark/timescaledb/benchmark.sh b/benchmark/timescaledb/benchmark.sh deleted file mode 100755 index e43509ba915..00000000000 --- a/benchmark/timescaledb/benchmark.sh +++ /dev/null @@ -1,41 +0,0 @@ -#!/bin/bash - -# Install - -export DEBIAN_FRONTEND=noninteractive -sudo apt-get update -sudo apt-get install -y gnupg postgresql-common apt-transport-https lsb-release wget -sudo /usr/share/postgresql-common/pgdg/apt.postgresql.org.sh -sudo bash -c 'echo "deb https://packagecloud.io/timescale/timescaledb/ubuntu/ $(lsb_release -c -s) main" > /etc/apt/sources.list.d/timescaledb.list' -wget --quiet -O - https://packagecloud.io/timescale/timescaledb/gpgkey | sudo apt-key add - -sudo apt-get update -sudo apt-get install -y timescaledb-2-postgresql-14 -sudo bash -c "echo \"shared_preload_libraries = 'timescaledb'\" >> /etc/postgresql/14/main/postgresql.conf" -sudo systemctl restart postgresql - -sudo -u postgres psql -c "CREATE DATABASE uncompressed" -sudo -u postgres psql uncompressed -c "CREATE EXTENSION IF NOT EXISTS timescaledb" - -# Import the data - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz -sudo chmod og+rX ~ -chmod 777 hits.tsv - -sudo -u postgres psql uncompressed < create.sql -sudo -u postgres psql uncompressed -c "SELECT create_hypertable('hits', 'eventtime')" -sudo -u postgres psql uncompressed -c "CREATE INDEX ix_counterid ON hits (counterid)" -sudo -u postgres psql uncompressed -c "ALTER TABLE hits SET (timescaledb.compress, timescaledb.compress_orderby = 'counterid, eventdate, userid, eventtime')" -sudo -u postgres psql uncompressed -c "SELECT add_compression_policy('hits', INTERVAL '1s')" - -sudo -u postgres psql uncompressed -t -c '\timing' -c "\\copy hits FROM 'hits.tsv'" - -# 1619875.288 ms (26:59.875) - -./run.sh 2>&1 | tee log.txt - -sudo du -bcs /var/lib/postgresql/14/main/ - -cat log.txt | grep -oP 'Time: \d+\.\d+ ms' | sed -r -e 's/Time: ([0-9]+\.[0-9]+) ms/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/timescaledb/create.sql b/benchmark/timescaledb/create.sql deleted file mode 100644 index 41c961c00fc..00000000000 --- a/benchmark/timescaledb/create.sql +++ /dev/null @@ -1,108 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title TEXT NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL TEXT NOT NULL, - Referer TEXT NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 TEXT NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel TEXT NOT NULL, - Params TEXT NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase TEXT NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset TEXT NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL TEXT NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage TEXT NOT NULL, - BrowserCountry TEXT NOT NULL, - SocialNetwork TEXT NOT NULL, - SocialAction TEXT NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage TEXT NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID TEXT NOT NULL, - ParamCurrency TEXT NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName TEXT NOT NULL, - OpenstatCampaignID TEXT NOT NULL, - OpenstatAdID TEXT NOT NULL, - OpenstatSourceID TEXT NOT NULL, - UTMSource TEXT NOT NULL, - UTMMedium TEXT NOT NULL, - UTMCampaign TEXT NOT NULL, - UTMContent TEXT NOT NULL, - UTMTerm TEXT NOT NULL, - FromTag TEXT NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -); diff --git a/benchmark/timescaledb/queries.sql b/benchmark/timescaledb/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/timescaledb/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/timescaledb/results/c6a.4xlarge.json b/benchmark/timescaledb/results/c6a.4xlarge.json deleted file mode 100644 index 63d57a74975..00000000000 --- a/benchmark/timescaledb/results/c6a.4xlarge.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "system": "TimescaleDB", - "date": "2022-07-01", - "machine": "c6a.4xlarge, 500gb gp2", - "cluster_size": 1, - "comment": "", - - "tags": ["C", "PostgreSQL compatible", "row-oriented", "time-series"], - - "load_time": 1620, - "data_size": 72882392030, - - "result": [ -[437.700,215.793,176.420], -[327.026,259.568,244.578], -[262.978,263.090,263.083], -[262.807,263.046,266.847], -[337.497,334.964,330.852], -[355.689,356.801,362.894], -[262.762,263.012,262.968], -[263.055,263.016,263.028], -[319.928,319.388,320.704], -[323.584,322.224,322.488], -[265.979,265.465,265.375], -[266.019,265.543,265.462], -[277.018,276.300,276.595], -[280.352,279.251,279.572], -[279.915,279.896,279.674], -[296.377,298.506,297.659], -[314.448,314.605,312.570], -[302.668,302.672,303.039], -[325.810,324.061,324.376], -[262.447,262.698,262.704], -[267.581,267.467,267.482], -[268.085,267.466,267.696], -[263.391,263.097,263.126], -[38.291,0.435,0.335], -[0.127,0.005,0.005], -[263.138,263.100,263.092], -[0.889,0.341,0.339], -[267.586,267.498,267.491], -[289.086,290.012,290.093], -[263.220,263.071,263.109], -[274.780,273.995,273.998], -[282.217,281.390,281.470], -[429.273,426.588,439.431], -[448.808,418.724,418.207], -[455.196,422.750,423.142], -[299.263,296.937,297.261], -[18.693,1.552,1.481], -[18.125,0.945,0.937], -[18.528,1.062,0.902], -[18.268,1.779,1.770], -[19.615,1.965,1.966], -[18.970,1.435,1.430], -[18.330,1.153,0.952] -] -} diff --git a/benchmark/timescaledb/run.sh b/benchmark/timescaledb/run.sh deleted file mode 100755 index 84edae1f3d3..00000000000 --- a/benchmark/timescaledb/run.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - echo "$query"; - for i in $(seq 1 $TRIES); do - sudo -u postgres psql uncompressed -t -c '\timing' -c "$query" | grep 'Time' - done; -done; diff --git a/benchmark/trino/README.md b/benchmark/trino/README.md deleted file mode 100644 index 82b0b2ff959..00000000000 --- a/benchmark/trino/README.md +++ /dev/null @@ -1 +0,0 @@ -Incomplete. diff --git a/benchmark/trino/benchmark.sh b/benchmark/trino/benchmark.sh deleted file mode 100755 index 71fab8922f0..00000000000 --- a/benchmark/trino/benchmark.sh +++ /dev/null @@ -1,10 +0,0 @@ -#!/bin/bash - -sudo apt-get update -sudo apt-get install -y docker.io -sudo docker run --network host -p 8080:8080 --name trino trinodb/trino - -sudo docker exec -it trino trino - -CREATE SCHEMA memory.test; -USE memory.test; diff --git a/benchmark/trino/create_partitioned.sql b/benchmark/trino/create_partitioned.sql deleted file mode 100644 index b0f9e5d0be5..00000000000 --- a/benchmark/trino/create_partitioned.sql +++ /dev/null @@ -1,112 +0,0 @@ -CREATE EXTERNAL TABLE IF NOT EXISTS `test`.`hits` ( -`watchid` bigint, -`javaenable` smallint, -`title` string, -`goodevent` smallint, -`eventtime` timestamp, -`eventdate` date, -`counterid` int, -`clientip` int, -`regionid` int, -`userid` bigint, -`counterclass` smallint, -`os` smallint, -`useragent` smallint, -`url` string, -`referer` string, -`isrefresh` smallint, -`referercategoryid` smallint, -`refererregionid` int, -`urlcategoryid` smallint, -`urlregionid` int, -`resolutionwidth` smallint, -`resolutionheight` smallint, -`resolutiondepth` smallint, -`flashmajor` smallint, -`flashminor` smallint, -`flashminor2` string, -`netmajor` smallint, -`netminor` smallint, -`useragentmajor` smallint, -`useragentminor` string, -`cookieenable` smallint, -`javascriptenable` smallint, -`ismobile` smallint, -`mobilephone` smallint, -`mobilephonemodel` string, -`params` string, -`ipnetworkid` int, -`traficsourceid` smallint, -`searchengineid` smallint, -`searchphrase` string, -`advengineid` smallint, -`isartifical` smallint, -`windowclientwidth` smallint, -`windowclientheight` smallint, -`clienttimezone` smallint, -`clienteventtime` timestamp, -`silverlightversion1` smallint, -`silverlightversion2` smallint, -`silverlightversion3` int, -`silverlightversion4` smallint, -`pagecharset` string, -`codeversion` int, -`islink` smallint, -`isdownload` smallint, -`isnotbounce` smallint, -`funiqid` bigint, -`originalurl` string, -`hid` int, -`isoldcounter` smallint, -`isevent` smallint, -`isparameter` smallint, -`dontcounthits` smallint, -`withhash` smallint, -`hitcolor` string, -`localeventtime` timestamp, -`age` smallint, -`sex` smallint, -`income` smallint, -`interests` smallint, -`robotness` smallint, -`remoteip` int, -`windowname` int, -`openername` int, -`historylength` smallint, -`browserlanguage` string, -`browsercountry` string, -`socialnetwork` string, -`socialaction` string, -`httperror` smallint, -`sendtiming` int, -`dnstiming` int, -`connecttiming` int, -`responsestarttiming` int, -`responseendtiming` int, -`fetchtiming` int, -`socialsourcenetworkid` smallint, -`socialsourcepage` string, -`paramprice` bigint, -`paramorderid` string, -`paramcurrency` string, -`paramcurrencyid` smallint, -`openstatservicename` string, -`openstatcampaignid` string, -`openstatadid` string, -`openstatsourceid` string, -`utmsource` string, -`utmmedium` string, -`utmcampaign` string, -`utmcontent` string, -`utmterm` string, -`fromtag` string, -`hasgclid` smallint, -`refererhash` bigint, -`urlhash` bigint, -`clid` int -) -ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' -WITH SERDEPROPERTIES ( -'serialization.format' = '1' -) LOCATION 's3://clickhouse-public-datasets/hits_compatible/athena_partitioned' -TBLPROPERTIES ('has_encrypted_data'='false'); diff --git a/benchmark/trino/create_single.sql b/benchmark/trino/create_single.sql deleted file mode 100644 index 4a24f246618..00000000000 --- a/benchmark/trino/create_single.sql +++ /dev/null @@ -1,107 +0,0 @@ -CREATE TABLE IF NOT EXISTS "memory"."test"."hits" ( -"watchid" bigint, -"javaenable" smallint, -"title" varchar(65535), -"goodevent" smallint, -"eventtime" timestamp, -"eventdate" date, -"counterid" int, -"clientip" int, -"regionid" int, -"userid" bigint, -"counterclass" smallint, -"os" smallint, -"useragent" smallint, -"url" varchar(65535), -"referer" varchar(65535), -"isrefresh" smallint, -"referercategoryid" smallint, -"refererregionid" int, -"urlcategoryid" smallint, -"urlregionid" int, -"resolutionwidth" smallint, -"resolutionheight" smallint, -"resolutiondepth" smallint, -"flashmajor" smallint, -"flashminor" smallint, -"flashminor2" varchar(65535), -"netmajor" smallint, -"netminor" smallint, -"useragentmajor" smallint, -"useragentminor" varchar(65535), -"cookieenable" smallint, -"javascriptenable" smallint, -"ismobile" smallint, -"mobilephone" smallint, -"mobilephonemodel" varchar(65535), -"params" varchar(65535), -"ipnetworkid" int, -"traficsourceid" smallint, -"searchengineid" smallint, -"searchphrase" varchar(65535), -"advengineid" smallint, -"isartifical" smallint, -"windowclientwidth" smallint, -"windowclientheight" smallint, -"clienttimezone" smallint, -"clienteventtime" timestamp, -"silverlightversion1" smallint, -"silverlightversion2" smallint, -"silverlightversion3" int, -"silverlightversion4" smallint, -"pagecharset" varchar(65535), -"codeversion" int, -"islink" smallint, -"isdownload" smallint, -"isnotbounce" smallint, -"funiqid" bigint, -"originalurl" varchar(65535), -"hid" int, -"isoldcounter" smallint, -"isevent" smallint, -"isparameter" smallint, -"dontcounthits" smallint, -"withhash" smallint, -"hitcolor" varchar(65535), -"localeventtime" timestamp, -"age" smallint, -"sex" smallint, -"income" smallint, -"interests" smallint, -"robotness" smallint, -"remoteip" int, -"windowname" int, -"openername" int, -"historylength" smallint, -"browserlanguage" varchar(65535), -"browsercountry" varchar(65535), -"socialnetwork" varchar(65535), -"socialaction" varchar(65535), -"httperror" smallint, -"sendtiming" int, -"dnstiming" int, -"connecttiming" int, -"responsestarttiming" int, -"responseendtiming" int, -"fetchtiming" int, -"socialsourcenetworkid" smallint, -"socialsourcepage" varchar(65535), -"paramprice" bigint, -"paramorderid" varchar(65535), -"paramcurrency" varchar(65535), -"paramcurrencyid" smallint, -"openstatservicename" varchar(65535), -"openstatcampaignid" varchar(65535), -"openstatadid" varchar(65535), -"openstatsourceid" varchar(65535), -"utmsource" varchar(65535), -"utmmedium" varchar(65535), -"utmcampaign" varchar(65535), -"utmcontent" varchar(65535), -"utmterm" varchar(65535), -"fromtag" varchar(65535), -"hasgclid" smallint, -"refererhash" bigint, -"urlhash" bigint, -"clid" int -); diff --git a/benchmark/trino/queries.sql b/benchmark/trino/queries.sql deleted file mode 100644 index ef8c727ead6..00000000000 --- a/benchmark/trino/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, extract(minute FROM EventTime), SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC OFFSET 1000 LIMIT 10; -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 >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN (SearchEngineID = 0 AND AdvEngineID = 0) THEN Referer ELSE '' END, URL ORDER BY PageViews DESC OFFSET 1000 LIMIT 10; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC OFFSET 100 LIMIT 10; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-01' AND EventDate <= DATE '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC OFFSET 10000 LIMIT 10; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= DATE '2013-07-14' AND EventDate <= DATE '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) OFFSET 1000 LIMIT 10; diff --git a/benchmark/vertica/.gitignore b/benchmark/vertica/.gitignore deleted file mode 100644 index 1a06816d838..00000000000 --- a/benchmark/vertica/.gitignore +++ /dev/null @@ -1 +0,0 @@ -results diff --git a/benchmark/vertica/README.md b/benchmark/vertica/README.md deleted file mode 100644 index 0adab4f41f9..00000000000 --- a/benchmark/vertica/README.md +++ /dev/null @@ -1,5 +0,0 @@ -Although Vertica EULA does not prevent doing benchmarks, it restricts from disclosing the results: - -> You may not disclose to any third-party performance information or analysis (including, without limitation, benchmarks and performance tests) from any source relating to the Software. - -https://www.vertica.com/end-user-license-agreement-ce-version/ diff --git a/benchmark/vertica/benchmark.sh b/benchmark/vertica/benchmark.sh deleted file mode 100755 index 86312a3a438..00000000000 --- a/benchmark/vertica/benchmark.sh +++ /dev/null @@ -1,26 +0,0 @@ -#!/bin/bash - -sudo apt-get update -sudo apt-get install -y docker.io - -sudo docker run -p 5433:5433 -p 5444:5444 --volume $(pwd):/workdir --mount type=volume,source=vertica-data,target=/data --name vertica_ce vertica/vertica-ce - -sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c "$(cat create.sql)" - -wget --continue 'https://datasets.clickhouse.com/hits_compatible/hits.tsv.gz' -gzip -d hits.tsv.gz - -time sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c "COPY hits FROM LOCAL '/workdir/hits.tsv' DELIMITER E'\\t' NULL E'\\001' DIRECT" - -sudo docker exec vertica_ce du -bcs /data/vertica/VMart - -./run.sh 2>&1 | tee log.txt - -# If you run the script on your own, you may get numbers like this: -# 200m00.000s -# 25000000000 - -# Note: the real numbers cannot be published. - -grep -F 'All rows formatted' logs.txt | sed -r -e 's/^.* ([0-9.]+) ms$/\1/' | - awk '{ if (i % 3 == 0) { printf "[" }; printf $1 / 1000; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }' diff --git a/benchmark/vertica/create.sql b/benchmark/vertica/create.sql deleted file mode 100644 index a6316c5e227..00000000000 --- a/benchmark/vertica/create.sql +++ /dev/null @@ -1,109 +0,0 @@ -CREATE TABLE hits -( - WatchID BIGINT NOT NULL, - JavaEnable SMALLINT NOT NULL, - Title VARCHAR(65000) NOT NULL, - GoodEvent SMALLINT NOT NULL, - EventTime TIMESTAMP NOT NULL, - EventDate Date NOT NULL, - CounterID INTEGER NOT NULL, - ClientIP INTEGER NOT NULL, - RegionID INTEGER NOT NULL, - UserID BIGINT NOT NULL, - CounterClass SMALLINT NOT NULL, - OS SMALLINT NOT NULL, - UserAgent SMALLINT NOT NULL, - URL VARCHAR(65000) NOT NULL, - Referer VARCHAR(65000) NOT NULL, - IsRefresh SMALLINT NOT NULL, - RefererCategoryID SMALLINT NOT NULL, - RefererRegionID INTEGER NOT NULL, - URLCategoryID SMALLINT NOT NULL, - URLRegionID INTEGER NOT NULL, - ResolutionWidth SMALLINT NOT NULL, - ResolutionHeight SMALLINT NOT NULL, - ResolutionDepth SMALLINT NOT NULL, - FlashMajor SMALLINT NOT NULL, - FlashMinor SMALLINT NOT NULL, - FlashMinor2 VARCHAR(65000) NOT NULL, - NetMajor SMALLINT NOT NULL, - NetMinor SMALLINT NOT NULL, - UserAgentMajor SMALLINT NOT NULL, - UserAgentMinor VARCHAR(255) NOT NULL, - CookieEnable SMALLINT NOT NULL, - JavascriptEnable SMALLINT NOT NULL, - IsMobile SMALLINT NOT NULL, - MobilePhone SMALLINT NOT NULL, - MobilePhoneModel VARCHAR(65000) NOT NULL, - Params VARCHAR(65000) NOT NULL, - IPNetworkID INTEGER NOT NULL, - TraficSourceID SMALLINT NOT NULL, - SearchEngineID SMALLINT NOT NULL, - SearchPhrase VARCHAR(65000) NOT NULL, - AdvEngineID SMALLINT NOT NULL, - IsArtifical SMALLINT NOT NULL, - WindowClientWidth SMALLINT NOT NULL, - WindowClientHeight SMALLINT NOT NULL, - ClientTimeZone SMALLINT NOT NULL, - ClientEventTime TIMESTAMP NOT NULL, - SilverlightVersion1 SMALLINT NOT NULL, - SilverlightVersion2 SMALLINT NOT NULL, - SilverlightVersion3 INTEGER NOT NULL, - SilverlightVersion4 SMALLINT NOT NULL, - PageCharset VARCHAR(65000) NOT NULL, - CodeVersion INTEGER NOT NULL, - IsLink SMALLINT NOT NULL, - IsDownload SMALLINT NOT NULL, - IsNotBounce SMALLINT NOT NULL, - FUniqID BIGINT NOT NULL, - OriginalURL VARCHAR(65000) NOT NULL, - HID INTEGER NOT NULL, - IsOldCounter SMALLINT NOT NULL, - IsEvent SMALLINT NOT NULL, - IsParameter SMALLINT NOT NULL, - DontCountHits SMALLINT NOT NULL, - WithHash SMALLINT NOT NULL, - HitColor CHAR NOT NULL, - LocalEventTime TIMESTAMP NOT NULL, - Age SMALLINT NOT NULL, - Sex SMALLINT NOT NULL, - Income SMALLINT NOT NULL, - Interests SMALLINT NOT NULL, - Robotness SMALLINT NOT NULL, - RemoteIP INTEGER NOT NULL, - WindowName INTEGER NOT NULL, - OpenerName INTEGER NOT NULL, - HistoryLength SMALLINT NOT NULL, - BrowserLanguage VARCHAR(65000) NOT NULL, - BrowserCountry VARCHAR(65000) NOT NULL, - SocialNetwork VARCHAR(65000) NOT NULL, - SocialAction VARCHAR(65000) NOT NULL, - HTTPError SMALLINT NOT NULL, - SendTiming INTEGER NOT NULL, - DNSTiming INTEGER NOT NULL, - ConnectTiming INTEGER NOT NULL, - ResponseStartTiming INTEGER NOT NULL, - ResponseEndTiming INTEGER NOT NULL, - FetchTiming INTEGER NOT NULL, - SocialSourceNetworkID SMALLINT NOT NULL, - SocialSourcePage VARCHAR(65000) NOT NULL, - ParamPrice BIGINT NOT NULL, - ParamOrderID VARCHAR(65000) NOT NULL, - ParamCurrency VARCHAR(65000) NOT NULL, - ParamCurrencyID SMALLINT NOT NULL, - OpenstatServiceName VARCHAR(65000) NOT NULL, - OpenstatCampaignID VARCHAR(65000) NOT NULL, - OpenstatAdID VARCHAR(65000) NOT NULL, - OpenstatSourceID VARCHAR(65000) NOT NULL, - UTMSource VARCHAR(65000) NOT NULL, - UTMMedium VARCHAR(65000) NOT NULL, - UTMCampaign VARCHAR(65000) NOT NULL, - UTMContent VARCHAR(65000) NOT NULL, - UTMTerm VARCHAR(65000) NOT NULL, - FromTag VARCHAR(65000) NOT NULL, - HasGCLID SMALLINT NOT NULL, - RefererHash BIGINT NOT NULL, - URLHash BIGINT NOT NULL, - CLID INTEGER NOT NULL -) -ORDER BY CounterID, EventDate, UserID, EventTime, WatchID; diff --git a/benchmark/vertica/queries.sql b/benchmark/vertica/queries.sql deleted file mode 100644 index 31f65fc898d..00000000000 --- a/benchmark/vertica/queries.sql +++ /dev/null @@ -1,43 +0,0 @@ -SELECT COUNT(*) FROM hits; -SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; -SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; -SELECT AVG(UserID) FROM hits; -SELECT COUNT(DISTINCT UserID) FROM hits; -SELECT COUNT(DISTINCT SearchPhrase) FROM hits; -SELECT MIN(EventDate), MAX(EventDate) FROM hits; -SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; -SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; -SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; -SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; -SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; -SELECT UserID FROM hits WHERE UserID = 435090932899640449; -SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; -SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; -SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -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; -SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits 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; -SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; -SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; -SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; -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; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 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 WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/benchmark/vertica/run.sh b/benchmark/vertica/run.sh deleted file mode 100755 index 7638dbb0299..00000000000 --- a/benchmark/vertica/run.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash - -TRIES=3 - -cat queries.sql | while read query; do - sync - echo 3 | sudo tee /proc/sys/vm/drop_caches - - echo "$query"; - for i in $(seq 1 $TRIES); do - sudo docker exec vertica_ce /opt/vertica/bin/vsql -U dbadmin -c '\timing' -c "$query" - done; -done; From 9e0cb37bd7d37b3122c1e54c9e79dcb88d569b10 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Jun 2022 11:36:06 +0000 Subject: [PATCH 512/627] Disable parallel hash join for asof --- src/Interpreters/TableJoin.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 1352e3777f1..17869e2084b 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -836,6 +836,8 @@ bool TableJoin::allowParallelHashJoin() const return false; if (table_join.kind != ASTTableJoin::Kind::Left && table_join.kind != ASTTableJoin::Kind::Inner) return false; + if (table_join.strictness == ASTTableJoin::Strictness::Asof) + return false; if (isSpecialStorage() || !oneDisjunct()) return false; return true; From 9f5e30a5d396eedf591c060f491044a10b040e3b Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 30 May 2022 12:43:47 +0000 Subject: [PATCH 513/627] Set default join_algorithm = parallel_hash --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5597d9076a4..12149688eba 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -350,7 +350,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \ M(OverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(Bool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \ - M(JoinAlgorithm, join_algorithm, JoinAlgorithm::HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge', 'parallel_hash'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \ + M(JoinAlgorithm, join_algorithm, JoinAlgorithm::PARALLEL_HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge', 'parallel_hash'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \ M(UInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \ M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ From 34d2a0a4db44a43e813cb5f2cc3b6b2f0112a87e Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Jun 2022 13:59:30 +0000 Subject: [PATCH 514/627] order by some join-related tests --- tests/queries/0_stateless/00050_any_left_join.sql | 3 ++- tests/queries/0_stateless/00053_all_inner_join.sql | 3 ++- tests/queries/0_stateless/00054_join_string.sql | 3 ++- tests/queries/0_stateless/00057_join_aliases.sql | 7 ++++++- .../queries/0_stateless/00169_join_constant_keys.sql | 3 ++- .../0_stateless/00689_join_table_function.sql | 2 +- tests/queries/0_stateless/00702_join_with_using.sql | 12 ++++++------ .../queries/0_stateless/01050_engine_join_crash.sql | 6 +++--- 8 files changed, 24 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/00050_any_left_join.sql b/tests/queries/0_stateless/00050_any_left_join.sql index 61ccb200307..5019de9fce7 100644 --- a/tests/queries/0_stateless/00050_any_left_join.sql +++ b/tests/queries/0_stateless/00050_any_left_join.sql @@ -6,4 +6,5 @@ ANY LEFT JOIN ( SELECT number * 2 AS k, number AS joined FROM system.numbers LIMIT 10 ) AS b -USING k; +USING k +ORDER BY k; diff --git a/tests/queries/0_stateless/00053_all_inner_join.sql b/tests/queries/0_stateless/00053_all_inner_join.sql index b2e65c80574..6cefb209682 100644 --- a/tests/queries/0_stateless/00053_all_inner_join.sql +++ b/tests/queries/0_stateless/00053_all_inner_join.sql @@ -6,4 +6,5 @@ ALL INNER JOIN ( SELECT intDiv(number, 2) AS k, number AS joined FROM system.numbers LIMIT 10 ) AS b -USING k; +USING k +ORDER BY k; diff --git a/tests/queries/0_stateless/00054_join_string.sql b/tests/queries/0_stateless/00054_join_string.sql index 274b19dc075..09a03f73614 100644 --- a/tests/queries/0_stateless/00054_join_string.sql +++ b/tests/queries/0_stateless/00054_join_string.sql @@ -6,4 +6,5 @@ ALL LEFT JOIN ( SELECT reinterpretAsString(intDiv(number, 2) + reinterpretAsUInt8('A')) AS k, number AS joined FROM system.numbers LIMIT 10 ) js2 -USING k; +USING k +ORDER BY k; diff --git a/tests/queries/0_stateless/00057_join_aliases.sql b/tests/queries/0_stateless/00057_join_aliases.sql index 6568e36fbda..481b0621ed7 100644 --- a/tests/queries/0_stateless/00057_join_aliases.sql +++ b/tests/queries/0_stateless/00057_join_aliases.sql @@ -1 +1,6 @@ -SELECT number, number / 2 AS n, j1, j2 FROM system.numbers ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 10) js2 USING n LIMIT 10 +SELECT * FROM ( + SELECT number, number / 2 AS n, j1, j2 + FROM system.numbers + ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 10) js2 + USING n LIMIT 10 +) ORDER BY n; diff --git a/tests/queries/0_stateless/00169_join_constant_keys.sql b/tests/queries/0_stateless/00169_join_constant_keys.sql index 16ab696c0f3..03c01c074f7 100644 --- a/tests/queries/0_stateless/00169_join_constant_keys.sql +++ b/tests/queries/0_stateless/00169_join_constant_keys.sql @@ -14,4 +14,5 @@ FROM arrayJoin([1, 3, 2]) AS key1, 0 AS key2, 999 AS table_1 -) js2 USING key2, key1; +) js2 USING key2, key1 +ORDER BY key1; diff --git a/tests/queries/0_stateless/00689_join_table_function.sql b/tests/queries/0_stateless/00689_join_table_function.sql index d44d1009ce8..c8e9100dcc0 100644 --- a/tests/queries/0_stateless/00689_join_table_function.sql +++ b/tests/queries/0_stateless/00689_join_table_function.sql @@ -1 +1 @@ -SELECT * FROM numbers(3) AS a ANY LEFT JOIN numbers(3) AS b ON a.number = b.number +SELECT * FROM numbers(3) AS a ANY LEFT JOIN numbers(3) AS b ON a.number = b.number ORDER BY a.number; diff --git a/tests/queries/0_stateless/00702_join_with_using.sql b/tests/queries/0_stateless/00702_join_with_using.sql index 24e57e1d555..5faa9a3fad3 100644 --- a/tests/queries/0_stateless/00702_join_with_using.sql +++ b/tests/queries/0_stateless/00702_join_with_using.sql @@ -26,14 +26,14 @@ values ('1', 'John'), ('2', 'Jack'), ('3', 'Daniel'), ('4', 'James'), ('5', 'Ama insert into children (id, childName) values ('1', 'Robert'), ('1', 'Susan'), ('3', 'Sarah'), ('4', 'David'), ('4', 'Joseph'), ('5', 'Robert'); -select * from persons all inner join children using id; -select * from persons all inner join (select * from children) as j using id; -select * from (select * from persons) as s all inner join (select * from children ) as j using id; +select * from persons all inner join children using id order by id; +select * from persons all inner join (select * from children) as j using id order by id; +select * from (select * from persons) as s all inner join (select * from children ) as j using id order by id; -- set joined_subquery_requires_alias = 0; -select * from persons all inner join (select * from children) using id; -select * from (select * from persons) all inner join (select * from children) using id; -select * from (select * from persons) as s all inner join (select * from children) using id; +select * from persons all inner join (select * from children) using id order by id; +select * from (select * from persons) all inner join (select * from children) using id order by id; +select * from (select * from persons) as s all inner join (select * from children) using id order by id; drop table persons; drop table children; diff --git a/tests/queries/0_stateless/01050_engine_join_crash.sql b/tests/queries/0_stateless/01050_engine_join_crash.sql index 285952056b4..3dd4bd2b798 100644 --- a/tests/queries/0_stateless/01050_engine_join_crash.sql +++ b/tests/queries/0_stateless/01050_engine_join_crash.sql @@ -15,7 +15,7 @@ SELECT * FROM testJoinTable; DROP TABLE testJoinTable; SELECT '-'; - + DROP TABLE IF EXISTS master; DROP TABLE IF EXISTS transaction; @@ -38,8 +38,8 @@ DROP TABLE IF EXISTS tbl; CREATE TABLE tbl (eventDate Date, id String) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY eventDate; CREATE TABLE some_join (id String, value String) ENGINE = Join(ANY, LEFT, id) SETTINGS any_join_distinct_right_table_keys = 1; -SELECT * FROM tbl AS t ANY LEFT JOIN some_join USING (id); -SELECT * FROM tbl AS t ANY LEFT JOIN some_join AS d USING (id); +SELECT * FROM tbl AS t ANY LEFT JOIN some_join USING (id) ORDER BY id; +SELECT * FROM tbl AS t ANY LEFT JOIN some_join AS d USING (id) ORDER BY id; -- TODO SELECT t.*, d.* FROM tbl AS t ANY LEFT JOIN some_join AS d USING (id); DROP TABLE some_join; From a7ea163e3b4af5542ef6a95f85def7eab06257af Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Jun 2022 15:10:41 +0000 Subject: [PATCH 515/627] Fix parallel hash join to lowcard keys --- src/Interpreters/ConcurrentHashJoin.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index f6ba9f95bbc..bc5f18a8dd9 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -192,7 +192,8 @@ Blocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, cons for (const auto & key_name : key_columns_names) { const auto & key_col = from_block.getByName(key_name).column; - key_col->updateWeakHash32(hash); + const auto & key_col_no_lc = recursiveRemoveLowCardinality(key_col); + key_col_no_lc->updateWeakHash32(hash); } auto selector = hashToSelector(hash, num_shards); From afc5b1a1cf66f06cea86283f39080d82dd910eb2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Jun 2022 15:17:56 +0000 Subject: [PATCH 516/627] workaroud: ignore order in reference of join related tests --- tests/clickhouse-test | 47 +++++++++++++------ .../0_stateless/00049_any_left_join.sql | 2 + .../0_stateless/00050_any_left_join.sql | 2 + .../0_stateless/00051_any_inner_join.sql | 2 + .../0_stateless/00052_all_left_join.sql | 2 + .../0_stateless/00053_all_inner_join.sql | 2 + .../queries/0_stateless/00054_join_string.sql | 2 + .../0_stateless/00057_join_aliases.sql | 2 + .../00099_join_many_blocks_segfault.sql | 2 + .../0_stateless/00169_join_constant_keys.sql | 2 + .../0_stateless/00689_join_table_function.sql | 2 + .../0_stateless/00702_join_with_using.sql | 2 + .../queries/0_stateless/00722_inner_join.sql | 2 + .../00800_low_cardinality_join.sql | 2 + .../0_stateless/00820_multiple_joins.sql | 2 + ...multiple_joins_subquery_requires_alias.sql | 2 + .../0_stateless/00845_join_on_aliases.sql | 2 + .../0_stateless/00850_global_join_dups.sql | 2 +- .../00855_join_with_array_join.sql | 2 + .../0_stateless/00863_comma_join_in.sql | 2 + .../0_stateless/00864_union_all_supertype.sql | 2 + .../00927_asof_join_correct_bt.sql | 2 + .../00927_asof_join_noninclusive.sql | 2 + .../queries/0_stateless/00927_asof_joins.sql | 2 + .../0_stateless/00976_asof_join_on.sql | 2 + .../0_stateless/01018_ambiguous_column.sql | 2 + .../0_stateless/01050_engine_join_crash.sql | 2 + .../0_stateless/01139_asof_join_types.sql | 2 + .../0_stateless/01428_nullable_asof_join.sql | 2 + .../01576_alias_column_rewrite.sql | 2 + .../01710_projection_with_joins.sql | 3 +- ...to_subqueries_rewriter_columns_matcher.sql | 2 + .../0_stateless/02007_join_use_nulls.sql | 2 + .../02236_explain_pipeline_join.sql | 2 + .../02240_asof_join_biginteger.sql | 2 + 35 files changed, 99 insertions(+), 17 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index cab6daf3a50..2db6dfbc348 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -819,22 +819,39 @@ class TestCase: description, ) - result_is_different = subprocess.call( - ["diff", "-q", self.reference_file, self.stdout_file], stdout=PIPE - ) + def get_diff(refence_file, stdout_file): + result_is_different = subprocess.call( + ["diff", "-q", refence_file, stdout_file], stdout=PIPE + ) - if result_is_different: - diff = Popen( - [ - "diff", - "-U", - str(self.testcase_args.unified), - self.reference_file, - self.stdout_file, - ], - stdout=PIPE, - universal_newlines=True, - ).communicate()[0] + if result_is_different: + return Popen( + [ + "diff", + "-U", + str(self.testcase_args.unified), + refence_file, + stdout_file, + ], + stdout=PIPE, + universal_newlines=True, + ).communicate()[0] + return None + + if "ignore-order" not in self.tags: + diff = get_diff(self.reference_file, self.stdout_file) + else: + sotred_refence_file = self.reference_file + ".sort" + sotred_stdout_file = self.stdout_file + ".sort" + subprocess.call( + ["sort", "-o", sotred_refence_file, self.reference_file], stdout=PIPE + ) + subprocess.call( + ["sort", "-o", sotred_stdout_file, self.stdout_file], stdout=PIPE + ) + diff = get_diff(sotred_refence_file, sotred_stdout_file) + + if diff is not None: description += f"\n{diff}\n" if debug_log: description += "\n" diff --git a/tests/queries/0_stateless/00049_any_left_join.sql b/tests/queries/0_stateless/00049_any_left_join.sql index ecd079a5085..5f014909ee4 100644 --- a/tests/queries/0_stateless/00049_any_left_join.sql +++ b/tests/queries/0_stateless/00049_any_left_join.sql @@ -1 +1,3 @@ +-- Tags: ignore-order + SELECT number, joined FROM system.numbers ANY LEFT JOIN (SELECT number * 2 AS number, number * 10 + 1 AS joined FROM system.numbers LIMIT 10) js2 USING number LIMIT 10 diff --git a/tests/queries/0_stateless/00050_any_left_join.sql b/tests/queries/0_stateless/00050_any_left_join.sql index 5019de9fce7..e380af3af27 100644 --- a/tests/queries/0_stateless/00050_any_left_join.sql +++ b/tests/queries/0_stateless/00050_any_left_join.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 diff --git a/tests/queries/0_stateless/00051_any_inner_join.sql b/tests/queries/0_stateless/00051_any_inner_join.sql index 566b5ad526b..7a364714ef8 100644 --- a/tests/queries/0_stateless/00051_any_inner_join.sql +++ b/tests/queries/0_stateless/00051_any_inner_join.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SET any_join_distinct_right_table_keys = 1; SELECT a.*, b.* FROM diff --git a/tests/queries/0_stateless/00052_all_left_join.sql b/tests/queries/0_stateless/00052_all_left_join.sql index 6d5a1ba073c..edafe0b6c7e 100644 --- a/tests/queries/0_stateless/00052_all_left_join.sql +++ b/tests/queries/0_stateless/00052_all_left_join.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SELECT * FROM ( SELECT number AS k FROM system.numbers LIMIT 10 diff --git a/tests/queries/0_stateless/00053_all_inner_join.sql b/tests/queries/0_stateless/00053_all_inner_join.sql index 6cefb209682..1d844926fbc 100644 --- a/tests/queries/0_stateless/00053_all_inner_join.sql +++ b/tests/queries/0_stateless/00053_all_inner_join.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 diff --git a/tests/queries/0_stateless/00054_join_string.sql b/tests/queries/0_stateless/00054_join_string.sql index 09a03f73614..4b7eda6b30d 100644 --- a/tests/queries/0_stateless/00054_join_string.sql +++ b/tests/queries/0_stateless/00054_join_string.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SELECT * FROM ( SELECT reinterpretAsString(number + reinterpretAsUInt8('A')) AS k FROM system.numbers LIMIT 10 diff --git a/tests/queries/0_stateless/00057_join_aliases.sql b/tests/queries/0_stateless/00057_join_aliases.sql index 481b0621ed7..b1dc71a2f14 100644 --- a/tests/queries/0_stateless/00057_join_aliases.sql +++ b/tests/queries/0_stateless/00057_join_aliases.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SELECT * FROM ( SELECT number, number / 2 AS n, j1, j2 FROM system.numbers diff --git a/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql b/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql index 7219bcecf55..2ef347b6769 100644 --- a/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql +++ b/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SELECT 1 AS DomainID, Domain diff --git a/tests/queries/0_stateless/00169_join_constant_keys.sql b/tests/queries/0_stateless/00169_join_constant_keys.sql index 03c01c074f7..abb37b1c9d1 100644 --- a/tests/queries/0_stateless/00169_join_constant_keys.sql +++ b/tests/queries/0_stateless/00169_join_constant_keys.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SELECT key1, key2, diff --git a/tests/queries/0_stateless/00689_join_table_function.sql b/tests/queries/0_stateless/00689_join_table_function.sql index c8e9100dcc0..1681aea7104 100644 --- a/tests/queries/0_stateless/00689_join_table_function.sql +++ b/tests/queries/0_stateless/00689_join_table_function.sql @@ -1 +1,3 @@ +-- Tags: ignore-order + SELECT * FROM numbers(3) AS a ANY LEFT JOIN numbers(3) AS b ON a.number = b.number ORDER BY a.number; diff --git a/tests/queries/0_stateless/00702_join_with_using.sql b/tests/queries/0_stateless/00702_join_with_using.sql index 5faa9a3fad3..8fed621671e 100644 --- a/tests/queries/0_stateless/00702_join_with_using.sql +++ b/tests/queries/0_stateless/00702_join_with_using.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS using1; DROP TABLE IF EXISTS using2; diff --git a/tests/queries/0_stateless/00722_inner_join.sql b/tests/queries/0_stateless/00722_inner_join.sql index eee164ed1e1..61dde70e3dd 100644 --- a/tests/queries/0_stateless/00722_inner_join.sql +++ b/tests/queries/0_stateless/00722_inner_join.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS one; CREATE TABLE one(dummy UInt8) ENGINE = Memory; diff --git a/tests/queries/0_stateless/00800_low_cardinality_join.sql b/tests/queries/0_stateless/00800_low_cardinality_join.sql index 30fb40a56e9..6d4fbc3c232 100644 --- a/tests/queries/0_stateless/00800_low_cardinality_join.sql +++ b/tests/queries/0_stateless/00800_low_cardinality_join.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + set joined_subquery_requires_alias = 0; select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) using val; diff --git a/tests/queries/0_stateless/00820_multiple_joins.sql b/tests/queries/0_stateless/00820_multiple_joins.sql index df82a199337..396bced1f76 100644 --- a/tests/queries/0_stateless/00820_multiple_joins.sql +++ b/tests/queries/0_stateless/00820_multiple_joins.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; DROP TABLE IF EXISTS table3; diff --git a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql index ad59e02ecad..ceaffabc419 100644 --- a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql +++ b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; DROP TABLE IF EXISTS table3; diff --git a/tests/queries/0_stateless/00845_join_on_aliases.sql b/tests/queries/0_stateless/00845_join_on_aliases.sql index 0800d27caa6..aede8138f42 100644 --- a/tests/queries/0_stateless/00845_join_on_aliases.sql +++ b/tests/queries/0_stateless/00845_join_on_aliases.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; diff --git a/tests/queries/0_stateless/00850_global_join_dups.sql b/tests/queries/0_stateless/00850_global_join_dups.sql index 5a3ab0f1adc..00de0b9fe70 100644 --- a/tests/queries/0_stateless/00850_global_join_dups.sql +++ b/tests/queries/0_stateless/00850_global_join_dups.sql @@ -1,4 +1,4 @@ --- Tags: global +-- Tags: global, ignore-order DROP TABLE IF EXISTS t_local; DROP TABLE IF EXISTS t1_00850; diff --git a/tests/queries/0_stateless/00855_join_with_array_join.sql b/tests/queries/0_stateless/00855_join_with_array_join.sql index 05180573525..3d957be02f3 100644 --- a/tests/queries/0_stateless/00855_join_with_array_join.sql +++ b/tests/queries/0_stateless/00855_join_with_array_join.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SET joined_subquery_requires_alias = 0; SELECT ax, c FROM (SELECT [1,2] ax, 0 c) ARRAY JOIN ax JOIN (SELECT 0 c) USING (c); diff --git a/tests/queries/0_stateless/00863_comma_join_in.sql b/tests/queries/0_stateless/00863_comma_join_in.sql index ebccd351c8a..d6f33caabce 100644 --- a/tests/queries/0_stateless/00863_comma_join_in.sql +++ b/tests/queries/0_stateless/00863_comma_join_in.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + drop table if exists test1_00863; drop table if exists test2_00863; drop table if exists test3_00863; diff --git a/tests/queries/0_stateless/00864_union_all_supertype.sql b/tests/queries/0_stateless/00864_union_all_supertype.sql index de7835e2929..86cd4b49f57 100644 --- a/tests/queries/0_stateless/00864_union_all_supertype.sql +++ b/tests/queries/0_stateless/00864_union_all_supertype.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SET joined_subquery_requires_alias = 0; select toTypeName(key), toTypeName(value) from ( diff --git a/tests/queries/0_stateless/00927_asof_join_correct_bt.sql b/tests/queries/0_stateless/00927_asof_join_correct_bt.sql index 281a81d51c0..a1a884eb981 100644 --- a/tests/queries/0_stateless/00927_asof_join_correct_bt.sql +++ b/tests/queries/0_stateless/00927_asof_join_correct_bt.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS A; DROP TABLE IF EXISTS B; diff --git a/tests/queries/0_stateless/00927_asof_join_noninclusive.sql b/tests/queries/0_stateless/00927_asof_join_noninclusive.sql index 5f15f3b593d..3b9de100134 100644 --- a/tests/queries/0_stateless/00927_asof_join_noninclusive.sql +++ b/tests/queries/0_stateless/00927_asof_join_noninclusive.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS A; DROP TABLE IF EXISTS B; diff --git a/tests/queries/0_stateless/00927_asof_joins.sql b/tests/queries/0_stateless/00927_asof_joins.sql index 2dd6ca07b15..4e4e2a2307e 100644 --- a/tests/queries/0_stateless/00927_asof_joins.sql +++ b/tests/queries/0_stateless/00927_asof_joins.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS md; DROP TABLE IF EXISTS tv; diff --git a/tests/queries/0_stateless/00976_asof_join_on.sql b/tests/queries/0_stateless/00976_asof_join_on.sql index 8060fb86831..af291d73119 100644 --- a/tests/queries/0_stateless/00976_asof_join_on.sql +++ b/tests/queries/0_stateless/00976_asof_join_on.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS A; DROP TABLE IF EXISTS B; diff --git a/tests/queries/0_stateless/01018_ambiguous_column.sql b/tests/queries/0_stateless/01018_ambiguous_column.sql index 54603aab810..70e3db374e8 100644 --- a/tests/queries/0_stateless/01018_ambiguous_column.sql +++ b/tests/queries/0_stateless/01018_ambiguous_column.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + select * from system.one cross join system.one; -- { serverError 352 } select * from system.one cross join system.one r; select * from system.one l cross join system.one; diff --git a/tests/queries/0_stateless/01050_engine_join_crash.sql b/tests/queries/0_stateless/01050_engine_join_crash.sql index 3dd4bd2b798..4b9b5261b49 100644 --- a/tests/queries/0_stateless/01050_engine_join_crash.sql +++ b/tests/queries/0_stateless/01050_engine_join_crash.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS testJoinTable; SET any_join_distinct_right_table_keys = 1; diff --git a/tests/queries/0_stateless/01139_asof_join_types.sql b/tests/queries/0_stateless/01139_asof_join_types.sql index 4cfde5d3210..dc1d687ced7 100644 --- a/tests/queries/0_stateless/01139_asof_join_types.sql +++ b/tests/queries/0_stateless/01139_asof_join_types.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + select * from (select 0 as k, toInt8(1) as v) t1 asof join (select 0 as k, toInt8(0) as v) t2 using(k, v); select * from (select 0 as k, toInt16(1) as v) t1 asof join (select 0 as k, toInt16(0) as v) t2 using(k, v); select * from (select 0 as k, toInt32(1) as v) t1 asof join (select 0 as k, toInt32(0) as v) t2 using(k, v); diff --git a/tests/queries/0_stateless/01428_nullable_asof_join.sql b/tests/queries/0_stateless/01428_nullable_asof_join.sql index e1b00158d68..26021303c72 100644 --- a/tests/queries/0_stateless/01428_nullable_asof_join.sql +++ b/tests/queries/0_stateless/01428_nullable_asof_join.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SET join_use_nulls = 1; select 'left asof using'; diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.sql b/tests/queries/0_stateless/01576_alias_column_rewrite.sql index 450127797cc..e48594f911c 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.sql +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + DROP TABLE IF EXISTS test_table; CREATE TABLE test_table ( diff --git a/tests/queries/0_stateless/01710_projection_with_joins.sql b/tests/queries/0_stateless/01710_projection_with_joins.sql index 472242e3043..0d78b1c4b0b 100644 --- a/tests/queries/0_stateless/01710_projection_with_joins.sql +++ b/tests/queries/0_stateless/01710_projection_with_joins.sql @@ -1,4 +1,5 @@ --- Tags: no-s3-storage +-- Tags: no-s3-storage, ignore-order + drop table if exists t; create table t (s UInt16, l UInt16, projection p (select s, l order by l)) engine MergeTree order by s; diff --git a/tests/queries/0_stateless/01839_join_to_subqueries_rewriter_columns_matcher.sql b/tests/queries/0_stateless/01839_join_to_subqueries_rewriter_columns_matcher.sql index 979debbcbb8..2f7599ded06 100644 --- a/tests/queries/0_stateless/01839_join_to_subqueries_rewriter_columns_matcher.sql +++ b/tests/queries/0_stateless/01839_join_to_subqueries_rewriter_columns_matcher.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SELECT COLUMNS('test') FROM (SELECT 1 AS id, 'a' AS test) a LEFT JOIN (SELECT 1 AS id, 'b' AS test) b ON b.id = a.id diff --git a/tests/queries/0_stateless/02007_join_use_nulls.sql b/tests/queries/0_stateless/02007_join_use_nulls.sql index e08fffce3b7..ccea9778152 100644 --- a/tests/queries/0_stateless/02007_join_use_nulls.sql +++ b/tests/queries/0_stateless/02007_join_use_nulls.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + SET join_use_nulls = 1; SELECT *, d.* FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id; diff --git a/tests/queries/0_stateless/02236_explain_pipeline_join.sql b/tests/queries/0_stateless/02236_explain_pipeline_join.sql index 7a92d6bfde0..bea9ad527e0 100644 --- a/tests/queries/0_stateless/02236_explain_pipeline_join.sql +++ b/tests/queries/0_stateless/02236_explain_pipeline_join.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + EXPLAIN PIPELINE SELECT * FROM ( diff --git a/tests/queries/0_stateless/02240_asof_join_biginteger.sql b/tests/queries/0_stateless/02240_asof_join_biginteger.sql index 6dc5b00f116..5b5cbfb5d96 100644 --- a/tests/queries/0_stateless/02240_asof_join_biginteger.sql +++ b/tests/queries/0_stateless/02240_asof_join_biginteger.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + select * from (select 0 as k, toInt128('18446744073709551617') as v) t1 asof join (select 0 as k, toInt128('18446744073709551616') as v) t2 using(k, v); select * from (select 0 as k, toInt256('340282366920938463463374607431768211457') as v) t1 asof join (select 0 as k, toInt256('340282366920938463463374607431768211456') as v) t2 using(k, v); From 18090aaed72e3d7bec17cf824fadd80cfc28bf3f Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Jun 2022 11:33:16 +0000 Subject: [PATCH 517/627] upd 00163_shard_join_with_empty_table --- .../00163_shard_join_with_empty_table.sql | 34 ++++++++++++++++--- 1 file changed, 29 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql b/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql index 71c570cf762..07825f83992 100644 --- a/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql +++ b/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql @@ -1,9 +1,33 @@ --- Tags: shard +-- Tags: shard, ignore-order SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; -SELECT number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.numbers) ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10; -SELECT dummy + 2 AS number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.one) ANY INNER JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10; -SELECT number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.numbers) GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10; -SELECT dummy + 2 AS number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.one) GLOBAL ANY INNER JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10; +SELECT * FROM ( + SELECT number, number / 2 AS n, j1, j2 + FROM remote('127.0.0.{2,3}', system.numbers) + ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) + USING n LIMIT 10 +) ORDER BY number; + +SELECT * FROM ( + SELECT dummy + 2 AS number, number / 2 AS n, j1, j2 + FROM remote('127.0.0.{2,3}', system.one) + ANY INNER JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) + USING n LIMIT 10 +) ORDER BY number; + + +SELECT * FROM ( + SELECT number, number / 2 AS n, j1, j2 + FROM remote('127.0.0.{2,3}', system.numbers) + GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) + USING n LIMIT 10; +) ORDER BY number; + +SELECT * FROM ( + SELECT dummy + 2 AS number, number / 2 AS n, j1, j2 + FROM remote('127.0.0.{2,3}', system.one) + GLOBAL ANY INNER JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) + USING n LIMIT 10; +) ORDER BY number; From f3e84d8fc6e826410f291775317626240625f7d2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Jun 2022 12:22:15 +0000 Subject: [PATCH 518/627] fix order in 01913_join_push_down_bug 00952_input_function --- src/Interpreters/ConcurrentHashJoin.cpp | 5 +++-- tests/queries/0_stateless/00952_input_function.sh | 8 ++++---- tests/queries/0_stateless/01913_join_push_down_bug.sql | 3 ++- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index bc5f18a8dd9..c23a9778c0d 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -191,8 +192,8 @@ Blocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, cons WeakHash32 hash(num_rows); for (const auto & key_name : key_columns_names) { - const auto & key_col = from_block.getByName(key_name).column; - const auto & key_col_no_lc = recursiveRemoveLowCardinality(key_col); + const auto & key_col = from_block.getByName(key_name).column->convertToFullColumnIfConst(); + const auto & key_col_no_lc = recursiveRemoveLowCardinality(recursiveRemoveSparse(key_col)); key_col_no_lc->updateWeakHash32(hash); } auto selector = hashToSelector(hash, num_shards); diff --git a/tests/queries/0_stateless/00952_input_function.sh b/tests/queries/0_stateless/00952_input_function.sh index 54496ba09e0..91aec7ab833 100755 --- a/tests/queries/0_stateless/00952_input_function.sh +++ b/tests/queries/0_stateless/00952_input_function.sh @@ -10,22 +10,22 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS input_function_table_1" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE input_function_table_1 (a String, b Date, c Int32, d Int16) ENGINE=Memory()" ${CLICKHOUSE_CLIENT} --query="SELECT number, number, number FROM numbers(5) FORMAT CSV" > "${CLICKHOUSE_TMP}"/data_for_input_function.csv cat "${CLICKHOUSE_TMP}"/data_for_input_function.csv | ${CLICKHOUSE_CLIENT} --query="INSERT INTO input_function_table_1 (a, b, c) SELECT a, b, c*c FROM input('a String, b Int32, c Int32') FORMAT CSV" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_1 FORMAT CSV" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_1 FORMAT CSV" | sort ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS input_function_table_2" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE input_function_table_2 (a String, b Date, c Int32, d Int16) ENGINE=Memory()" cat "${CLICKHOUSE_TMP}"/data_for_input_function.csv | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT%20INTO%20input_function_table_2%20%28a%2C%20b%2C%20c%29%20SELECT%20a%2C%20b%2C%20c%2Ac%20FROM%20input%28%27a%20String%2C%20b%20Int32%2C%20c%20Int32%27%29%20FORMAT%20CSV" --data-binary @- -${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_2 FORMAT CSV" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_2 FORMAT CSV" | sort ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS input_function_table_3" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE input_function_table_3 (a String, b Date, c Int32, d Int16) ENGINE=Memory()" cat "${CLICKHOUSE_TMP}"/data_for_input_function.csv | ${CLICKHOUSE_CLIENT} --query="INSERT INTO input_function_table_3 (a, b, c) SELECT * FROM (SELECT s, b, c*c FROM input('s String, b Int32, c Int32') js1 JOIN input_function_table_1 ON s=input_function_table_1.a) FORMAT CSV" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_3 FORMAT CSV" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_3 FORMAT CSV" | sort ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS input_function_table_4" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE input_function_table_4 (a String, b Date, c Int32, d Int16) ENGINE=Memory()" cat "${CLICKHOUSE_TMP}"/data_for_input_function.csv | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT%20INTO%20input_function_table_4%20%28a%2C%20b%2C%20c%29%20SELECT%20%2A%20FROM%20%28SELECT%20s%2C%20b%2C%20c%2Ac%20FROM%20input%28%27s%20String%2C%20b%20Int32%2C%20c%20Int32%27%29%20js1%20JOIN%20input_function_table_1%20ON%20s%3Dinput_function_table_1.a%29%20FORMAT%20CSV" --data-binary @- -${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_4 FORMAT CSV" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_4 FORMAT CSV" | sort ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS input_function_table_5" diff --git a/tests/queries/0_stateless/01913_join_push_down_bug.sql b/tests/queries/0_stateless/01913_join_push_down_bug.sql index 00fd8cb5ddb..b945530b32a 100644 --- a/tests/queries/0_stateless/01913_join_push_down_bug.sql +++ b/tests/queries/0_stateless/01913_join_push_down_bug.sql @@ -17,6 +17,7 @@ set query_plan_filter_push_down = true; SELECT id, flag FROM test t1 INNER JOIN (SELECT DISTINCT id FROM test) AS t2 ON t1.id = t2.id -WHERE flag = 0 and t = 1 AND id NOT IN (SELECT 1 WHERE 0); +WHERE flag = 0 and t = 1 AND id NOT IN (SELECT 1 WHERE 0) +ORDER BY id; DROP TABLE IF EXISTS test; From 578494660c192c57972e79229ce13a5a84ff8226 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Jun 2022 14:18:54 +0000 Subject: [PATCH 519/627] upd 00163_shard_join_with_empty_table.sql --- .../0_stateless/00163_shard_join_with_empty_table.sql | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql b/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql index 07825f83992..777b2443416 100644 --- a/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql +++ b/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql @@ -1,4 +1,4 @@ --- Tags: shard, ignore-order +-- Tags: shard SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; @@ -17,17 +17,16 @@ SELECT * FROM ( USING n LIMIT 10 ) ORDER BY number; - SELECT * FROM ( SELECT number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.numbers) GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) - USING n LIMIT 10; + USING n LIMIT 10 ) ORDER BY number; SELECT * FROM ( SELECT dummy + 2 AS number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.one) GLOBAL ANY INNER JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) - USING n LIMIT 10; + USING n LIMIT 10 ) ORDER BY number; From 2e38bb5f23d8532661265296b08a2b79b2e8ec59 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Jun 2022 14:29:52 +0000 Subject: [PATCH 520/627] adjust more tests for parallel_join --- .../0_stateless/01655_plan_optimizations.sh | 2 +- .../0_stateless/01910_view_dictionary.sql | 6 +++-- ...ocal_join_right_distribute_table.reference | 13 ++++++----- ...rite_local_join_right_distribute_table.sql | 10 ++++---- .../02236_explain_pipeline_join.reference | 23 +++++++++++-------- .../02236_explain_pipeline_join.sql | 2 -- 6 files changed, 31 insertions(+), 25 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 3a3ce95460c..aaecdc390cb 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -172,7 +172,7 @@ $CLICKHOUSE_CLIENT -q " $CLICKHOUSE_CLIENT -q " select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) - ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" + ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | sort echo "> one condition of filter is pushed down before INNER JOIN" $CLICKHOUSE_CLIENT -q " diff --git a/tests/queries/0_stateless/01910_view_dictionary.sql b/tests/queries/0_stateless/01910_view_dictionary.sql index 7b807c7fbba..1f9928735b4 100644 --- a/tests/queries/0_stateless/01910_view_dictionary.sql +++ b/tests/queries/0_stateless/01910_view_dictionary.sql @@ -21,9 +21,11 @@ CREATE TABLE dictionary_source_ru INSERT INTO dictionary_source_ru VALUES (1, 'Один'), (2,'Два'), (3, 'Три'); -CREATE VIEW dictionary_source_view AS SELECT id, dictionary_source_en.value as value_en, dictionary_source_ru.value as value_ru FROM dictionary_source_en LEFT JOIN dictionary_source_ru USING (id); +CREATE VIEW dictionary_source_view AS + SELECT id, dictionary_source_en.value as value_en, dictionary_source_ru.value as value_ru + FROM dictionary_source_en LEFT JOIN dictionary_source_ru USING (id); -select * from dictionary_source_view; +select * from dictionary_source_view ORDER BY id; CREATE DICTIONARY flat_dictionary ( diff --git a/tests/queries/0_stateless/02115_rewrite_local_join_right_distribute_table.reference b/tests/queries/0_stateless/02115_rewrite_local_join_right_distribute_table.reference index b9119b9d087..2b974451e74 100644 --- a/tests/queries/0_stateless/02115_rewrite_local_join_right_distribute_table.reference +++ b/tests/queries/0_stateless/02115_rewrite_local_join_right_distribute_table.reference @@ -8,11 +8,12 @@ SELECT a FROM t1_all AS t1 ALL INNER JOIN test_02115.t2_local AS t2 ON a = t2.a 1 -2 -3 1 2 +2 3 +3 +- 1 2 3 @@ -24,13 +25,13 @@ FROM t1_all AS t1 GLOBAL ALL INNER JOIN t2_all AS t2 ON a = t2.a 1 1 -2 -2 -3 -3 1 1 2 2 +2 +2 +3 +3 3 3 diff --git a/tests/queries/0_stateless/02115_rewrite_local_join_right_distribute_table.sql b/tests/queries/0_stateless/02115_rewrite_local_join_right_distribute_table.sql index 2eebb14a46f..d5ab82ba064 100644 --- a/tests/queries/0_stateless/02115_rewrite_local_join_right_distribute_table.sql +++ b/tests/queries/0_stateless/02115_rewrite_local_join_right_distribute_table.sql @@ -13,18 +13,20 @@ create table t2_local as t1_local; create table t1_all as t1_local engine Distributed(test_cluster_two_shards_localhost, test_02115, t1_local, rand()); create table t2_all as t2_local engine Distributed(test_cluster_two_shards_localhost, test_02115, t2_local, rand()); -insert into t1_local values(1), (2), (3); -insert into t2_local values(1), (2), (3); +insert into t1_local values (1), (2), (3); +insert into t2_local values (1), (2), (3); set distributed_product_mode = 'local'; select * from t1_all t1 where t1.a in (select t2.a from t2_all t2); explain syntax select t1.* from t1_all t1 join t2_all t2 on t1.a = t2.a; -select t1.* from t1_all t1 join t2_all t2 on t1.a = t2.a; +select t1.* from t1_all t1 join t2_all t2 on t1.a = t2.a ORDER BY t1.a; + +SELECT '-'; set distributed_product_mode = 'global'; select * from t1_all t1 where t1.a in (select t2.a from t2_all t2); explain syntax select t1.* from t1_all t1 join t2_all t2 on t1.a = t2.a; -select t1.* from t1_all t1 join t2_all t2 on t1.a = t2.a; +select t1.* from t1_all t1 join t2_all t2 on t1.a = t2.a ORDER BY t1.a; DROP TABLE t1_local; DROP TABLE t2_local; diff --git a/tests/queries/0_stateless/02236_explain_pipeline_join.reference b/tests/queries/0_stateless/02236_explain_pipeline_join.reference index 5d7a7bfc488..0f4b6d3044b 100644 --- a/tests/queries/0_stateless/02236_explain_pipeline_join.reference +++ b/tests/queries/0_stateless/02236_explain_pipeline_join.reference @@ -1,17 +1,20 @@ (Expression) -ExpressionTransform +ExpressionTransform × 16 (Join) - JoiningTransform 2 → 1 - (Expression) - ExpressionTransform - (Limit) - Limit - (ReadFromStorage) - Numbers 0 → 1 - (Expression) - FillingRightJoinSide + JoiningTransform × 16 2 → 1 + Resize 1 → 16 + (Expression) ExpressionTransform (Limit) Limit (ReadFromStorage) Numbers 0 → 1 + (Expression) + Resize × 2 16 → 1 + FillingRightJoinSide × 16 + Resize 1 → 16 + ExpressionTransform + (Limit) + Limit + (ReadFromStorage) + Numbers 0 → 1 diff --git a/tests/queries/0_stateless/02236_explain_pipeline_join.sql b/tests/queries/0_stateless/02236_explain_pipeline_join.sql index bea9ad527e0..7a92d6bfde0 100644 --- a/tests/queries/0_stateless/02236_explain_pipeline_join.sql +++ b/tests/queries/0_stateless/02236_explain_pipeline_join.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - EXPLAIN PIPELINE SELECT * FROM ( From f51b25b2621ed263a93e6fc0164eaebfd3e3fc58 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Jun 2022 17:06:34 +0000 Subject: [PATCH 521/627] clickhouse test ignore order via special format --- .../Impl/TabSeparatedRowOutputFormat.cpp | 45 ++++++++++++++ .../Impl/TabSeparatedRowOutputFormat.h | 18 ++++++ tests/clickhouse-test | 58 +++++++------------ 3 files changed, 83 insertions(+), 38 deletions(-) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 03a3ea99b28..fa176856ce4 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -3,6 +3,10 @@ #include #include +#include +#include +#include + namespace DB { @@ -77,6 +81,37 @@ void TabSeparatedRowOutputFormat::writeBeforeExtremes() writeChar('\n', out); } +TabSeparatedSortedRowOutputFormat::TabSeparatedSortedRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + const RowOutputFormatParams & params_, + const FormatSettings & format_settings_) + : TabSeparatedRowOutputFormat(tmp_buf, header_, false, false, false, params_, format_settings_) + , real_out(out_) +{} + +static String sortLines(const String & str) +{ + Strings lines; + boost::split(lines, str, boost::is_any_of("\n")); + /// sort alphabetically, empty lines at the end + std::sort(lines.begin(), lines.end(), [](const String & a, const String & b) { return a.empty() ? !b.empty() : a < b; }); + return boost::join(lines, "\n"); +} + +void TabSeparatedSortedRowOutputFormat::writeSuffix() +{ + tmp_buf.finalize(); + String sorted_lines = sortLines(tmp_buf.str()); + tmp_buf.restart(); + writeString(sorted_lines, real_out); +} + +void TabSeparatedSortedRowOutputFormat::finalizeImpl() +{ + tmp_buf.finalize(); + writeString(tmp_buf.str(), real_out); +} void registerOutputFormatTabSeparated(FormatFactory & factory) { @@ -101,6 +136,16 @@ void registerOutputFormatTabSeparated(FormatFactory & factory) if (is_raw) registerWithNamesAndTypes("LineAsString", register_func); } + + /// Used in tests + factory.registerOutputFormat("TabSeparatedSorted", []( + WriteBuffer & buf, + const Block & sample, + const RowOutputFormatParams & params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, params, settings); + }); } } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index 8aac94812e2..a463fde003c 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -48,4 +49,21 @@ protected: const FormatSettings format_settings; }; +class TabSeparatedSortedRowOutputFormat : public TabSeparatedRowOutputFormat +{ +public: + TabSeparatedSortedRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + const RowOutputFormatParams & params_, + const FormatSettings & format_settings_); + + void writeSuffix() override; + void finalizeImpl() override; + +private: + WriteBufferFromOwnString tmp_buf; + WriteBuffer & real_out; +}; + } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2db6dfbc348..67e91282f2e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -819,39 +819,22 @@ class TestCase: description, ) - def get_diff(refence_file, stdout_file): - result_is_different = subprocess.call( - ["diff", "-q", refence_file, stdout_file], stdout=PIPE - ) + result_is_different = subprocess.call( + ["diff", "-q", self.reference_file, self.stdout_file], stdout=PIPE + ) - if result_is_different: - return Popen( - [ - "diff", - "-U", - str(self.testcase_args.unified), - refence_file, - stdout_file, - ], - stdout=PIPE, - universal_newlines=True, - ).communicate()[0] - return None - - if "ignore-order" not in self.tags: - diff = get_diff(self.reference_file, self.stdout_file) - else: - sotred_refence_file = self.reference_file + ".sort" - sotred_stdout_file = self.stdout_file + ".sort" - subprocess.call( - ["sort", "-o", sotred_refence_file, self.reference_file], stdout=PIPE - ) - subprocess.call( - ["sort", "-o", sotred_stdout_file, self.stdout_file], stdout=PIPE - ) - diff = get_diff(sotred_refence_file, sotred_stdout_file) - - if diff is not None: + if result_is_different: + diff = Popen( + [ + "diff", + "-U", + str(self.testcase_args.unified), + self.reference_file, + self.stdout_file, + ], + stdout=PIPE, + universal_newlines=True, + ).communicate()[0] description += f"\n{diff}\n" if debug_log: description += "\n" @@ -929,6 +912,9 @@ class TestCase: # This is for .sh tests os.environ["CLICKHOUSE_LOG_COMMENT"] = args.testcase_basename + if "ignore-order" in self.tags: + client = client + " --format=TabSeparatedSorted" + params = { "client": client + " --database=" + database, "logs_level": server_logs_level, @@ -939,12 +925,8 @@ class TestCase: "secure": "--secure" if args.secure else "", } - # >> append to stderr (but not stdout since it is not used there), - # because there are also output of per test database creation - if not args.database: - pattern = "{test} > {stdout} 2> {stderr}" - else: - pattern = "{test} > {stdout} 2> {stderr}" + pattern = "{test} > {stdout} 2> {stderr}" + if self.ext == ".sql": pattern = ( From f64a8334493692a2f70c621f6a83979d2789c852 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Jun 2022 17:11:39 +0000 Subject: [PATCH 522/627] remove ignore-order from some tests --- tests/queries/0_stateless/00053_all_inner_join.sql | 2 -- tests/queries/0_stateless/00054_join_string.sql | 2 -- tests/queries/0_stateless/00057_join_aliases.sql | 2 -- tests/queries/0_stateless/00169_join_constant_keys.sql | 2 -- tests/queries/0_stateless/00702_join_with_using.sql | 2 -- tests/queries/0_stateless/01050_engine_join_crash.sql | 2 -- 6 files changed, 12 deletions(-) diff --git a/tests/queries/0_stateless/00053_all_inner_join.sql b/tests/queries/0_stateless/00053_all_inner_join.sql index 1d844926fbc..6cefb209682 100644 --- a/tests/queries/0_stateless/00053_all_inner_join.sql +++ b/tests/queries/0_stateless/00053_all_inner_join.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 diff --git a/tests/queries/0_stateless/00054_join_string.sql b/tests/queries/0_stateless/00054_join_string.sql index 4b7eda6b30d..09a03f73614 100644 --- a/tests/queries/0_stateless/00054_join_string.sql +++ b/tests/queries/0_stateless/00054_join_string.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SELECT * FROM ( SELECT reinterpretAsString(number + reinterpretAsUInt8('A')) AS k FROM system.numbers LIMIT 10 diff --git a/tests/queries/0_stateless/00057_join_aliases.sql b/tests/queries/0_stateless/00057_join_aliases.sql index b1dc71a2f14..481b0621ed7 100644 --- a/tests/queries/0_stateless/00057_join_aliases.sql +++ b/tests/queries/0_stateless/00057_join_aliases.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SELECT * FROM ( SELECT number, number / 2 AS n, j1, j2 FROM system.numbers diff --git a/tests/queries/0_stateless/00169_join_constant_keys.sql b/tests/queries/0_stateless/00169_join_constant_keys.sql index abb37b1c9d1..03c01c074f7 100644 --- a/tests/queries/0_stateless/00169_join_constant_keys.sql +++ b/tests/queries/0_stateless/00169_join_constant_keys.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SELECT key1, key2, diff --git a/tests/queries/0_stateless/00702_join_with_using.sql b/tests/queries/0_stateless/00702_join_with_using.sql index 8fed621671e..5faa9a3fad3 100644 --- a/tests/queries/0_stateless/00702_join_with_using.sql +++ b/tests/queries/0_stateless/00702_join_with_using.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS using1; DROP TABLE IF EXISTS using2; diff --git a/tests/queries/0_stateless/01050_engine_join_crash.sql b/tests/queries/0_stateless/01050_engine_join_crash.sql index 4b9b5261b49..3dd4bd2b798 100644 --- a/tests/queries/0_stateless/01050_engine_join_crash.sql +++ b/tests/queries/0_stateless/01050_engine_join_crash.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS testJoinTable; SET any_join_distinct_right_table_keys = 1; From 46df417c2e6031858cde07ee18d042b1aeddc1d2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 14 Jun 2022 11:12:01 +0000 Subject: [PATCH 523/627] Fix empty line sorting in TabSeparatedSorted --- src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index fa176856ce4..09839379a65 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -95,7 +95,7 @@ static String sortLines(const String & str) Strings lines; boost::split(lines, str, boost::is_any_of("\n")); /// sort alphabetically, empty lines at the end - std::sort(lines.begin(), lines.end(), [](const String & a, const String & b) { return a.empty() ? !b.empty() : a < b; }); + std::sort(lines.begin(), lines.end(), [](const String & a, const String & b) { return (a.empty() || b.empty()) ? b.empty() : a < b; }); return boost::join(lines, "\n"); } From 2ec373c6a7c5cb7fee8f77d9d0dc2491e85849cd Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 14 Jun 2022 12:13:39 +0000 Subject: [PATCH 524/627] Number of shards in parallel hash join is always power of two --- src/Interpreters/ConcurrentHashJoin.cpp | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index c23a9778c0d..4213aa4090b 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -30,15 +30,21 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +static UInt32 toPowerOfTwo(UInt32 x) +{ + x = x - 1; + for (UInt32 i = 1; i < sizeof(UInt32) * 8; i <<= 1) + x = x | x >> i; + return x + 1; +} + ConcurrentHashJoin::ConcurrentHashJoin(ContextPtr context_, std::shared_ptr table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_) : context(context_) , table_join(table_join_) , slots(slots_) { - if (slots < 1 || 255 < slots) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Number of slots should be [1, 255], got {}", slots); - } + slots = std::min(std::max(1, slots), 255); + slots = toPowerOfTwo(slots); for (size_t i = 0; i < slots; ++i) { @@ -174,12 +180,13 @@ std::shared_ptr ConcurrentHashJoin::getNonJoinedBlocks( static IColumn::Selector hashToSelector(const WeakHash32 & hash, size_t num_shards) { + assert(num_shards > 0 && (num_shards & (num_shards - 1)) == 0); const auto & data = hash.getData(); size_t num_rows = data.size(); IColumn::Selector selector(num_rows); for (size_t i = 0; i < num_rows; ++i) - selector[i] = data[i] % num_shards; + selector[i] = data[i] & (num_shards - 1); return selector; } From 218288ee6c3ad451a58744fa1d12806d5f1d04fd Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 14 Jun 2022 14:13:01 +0000 Subject: [PATCH 525/627] upd ConcurrentHashJoin --- src/Interpreters/ConcurrentHashJoin.cpp | 25 +++++++++++++++---------- src/Interpreters/ConcurrentHashJoin.h | 7 ++----- src/Interpreters/HashJoin.cpp | 2 +- 3 files changed, 18 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 4213aa4090b..2c1f68c5bf7 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -27,7 +27,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int SET_SIZE_LIMIT_EXCEEDED; - extern const int BAD_ARGUMENTS; } static UInt32 toPowerOfTwo(UInt32 x) @@ -101,6 +100,7 @@ bool ConcurrentHashJoin::addJoinedBlock(const Block & right_block, bool check_li void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr & /*not_processed*/) { Blocks dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, block); + block = {}; for (size_t i = 0; i < dispatched_blocks.size(); ++i) { std::shared_ptr none_extra_block; @@ -178,7 +178,7 @@ std::shared_ptr ConcurrentHashJoin::getNonJoinedBlocks( throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid join type. join kind: {}, strictness: {}", table_join->kind(), table_join->strictness()); } -static IColumn::Selector hashToSelector(const WeakHash32 & hash, size_t num_shards) +static ALWAYS_INLINE IColumn::Selector hashToSelector(const WeakHash32 & hash, size_t num_shards) { assert(num_shards > 0 && (num_shards & (num_shards - 1)) == 0); const auto & data = hash.getData(); @@ -190,11 +190,10 @@ static IColumn::Selector hashToSelector(const WeakHash32 & hash, size_t num_shar return selector; } -Blocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block) +IColumn::Selector ConcurrentHashJoin::selectDispatchBlock(const Strings & key_columns_names, const Block & from_block) { - size_t num_shards = hash_joins.size(); size_t num_rows = from_block.rows(); - size_t num_cols = from_block.columns(); + size_t num_shards = hash_joins.size(); WeakHash32 hash(num_rows); for (const auto & key_name : key_columns_names) @@ -203,13 +202,19 @@ Blocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, cons const auto & key_col_no_lc = recursiveRemoveLowCardinality(recursiveRemoveSparse(key_col)); key_col_no_lc->updateWeakHash32(hash); } - auto selector = hashToSelector(hash, num_shards); + return hashToSelector(hash, num_shards); +} - Blocks result; +Blocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block) +{ + size_t num_shards = hash_joins.size(); + size_t num_cols = from_block.columns(); + + IColumn::Selector selector = selectDispatchBlock(key_columns_names, from_block); + + Blocks result(num_shards); for (size_t i = 0; i < num_shards; ++i) - { - result.emplace_back(from_block.cloneEmpty()); - } + result[i] = from_block.cloneEmpty(); for (size_t i = 0; i < num_cols; ++i) { diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index fb226c39a0c..705e6ba81b7 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -62,13 +62,10 @@ private: size_t slots; std::vector> hash_joins; - std::mutex finished_add_joined_blocks_tasks_mutex; - std::condition_variable finished_add_joined_blocks_tasks_cond; - std::atomic finished_add_joined_blocks_tasks = 0; - - mutable std::mutex totals_mutex; + std::mutex totals_mutex; Block totals; + IColumn::Selector selectDispatchBlock(const Strings & key_columns_names, const Block & from_block); Blocks dispatchBlock(const Strings & key_columns_names, const Block & from_block); }; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index a5bbcf9a373..b54c77b385f 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -411,7 +411,7 @@ HashJoin::Type HashJoin::chooseMethod(ASTTableJoin::Kind kind, const ColumnRawPt return Type::hashed; } -template +template static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes) { if constexpr (is_asof_join) From 816d8477f74c2d3543c78c08c3b97330e1eb8040 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 14 Jun 2022 17:33:06 +0000 Subject: [PATCH 526/627] Fix more tests for parallel hash join --- .../0_stateless/00820_multiple_joins.sql | 43 +++++++++++-------- ...multiple_joins_subquery_requires_alias.sql | 35 ++++++++------- .../0_stateless/00864_union_all_supertype.sql | 2 - .../0_stateless/01428_nullable_asof_join.sql | 2 - .../01576_alias_column_rewrite.sql | 4 -- 5 files changed, 45 insertions(+), 41 deletions(-) diff --git a/tests/queries/0_stateless/00820_multiple_joins.sql b/tests/queries/0_stateless/00820_multiple_joins.sql index 396bced1f76..af13e6f1d8b 100644 --- a/tests/queries/0_stateless/00820_multiple_joins.sql +++ b/tests/queries/0_stateless/00820_multiple_joins.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; DROP TABLE IF EXISTS table3; @@ -16,69 +14,80 @@ INSERT INTO table2 SELECT number * 2, number * 20 FROM numbers(11); INSERT INTO table3 SELECT number * 30, number * 300 FROM numbers(10); INSERT INTO table5 SELECT number * 5, number * 50, number * 500 FROM numbers(10); -select t1.a, t2.b, t3.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b; -select t1.a, t2.b, t5.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on t1.a = t5.a AND t2.b = t5.b; +select t1.a, t2.b, t3.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b ORDER BY t1.a; +select t1.a, t2.b, t5.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on t1.a = t5.a AND t2.b = t5.b ORDER BY t1.a; select t1.a, t2.a, t2.b, t3.b, t3.c, t5.a, t5.b, t5.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b join table5 as t5 on t3.c = t5.c +ORDER BY t1.a FORMAT PrettyCompactNoEscapes; select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b from table1 as t1 join table2 as t2 on t1_a = t2_a -join table3 as t3 on t2_b = t3_b; +join table3 as t3 on t2_b = t3_b +ORDER BY t1.a +; select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b from table1 as t1 join table2 as t2 on t1.a = t2.a -join table3 as t3 on t2.b = t3.b; +join table3 as t3 on t2.b = t3.b +ORDER BY t1.a +; select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b from table1 as t1 join table2 as t2 on table1.a = table2.a -join table3 as t3 on table2.b = table3.b; +join table3 as t3 on table2.b = table3.b +ORDER BY t1.a +; select t1.a, t2.a, t2.b, t3.b from table1 as t1 join table2 as t2 on table1.a = table2.a -join table3 as t3 on table2.b = table3.b; +join table3 as t3 on table2.b = table3.b +ORDER BY t1.a +; select t1.a, t2.a, t2.b, t3.b from table1 as t1 join table2 as t2 on t1.a = t2.a -join table3 as t3 on t2.b = t3.b; +join table3 as t3 on t2.b = t3.b +ORDER BY t1.a +; select table1.a, table2.a, table2.b, table3.b from table1 as t1 join table2 as t2 on table1.a = table2.a -join table3 as t3 on table2.b = table3.b; +join table3 as t3 on table2.b = table3.b +ORDER BY t1.a +; select t1.*, t2.*, t3.* from table1 as t1 join table2 as t2 on table1.a = table2.a join table3 as t3 on table2.b = table3.b +ORDER BY t1.a FORMAT PrettyCompactNoEscapes; select * from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b +ORDER BY t1.a FORMAT PrettyCompactNoEscapes; select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b, (t1.a + table2.b) as t1_t2_x, (table1.a + table3.b) as t1_t3_x, (t2.b + t3.b) as t2_t3_x from table1 as t1 join table2 as t2 on t1_a = t2_a -join table3 as t3 on t2_b = t3_b; - ---select (t1.a + table2.b) as t1_t2_x, (table1.a + table3.b) as t1_t3_x, (t2.b + t3.b) as t2_t3_x ---from table1 as t1 ---join table2 as t2 on t1_t2_x = t2.a ---join table3 as t3 on t1_t3_x = t2_t3_x; - +join table3 as t3 on t2_b = t3_b +ORDER BY t1.a +; CREATE TABLE table_set ( x UInt32 ) ENGINE = Set; INSERT INTO table_set VALUES (0), (1), (2); diff --git a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql index ceaffabc419..612e75e6d4d 100644 --- a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql +++ b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; DROP TABLE IF EXISTS table3; @@ -17,68 +15,73 @@ INSERT INTO table5 SELECT number * 5, number * 50, number * 500 FROM numbers(10) SET joined_subquery_requires_alias = 1; -select t1.a, t2.b, t3.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b; -select t1.a, t2.b, t5.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on t1.a = t5.a AND t2.b = t5.b; +select t1.a, t2.b, t3.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b ORDER BY t1.a; +select t1.a, t2.b, t5.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on t1.a = t5.a AND t2.b = t5.b ORDER BY t1.a; select t1.a, t2.a, t2.b, t3.b, t3.c, t5.a, t5.b, t5.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b join table5 as t5 on t3.c = t5.c +ORDER BY t1.a FORMAT PrettyCompactNoEscapes; select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b from table1 as t1 join table2 as t2 on t1_a = t2_a -join table3 as t3 on t2_b = t3_b; +join table3 as t3 on t2_b = t3_b +ORDER BY t1.a; select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b from table1 as t1 join table2 as t2 on t1.a = t2.a -join table3 as t3 on t2.b = t3.b; +join table3 as t3 on t2.b = t3.b +ORDER BY t1.a; select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b from table1 as t1 join table2 as t2 on table1.a = table2.a -join table3 as t3 on table2.b = table3.b; +join table3 as t3 on table2.b = table3.b +ORDER BY t1.a; select t1.a, t2.a, t2.b, t3.b from table1 as t1 join table2 as t2 on table1.a = table2.a -join table3 as t3 on table2.b = table3.b; +join table3 as t3 on table2.b = table3.b +ORDER BY t1.a; select t1.a, t2.a, t2.b, t3.b from table1 as t1 join table2 as t2 on t1.a = t2.a -join table3 as t3 on t2.b = t3.b; +join table3 as t3 on t2.b = t3.b +ORDER BY t1.a; select table1.a, table2.a, table2.b, table3.b from table1 as t1 join table2 as t2 on table1.a = table2.a -join table3 as t3 on table2.b = table3.b; +join table3 as t3 on table2.b = table3.b +ORDER BY t1.a; select t1.*, t2.*, t3.* from table1 as t1 join table2 as t2 on table1.a = table2.a join table3 as t3 on table2.b = table3.b +ORDER BY t1.a FORMAT PrettyCompactNoEscapes; select * from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b +ORDER BY t1.a FORMAT PrettyCompactNoEscapes; select t1.a as t1_a, t2.a as t2_a, t2.b as t2_b, t3.b as t3_b, (t1.a + table2.b) as t1_t2_x, (table1.a + table3.b) as t1_t3_x, (t2.b + t3.b) as t2_t3_x from table1 as t1 join table2 as t2 on t1_a = t2_a -join table3 as t3 on t2_b = t3_b; - ---select (t1.a + table2.b) as t1_t2_x, (table1.a + table3.b) as t1_t3_x, (t2.b + t3.b) as t2_t3_x ---from table1 as t1 ---join table2 as t2 on t1_t2_x = t2.a ---join table3 as t3 on t1_t3_x = t2_t3_x; +join table3 as t3 on t2_b = t3_b +ORDER BY t1.a; DROP TABLE table1; DROP TABLE table2; diff --git a/tests/queries/0_stateless/00864_union_all_supertype.sql b/tests/queries/0_stateless/00864_union_all_supertype.sql index 86cd4b49f57..de7835e2929 100644 --- a/tests/queries/0_stateless/00864_union_all_supertype.sql +++ b/tests/queries/0_stateless/00864_union_all_supertype.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SET joined_subquery_requires_alias = 0; select toTypeName(key), toTypeName(value) from ( diff --git a/tests/queries/0_stateless/01428_nullable_asof_join.sql b/tests/queries/0_stateless/01428_nullable_asof_join.sql index 26021303c72..e1b00158d68 100644 --- a/tests/queries/0_stateless/01428_nullable_asof_join.sql +++ b/tests/queries/0_stateless/01428_nullable_asof_join.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SET join_use_nulls = 1; select 'left asof using'; diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.sql b/tests/queries/0_stateless/01576_alias_column_rewrite.sql index e48594f911c..8424eb11f9b 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.sql +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS test_table; CREATE TABLE test_table ( @@ -27,8 +25,6 @@ SELECT t = '2020-01-03' FROM (SELECT day AS t FROM test_table WHERE t = '2020-01 SELECT COUNT() = 10 FROM test_table WHERE day = '2020-01-01' UNION ALL SELECT 1 FROM numbers(1) SETTINGS max_rows_to_read = 11; SELECT COUNT() = 0 FROM (SELECT toDate('2019-01-01') AS day, day AS t FROM test_table PREWHERE t = '2020-01-03' WHERE t = '2020-01-03' GROUP BY t ); - - SELECT 'test-join'; SELECT day = '2020-01-03' FROM From 86ac6386f7cc4cff153f24e5e0a6b1929cb52f10 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 15 Jun 2022 11:25:36 +0000 Subject: [PATCH 527/627] adjust more tests for parallel hash join --- tests/queries/0_stateless/00725_join_on_bug_2.sql | 2 ++ tests/queries/0_stateless/00725_join_on_bug_3.sql | 2 +- tests/queries/0_stateless/00818_join_bug_4271.sql | 10 +++++----- .../queries/0_stateless/00826_cross_to_inner_join.sql | 6 +++--- 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00725_join_on_bug_2.sql b/tests/queries/0_stateless/00725_join_on_bug_2.sql index 14fedfa14e7..02fc3952cb0 100644 --- a/tests/queries/0_stateless/00725_join_on_bug_2.sql +++ b/tests/queries/0_stateless/00725_join_on_bug_2.sql @@ -1,3 +1,5 @@ +-- Tags: ignore-order + set joined_subquery_requires_alias = 0; drop table if exists t_00725_2; diff --git a/tests/queries/0_stateless/00725_join_on_bug_3.sql b/tests/queries/0_stateless/00725_join_on_bug_3.sql index 08b39d899cf..678516b4fc5 100644 --- a/tests/queries/0_stateless/00725_join_on_bug_3.sql +++ b/tests/queries/0_stateless/00725_join_on_bug_3.sql @@ -7,7 +7,7 @@ insert into t_00725_3 values(2,2); create table z_00725_3(c Int64, d Int64, e Int64) engine = TinyLog; insert into z_00725_3 values(1,1,1); -select * from t_00725_3 all left join z_00725_3 on (z_00725_3.c = t_00725_3.a and z_00725_3.d = t_00725_3.b); +select * from t_00725_3 all left join z_00725_3 on (z_00725_3.c = t_00725_3.a and z_00725_3.d = t_00725_3.b) ORDER BY t_00725_3.a; drop table if exists t_00725_3; drop table if exists z_00725_3; diff --git a/tests/queries/0_stateless/00818_join_bug_4271.sql b/tests/queries/0_stateless/00818_join_bug_4271.sql index ce11088fd95..7bf3b4bffd6 100644 --- a/tests/queries/0_stateless/00818_join_bug_4271.sql +++ b/tests/queries/0_stateless/00818_join_bug_4271.sql @@ -7,11 +7,11 @@ create table s_00818(a Nullable(Int64), b Nullable(Int64), c Nullable(String)) e insert into t_00818 values(1,1,'a'), (2,2,'b'); insert into s_00818 values(1,1,'a'); -select * from t_00818 left join s_00818 on t_00818.a = s_00818.a; -select * from t_00818 left join s_00818 on t_00818.a = s_00818.a and t_00818.a = s_00818.b; -select * from t_00818 left join s_00818 on t_00818.a = s_00818.a where s_00818.a = 1; -select * from t_00818 left join s_00818 on t_00818.a = s_00818.a and t_00818.a = s_00818.a; -select * from t_00818 left join s_00818 on t_00818.a = s_00818.a and t_00818.b = s_00818.a; +select * from t_00818 left join s_00818 on t_00818.a = s_00818.a ORDER BY t_00818.a; +select * from t_00818 left join s_00818 on t_00818.a = s_00818.a and t_00818.a = s_00818.b ORDER BY t_00818.a; +select * from t_00818 left join s_00818 on t_00818.a = s_00818.a where s_00818.a = 1 ORDER BY t_00818.a; +select * from t_00818 left join s_00818 on t_00818.a = s_00818.a and t_00818.a = s_00818.a ORDER BY t_00818.a; +select * from t_00818 left join s_00818 on t_00818.a = s_00818.a and t_00818.b = s_00818.a ORDER BY t_00818.a; drop table t_00818; drop table s_00818; diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/tests/queries/0_stateless/00826_cross_to_inner_join.sql index ce0c8ea2bfc..e9f9e13e2d3 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -19,9 +19,9 @@ SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a; SELECT '--- cross nullable ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b; SELECT '--- cross nullable vs not nullable ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b ORDER BY t1_00826.a; SELECT '--- cross self ---'; -SELECT * FROM t1_00826 x cross join t1_00826 y where x.a = y.a and x.b = y.b; +SELECT * FROM t1_00826 x cross join t1_00826 y where x.a = y.a and x.b = y.b ORDER BY x.a; SELECT '--- cross one table expr ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t1_00826.b order by (t1_00826.a, t2_00826.a, t2_00826.b); SELECT '--- cross multiple ands ---'; @@ -38,7 +38,7 @@ SELECT '--- arithmetic expr ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a + 1 = t2_00826.a + t2_00826.b AND (t1_00826.a + t1_00826.b + t2_00826.a + t2_00826.b > 5); SELECT '--- is null or ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a); +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a; SELECT '--- do not rewrite alias ---'; SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0; From 63aebd17b2f165af6e8fcfd81cc32231bc706649 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Jul 2022 12:28:48 +0000 Subject: [PATCH 528/627] Remove TabSeparatedSorted --- .../Impl/TabSeparatedRowOutputFormat.cpp | 46 ------------------- .../Impl/TabSeparatedRowOutputFormat.h | 17 ------- tests/clickhouse-test | 11 +++-- 3 files changed, 6 insertions(+), 68 deletions(-) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 09839379a65..3bd0fd7e3d6 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -3,10 +3,6 @@ #include #include -#include -#include -#include - namespace DB { @@ -81,38 +77,6 @@ void TabSeparatedRowOutputFormat::writeBeforeExtremes() writeChar('\n', out); } -TabSeparatedSortedRowOutputFormat::TabSeparatedSortedRowOutputFormat( - WriteBuffer & out_, - const Block & header_, - const RowOutputFormatParams & params_, - const FormatSettings & format_settings_) - : TabSeparatedRowOutputFormat(tmp_buf, header_, false, false, false, params_, format_settings_) - , real_out(out_) -{} - -static String sortLines(const String & str) -{ - Strings lines; - boost::split(lines, str, boost::is_any_of("\n")); - /// sort alphabetically, empty lines at the end - std::sort(lines.begin(), lines.end(), [](const String & a, const String & b) { return (a.empty() || b.empty()) ? b.empty() : a < b; }); - return boost::join(lines, "\n"); -} - -void TabSeparatedSortedRowOutputFormat::writeSuffix() -{ - tmp_buf.finalize(); - String sorted_lines = sortLines(tmp_buf.str()); - tmp_buf.restart(); - writeString(sorted_lines, real_out); -} - -void TabSeparatedSortedRowOutputFormat::finalizeImpl() -{ - tmp_buf.finalize(); - writeString(tmp_buf.str(), real_out); -} - void registerOutputFormatTabSeparated(FormatFactory & factory) { for (bool is_raw : {false, true}) @@ -136,16 +100,6 @@ void registerOutputFormatTabSeparated(FormatFactory & factory) if (is_raw) registerWithNamesAndTypes("LineAsString", register_func); } - - /// Used in tests - factory.registerOutputFormat("TabSeparatedSorted", []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, params, settings); - }); } } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index a463fde003c..8781b7be0b1 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -49,21 +49,4 @@ protected: const FormatSettings format_settings; }; -class TabSeparatedSortedRowOutputFormat : public TabSeparatedRowOutputFormat -{ -public: - TabSeparatedSortedRowOutputFormat( - WriteBuffer & out_, - const Block & header_, - const RowOutputFormatParams & params_, - const FormatSettings & format_settings_); - - void writeSuffix() override; - void finalizeImpl() override; - -private: - WriteBufferFromOwnString tmp_buf; - WriteBuffer & real_out; -}; - } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 67e91282f2e..cab6daf3a50 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -912,9 +912,6 @@ class TestCase: # This is for .sh tests os.environ["CLICKHOUSE_LOG_COMMENT"] = args.testcase_basename - if "ignore-order" in self.tags: - client = client + " --format=TabSeparatedSorted" - params = { "client": client + " --database=" + database, "logs_level": server_logs_level, @@ -925,8 +922,12 @@ class TestCase: "secure": "--secure" if args.secure else "", } - pattern = "{test} > {stdout} 2> {stderr}" - + # >> append to stderr (but not stdout since it is not used there), + # because there are also output of per test database creation + if not args.database: + pattern = "{test} > {stdout} 2> {stderr}" + else: + pattern = "{test} > {stdout} 2> {stderr}" if self.ext == ".sql": pattern = ( From dfc0d4983455af93b51dfa694c6928996a8599d9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Jul 2022 12:34:11 +0000 Subject: [PATCH 529/627] remove tag ignore-order from tests --- tests/queries/0_stateless/00049_any_left_join.sql | 2 -- tests/queries/0_stateless/00050_any_left_join.sql | 2 -- tests/queries/0_stateless/00051_any_inner_join.sql | 2 -- tests/queries/0_stateless/00052_all_left_join.sql | 2 -- tests/queries/0_stateless/00099_join_many_blocks_segfault.sql | 2 -- tests/queries/0_stateless/00689_join_table_function.sql | 2 -- tests/queries/0_stateless/00722_inner_join.sql | 2 -- tests/queries/0_stateless/00725_join_on_bug_2.sql | 2 -- tests/queries/0_stateless/00800_low_cardinality_join.sql | 2 -- tests/queries/0_stateless/00845_join_on_aliases.sql | 2 -- tests/queries/0_stateless/00850_global_join_dups.sql | 2 +- tests/queries/0_stateless/00855_join_with_array_join.sql | 2 -- tests/queries/0_stateless/00863_comma_join_in.sql | 2 -- tests/queries/0_stateless/00927_asof_join_correct_bt.sql | 2 -- tests/queries/0_stateless/00927_asof_join_noninclusive.sql | 2 -- tests/queries/0_stateless/00927_asof_joins.sql | 2 -- tests/queries/0_stateless/00976_asof_join_on.sql | 2 -- tests/queries/0_stateless/01018_ambiguous_column.sql | 2 -- tests/queries/0_stateless/01139_asof_join_types.sql | 2 -- tests/queries/0_stateless/01710_projection_with_joins.sql | 3 +-- .../01839_join_to_subqueries_rewriter_columns_matcher.sql | 2 -- tests/queries/0_stateless/02007_join_use_nulls.sql | 2 -- tests/queries/0_stateless/02240_asof_join_biginteger.sql | 2 -- 23 files changed, 2 insertions(+), 45 deletions(-) diff --git a/tests/queries/0_stateless/00049_any_left_join.sql b/tests/queries/0_stateless/00049_any_left_join.sql index 5f014909ee4..ecd079a5085 100644 --- a/tests/queries/0_stateless/00049_any_left_join.sql +++ b/tests/queries/0_stateless/00049_any_left_join.sql @@ -1,3 +1 @@ --- Tags: ignore-order - SELECT number, joined FROM system.numbers ANY LEFT JOIN (SELECT number * 2 AS number, number * 10 + 1 AS joined FROM system.numbers LIMIT 10) js2 USING number LIMIT 10 diff --git a/tests/queries/0_stateless/00050_any_left_join.sql b/tests/queries/0_stateless/00050_any_left_join.sql index e380af3af27..5019de9fce7 100644 --- a/tests/queries/0_stateless/00050_any_left_join.sql +++ b/tests/queries/0_stateless/00050_any_left_join.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 diff --git a/tests/queries/0_stateless/00051_any_inner_join.sql b/tests/queries/0_stateless/00051_any_inner_join.sql index 7a364714ef8..566b5ad526b 100644 --- a/tests/queries/0_stateless/00051_any_inner_join.sql +++ b/tests/queries/0_stateless/00051_any_inner_join.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SET any_join_distinct_right_table_keys = 1; SELECT a.*, b.* FROM diff --git a/tests/queries/0_stateless/00052_all_left_join.sql b/tests/queries/0_stateless/00052_all_left_join.sql index edafe0b6c7e..6d5a1ba073c 100644 --- a/tests/queries/0_stateless/00052_all_left_join.sql +++ b/tests/queries/0_stateless/00052_all_left_join.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SELECT * FROM ( SELECT number AS k FROM system.numbers LIMIT 10 diff --git a/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql b/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql index 2ef347b6769..7219bcecf55 100644 --- a/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql +++ b/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SELECT 1 AS DomainID, Domain diff --git a/tests/queries/0_stateless/00689_join_table_function.sql b/tests/queries/0_stateless/00689_join_table_function.sql index 1681aea7104..c8e9100dcc0 100644 --- a/tests/queries/0_stateless/00689_join_table_function.sql +++ b/tests/queries/0_stateless/00689_join_table_function.sql @@ -1,3 +1 @@ --- Tags: ignore-order - SELECT * FROM numbers(3) AS a ANY LEFT JOIN numbers(3) AS b ON a.number = b.number ORDER BY a.number; diff --git a/tests/queries/0_stateless/00722_inner_join.sql b/tests/queries/0_stateless/00722_inner_join.sql index 61dde70e3dd..eee164ed1e1 100644 --- a/tests/queries/0_stateless/00722_inner_join.sql +++ b/tests/queries/0_stateless/00722_inner_join.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS one; CREATE TABLE one(dummy UInt8) ENGINE = Memory; diff --git a/tests/queries/0_stateless/00725_join_on_bug_2.sql b/tests/queries/0_stateless/00725_join_on_bug_2.sql index 02fc3952cb0..14fedfa14e7 100644 --- a/tests/queries/0_stateless/00725_join_on_bug_2.sql +++ b/tests/queries/0_stateless/00725_join_on_bug_2.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - set joined_subquery_requires_alias = 0; drop table if exists t_00725_2; diff --git a/tests/queries/0_stateless/00800_low_cardinality_join.sql b/tests/queries/0_stateless/00800_low_cardinality_join.sql index 6d4fbc3c232..30fb40a56e9 100644 --- a/tests/queries/0_stateless/00800_low_cardinality_join.sql +++ b/tests/queries/0_stateless/00800_low_cardinality_join.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - set joined_subquery_requires_alias = 0; select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) using val; diff --git a/tests/queries/0_stateless/00845_join_on_aliases.sql b/tests/queries/0_stateless/00845_join_on_aliases.sql index aede8138f42..0800d27caa6 100644 --- a/tests/queries/0_stateless/00845_join_on_aliases.sql +++ b/tests/queries/0_stateless/00845_join_on_aliases.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; diff --git a/tests/queries/0_stateless/00850_global_join_dups.sql b/tests/queries/0_stateless/00850_global_join_dups.sql index 00de0b9fe70..5a3ab0f1adc 100644 --- a/tests/queries/0_stateless/00850_global_join_dups.sql +++ b/tests/queries/0_stateless/00850_global_join_dups.sql @@ -1,4 +1,4 @@ --- Tags: global, ignore-order +-- Tags: global DROP TABLE IF EXISTS t_local; DROP TABLE IF EXISTS t1_00850; diff --git a/tests/queries/0_stateless/00855_join_with_array_join.sql b/tests/queries/0_stateless/00855_join_with_array_join.sql index 3d957be02f3..05180573525 100644 --- a/tests/queries/0_stateless/00855_join_with_array_join.sql +++ b/tests/queries/0_stateless/00855_join_with_array_join.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SET joined_subquery_requires_alias = 0; SELECT ax, c FROM (SELECT [1,2] ax, 0 c) ARRAY JOIN ax JOIN (SELECT 0 c) USING (c); diff --git a/tests/queries/0_stateless/00863_comma_join_in.sql b/tests/queries/0_stateless/00863_comma_join_in.sql index d6f33caabce..ebccd351c8a 100644 --- a/tests/queries/0_stateless/00863_comma_join_in.sql +++ b/tests/queries/0_stateless/00863_comma_join_in.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - drop table if exists test1_00863; drop table if exists test2_00863; drop table if exists test3_00863; diff --git a/tests/queries/0_stateless/00927_asof_join_correct_bt.sql b/tests/queries/0_stateless/00927_asof_join_correct_bt.sql index a1a884eb981..281a81d51c0 100644 --- a/tests/queries/0_stateless/00927_asof_join_correct_bt.sql +++ b/tests/queries/0_stateless/00927_asof_join_correct_bt.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS A; DROP TABLE IF EXISTS B; diff --git a/tests/queries/0_stateless/00927_asof_join_noninclusive.sql b/tests/queries/0_stateless/00927_asof_join_noninclusive.sql index 3b9de100134..5f15f3b593d 100644 --- a/tests/queries/0_stateless/00927_asof_join_noninclusive.sql +++ b/tests/queries/0_stateless/00927_asof_join_noninclusive.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS A; DROP TABLE IF EXISTS B; diff --git a/tests/queries/0_stateless/00927_asof_joins.sql b/tests/queries/0_stateless/00927_asof_joins.sql index 4e4e2a2307e..2dd6ca07b15 100644 --- a/tests/queries/0_stateless/00927_asof_joins.sql +++ b/tests/queries/0_stateless/00927_asof_joins.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS md; DROP TABLE IF EXISTS tv; diff --git a/tests/queries/0_stateless/00976_asof_join_on.sql b/tests/queries/0_stateless/00976_asof_join_on.sql index af291d73119..8060fb86831 100644 --- a/tests/queries/0_stateless/00976_asof_join_on.sql +++ b/tests/queries/0_stateless/00976_asof_join_on.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - DROP TABLE IF EXISTS A; DROP TABLE IF EXISTS B; diff --git a/tests/queries/0_stateless/01018_ambiguous_column.sql b/tests/queries/0_stateless/01018_ambiguous_column.sql index 70e3db374e8..54603aab810 100644 --- a/tests/queries/0_stateless/01018_ambiguous_column.sql +++ b/tests/queries/0_stateless/01018_ambiguous_column.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - select * from system.one cross join system.one; -- { serverError 352 } select * from system.one cross join system.one r; select * from system.one l cross join system.one; diff --git a/tests/queries/0_stateless/01139_asof_join_types.sql b/tests/queries/0_stateless/01139_asof_join_types.sql index dc1d687ced7..4cfde5d3210 100644 --- a/tests/queries/0_stateless/01139_asof_join_types.sql +++ b/tests/queries/0_stateless/01139_asof_join_types.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - select * from (select 0 as k, toInt8(1) as v) t1 asof join (select 0 as k, toInt8(0) as v) t2 using(k, v); select * from (select 0 as k, toInt16(1) as v) t1 asof join (select 0 as k, toInt16(0) as v) t2 using(k, v); select * from (select 0 as k, toInt32(1) as v) t1 asof join (select 0 as k, toInt32(0) as v) t2 using(k, v); diff --git a/tests/queries/0_stateless/01710_projection_with_joins.sql b/tests/queries/0_stateless/01710_projection_with_joins.sql index 0d78b1c4b0b..472242e3043 100644 --- a/tests/queries/0_stateless/01710_projection_with_joins.sql +++ b/tests/queries/0_stateless/01710_projection_with_joins.sql @@ -1,5 +1,4 @@ --- Tags: no-s3-storage, ignore-order - +-- Tags: no-s3-storage drop table if exists t; create table t (s UInt16, l UInt16, projection p (select s, l order by l)) engine MergeTree order by s; diff --git a/tests/queries/0_stateless/01839_join_to_subqueries_rewriter_columns_matcher.sql b/tests/queries/0_stateless/01839_join_to_subqueries_rewriter_columns_matcher.sql index 2f7599ded06..979debbcbb8 100644 --- a/tests/queries/0_stateless/01839_join_to_subqueries_rewriter_columns_matcher.sql +++ b/tests/queries/0_stateless/01839_join_to_subqueries_rewriter_columns_matcher.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SELECT COLUMNS('test') FROM (SELECT 1 AS id, 'a' AS test) a LEFT JOIN (SELECT 1 AS id, 'b' AS test) b ON b.id = a.id diff --git a/tests/queries/0_stateless/02007_join_use_nulls.sql b/tests/queries/0_stateless/02007_join_use_nulls.sql index ccea9778152..e08fffce3b7 100644 --- a/tests/queries/0_stateless/02007_join_use_nulls.sql +++ b/tests/queries/0_stateless/02007_join_use_nulls.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - SET join_use_nulls = 1; SELECT *, d.* FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id; diff --git a/tests/queries/0_stateless/02240_asof_join_biginteger.sql b/tests/queries/0_stateless/02240_asof_join_biginteger.sql index 5b5cbfb5d96..6dc5b00f116 100644 --- a/tests/queries/0_stateless/02240_asof_join_biginteger.sql +++ b/tests/queries/0_stateless/02240_asof_join_biginteger.sql @@ -1,5 +1,3 @@ --- Tags: ignore-order - select * from (select 0 as k, toInt128('18446744073709551617') as v) t1 asof join (select 0 as k, toInt128('18446744073709551616') as v) t2 using(k, v); select * from (select 0 as k, toInt256('340282366920938463463374607431768211457') as v) t1 asof join (select 0 as k, toInt256('340282366920938463463374607431768211456') as v) t2 using(k, v); From b4176828c680c866b99238f15c1306cc6e11cc92 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Jul 2022 13:16:40 +0000 Subject: [PATCH 530/627] Apply intHash64 on top of WeakHash32 for parallel join --- src/Interpreters/ConcurrentHashJoin.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 2c1f68c5bf7..56468ff5de3 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -186,7 +186,9 @@ static ALWAYS_INLINE IColumn::Selector hashToSelector(const WeakHash32 & hash, s IColumn::Selector selector(num_rows); for (size_t i = 0; i < num_rows; ++i) - selector[i] = data[i] & (num_shards - 1); + /// Apply intHash64 to mix bits in data. + /// HashTable internally uses WeakHash32, and we need to get different lower bits not to cause collisions. + selector[i] = intHash64(data[i]) & (num_shards - 1); return selector; } From b4cb2a847e8c1fa6d1a340d3d03a8c036f8754c1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Jul 2022 13:19:04 +0000 Subject: [PATCH 531/627] Default join algorithm set to hash --- src/Core/Settings.h | 2 +- .../02236_explain_pipeline_join.reference | 23 ++++++++----------- 2 files changed, 11 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 12149688eba..5597d9076a4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -350,7 +350,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \ M(OverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(Bool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \ - M(JoinAlgorithm, join_algorithm, JoinAlgorithm::PARALLEL_HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge', 'parallel_hash'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \ + M(JoinAlgorithm, join_algorithm, JoinAlgorithm::HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge', 'parallel_hash'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \ M(UInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \ M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ diff --git a/tests/queries/0_stateless/02236_explain_pipeline_join.reference b/tests/queries/0_stateless/02236_explain_pipeline_join.reference index 0f4b6d3044b..5d7a7bfc488 100644 --- a/tests/queries/0_stateless/02236_explain_pipeline_join.reference +++ b/tests/queries/0_stateless/02236_explain_pipeline_join.reference @@ -1,20 +1,17 @@ (Expression) -ExpressionTransform × 16 +ExpressionTransform (Join) - JoiningTransform × 16 2 → 1 - Resize 1 → 16 - (Expression) + JoiningTransform 2 → 1 + (Expression) + ExpressionTransform + (Limit) + Limit + (ReadFromStorage) + Numbers 0 → 1 + (Expression) + FillingRightJoinSide ExpressionTransform (Limit) Limit (ReadFromStorage) Numbers 0 → 1 - (Expression) - Resize × 2 16 → 1 - FillingRightJoinSide × 16 - Resize 1 → 16 - ExpressionTransform - (Limit) - Limit - (ReadFromStorage) - Numbers 0 → 1 From af55f1c1e3dddf7321f0113f4e2d181dd4cdc77e Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 7 Jul 2022 11:10:27 +0000 Subject: [PATCH 532/627] Update toPowerOfTwo in ConcurrentHashJoin --- src/Interpreters/ConcurrentHashJoin.cpp | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 56468ff5de3..5d6318a8df1 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -31,20 +31,16 @@ namespace ErrorCodes static UInt32 toPowerOfTwo(UInt32 x) { - x = x - 1; - for (UInt32 i = 1; i < sizeof(UInt32) * 8; i <<= 1) - x = x | x >> i; - return x + 1; + if (x <= 1) + return 1; + return static_cast(1) << (32 - __builtin_clz(x - 1)); } ConcurrentHashJoin::ConcurrentHashJoin(ContextPtr context_, std::shared_ptr table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_) : context(context_) , table_join(table_join_) - , slots(slots_) + , slots(toPowerOfTwo(std::min(slots_, 256))) { - slots = std::min(std::max(1, slots), 255); - slots = toPowerOfTwo(slots); - for (size_t i = 0; i < slots; ++i) { auto inner_hash_join = std::make_shared(); From d636674a9ed33de54172ca0b435315a8d89576ef Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 7 Jul 2022 15:57:15 +0000 Subject: [PATCH 533/627] Add order by to some join tests --- tests/queries/0_stateless/00053_all_inner_join.sql | 2 +- tests/queries/0_stateless/00054_join_string.sql | 3 ++- tests/queries/0_stateless/00702_join_with_using.sql | 12 ++++++------ 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/00053_all_inner_join.sql b/tests/queries/0_stateless/00053_all_inner_join.sql index 6cefb209682..f4f84069d05 100644 --- a/tests/queries/0_stateless/00053_all_inner_join.sql +++ b/tests/queries/0_stateless/00053_all_inner_join.sql @@ -7,4 +7,4 @@ ALL INNER JOIN SELECT intDiv(number, 2) AS k, number AS joined FROM system.numbers LIMIT 10 ) AS b USING k -ORDER BY k; +ORDER BY k, joined; diff --git a/tests/queries/0_stateless/00054_join_string.sql b/tests/queries/0_stateless/00054_join_string.sql index 09a03f73614..0e7a5520e78 100644 --- a/tests/queries/0_stateless/00054_join_string.sql +++ b/tests/queries/0_stateless/00054_join_string.sql @@ -7,4 +7,5 @@ ALL LEFT JOIN SELECT reinterpretAsString(intDiv(number, 2) + reinterpretAsUInt8('A')) AS k, number AS joined FROM system.numbers LIMIT 10 ) js2 USING k -ORDER BY k; +ORDER BY k, joined +; diff --git a/tests/queries/0_stateless/00702_join_with_using.sql b/tests/queries/0_stateless/00702_join_with_using.sql index 5faa9a3fad3..4f81e757dfc 100644 --- a/tests/queries/0_stateless/00702_join_with_using.sql +++ b/tests/queries/0_stateless/00702_join_with_using.sql @@ -26,14 +26,14 @@ values ('1', 'John'), ('2', 'Jack'), ('3', 'Daniel'), ('4', 'James'), ('5', 'Ama insert into children (id, childName) values ('1', 'Robert'), ('1', 'Susan'), ('3', 'Sarah'), ('4', 'David'), ('4', 'Joseph'), ('5', 'Robert'); -select * from persons all inner join children using id order by id; -select * from persons all inner join (select * from children) as j using id order by id; -select * from (select * from persons) as s all inner join (select * from children ) as j using id order by id; +select * from persons all inner join children using id order by id, name, childName; +select * from persons all inner join (select * from children) as j using id order by id, name, childName; +select * from (select * from persons) as s all inner join (select * from children ) as j using id order by id, name, childName; -- set joined_subquery_requires_alias = 0; -select * from persons all inner join (select * from children) using id order by id; -select * from (select * from persons) all inner join (select * from children) using id order by id; -select * from (select * from persons) as s all inner join (select * from children) using id order by id; +select * from persons all inner join (select * from children) using id order by id, name, childName; +select * from (select * from persons) all inner join (select * from children) using id order by id, name, childName; +select * from (select * from persons) as s all inner join (select * from children) using id order by id, name, childName; drop table persons; drop table children; From 6d3b0032c343b8f601bd01b1c6159c07e384af91 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 20:25:40 +0200 Subject: [PATCH 534/627] Change the link to Hardware benchmark script --- docs/en/operations/performance-test.md | 2 +- docs/zh/operations/performance-test.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/performance-test.md b/docs/en/operations/performance-test.md index 5be317bbd6b..5b2a772a009 100644 --- a/docs/en/operations/performance-test.md +++ b/docs/en/operations/performance-test.md @@ -14,7 +14,7 @@ You can run benchmark with a single script. 1. Download the script. ``` -wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/hardware.sh +wget https://raw.githubusercontent.com/ClickHouse/ClickBench/main/hardware/hardware.sh ``` 2. Run the script. diff --git a/docs/zh/operations/performance-test.md b/docs/zh/operations/performance-test.md index f7e79d606f7..005e2a9d657 100644 --- a/docs/zh/operations/performance-test.md +++ b/docs/zh/operations/performance-test.md @@ -14,7 +14,7 @@ sidebar_label: "\u6D4B\u8BD5\u786C\u4EF6" 1. 下载脚本 ``` -wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/hardware.sh +wget https://raw.githubusercontent.com/ClickHouse/ClickBench/main/hardware/hardware.sh ``` 2. 运行脚本 From e93e3212a715a584e857aabf4d30bc7ea557457f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 20:28:02 +0200 Subject: [PATCH 535/627] Remove old file --- benchmark/hardware.sh | 247 ------------------------------------------ 1 file changed, 247 deletions(-) delete mode 100755 benchmark/hardware.sh diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh deleted file mode 100755 index 635f3a9eb61..00000000000 --- a/benchmark/hardware.sh +++ /dev/null @@ -1,247 +0,0 @@ -#!/bin/bash -e - -QUERIES_FILE="queries.sql" -TRIES=3 - -mkdir -p clickhouse-benchmark -pushd clickhouse-benchmark - -# Download the binary -if [[ ! -x clickhouse ]]; then - curl https://clickhouse.com/ | sh -fi - -if [[ ! -f $QUERIES_FILE ]]; then - wget "https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/hardware/$QUERIES_FILE" -fi - -uptime - -echo "Starting clickhouse-server" - -./clickhouse server >/dev/null 2>&1 & -PID=$! - -function finish { - kill $PID - wait -} -trap finish EXIT - -echo "Waiting for clickhouse-server to start" - -for i in {1..30}; do - sleep 1 - ./clickhouse client --query "SELECT 'Ok.'" 2>/dev/null && break || echo -n '.' - if [[ $i == 30 ]]; then exit 1; fi -done - -if [[ $(./clickhouse client --query "EXISTS hits") == '1' && $(./clickhouse client --query "SELECT count() FROM hits") == '100000000' ]]; then - echo "Dataset already downloaded" -else - echo "Will download the dataset" - if [ "`uname`" = "Darwin" ] - then - ./clickhouse client --receive_timeout 1000 --max_insert_threads $(sysctl -n hw.ncpu) --progress --query " - CREATE OR REPLACE TABLE hits ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) - AS SELECT * FROM url('https://datasets.clickhouse.com/hits/native/hits_100m_obfuscated_{0..255}.native.zst')" - else - ./clickhouse client --receive_timeout 1000 --max_insert_threads $(nproc || 4) --progress --query " - CREATE OR REPLACE TABLE hits ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) - AS SELECT * FROM url('https://datasets.clickhouse.com/hits/native/hits_100m_obfuscated_{0..255}.native.zst')" - fi - ./clickhouse client --query "SELECT 'The dataset size is: ', count() FROM hits" -fi - -if [[ $(./clickhouse client --query "SELECT count() FROM system.parts WHERE table = 'hits' AND database = 'default' AND active") == '1' ]]; then - echo "Dataset already prepared" -else - echo "Will prepare the dataset" - ./clickhouse client --receive_timeout 1000 --query "OPTIMIZE TABLE hits FINAL" -fi - -echo -echo "Will perform benchmark. Results:" -echo - ->result.csv -QUERY_NUM=1 - -cat "$QUERIES_FILE" | sed "s/{table}/hits/g" | while read query; do - sync - if [ "`uname`" = "Darwin" ] - then - sudo purge > /dev/null - else - echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null - fi - - echo -n "[" - for i in $(seq 1 $TRIES); do - RES=$(./clickhouse client --time --format=Null --query="$query" 2>&1 ||:) - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " - - echo "${QUERY_NUM},${i},${RES}" >> result.csv - done - echo "]," - - QUERY_NUM=$((QUERY_NUM + 1)) -done - - -echo -echo "Benchmark complete. System info:" -echo - -touch {cpu_model,cpu,df,memory,memory_total,blk,mdstat,instance}.txt - -if [ "`uname`" = "Darwin" ] -then - echo '----Version, build id-----------' - ./clickhouse local --query "SELECT format('Version: {}', version())" - ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw - ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" - echo '----CPU-------------------------' - sysctl hw.model | tee cpu_model.txt - sysctl -a | grep -E 'hw.activecpu|hw.memsize|hw.byteorder|cachesize' | tee cpu.txt - echo '----Disk Free and Total--------' - df -h . | tee df.txt - echo '----Memory Free and Total-------' - vm_stat | tee memory.txt - echo '----Physical Memory Amount------' - ls -l /var/vm | tee memory_total.txt - echo '--------------------------------' -else - echo '----Version, build id-----------' - ./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())" - ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw - ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" - echo '----CPU-------------------------' - cat /proc/cpuinfo | grep -i -F 'model name' | uniq | tee cpu_model.txt - lscpu | tee cpu.txt - echo '----Block Devices---------------' - lsblk | tee blk.txt - echo '----Disk Free and Total--------' - df -h . | tee df.txt - echo '----Memory Free and Total-------' - free -h | tee memory.txt - echo '----Physical Memory Amount------' - cat /proc/meminfo | grep MemTotal | tee memory_total.txt - echo '----RAID Info-------------------' - cat /proc/mdstat| tee mdstat.txt - echo '--------------------------------' -fi -echo - -echo "Instance type from IMDS (if available):" -curl -s --connect-timeout 1 'http://169.254.169.254/latest/meta-data/instance-type' | tee instance.txt -echo - -echo "Uploading the results (if possible)" - -UUID=$(./clickhouse local --query "SELECT generateUUIDv4()") - -./clickhouse local --query " - SELECT - '${UUID}' AS run_id, - version() AS version, - now() AS test_time, - (SELECT value FROM system.settings WHERE name = 'max_threads') AS threads, - filesystemCapacity() AS fs_capacity, - filesystemAvailable() AS fs_available, - file('cpu_model.txt') AS cpu_model, - file('cpu.txt') AS cpu, - file('df.txt') AS df, - file('memory.txt') AS memory, - file('memory_total.txt') AS memory_total, - file('blk.txt') AS blk, - file('mdstat.txt') AS mdstat, - file('instance.txt') AS instance -" | tee meta.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query " - INSERT INTO benchmark_runs - (run_id, version, test_time, threads, fs_capacity, fs_available, cpu_model, cpu, df, memory, memory_total, blk, mdstat, instance) - FORMAT TSV" || echo "Cannot upload results." - -./clickhouse local --query " - SELECT - '${UUID}' AS run_id, - c1 AS query_num, - c2 AS try_num, - c3 AS time - FROM file('result.csv') -" | tee results.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query " - INSERT INTO benchmark_results - (run_id, query_num, try_num, time) - FORMAT TSV" || echo "Cannot upload results. Please send the output to feedback@clickhouse.com" - -< 1 - ) - GROUP BY run_id - ORDER BY k ASC -) AS t -INNER JOIN benchmark_runs USING (run_id) - -//// From 1760417bfcf01fccd8ec026558f80846e7cd9dee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 20:28:43 +0200 Subject: [PATCH 536/627] Update index.html --- website/benchmark/dbms/index.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/benchmark/dbms/index.html b/website/benchmark/dbms/index.html index cbf48fd021d..d53bd3ff195 100644 --- a/website/benchmark/dbms/index.html +++ b/website/benchmark/dbms/index.html @@ -365,7 +365,7 @@ const data = [ ,{"system":"Snowflake","date":"2022-07-01","machine":"XL","cluster_size":16,"comment":"","tags":["managed","column-oriented"],"load_time":2524,"data_size":12300000000,"result":[[0.071,0.053,0.057],[0.998,0.61,0.24],[0.42,1.138,1.051],[0.653,0.264,0.178],[0.352,0.312,0.349],[1.126,0.431,0.42],[0.067,0.057,0.054],[0.225,0.217,0.2],[0.617,0.366,0.371],[1.006,0.541,0.498],[0.463,0.425,0.293],[0.431,0.36,0.339],[0.392,0.371,0.386],[0.588,0.581,0.59],[0.634,0.414,0.4],[0.368,0.41,0.388],[0.594,0.639,0.663],[0.616,0.581,0.569],[1.092,0.933,0.901],[0.493,0.213,0.16],[0.886,0.48,0.442],[0.448,0.337,0.399],[0.84,0.572,0.505],[2.251,1.23,0.959],[0.295,0.253,0.241],[0.214,0.239,0.278],[0.261,0.232,0.314],[0.422,0.429,0.403],[0.892,0.934,0.883],[1.041,1.017,1.009],[0.715,0.442,0.363],[0.845,0.413,0.461],[1.101,1.085,1.102],[1.294,1.272,1.339],[1.839,1.327,1.241],[0.439,0.399,0.393],[0.199,0.211,0.19],[0.157,0.143,0.14],[0.145,0.157,0.141],[0.331,0.291,0.333],[0.173,0.214,0.138],[0.189,0.15,0.159],[0.135,0.149,0.138]],"source":"snowflake/results/xl.json"} ,{"system":"Snowflake","date":"2022-07-01","machine":"XS","cluster_size":1,"comment":"","tags":["managed","column-oriented"],"load_time":2524,"data_size":12300000000,"result":[[0.169,0.055,0.056],[1.184,0.582,0.386],[1.35,0.56,0.568],[1.27,0.554,0.538],[2.516,2.564,2.506],[2.935,2.649,2.67],[0.052,0.05,0.064],[0.383,0.387,0.397],[3.249,2.993,3.014],[3.589,3.627,3.887],[1.243,0.986,0.966],[1.325,1.08,1.073],[2.038,2.046,2.035],[3.738,3.626,3.718],[2.318,2.159,2.176],[2.733,2.637,2.668],[5.607,5.683,5.667],[3.978,3.923,3.879],[10.085,9.871,9.844],[0.45,0.375,0.469],[5.474,3.103,3.06],[2.012,1.982,1.971],[3.365,2.471,2.501],[11.96,10.619,9.518],[1.074,1.059,1.026],[0.856,0.846,0.879],[1.1,1.085,1.083],[3.057,3.228,3.117],[9.406,9.019,9.158],[6.196,6.243,6.911],[2.906,2.343,2.017],[2.954,2.666,2.565],[9.459,9.565,9.557],[9.555,9.529,9.368],[9.409,9.185,9.294],[2.796,2.88,2.685],[0.299,0.249,0.262],[0.156,0.145,0.18],[0.147,0.146,0.16],[0.371,0.357,0.356],[0.166,0.133,0.155],[0.218,0.14,0.135],[0.14,0.152,0.158]],"source":"snowflake/results/xs.json"} ,{"system":"SQLite","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","embedded","row-oriented"],"load_time":2608,"data_size":75776589824,"result":[[752.739,2.003,1.2],[304.302,291.521,286.965],[293.964,287.619,287.219],[758.302,5.879,5.65],[836.393,48.593,48.452],[362.605,344.884,356.245],[763.993,11.602,10.795],[296.348,286.879,287.557],[365.816,360.339,354.126],[374.403,365.196,362.261],[302.989,293.888,298.432],[303.64,291.729,295.347],[316.824,298.18,301.006],[320.665,301.15,305.227],[313.593,301.021,301.626],[794.881,47,47.225],[355.346,344.615,342.442],[316.499,305.971,305.007],[398.177,380.383,385.571],[751.82,5.082,4.913],[295.522,286.573,287.368],[298.58,287.182,288.303],[296.474,288.747,288.638],[296.579,287.127,287.361],[304.709,286.865,287.56],[300.391,290.633,288.587],[294.605,286.91,287.799],[305.986,312.111,305.626],[null,null,null],[411.225,397.614,394.253],[307.711,295.181,300.266],[312.472,299.079,298.19],[386.674,378.347,376.963],[409.742,409.554,420.273],[468.73,453.709,458.149],[366.015,347.446,346.728],[2.911,0.781,0.757],[1.599,0.609,0.587],[1.288,0.256,0.238],[2.469,1.582,1.52],[1.274,0.303,0.283],[1.322,0.317,0.314],[1.498,0.602,0.613]],"source":"sqlite/results/c6a.4xlarge.json"} -,{"system":"TimescaleDB (compression)","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","PostgreSQL compatible","column-oriented","time-series"],"load_time":4605,"data_size":20333747165,"result":[[2.28686,1.63642,1.64263],[32.6848,1.63476,1.40052],[60.8633,3.70484,3.59342],[36.4029,2.87091,2.80739],[110.391,38.9688,38.0549],[147.379,66.2513,65.6379],[33.0294,2.92031,2.84375],[33.0221,1.2984,1.19227],[115.694,47.4651,47.0125],[156.347,51.577,51.2694],[68.3301,4.75521,4.68007],[77.4356,5.55128,5.56577],[49.7741,11.2911,11.3265],[81.1014,14.9111,14.9541],[82.9569,14.6156,14.6331],[62.0338,26.399,26.3351],[103.259,36.4122,36.6076],[92.8828,26.2395,25.8991],[144.281,63.5102,63.7661],[7.00679,0.573073,0.536283],[75.0203,7.86344,7.90495],[81.2825,9.15868,9.01775],[104.084,13.9528,13.8435],[132.531,81.522,82.1561],[80.6965,3.28231,3.16574],[39.7693,2.51443,2.43849],[80.4245,3.26941,3.13916],[104.015,13.7044,13.5313],[307.26,253.127,252.147],[42.8549,22.4187,22.0325],[137.601,14.9592,14.6804],[136.767,22.8007,22.131],[263.005,168.551,163.355],[156.919,92.6308,91.702],[160.842,96.0512,97.1773],[62.8357,28.0336,28.7397],[1.75869,0.561604,0.541215],[0.46607,0.191863,0.19021],[0.303671,0.137579,0.136615],[2.32031,1.49223,1.52369],[0.563764,0.14192,0.138234],[0.372428,0.122989,0.123709],[0.448574,0.159092,0.154687]],"source":"timescaledb/results/c6a.4xlarge.compression.json"} +,{"system":"TimescaleDB (compression)","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","PostgreSQL compatible","column-oriented","time-series"],"load_time":4605,"data_size":20333747165,"result":[[2.28686,1.63642,1.64263],[32.6848,1.63476,1.40052],[60.8633,3.70484,3.59342],[36.4029,2.87091,2.80739],[110.391,38.9688,38.0549],[147.379,66.2513,65.6379],[33.0294,2.92031,2.84375],[33.0221,1.2984,1.19227],[115.694,47.4651,47.0125],[156.347,51.577,51.2694],[68.3301,4.75521,4.68007],[77.4356,5.55128,5.56577],[49.7741,11.2911,11.3265],[81.1014,14.9111,14.9541],[82.9569,14.6156,14.6331],[62.0338,26.399,26.3351],[103.259,36.4122,36.6076],[92.8828,26.2395,25.8991],[144.281,63.5102,63.7661],[7.00679,0.573073,0.536283],[75.0203,7.86344,7.90495],[81.2825,9.15868,9.01775],[104.084,13.9528,13.8435],[132.531,81.522,82.1561],[80.6965,3.28231,3.16574],[39.7693,2.51443,2.43849],[80.4245,3.26941,3.13916],[104.015,13.7044,13.5313],[307.26,253.127,252.147],[42.8549,22.4187,22.0325],[137.601,14.9592,14.6804],[136.767,22.8007,22.131],[263.005,168.551,163.355],[156.919,92.6308,91.702],[160.842,96.0512,97.1773],[62.8357,28.0336,28.7397],[1.75869,0.561604,0.541215],[0.46607,0.191863,0.19021],[0.303671,0.137579,0.136615],[2.32031,1.49223,1.52369],[0.563764,0.14192,0.138234],[0.372428,0.122989,0.123709],[0.448574,0.159092,0.154687]],"source":"timescaledb-compressed/results/c6a.4xlarge.json"} ,{"system":"TimescaleDB","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C","PostgreSQL compatible","row-oriented","time-series"],"load_time":1620,"data_size":72882392030,"result":[[437.7,215.793,176.42],[327.026,259.568,244.578],[262.978,263.09,263.083],[262.807,263.046,266.847],[337.497,334.964,330.852],[355.689,356.801,362.894],[262.762,263.012,262.968],[263.055,263.016,263.028],[319.928,319.388,320.704],[323.584,322.224,322.488],[265.979,265.465,265.375],[266.019,265.543,265.462],[277.018,276.3,276.595],[280.352,279.251,279.572],[279.915,279.896,279.674],[296.377,298.506,297.659],[314.448,314.605,312.57],[302.668,302.672,303.039],[325.81,324.061,324.376],[262.447,262.698,262.704],[267.581,267.467,267.482],[268.085,267.466,267.696],[263.391,263.097,263.126],[38.291,0.435,0.335],[0.127,0.005,0.005],[263.138,263.1,263.092],[0.889,0.341,0.339],[267.586,267.498,267.491],[289.086,290.012,290.093],[263.22,263.071,263.109],[274.78,273.995,273.998],[282.217,281.39,281.47],[429.273,426.588,439.431],[448.808,418.724,418.207],[455.196,422.75,423.142],[299.263,296.937,297.261],[18.693,1.552,1.481],[18.125,0.945,0.937],[18.528,1.062,0.902],[18.268,1.779,1.77],[19.615,1.965,1.966],[18.97,1.435,1.43],[18.33,1.153,0.952]],"source":"timescaledb/results/c6a.4xlarge.json"} ]; // end of data From dcfe2999665f1b282cdec7d92f15165436b1d4ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Jul 2022 23:51:12 +0200 Subject: [PATCH 537/627] Add ClickHouse with zstd to benchmark --- website/benchmark/dbms/index.html | 1 + 1 file changed, 1 insertion(+) diff --git a/website/benchmark/dbms/index.html b/website/benchmark/dbms/index.html index d53bd3ff195..8fae6311a78 100644 --- a/website/benchmark/dbms/index.html +++ b/website/benchmark/dbms/index.html @@ -338,6 +338,7 @@ const data = [ ,{"system":"clickhouse-local (partitioned)","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","column-oriented","embedded","stateless","ClickHouse derivative"],"load_time":0,"data_size":14737666736,"result":[[0.85,0.102,0.114],[1.342,0.09,0.099],[2.547,0.179,0.189],[1.681,0.245,0.252],[2.704,1.68,1.648],[2.194,1.46,1.489],[0.832,0.113,0.102],[1.371,0.106,0.101],[2.24,0.79,0.825],[4.548,1.021,1.026],[3.094,0.552,0.552],[3.088,0.623,0.63],[2.017,1.17,1.165],[4.319,1.677,1.708],[2.157,1.496,1.5],[1.629,1.138,1.139],[5.026,3.267,3.241],[4.142,2.303,2.319],[8.295,5.569,5.629],[1.331,0.255,0.252],[10.712,3.668,3.786],[13.053,4.185,4.202],[24.17,7.935,8.008],[55.965,23.933,23.071],[4.417,0.947,0.974],[1.793,0.698,0.69],[4.376,0.955,0.956],[11.731,4.385,4.321],[11.403,8.549,8.288],[2.764,2.754,2.735],[5.096,1.262,1.273],[9.515,1.682,1.688],[10.325,6.745,6.608],[11.686,6.261,6.242],[11.769,6.301,6.364],[1.675,1.49,1.495],[14.937,3.631,3.604],[14.187,3.609,3.631],[14.842,3.769,3.741],[22.222,6.355,6.263],[7.212,0.836,0.838],[7.863,0.716,0.718],[5.12,0.587,0.574]],"source":"clickhouse-local/results/c6a.4xlarge.partitioned.json"} ,{"system":"clickhouse-local (single)","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","column-oriented","embedded","stateless","ClickHouse derivative"],"load_time":0,"data_size":14779976446,"result":[[1.176,0.251,0.249],[1.037,0.134,0.111],[1.609,0.403,0.369],[1.616,0.372,0.37],[3.008,2.338,2.266],[9.061,7.537,7.535],[1.206,0.191,0.187],[0.882,0.144,0.135],[4.61,3.406,3.256],[6.712,4.479,4.469],[4.081,2.413,2.394],[3.894,2.719,2.691],[7.651,6.436,6.243],[10.765,8.043,7.894],[9.86,8.945,8.235],[7.159,5.815,5.814],[20.916,18.159,18.013],[20.952,17.862,17.85],[37.585,32.649,32.487],[1.767,0.401,0.393],[23.713,15.687,15.755],[28.7,19.241,19.198],[50.74,33.161,33.011],[152.485,117.417,118.178],[7.606,4.491,5.326],[4.331,4.214,3.587],[6.743,4.486,5.357],[22.91,15.043,15.183],[43.342,37.167,36.842],[11.807,4.49,4.546],[9.557,6.349,6.263],[13.964,8.493,8.464],[38.11,33.642,33.996],[41.266,35.08,27.073],[34.056,26.814,26.902],[8.855,7.548,7.475],[22.596,12.615,12.669],[23.217,13.956,13.831],[22.528,21.601,13.207],[37.89,23.115,22.955],[6.49,1.548,1.522],[6.413,1.474,1.416],[3.796,1.339,1.316]],"source":"clickhouse-local/results/c6a.4xlarge.single.json"} ,{"system":"ClickHouse","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","column-oriented","ClickHouse derivative"],"load_time":475.529,"data_size":14345515782,"result":[[0.027,0.001,0.001],[0.035,0.015,0.021],[0.083,0.034,0.033],[0.171,0.044,0.045],[1.552,1.495,1.574],[1.27,1.075,1.063],[0.045,0.026,0.025],[0.032,0.016,0.015],[0.717,0.615,0.607],[0.843,0.821,0.747],[0.293,0.219,0.216],[0.312,0.226,0.235],[0.804,0.694,0.702],[1.476,1.047,1.029],[1.013,0.898,0.911],[1.043,0.964,1.453],[3.632,2.715,2.711],[1.867,1.75,1.714],[5.187,4.797,4.953],[0.112,0.068,0.041],[8.637,1.761,1.212],[9.902,0.902,0.869],[18.831,2.067,1.829],[41.903,4.476,3.486],[1.801,0.254,0.238],[0.627,0.214,0.207],[2.181,0.241,0.246],[8.868,0.748,0.733],[9.674,6.891,5.77],[2.62,2.355,2.368],[1.395,0.533,0.525],[4.454,0.73,0.712],[5.453,4.99,5.922],[9.955,3.968,4.096],[9.987,4.035,4.476],[1.695,1.236,1.241],[0.142,0.079,0.103],[0.066,0.033,0.04],[0.065,0.03,0.033],[0.246,0.207,0.192],[0.044,0.019,0.02],[0.03,0.023,0.012],[0.03,0.018,0.013]],"source":"clickhouse/results/c6a.4xlarge.json"} +,{"system":"ClickHouse (zstd)","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","column-oriented","ClickHouse derivative"],"load_time":417.533,"data_size":9997826232,"result":[[0.004,0.002,0.001],[0.069,0.017,0.016],[0.073,0.051,0.048],[0.158,0.067,0.066],[1.603,1.545,1.709],[1.225,1.105,1.1],[0.033,0.026,0.024],[0.023,0.021,0.02],[0.71,0.621,0.614],[0.813,0.715,0.713],[0.292,0.231,0.245],[0.35,0.269,0.271],[0.785,0.692,0.704],[1.209,1.044,1.066],[1.034,0.908,0.897],[0.993,0.94,0.909],[2.818,2.595,2.611],[1.804,1.717,1.711],[4.846,4.692,4.714],[0.081,0.025,0.02],[5.511,1.388,1.385],[6.304,1.306,1.293],[11.938,2.406,2.4],[30.031,4.268,4.242],[0.964,0.337,0.335],[0.326,0.264,0.26],[0.997,0.338,0.338],[5.523,1.148,1.14],[7.319,5.744,5.796],[2.235,2.236,2.199],[0.86,0.66,0.664],[3.695,0.772,0.764],[5.044,4.82,4.821],[6.786,4.133,4.093],[6.769,4.11,4.17],[1.289,1.188,1.221],[0.093,0.069,0.075],[0.037,0.033,0.029],[0.047,0.033,0.03],[0.185,0.151,0.155],[0.027,0.021,0.017],[0.02,0.012,0.015],[0.017,0.013,0.013]],"source":"clickhouse/results/c6a.4xlarge.zstd.json"} ,{"system":"ClickHouse","date":"2022-07-01","machine":"c6a.metal, 500gb gp2","cluster_size":1,"comment":"","tags":["C++","column-oriented","ClickHouse derivative"],"load_time":136.869,"data_size":14571706777,"result":[[0.011,0.001,0.001],[0.04,0.015,0.013],[0.045,0.021,0.023],[0.09,0.023,0.023],[1.922,1.565,1.576],[0.961,0.737,0.739],[0.04,0.023,0.018],[0.032,0.028,0.028],[0.321,0.287,0.275],[0.632,0.284,0.287],[0.166,0.124,0.118],[0.235,0.1,0.102],[1.006,0.182,0.159],[1.637,0.216,0.213],[0.871,0.174,0.177],[0.258,0.148,0.148],[1.804,0.37,0.358],[1.235,0.275,0.278],[3.143,0.854,0.815],[0.071,0.024,0.016],[8.816,0.215,0.155],[10.239,0.203,0.173],[19.179,0.388,0.357],[43.152,0.824,0.823],[1.821,0.059,0.052],[0.992,0.045,0.051],[2.539,0.063,0.058],[9.258,0.3,0.278],[7.923,0.961,0.936],[0.445,0.431,0.428],[1.367,0.131,0.113],[4.819,0.205,0.175],[3.808,0.739,0.726],[8.935,0.607,0.6],[8.988,0.634,0.615],[0.242,0.22,0.226],[0.075,0.058,0.056],[0.038,0.028,0.026],[0.043,0.028,0.021],[0.172,0.127,0.119],[0.028,0.018,0.017],[0.027,0.019,0.014],[0.018,0.026,0.015]],"source":"clickhouse/results/c6a.metal.json"} ,{"system":"CrateDB","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"For some queries it gives \"Data too large\".","tags":["Java","column-oriented"],"load_time":10687,"data_size":109636633416,"result":[[0.008162,0.005118,0.002553],[0.350014,0.39977,0.133775],[2.58426,2.47192,2.59779],[2.12939,0.532981,0.507246],[null,null,null],[null,null,null],[1.18488,1.06603,1.07219],[0.209264,0.073284,0.067912],[null,null,null],[null,null,null],[1.68892,1.2866,1.47428],[1.62976,1.43073,1.26904],[12.7517,13.0334,13.2685],[18.8587,null,18.6951],[11.2982,11.2108,11.577],[20.2964,20.4035,19.1076],[null,null,null],[null,null,null],[null,null,null],[0.202044,0.010009,0.005566],[9.22964,4.54606,0.774149],[1.41673,1.09885,0.789775],[12.3933,8.06911,1.69671],[1.45018,0.969528,0.979718],[0.357589,0.14887,0.153326],[0.189282,0.133963,0.130279],[0.153222,0.140756,0.139861],[27.5195,19.6862,20.1825],[72.7575,68.2,67.1238],[144.533,146.579,152.144],[8.76866,9.00563,8.46917],[17.6652,16.6755,16.0558],[null,null,null],[null,null,null],[null,null,null],[42.2967,44.9621,44.4386],[0.786911,0.4904,0.508416],[0.602075,0.226261,0.182399],[0.131407,0.058958,0.054518],[0.954736,1.1361,1.14233],[0.23764,0.139109,0.134472],[0.110253,0.057695,0.056073],[0.124285,0.150479,0.066226]],"source":"cratedb/results/c6a.4xlarge.json"} ,{"system":"Databend","date":"2022-07-01","machine":"c6a.4xlarge, 500gb gp2","cluster_size":1,"comment":"Only 90% of data successfully loaded.","tags":["Rust","column-oriented","ClickHouse derivative"],"load_time":484,"data_size":43016643271,"result":[[0.010087,0.002961,0.003271],[0.127964,0.080012,0.075741],[0.162388,0.143967,0.144762],[0.252904,0.217471,0.217369],[34.281026,34.844158,34.526942],[25.290307,25.793068,25.620563],[0.112484,0.093867,0.090891],[0.086604,0.07796,0.076448],[20.723203,20.7483,20.354869],[20.81994,20.72446,20.696573],[1.964378,1.93559,1.893824],[1.846866,1.789111,1.763664],[4.468158,4.407959,4.438036],[19.947276,19.8859,19.853514],[5.478573,5.474461,5.460604],[5.509521,5.513413,5.363123],[15.430359,15.5406,15.461211],[14.905998,15.029721,15.019642],[31.069663,30.811763,30.737336],[0.281067,0.220021,0.217741],[8.89374,4.12692,4.131689],[10.38448,4.603694,4.571757],[19.980572,8.836322,8.892694],[59.786474,52.452881,39.941988],[2.804019,0.994794,0.958224],[0.765299,0.730434,0.723964],[2.784648,0.94665,0.936684],[8.905027,5.418438,5.386109],[12.187652,12.230066,12.164123],[3.35748,3.395991,3.319434],[4.309389,3.854977,3.772506],[9.958201,7.027432,6.888253],[50.200569,50.535126,50.283066],[24.469412,21.222713,21.010188],[26.115852,23.93507,24.835342],[7.511517,7.296179,7.324549],[2.156784,1.298258,1.278441],[2.155447,1.314499,1.331237],[2.007053,1.181676,1.155612],[null,null,null],[0.485363,0.420291,0.416819],[0.372131,0.322068,0.323578],[null,null,null]],"source":"databend/results/c6a.4xlarge.json"} From daae55b4ac5771c926d6212892d39e723d65c9e0 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 13 Jul 2022 01:41:27 +0000 Subject: [PATCH 538/627] Better --- src/Functions/translate.cpp | 133 ++++++++++++------ .../0_stateless/02353_translate.reference | 2 + tests/queries/0_stateless/02353_translate.sql | 6 + 3 files changed, 95 insertions(+), 46 deletions(-) diff --git a/src/Functions/translate.cpp b/src/Functions/translate.cpp index e509ea96a05..d2757c9f2d4 100644 --- a/src/Functions/translate.cpp +++ b/src/Functions/translate.cpp @@ -20,16 +20,17 @@ namespace ErrorCodes struct TranslateImpl { + using Map = std::array; + static void fillMapWithValues( - std::array & map, + Map & map, const std::string & map_from, const std::string & map_to) { if (map_from.size() != map_to.size()) - throw Exception("Second and trird arguments must be the same size", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Second and trird arguments must be the same length", ErrorCodes::BAD_ARGUMENTS); - for (size_t i = 0; i < 256; ++i) - map[i] = i; + std::iota(map.begin(), map.end(), 0); for (size_t i = 0; i < map_from.size(); ++i) { @@ -48,7 +49,7 @@ struct TranslateImpl ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { - std::array map; + Map map; fillMapWithValues(map, map_from, map_to); res_data.resize(data.size()); @@ -63,7 +64,10 @@ struct TranslateImpl while (src < src_end) { - *dst = map[*src]; + if (*src <= ascii_upper_bound) + *dst = map[*src]; + else + *dst = *src; ++src; ++dst; @@ -82,7 +86,7 @@ struct TranslateImpl const std::string & map_to, ColumnString::Chars & res_data) { - std::array map; + std::array map; fillMapWithValues(map, map_from, map_to); res_data.resize(data.size()); @@ -93,16 +97,73 @@ struct TranslateImpl while (src < src_end) { - *dst = map[*src]; + if (*src <= ascii_upper_bound) + *dst = map[*src]; + else + *dst = *src; ++src; ++dst; } } + +private: + static constexpr auto ascii_upper_bound = '\x7f'; }; struct TranslateUTF8Impl { + using MapASCII = std::array; + using MapUTF8 = HashMap>; + + static void fillMapWithValues( + MapASCII & map_ascii, + MapUTF8 & map, + const std::string & map_from, + const std::string & map_to) + { + auto map_from_size = UTF8::countCodePoints(reinterpret_cast(map_from.data()), map_from.size()); + auto map_to_size = UTF8::countCodePoints(reinterpret_cast(map_to.data()), map_to.size()); + + if (map_from_size != map_to_size) + throw Exception("Second and trird arguments must be the same length", ErrorCodes::BAD_ARGUMENTS); + + std::iota(map_ascii.begin(), map_ascii.end(), 0); + + const UInt8 * map_from_ptr = reinterpret_cast(map_from.data()); + const UInt8 * map_from_end = map_from_ptr + map_from.size(); + const UInt8 * map_to_ptr = reinterpret_cast(map_to.data()); + const UInt8 * map_to_end = map_to_ptr + map_to.size(); + + while (map_from_ptr < map_from_end && map_to_ptr < map_to_end) + { + size_t len_from = UTF8::seqLength(*map_from_ptr); + size_t len_to = UTF8::seqLength(*map_to_ptr); + + std::optional res_from, res_to; + + if (map_from_ptr + len_from <= map_from_end) + res_from = UTF8::convertUTF8ToCodePoint(map_from_ptr, len_from); + + if (map_to_ptr + len_to <= map_to_end) + res_to = UTF8::convertUTF8ToCodePoint(map_to_ptr, len_to); + + if (!res_from) + throw Exception("Second argument must be a valid UTF-8 string", ErrorCodes::BAD_ARGUMENTS); + + if (!res_to) + throw Exception("Third argument must be a valid UTF-8 string", ErrorCodes::BAD_ARGUMENTS); + + if (*map_from_ptr <= ascii_upper_bound) + map_ascii[*map_from_ptr] = *res_to; + else + map[*res_from] = *res_to; + + map_from_ptr += len_from; + map_to_ptr += len_to; + } + } + static void vector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, @@ -111,37 +172,9 @@ struct TranslateUTF8Impl ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { - auto map_from_size = UTF8::countCodePoints(reinterpret_cast(map_from.data()), map_from.size()); - auto map_to_size = UTF8::countCodePoints(reinterpret_cast(map_to.data()), map_to.size()); - - if (map_from_size != map_to_size) - throw Exception("Second and trird arguments must be the same size", ErrorCodes::BAD_ARGUMENTS); - - HashMap> map(map_from_size); - - const UInt8 * map_from_ptr = reinterpret_cast(map_from.data()); - const UInt8 * map_from_end = map_from_ptr + map_from.size(); - const UInt8 * map_to_ptr = reinterpret_cast(map_to.data()); - - while (map_from_ptr < map_from_end) - { - size_t len_a = UTF8::seqLength(*map_from_ptr); - auto res_a = UTF8::convertUTF8ToCodePoint(map_from_ptr, len_a); - - size_t len_b = UTF8::seqLength(*map_to_ptr); - auto res_b = UTF8::convertUTF8ToCodePoint(map_to_ptr, len_b); - - if (!res_a) - throw Exception("Second argument must be a valid UTF-8 string", ErrorCodes::BAD_ARGUMENTS); - - if (!res_b) - throw Exception("Third argument must be a valid UTF-8 string", ErrorCodes::BAD_ARGUMENTS); - - map[*res_a] = *res_b; - - map_from_ptr += len_a; - map_to_ptr += len_b; - } + MapASCII map_ascii; + MapUTF8 map; + fillMapWithValues(map_ascii, map, map_from, map_to); res_data.resize(data.size()); res_offsets.resize(offsets.size()); @@ -156,23 +189,34 @@ struct TranslateUTF8Impl while (src < src_end) { - /// Maximum length of UTF-8 sequence is 4 byte + 1 zero byte + /// Maximum length of UTF-8 sequence is 4 bytes + 1 zero byte if (data_size + 5 > res_data.size()) { res_data.resize(data_size * 2 + 5); dst = res_data.data() + data_size; } + if (*src <= ascii_upper_bound) + { + size_t dst_len = UTF8::convertCodePointToUTF8(map_ascii[*src], dst, 4); + assert(0 < dst_len && dst_len <= 4); + + src += 1; + dst += dst_len; + data_size += dst_len; + continue; + } + size_t src_len = UTF8::seqLength(*src); assert(0 < src_len && src_len <= 4); if (src + src_len <= src_end) { - auto res = UTF8::convertUTF8ToCodePoint(src, src_len); + auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_len); - if (res) + if (src_code_point) { - auto * it = map.find(*res); + auto * it = map.find(*src_code_point); if (it != map.end()) { size_t dst_len = UTF8::convertCodePointToUTF8(it->getMapped(), dst, 4); @@ -274,9 +318,6 @@ public: String map_from = c1_const->getValue(); String map_to = c2_const->getValue(); - if (map_from.empty()) - throw Exception("Length of the second argument of function " + getName() + " must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - if (const ColumnString * col = checkAndGetColumn(column_src.get())) { auto col_res = ColumnString::create(); diff --git a/tests/queries/0_stateless/02353_translate.reference b/tests/queries/0_stateless/02353_translate.reference index c4a06bb21c7..557b5182127 100644 --- a/tests/queries/0_stateless/02353_translate.reference +++ b/tests/queries/0_stateless/02353_translate.reference @@ -12,3 +12,5 @@ HotelGenev ¿йðՅনՅ ¿йðՅনð ¿йðՅনй +abc +abc diff --git a/tests/queries/0_stateless/02353_translate.sql b/tests/queries/0_stateless/02353_translate.sql index bc63cc71e69..a7059ec85a7 100644 --- a/tests/queries/0_stateless/02353_translate.sql +++ b/tests/queries/0_stateless/02353_translate.sql @@ -5,3 +5,9 @@ SELECT translate(toString(number), '0123456789', 'abcdefghij') FROM numbers(9876 SELECT translateUTF8('HôtelGenèv', 'Ááéíóúôè', 'aaeiouoe'); SELECT translateUTF8('中文内码', '久标准中文内码', 'ユニコードとは'); SELECT translateUTF8(toString(number), '1234567890', 'ዩय𐑿𐐏নՅðй¿ค') FROM numbers(987654, 5); + +SELECT translate('abc', '', ''); +SELECT translateUTF8('abc', '', ''); + +SELECT translate('abc', 'Ááéíóúôè', 'aaeiouoe'); -- { serverError 36 } +SELECT translateUTF8('abc', 'efg', ''); -- { serverError 36 } From 185978bfbef353d80a09737f2b011a435ae30bf1 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 13 Jul 2022 01:52:25 +0000 Subject: [PATCH 539/627] Add docs --- .../functions/string-replace-functions.md | 32 +++++++++++++++++++ .../functions/string-replace-functions.md | 31 ++++++++++++++++++ 2 files changed, 63 insertions(+) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 08fd979ce8f..3cd9bdf8ff5 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -93,3 +93,35 @@ Predefined characters: `\0`, `\\`, `|`, `(`, `)`, `^`, `$`, `.`, `[`, `]`, `?`, 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) + +## translate(s, from, to) + +The function replaces characters in the string ‘s’ in accordance with one-to-one character mapping defined by ‘from’ and ‘to’ strings. ‘from’ and ‘to’ must be ASCII strings of the same size. Non-ASCII characters in the original string are not modified. + +Example: + +``` sql +SELECT translate('Hello, World!', 'delor', 'DELOR') AS res +``` + +``` text +┌─res───────────┐ +│ HELLO, WORLD! │ +└───────────────┘ +``` + +## translateUTF8(string, from, to) + +Similar to previous function, but works with UTF-8 arguments. ‘from’ and ‘to’ must be valid UTF-8 strings of the same size. + +Example: + +``` sql +SELECT translateUTF8('Hélló, Wórld¡', 'óé¡', 'oe!') AS res +``` + +``` text +┌─res───────────┐ +│ Hello, World! │ +└───────────────┘ +``` diff --git a/docs/ru/sql-reference/functions/string-replace-functions.md b/docs/ru/sql-reference/functions/string-replace-functions.md index dbcdb2e902b..1bbb68f9014 100644 --- a/docs/ru/sql-reference/functions/string-replace-functions.md +++ b/docs/ru/sql-reference/functions/string-replace-functions.md @@ -83,3 +83,34 @@ SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res └─────────────────────┘ ``` +## translate(s, from, to) + +Данная функция заменяет символы в строке ‘s’ в соответствии с поэлементным отображением определяемым строками ‘from’ и ‘to’. ‘from’ и ‘to’ должны быть корректными ASCII строками одного размера. Не ASCII символы в оригинальной строке не изменяются. + +Example: + +``` sql +SELECT translate('Hello, World!', 'delor', 'DELOR') AS res +``` + +``` text +┌─res───────────┐ +│ HELLO, WORLD! │ +└───────────────┘ +``` + +## translateUTF8(string, from, to) + +Аналогично предыдущей функции, но работает со строками, состоящими из UTF-8 символов. ‘from’ и ‘to’ должны быть корректными UTF-8 строками одного размера. + +Example: + +``` sql +SELECT translateUTF8('Hélló, Wórld¡', 'óé¡', 'oe!') AS res +``` + +``` text +┌─res───────────┐ +│ Hello, World! │ +└───────────────┘ +``` From 0795cdd36a190bdc9aba2d4b7d7719481cbf438d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 13 Jul 2022 12:34:12 +0800 Subject: [PATCH 540/627] Slightly better interface of waitForMutation --- src/Storages/StorageMergeTree.cpp | 10 ++++++++-- src/Storages/StorageMergeTree.h | 1 + 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 64d929373fe..6825698f006 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -505,12 +505,18 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re void StorageMergeTree::waitForMutation(Int64 version) { - waitForMutation(MergeTreeMutationEntry::versionToFileName(version)); + String mutation_id = MergeTreeMutationEntry::versionToFileName(version); + waitForMutation(version, mutation_id); } void StorageMergeTree::waitForMutation(const String & mutation_id) { - UInt64 version = MergeTreeMutationEntry::parseFileName(mutation_id); + Int64 version = MergeTreeMutationEntry::parseFileName(mutation_id); + waitForMutation(version, mutation_id); +} + +void StorageMergeTree::waitForMutation(Int64 version, const String & mutation_id) +{ LOG_INFO(log, "Waiting mutation: {}", mutation_id); { auto check = [version, this]() diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index fb9478c5eca..8ca8ab5d11e 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -184,6 +184,7 @@ private: /// Wait until mutation with version will finish mutation for all parts void waitForMutation(Int64 version); void waitForMutation(const String & mutation_id) override; + void waitForMutation(Int64 version, const String & mutation_id); void setMutationCSN(const String & mutation_id, CSN csn) override; From 0595c50c924cfc9a0eaac49ad92c18bfe90c5ca6 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 13 Jul 2022 12:54:52 +0800 Subject: [PATCH 541/627] Fix trivial count with empty set --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + .../02356_trivial_count_with_empty_set.reference | 0 .../0_stateless/02356_trivial_count_with_empty_set.sql | 9 +++++++++ 3 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/02356_trivial_count_with_empty_set.reference create mode 100644 tests/queries/0_stateless/02356_trivial_count_with_empty_set.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 011dc11ae97..d8e10c8a173 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1933,6 +1933,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc syntax_analyzer_result->optimize_trivial_count && (settings.max_parallel_replicas <= 1) && !settings.allow_experimental_query_deduplication + && !settings.empty_result_for_aggregation_by_empty_set && storage && storage->getName() != "MaterializedMySQL" && !row_policy_filter diff --git a/tests/queries/0_stateless/02356_trivial_count_with_empty_set.reference b/tests/queries/0_stateless/02356_trivial_count_with_empty_set.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02356_trivial_count_with_empty_set.sql b/tests/queries/0_stateless/02356_trivial_count_with_empty_set.sql new file mode 100644 index 00000000000..89630d1aa4a --- /dev/null +++ b/tests/queries/0_stateless/02356_trivial_count_with_empty_set.sql @@ -0,0 +1,9 @@ +drop table if exists test; + +create table test(a Int64) Engine=MergeTree order by tuple(); + +set optimize_trivial_count_query=1, empty_result_for_aggregation_by_empty_set=1; + +select count() from test; + +drop table test; From 1f57f92bf775601436b74d1817ccf67d5c8fe0c2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 13 Jul 2022 03:01:23 -0400 Subject: [PATCH 542/627] prewhere action should preserve sorting columns --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b79dc9c80e9..e75fdcd6e3f 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1851,7 +1851,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { ExpressionActionsChain chain(context); - Names additional_required_columns_after_prewhere; + Names additional_required_columns_after_prewhere = metadata_snapshot->getColumnsRequiredForSortingKey(); if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { From 9973fb2f1de4e113cb0462175190c4aec84b3e21 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Jul 2022 11:19:14 +0200 Subject: [PATCH 543/627] Fix merge --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/S3/registerDiskS3.cpp | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index c07d56eaaf8..ee9111b7a43 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -253,7 +253,7 @@ void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exis Aws::S3::Model::DeleteObjectRequest request; request.SetBucket(bucket); - request.SetKey(path); + request.SetKey(object.absolute_path); auto outcome = client_ptr->DeleteObject(request); throwIfUnexpectedError(outcome, if_exists); diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 3f6cda24ce3..f946e9757c1 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -75,10 +75,10 @@ void checkRemoveAccess(IDisk & disk) bool checkBatchRemoveIsMissing(S3ObjectStorage & storage, const String & key_with_trailing_slash) { - const String path = key_with_trailing_slash + "_test_remove_objects_capability"; + StoredObject object(key_with_trailing_slash + "_test_remove_objects_capability"); try { - auto file = storage.writeObject(path, WriteMode::Rewrite); + auto file = storage.writeObject(object, WriteMode::Rewrite); file->write("test", 4); file->finalize(); } @@ -86,7 +86,7 @@ bool checkBatchRemoveIsMissing(S3ObjectStorage & storage, const String & key_wit { try { - storage.removeObject(path); + storage.removeObject(object); } catch (...) { @@ -96,14 +96,14 @@ bool checkBatchRemoveIsMissing(S3ObjectStorage & storage, const String & key_wit try { /// Uses `DeleteObjects` request (batch delete). - storage.removeObjects({{ path, 0 }}); + storage.removeObjects({object}); return false; } catch (const Exception &) { try { - storage.removeObject(path); + storage.removeObject(object); } catch (...) { From 1f0280bcde36f5eb7661dbe89111c79939dbc9df Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 13 Jul 2022 11:25:34 +0200 Subject: [PATCH 544/627] Fix path retrieval for Keeper's state (#39148) --- src/Coordination/CoordinationSettings.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 20e5a8d4463..34d69967828 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -225,6 +225,15 @@ String KeeperConfigurationAndSettings::getSnapshotsPathFromConfig(const Poco::Ut String KeeperConfigurationAndSettings::getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) { + if (config.has("keeper_server.storage_path")) + return std::filesystem::path{config.getString("keeper_server.storage_path")} / "state"; + + if (config.has("keeper_server.snapshot_storage_path")) + return std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path() / "state"; + + if (config.has("keeper_server.log_storage_path")) + return std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path() / "state"; + if (standalone_keeper_) return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "state"; else From ae324622d357d098292e54ec61c16d8d4094176e Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Jul 2022 11:27:43 +0200 Subject: [PATCH 545/627] Add one more test --- src/Storages/StorageMongoDB.cpp | 11 +++++--- .../integration/test_storage_mongodb/test.py | 26 +++++++++++++++++++ 2 files changed, 33 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index a33f17191eb..67cb7722414 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -94,8 +94,7 @@ public: const std::string & collection_name_, const std::string & db_name_, const StorageMetadataPtr & metadata_snapshot_, - std::shared_ptr connection_ - ) + std::shared_ptr connection_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) , collection_name(collection_name_) , db_name(db_name_) @@ -108,14 +107,18 @@ public: void consume(Chunk chunk) override { - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); Poco::MongoDB::Database db(db_name); Poco::MongoDB::Document::Ptr index = new Poco::MongoDB::Document(); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + + size_t num_rows = block.rows(); + size_t num_cols = block.columns(); + const auto columns = block.getColumns(); - const size_t num_rows = block.rows(), num_cols = block.columns(); const auto data_types = block.getDataTypes(); const auto data_names = block.getNames(); + std::vector row(num_cols); for (const auto i : collections::range(0, num_rows)) { diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index d8ca207d0a6..1fc3e255a5b 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -253,3 +253,29 @@ def test_missing_columns(started_cluster): result = node.query("SELECT count() FROM simple_mongo_table WHERE isNull(data)") assert result == "10\n" simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_simple_insert_select(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + + node = started_cluster.instances["node"] + node.query( + "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" + ) + node.query("INSERT INTO simple_mongo_table SELECT 1, 'kek'") + + assert ( + node.query("SELECT data from simple_mongo_table where key = 1").strip() == "kek" + ) + node.query("INSERT INTO simple_mongo_table(key) SELECT 12") + assert int(node.query("SELECT count() from simple_mongo_table")) == 2 + assert ( + node.query("SELECT data from simple_mongo_table where key = 12").strip() == "" + ) + + node.query("DROP TABLE simple_mongo_table") + simple_mongo_table.drop() From d6c145e5e1aecc60dadf7e792af6d668dc88da0a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Jul 2022 11:48:17 +0200 Subject: [PATCH 546/627] Update CachedReadBufferFromRemoteFS.cpp --- src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index 5d629a75259..b929cea0236 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -707,7 +707,7 @@ bool CachedReadBufferFromRemoteFS::writeCache(char * data, size_t size, size_t o int code = e.getErrno(); if (code == /* No space left on device */28 || code == /* Quota exceeded */122) { - LOG_INFO(log, "Not enough disk space to write cache, will skip cache download. ({})", e.displayText()); + LOG_INFO(log, "Insert into cache is skipped due to insufficient disk space. ({})", e.displayText()); return false; } throw; From 48cb1ed9094a7ff2e2cba5be57197e07e1707fb8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Jul 2022 11:48:39 +0200 Subject: [PATCH 547/627] Update test.py --- tests/integration/test_merge_tree_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 64d70322a9c..5c3885851fb 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -694,6 +694,6 @@ def test_cache_with_full_disk_space(cluster, node_name): "SELECT * FROM s3_test WHERE value LIKE '%abc%' ORDER BY value FORMAT Null" ) assert node.contains_in_log( - "Not enough disk space to write cache, will skip cache download" + "Insert into cache is skipped due to insufficient disk space" ) node.query("DROP TABLE IF EXISTS s3_test NO DELAY") From 90fa4863f9beabe094474e4d8516ddc1b0eac518 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 13 Jul 2022 16:15:37 +0800 Subject: [PATCH 548/627] Text log source regexp --- src/Core/Settings.h | 1 + src/Interpreters/InternalTextLogsQueue.cpp | 15 +++++++++++++++ src/Interpreters/InternalTextLogsQueue.h | 9 ++++++++- src/Loggers/OwnSplitChannel.cpp | 4 ++-- src/Server/GRPCServer.cpp | 1 + src/Server/TCPHandler.cpp | 1 + .../02359_send_logs_source_regexp.reference | 1 + .../0_stateless/02359_send_logs_source_regexp.sh | 11 +++++++++++ 8 files changed, 40 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02359_send_logs_source_regexp.reference create mode 100755 tests/queries/0_stateless/02359_send_logs_source_regexp.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5597d9076a4..0257c8e01e4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -391,6 +391,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, log_query_views, true, "Log query dependent views into system.query_views_log table. This setting have effect only when 'log_queries' is true.", 0) \ M(String, log_comment, "", "Log comment into system.query_log table and server log. It can be set to arbitrary string no longer than max_query_size.", 0) \ M(LogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \ + M(String, send_logs_source_regexp, "", "Send server text logs with specified regexp to match log source name. Empty means all sources.", 0) \ M(Bool, enable_optimize_predicate_expression, true, "If it is set to true, optimize predicates to subqueries.", 0) \ M(Bool, enable_optimize_predicate_expression_to_final_subquery, true, "Allow push predicate to final subquery.", 0) \ M(Bool, allow_push_predicate_when_subquery_contains_with, true, "Allows push predicate when subquery contains WITH clause", 0) \ diff --git a/src/Interpreters/InternalTextLogsQueue.cpp b/src/Interpreters/InternalTextLogsQueue.cpp index 2172a6f4261..8e689069cec 100644 --- a/src/Interpreters/InternalTextLogsQueue.cpp +++ b/src/Interpreters/InternalTextLogsQueue.cpp @@ -66,4 +66,19 @@ const char * InternalTextLogsQueue::getPriorityName(int priority) return (priority >= 1 && priority <= 8) ? PRIORITIES[priority] : PRIORITIES[0]; } +bool InternalTextLogsQueue::isNeeded(int priority, const String & source) const +{ + bool is_needed = priority <= max_priority; + + if (is_needed && source_regexp) + is_needed = re2::RE2::PartialMatch(source, *source_regexp); + + return is_needed; +} + +void InternalTextLogsQueue::setSourceRegexp(const String & regexp) +{ + source_regexp = std::make_unique(regexp); +} + } diff --git a/src/Interpreters/InternalTextLogsQueue.h b/src/Interpreters/InternalTextLogsQueue.h index 53710fa3bd2..9c3c9f9e707 100644 --- a/src/Interpreters/InternalTextLogsQueue.h +++ b/src/Interpreters/InternalTextLogsQueue.h @@ -2,7 +2,7 @@ #include #include #include - +#include namespace DB { @@ -15,6 +15,8 @@ public: InternalTextLogsQueue(); + bool isNeeded(int priority, const String & source) const; + static Block getSampleBlock(); static MutableColumns getSampleColumns(); @@ -23,6 +25,11 @@ public: /// Converts priority from Poco::Message::Priority to a string static const char * getPriorityName(int priority); + + void setSourceRegexp(const String & regexp); +private: + /// If not null, you should only push logs which are matched with this regexp + std::unique_ptr source_regexp; }; using InternalTextLogsQueuePtr = std::shared_ptr; diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 71be8007d85..355b733b624 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -24,7 +24,7 @@ void OwnSplitChannel::log(const Poco::Message & msg) #ifdef WITH_TEXT_LOG auto logs_queue = CurrentThread::getInternalTextLogsQueue(); - if (channels.empty() && (logs_queue == nullptr || msg.getPriority() > logs_queue->max_priority)) + if (channels.empty() && (logs_queue == nullptr || !logs_queue->isNeeded(msg.getPriority(), msg.getSource()))) return; #endif @@ -93,7 +93,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) auto logs_queue = CurrentThread::getInternalTextLogsQueue(); /// Log to "TCP queue" if message is not too noisy - if (logs_queue && msg.getPriority() <= logs_queue->max_priority) + if (logs_queue && logs_queue->isNeeded(msg.getPriority(), msg.getSource())) { MutableColumns columns = InternalTextLogsQueue::getSampleColumns(); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index e1c73b7ebbb..4178d0d62da 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -848,6 +848,7 @@ namespace { logs_queue = std::make_shared(); logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString()); + logs_queue->setSourceRegexp(settings.send_logs_source_regexp); CurrentThread::attachInternalTextLogsQueue(logs_queue, client_logs_level); CurrentThread::setFatalErrorCallback([this]{ onFatalError(); }); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d1d762d3c61..05565063893 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -241,6 +241,7 @@ void TCPHandler::runImpl() { state.logs_queue = std::make_shared(); state.logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString()); + state.logs_queue->setSourceRegexp(query_context->getSettingsRef().send_logs_source_regexp); CurrentThread::attachInternalTextLogsQueue(state.logs_queue, client_logs_level); CurrentThread::setFatalErrorCallback([this] { diff --git a/tests/queries/0_stateless/02359_send_logs_source_regexp.reference b/tests/queries/0_stateless/02359_send_logs_source_regexp.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02359_send_logs_source_regexp.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02359_send_logs_source_regexp.sh b/tests/queries/0_stateless/02359_send_logs_source_regexp.sh new file mode 100755 index 00000000000..d3b60bc59f4 --- /dev/null +++ b/tests/queries/0_stateless/02359_send_logs_source_regexp.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +[ ! -z "$CLICKHOUSE_CLIENT_REDEFINED" ] && CLICKHOUSE_CLIENT=$CLICKHOUSE_CLIENT_REDEFINED + +CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') +regexp="executeQuery|InterpreterSelectQuery" +$CLICKHOUSE_CLIENT --send_logs_source_regexp "$regexp" -q "SELECT 1;" 2> >(grep -v -E "$regexp" 1>&2) From 411a08630c1bb83252ff3a424315b4681edde653 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Wed, 13 Jul 2022 10:29:13 +0000 Subject: [PATCH 549/627] add docs --- .../functions/array-functions.md | 4 + .../functions/distance-functions.md | 502 ++++++++++++++++++ .../functions/tuple-functions.md | 499 +---------------- 3 files changed, 508 insertions(+), 497 deletions(-) create mode 100644 docs/en/sql-reference/functions/distance-functions.md diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 53c2e929d15..50a394e45fd 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1715,3 +1715,7 @@ Return value type is always [Float64](../../sql-reference/data-types/float.md). │ 6 │ Float64 │ └─────┴──────────────────────────────────────────────────────────────────────────────────────────┘ ``` + +## Distance functions + +All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). diff --git a/docs/en/sql-reference/functions/distance-functions.md b/docs/en/sql-reference/functions/distance-functions.md new file mode 100644 index 00000000000..4af264f27ca --- /dev/null +++ b/docs/en/sql-reference/functions/distance-functions.md @@ -0,0 +1,502 @@ +# Distance functions + +## L1Norm + +Calculates the sum of absolute values of a vector. + +**Syntax** + +```sql +L1Norm(vector) +``` + +Alias: `normL1`. + +**Arguments** + +- `vector` — [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). + +**Returned value** + +- L1-norm or [taxicab geometry](https://en.wikipedia.org/wiki/Taxicab_geometry) distance. + +Type: [UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). + +**Examples** + +Query: + +```sql +SELECT L1Norm((1, 2)); +``` + +Result: + +```text +┌─L1Norm((1, 2))─┐ +│ 3 │ +└────────────────┘ +``` + +## L2Norm + +Calculates the square root of the sum of the squares of the vector values. + +**Syntax** + +```sql +L2Norm(vector) +``` + +Alias: `normL2`. + +**Arguments** + +- `vector` — [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). + +**Returned value** + +- L2-norm or [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance). + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT L2Norm((1, 2)); +``` + +Result: + +```text +┌───L2Norm((1, 2))─┐ +│ 2.23606797749979 │ +└──────────────────┘ +``` + +## LinfNorm + +Calculates the maximum of absolute values of a vector. + +**Syntax** + +```sql +LinfNorm(vector) +``` + +Alias: `normLinf`. + +**Arguments** + +- `vector` — [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). + +**Returned value** + +- Linf-norm or the maximum absolute value. + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT LinfNorm((1, -2)); +``` + +Result: + +```text +┌─LinfNorm((1, -2))─┐ +│ 2 │ +└───────────────────┘ +``` + +## LpNorm + +Calculates the root of `p`-th power of the sum of the absolute values of a vector in the power of `p`. + +**Syntax** + +```sql +LpNorm(vector, p) +``` + +Alias: `normLp`. + +**Arguments** + +- `vector` — [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). +- `p` — The power. Possible values: real number in `[1; inf)`. [UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). + +**Returned value** + +- [Lp-norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#p-norm) + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT LpNorm((1, -2), 2); +``` + +Result: + +```text +┌─LpNorm((1, -2), 2)─┐ +│ 2.23606797749979 │ +└────────────────────┘ +``` + +## L1Distance + +Calculates the distance between two points (the values of the vectors are the coordinates) in `L1` space (1-norm ([taxicab geometry](https://en.wikipedia.org/wiki/Taxicab_geometry) distance)). + +**Syntax** + +```sql +L1Distance(vector1, vector2) +``` + +Alias: `distanceL1`. + +**Arguments** + +- `vector1` — First vector. [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). +- `vector2` — Second vector. [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). + +**Returned value** + +- 1-norm distance. + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT L1Distance((1, 2), (2, 3)); +``` + +Result: + +```text +┌─L1Distance((1, 2), (2, 3))─┐ +│ 2 │ +└────────────────────────────┘ +``` + +## L2Distance + +Calculates the distance between two points (the values of the vectors are the coordinates) in Euclidean space ([Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance)). + +**Syntax** + +```sql +L2Distance(vector1, vector2) +``` + +Alias: `distanceL2`. + +**Arguments** + +- `vector1` — First vector. [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). +- `vector2` — Second vector. [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). + +**Returned value** + +- 2-norm distance. + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT L2Distance((1, 2), (2, 3)); +``` + +Result: + +```text +┌─L2Distance((1, 2), (2, 3))─┐ +│ 1.4142135623730951 │ +└────────────────────────────┘ +``` + +## LinfDistance + +Calculates the distance between two points (the values of the vectors are the coordinates) in `L_{inf}` space ([maximum norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#Maximum_norm_(special_case_of:_infinity_norm,_uniform_norm,_or_supremum_norm))). + +**Syntax** + +```sql +LinfDistance(vector1, vector2) +``` + +Alias: `distanceLinf`. + +**Arguments** + +- `vector1` — First vector. [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). +- `vector1` — Second vector. [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). + +**Returned value** + +- Infinity-norm distance. + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT LinfDistance((1, 2), (2, 3)); +``` + +Result: + +```text +┌─LinfDistance((1, 2), (2, 3))─┐ +│ 1 │ +└──────────────────────────────┘ +``` + +## LpDistance + +Calculates the distance between two points (the values of the vectors are the coordinates) in `Lp` space ([p-norm distance](https://en.wikipedia.org/wiki/Norm_(mathematics)#p-norm)). + +**Syntax** + +```sql +LpDistance(vector1, vector2, p) +``` + +Alias: `distanceLp`. + +**Arguments** + +- `vector1` — First vector. [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). +- `vector2` — Second vector. [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). +- `p` — The power. Possible values: real number from `[1; inf)`. [UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). + +**Returned value** + +- p-norm distance. + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT LpDistance((1, 2), (2, 3), 3); +``` + +Result: + +```text +┌─LpDistance((1, 2), (2, 3), 3)─┐ +│ 1.2599210498948732 │ +└───────────────────────────────┘ +``` + + +## L1Normalize + +Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in `L1` space ([taxicab geometry](https://en.wikipedia.org/wiki/Taxicab_geometry)). + +**Syntax** + +```sql +L1Normalize(tuple) +``` + +Alias: `normalizeL1`. + +**Arguments** + +- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). + +**Returned value** + +- Unit vector. + +Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT L1Normalize((1, 2)); +``` + +Result: + +```text +┌─L1Normalize((1, 2))─────────────────────┐ +│ (0.3333333333333333,0.6666666666666666) │ +└─────────────────────────────────────────┘ +``` + +## L2Normalize + +Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in Euclidean space (using [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance)). + +**Syntax** + +```sql +L2Normalize(tuple) +``` + +Alias: `normalizeL1`. + +**Arguments** + +- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). + +**Returned value** + +- Unit vector. + +Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT L2Normalize((3, 4)); +``` + +Result: + +```text +┌─L2Normalize((3, 4))─┐ +│ (0.6,0.8) │ +└─────────────────────┘ +``` + +## LinfNormalize + +Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in `L_{inf}` space (using [maximum norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#Maximum_norm_(special_case_of:_infinity_norm,_uniform_norm,_or_supremum_norm))). + +**Syntax** + +```sql +LinfNormalize(tuple) +``` + +Alias: `normalizeLinf `. + +**Arguments** + +- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). + +**Returned value** + +- Unit vector. + +Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT LinfNormalize((3, 4)); +``` + +Result: + +```text +┌─LinfNormalize((3, 4))─┐ +│ (0.75,1) │ +└───────────────────────┘ +``` + +## LpNormalize + +Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in `Lp` space (using [p-norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#p-norm)). + +**Syntax** + +```sql +LpNormalize(tuple, p) +``` + +Alias: `normalizeLp `. + +**Arguments** + +- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). +- `p` — The power. Possible values: any number from [1;inf). [UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). + +**Returned value** + +- Unit vector. + +Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT LpNormalize((3, 4),5); +``` + +Result: + +```text +┌─LpNormalize((3, 4), 5)──────────────────┐ +│ (0.7187302630182624,0.9583070173576831) │ +└─────────────────────────────────────────┘ +``` + +## cosineDistance + +Calculates the cosine distance between two vectors (the values of the tuples are the coordinates). The less the returned value is, the more similar are the vectors. + +**Syntax** + +```sql +cosineDistance(tuple1, tuple2) +``` + +**Arguments** + +- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). +- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). + +**Returned value** + +- Cosine of the angle between two vectors substracted from one. + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT cosineDistance((1, 2), (2, 3)); +``` + +Result: + +```text +┌─cosineDistance((1, 2), (2, 3))─┐ +│ 0.007722123286332261 │ +└────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 64bdf9b9f33..d3aac635841 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -559,502 +559,7 @@ Result: └────────────────────────────┘ ``` -## L1Norm -Calculates the sum of absolute values of a tuple. +## Distance functions -**Syntax** - -```sql -L1Norm(tuple) -``` - -Alias: `normL1`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- L1-norm or [taxicab geometry](https://en.wikipedia.org/wiki/Taxicab_geometry) distance. - -Type: [UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). - -**Example** - -Query: - -```sql -SELECT L1Norm((1, 2)); -``` - -Result: - -```text -┌─L1Norm((1, 2))─┐ -│ 3 │ -└────────────────┘ -``` - -## L2Norm - -Calculates the square root of the sum of the squares of the tuple values. - -**Syntax** - -```sql -L2Norm(tuple) -``` - -Alias: `normL2`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- L2-norm or [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance). - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L2Norm((1, 2)); -``` - -Result: - -```text -┌───L2Norm((1, 2))─┐ -│ 2.23606797749979 │ -└──────────────────┘ -``` - -## LinfNorm - -Calculates the maximum of absolute values of a tuple. - -**Syntax** - -```sql -LinfNorm(tuple) -``` - -Alias: `normLinf`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Linf-norm or the maximum absolute value. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LinfNorm((1, -2)); -``` - -Result: - -```text -┌─LinfNorm((1, -2))─┐ -│ 2 │ -└───────────────────┘ -``` - -## LpNorm - -Calculates the root of `p`-th power of the sum of the absolute values of a tuple in the power of `p`. - -**Syntax** - -```sql -LpNorm(tuple, p) -``` - -Alias: `normLp`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). -- `p` — The power. Possible values: real number in `[1; inf)`. [UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). - -**Returned value** - -- [Lp-norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#p-norm) - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LpNorm((1, -2), 2); -``` - -Result: - -```text -┌─LpNorm((1, -2), 2)─┐ -│ 2.23606797749979 │ -└────────────────────┘ -``` - -## L1Distance - -Calculates the distance between two points (the values of the tuples are the coordinates) in `L1` space (1-norm ([taxicab geometry](https://en.wikipedia.org/wiki/Taxicab_geometry) distance)). - -**Syntax** - -```sql -L1Distance(tuple1, tuple2) -``` - -Alias: `distanceL1`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple1` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- 1-norm distance. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L1Distance((1, 2), (2, 3)); -``` - -Result: - -```text -┌─L1Distance((1, 2), (2, 3))─┐ -│ 2 │ -└────────────────────────────┘ -``` - -## L2Distance - -Calculates the distance between two points (the values of the tuples are the coordinates) in Euclidean space ([Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance)). - -**Syntax** - -```sql -L2Distance(tuple1, tuple2) -``` - -Alias: `distanceL2`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple1` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- 2-norm distance. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L2Distance((1, 2), (2, 3)); -``` - -Result: - -```text -┌─L2Distance((1, 2), (2, 3))─┐ -│ 1.4142135623730951 │ -└────────────────────────────┘ -``` - -## LinfDistance - -Calculates the distance between two points (the values of the tuples are the coordinates) in `L_{inf}` space ([maximum norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#Maximum_norm_(special_case_of:_infinity_norm,_uniform_norm,_or_supremum_norm))). - -**Syntax** - -```sql -LinfDistance(tuple1, tuple2) -``` - -Alias: `distanceLinf`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple1` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Infinity-norm distance. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LinfDistance((1, 2), (2, 3)); -``` - -Result: - -```text -┌─LinfDistance((1, 2), (2, 3))─┐ -│ 1 │ -└──────────────────────────────┘ -``` - -## LpDistance - -Calculates the distance between two points (the values of the tuples are the coordinates) in `Lp` space ([p-norm distance](https://en.wikipedia.org/wiki/Norm_(mathematics)#p-norm)). - -**Syntax** - -```sql -LpDistance(tuple1, tuple2, p) -``` - -Alias: `distanceLp`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple1` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `p` — The power. Possible values: real number from `[1; inf)`. [UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). - -**Returned value** - -- p-norm distance. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LpDistance((1, 2), (2, 3), 3); -``` - -Result: - -```text -┌─LpDistance((1, 2), (2, 3), 3)─┐ -│ 1.2599210498948732 │ -└───────────────────────────────┘ -``` - -## L1Normalize - -Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in `L1` space ([taxicab geometry](https://en.wikipedia.org/wiki/Taxicab_geometry)). - -**Syntax** - -```sql -L1Normalize(tuple) -``` - -Alias: `normalizeL1`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Unit vector. - -Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L1Normalize((1, 2)); -``` - -Result: - -```text -┌─L1Normalize((1, 2))─────────────────────┐ -│ (0.3333333333333333,0.6666666666666666) │ -└─────────────────────────────────────────┘ -``` - -## L2Normalize - -Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in Euclidean space (using [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance)). - -**Syntax** - -```sql -L2Normalize(tuple) -``` - -Alias: `normalizeL1`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Unit vector. - -Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L2Normalize((3, 4)); -``` - -Result: - -```text -┌─L2Normalize((3, 4))─┐ -│ (0.6,0.8) │ -└─────────────────────┘ -``` - -## LinfNormalize - -Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in `L_{inf}` space (using [maximum norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#Maximum_norm_(special_case_of:_infinity_norm,_uniform_norm,_or_supremum_norm))). - -**Syntax** - -```sql -LinfNormalize(tuple) -``` - -Alias: `normalizeLinf `. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Unit vector. - -Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LinfNormalize((3, 4)); -``` - -Result: - -```text -┌─LinfNormalize((3, 4))─┐ -│ (0.75,1) │ -└───────────────────────┘ -``` - -## LpNormalize - -Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in `Lp` space (using [p-norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#p-norm)). - -**Syntax** - -```sql -LpNormalize(tuple, p) -``` - -Alias: `normalizeLp `. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). -- `p` — The power. Possible values: any number from [1;inf). [UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). - -**Returned value** - -- Unit vector. - -Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LpNormalize((3, 4),5); -``` - -Result: - -```text -┌─LpNormalize((3, 4), 5)──────────────────┐ -│ (0.7187302630182624,0.9583070173576831) │ -└─────────────────────────────────────────┘ -``` - -## cosineDistance - -Calculates the cosine distance between two vectors (the values of the tuples are the coordinates). The less the returned value is, the more similar are the vectors. - -**Syntax** - -```sql -cosineDistance(tuple1, tuple2) -``` - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Cosine of the angle between two vectors substracted from one. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT cosineDistance((1, 2), (2, 3)); -``` - -Result: - -```text -┌─cosineDistance((1, 2), (2, 3))─┐ -│ 0.007722123286332261 │ -└────────────────────────────────┘ -``` +All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). From d886aea351e4ba6f9a9d3a5173cd12ae9b81984d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 13 Jul 2022 11:31:24 +0000 Subject: [PATCH 550/627] Add setting opentelemetry_trace_processors --- src/Core/Settings.h | 1 + .../Executors/ExecutionThreadContext.cpp | 23 +++++++++++-------- .../Executors/ExecutionThreadContext.h | 4 +++- src/Processors/Executors/ExecutorTasks.cpp | 4 ++-- src/Processors/Executors/ExecutorTasks.h | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 6 +++-- src/Processors/Executors/PipelineExecutor.h | 2 ++ 7 files changed, 27 insertions(+), 15 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5597d9076a4..a5f46a22eb7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -286,6 +286,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \ M(String, metrics_perf_events_list, "", "Comma separated list of perf metrics that will be measured throughout queries' execution. Empty means all events. See PerfEventInfo in sources for the available events.", 0) \ M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \ + M(Bool, opentelemetry_trace_processors, false, "Collect OpenTelemetry spans for processors.", 0) \ M(Bool, prefer_column_name_to_alias, false, "Prefer using column names instead of aliases if possible.", 0) \ M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \ \ diff --git a/src/Processors/Executors/ExecutionThreadContext.cpp b/src/Processors/Executors/ExecutionThreadContext.cpp index 5a5c1826c61..7631cb09f61 100644 --- a/src/Processors/Executors/ExecutionThreadContext.cpp +++ b/src/Processors/Executors/ExecutionThreadContext.cpp @@ -71,7 +71,13 @@ static void executeJob(ExecutingGraph::Node * node, ReadProgressCallback * read_ bool ExecutionThreadContext::executeTask() { - OpenTelemetrySpanHolder span("ExecutionThreadContext::executeTask() " + node->processor->getName()); + std::unique_ptr span; + + if (trace_processors) + { + span = std::make_unique("ExecutionThreadContext::executeTask() " + node->processor->getName()); + span->addAttribute("thread_number", thread_number); + } std::optional execution_time_watch; #ifndef NDEBUG @@ -93,17 +99,16 @@ bool ExecutionThreadContext::executeTask() if (profile_processors) { - UInt64 elapsed_microseconds = execution_time_watch->elapsedMicroseconds(); - node->processor->elapsed_us += elapsed_microseconds; - span.addAttribute("execution_time_ms", elapsed_microseconds); - } + UInt64 elapsed_microseconds = execution_time_watch->elapsedMicroseconds(); + node->processor->elapsed_us += elapsed_microseconds; + if (trace_processors) + span->addAttribute("execution_time_ms", elapsed_microseconds); + } #ifndef NDEBUG execution_time_ns += execution_time_watch->elapsed(); - span.addAttribute("execution_time_ns", execution_time_watch->elapsed()); + if (trace_processors) + span->addAttribute("execution_time_ns", execution_time_watch->elapsed()); #endif - - span.addAttribute("thread_number", thread_number); - return node->exception == nullptr; } diff --git a/src/Processors/Executors/ExecutionThreadContext.h b/src/Processors/Executors/ExecutionThreadContext.h index f0341333117..eb048f8ab09 100644 --- a/src/Processors/Executors/ExecutionThreadContext.h +++ b/src/Processors/Executors/ExecutionThreadContext.h @@ -41,6 +41,7 @@ public: const size_t thread_number; const bool profile_processors; + const bool trace_processors; void wait(std::atomic_bool & finished); void wakeUp(); @@ -61,10 +62,11 @@ public: void setException(std::exception_ptr exception_) { exception = exception_; } void rethrowExceptionIfHas(); - explicit ExecutionThreadContext(size_t thread_number_, bool profile_processors_, ReadProgressCallback * callback) + explicit ExecutionThreadContext(size_t thread_number_, bool profile_processors_, bool trace_processors_, ReadProgressCallback * callback) : read_progress_callback(callback) , thread_number(thread_number_) , profile_processors(profile_processors_) + , trace_processors(trace_processors_) {} }; diff --git a/src/Processors/Executors/ExecutorTasks.cpp b/src/Processors/Executors/ExecutorTasks.cpp index f2287e467dc..824b4e962d2 100644 --- a/src/Processors/Executors/ExecutorTasks.cpp +++ b/src/Processors/Executors/ExecutorTasks.cpp @@ -128,7 +128,7 @@ void ExecutorTasks::pushTasks(Queue & queue, Queue & async_queue, ExecutionThrea } } -void ExecutorTasks::init(size_t num_threads_, bool profile_processors, ReadProgressCallback * callback) +void ExecutorTasks::init(size_t num_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback) { num_threads = num_threads_; threads_queue.init(num_threads); @@ -139,7 +139,7 @@ void ExecutorTasks::init(size_t num_threads_, bool profile_processors, ReadProgr executor_contexts.reserve(num_threads); for (size_t i = 0; i < num_threads; ++i) - executor_contexts.emplace_back(std::make_unique(i, profile_processors, callback)); + executor_contexts.emplace_back(std::make_unique(i, profile_processors, trace_processors, callback)); } } diff --git a/src/Processors/Executors/ExecutorTasks.h b/src/Processors/Executors/ExecutorTasks.h index caff1a35d98..668470e7b11 100644 --- a/src/Processors/Executors/ExecutorTasks.h +++ b/src/Processors/Executors/ExecutorTasks.h @@ -54,7 +54,7 @@ public: void tryGetTask(ExecutionThreadContext & context); void pushTasks(Queue & queue, Queue & async_queue, ExecutionThreadContext & context); - void init(size_t num_threads_, bool profile_processors, ReadProgressCallback * callback); + void init(size_t num_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback); void fill(Queue & queue); void processAsyncTasks(); diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index cccd08b2273..68225d73ff1 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -29,8 +29,10 @@ PipelineExecutor::PipelineExecutor(Processors & processors, QueryStatus * elem) : process_list_element(elem) { if (process_list_element) + { profile_processors = process_list_element->getContext()->getSettingsRef().log_processors_profiles; - + trace_processors = process_list_element->getContext()->getSettingsRef().opentelemetry_trace_processors; + } try { graph = std::make_unique(processors, profile_processors); @@ -268,7 +270,7 @@ void PipelineExecutor::initializeExecution(size_t num_threads) Queue queue; graph->initializeExecution(queue); - tasks.init(num_threads, profile_processors, read_progress_callback.get()); + tasks.init(num_threads, profile_processors, trace_processors, read_progress_callback.get()); tasks.fill(queue); } diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index 80ba21a8adf..c4d11ef688d 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -65,6 +65,8 @@ private: bool is_execution_initialized = false; /// system.processors_profile_log bool profile_processors = false; + /// system.opentelemetry_span_log + bool trace_processors = false; std::atomic_bool cancelled = false; From bddf6c1b3249fddb54c4569e3edc4b09777ce7c9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 12 Jul 2022 13:19:28 +0000 Subject: [PATCH 551/627] Pushdown filter to the right side of sorting join --- src/Interpreters/FullSortingMergeJoin.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 8 +-- src/Processors/QueryPlan/JoinStep.cpp | 22 ++++-- src/Processors/QueryPlan/JoinStep.h | 3 +- .../Optimizations/filterPushDown.cpp | 68 ++++++++++++------- 5 files changed, 66 insertions(+), 37 deletions(-) diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 90ec67f54b1..3ee6ce1c1fb 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -95,7 +95,7 @@ public: } /// Left and right streams have the same priority and are processed simultaneously - virtual JoinPipelineType pipelineType() const override { return JoinPipelineType::YShaped; } + JoinPipelineType pipelineType() const override { return JoinPipelineType::YShaped; } private: std::shared_ptr table_join; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d8e10c8a173..ac31588d210 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1350,7 +1350,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

context->getTemporaryVolume(), settings.min_free_disk_space_for_temporary_data); - sorting_step->setStepDescription("Sort before JOIN"); + sorting_step->setStepDescription(fmt::format("Sort {} before JOIN", is_right ? "right" : "left")); plan.addStep(std::move(sorting_step)); }; if (expressions.join->pipelineType() == JoinPipelineType::YShaped) { const auto & join_clause = expressions.join->getTableJoin().getOnlyClause(); - add_sorting(query_plan, join_clause.key_names_left); - add_sorting(*joined_plan, join_clause.key_names_right); + add_sorting(query_plan, join_clause.key_names_left, false); + add_sorting(*joined_plan, join_clause.key_names_right, true); } QueryPlanStepPtr join_step = std::make_unique( diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index d7ccf003937..909933fbed2 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -48,18 +48,30 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines &processors); } +bool JoinStep::allowPushDownToRight() const +{ + return join->pipelineType() == JoinPipelineType::YShaped; +} + void JoinStep::describePipeline(FormatSettings & settings) const { IQueryPlanStep::describePipeline(processors, settings); } -void JoinStep::updateLeftStream(const DataStream & left_stream_) +void JoinStep::updateInputStream(const DataStream & new_input_stream_, size_t idx) { - input_streams = {left_stream_, input_streams.at(1)}; - output_stream = DataStream + if (idx == 0) { - .header = JoiningTransform::transformHeader(left_stream_.header, join), - }; + input_streams = {new_input_stream_, input_streams.at(1)}; + output_stream = DataStream + { + .header = JoiningTransform::transformHeader(new_input_stream_.header, join), + }; + } + else + { + input_streams = {input_streams.at(0), new_input_stream_}; + } } static ITransformingStep::Traits getStorageJoinTraits() diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 1ea0e9b366d..fc7f74d4fe8 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -28,8 +28,9 @@ public: void describePipeline(FormatSettings & settings) const override; const JoinPtr & getJoin() const { return join; } + bool allowPushDownToRight() const; - void updateLeftStream(const DataStream & left_stream_); + void updateInputStream(const DataStream & new_input_stream_, size_t idx); private: JoinPtr join; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 0c17c27e7aa..e9aa7ae5e12 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -39,7 +39,8 @@ static bool filterColumnIsNotAmongAggregatesArguments(const AggregateDescription } static size_t -tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const Names & allowed_inputs, bool can_remove_filter = true) +tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const Names & allowed_inputs, + bool can_remove_filter = true, size_t child_idx = 0) { QueryPlan::Node * child_node = parent_node->children.front(); @@ -53,7 +54,10 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con // std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; - const auto & all_inputs = child->getInputStreams().front().header.getColumnsWithTypeAndName(); + if (child_idx >= child->getInputStreams().size() || child_idx >= child_node->children.size()) + return 0; + + const auto & all_inputs = child->getInputStreams()[child_idx].header.getColumnsWithTypeAndName(); auto split_filter = expression->cloneActionsForFilterPushDown(filter_column_name, removes_filter, allowed_inputs, all_inputs); if (!split_filter) @@ -75,7 +79,8 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con /// Expression/Filter -> Aggregating -> Something auto & node = nodes.emplace_back(); node.children.emplace_back(&node); - std::swap(node.children[0], child_node->children[0]); + + std::swap(node.children[0], child_node->children[child_idx]); /// Expression/Filter -> Aggregating -> Filter -> Something /// New filter column is the first one. @@ -90,7 +95,9 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con else { if (auto * join = typeid_cast(child.get())) - join->updateLeftStream(node.step->getOutputStream()); + { + join->updateInputStream(node.step->getOutputStream(), child_idx); + } else throw Exception( ErrorCodes::LOGICAL_ERROR, "We are trying to push down a filter through a step for which we cannot update input stream"); @@ -208,34 +215,43 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * join = typeid_cast(child.get())) { - const auto & table_join = join->getJoin()->getTableJoin(); - /// Push down is for left table only. We need to update JoinStep for push down into right. - /// Only inner and left join are supported. Other types may generate default values for left table keys. + const auto & table_join = join->getJoin()->getTableJoin(); + std::vector kinds = {ASTTableJoin::Kind::Left}; + + /// For not full sorting merge join push down is for left table only, because left and right streams are not independent. + /// Only inner and left(/right) join are supported. Other types may generate default values for left table keys. /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (table_join.kind() == ASTTableJoin::Kind::Inner || table_join.kind() == ASTTableJoin::Kind::Left) + if (join->allowPushDownToRight()) + kinds.emplace_back(ASTTableJoin::Kind::Right); + + for (const auto kind : kinds) { - const auto & left_header = join->getInputStreams().front().header; - const auto & res_header = join->getOutputStream().header; - Names allowed_keys; - const auto & source_columns = left_header.getNames(); - for (const auto & name : source_columns) + if (table_join.kind() == ASTTableJoin::Kind::Inner || table_join.kind() == kind) { - /// Skip key if it is renamed. - /// I don't know if it is possible. Just in case. - if (!left_header.has(name) || !res_header.has(name)) - continue; + const auto & streams = join->getInputStreams(); + const auto & input_header = kind == ASTTableJoin::Kind::Left ? streams.front().header : streams.back().header; + const auto & res_header = join->getOutputStream().header; + Names allowed_keys; + const auto & source_columns = input_header.getNames(); + for (const auto & name : source_columns) + { + /// Skip key if it is renamed. + /// I don't know if it is possible. Just in case. + if (!input_header.has(name) || !res_header.has(name)) + continue; - /// Skip if type is changed. Push down expression expect equal types. - if (!left_header.getByName(name).type->equals(*res_header.getByName(name).type)) - continue; + /// Skip if type is changed. Push down expression expect equal types. + if (!input_header.getByName(name).type->equals(*res_header.getByName(name).type)) + continue; - allowed_keys.push_back(name); + allowed_keys.push_back(name); + } + + const bool can_remove_filter + = std::find(source_columns.begin(), source_columns.end(), filter->getFilterColumnName()) == source_columns.end(); + if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_keys, can_remove_filter, kind == ASTTableJoin::Kind::Left ? 0 : 1)) + return updated_steps; } - - const bool can_remove_filter - = std::find(source_columns.begin(), source_columns.end(), filter->getFilterColumnName()) == source_columns.end(); - if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_keys, can_remove_filter)) - return updated_steps; } } From c344e2de848cdb5695972ebf93257570022c86ca Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 13 Jul 2022 11:41:08 +0000 Subject: [PATCH 552/627] Fix style --- src/Functions/translate.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/translate.cpp b/src/Functions/translate.cpp index d2757c9f2d4..8342bfe236b 100644 --- a/src/Functions/translate.cpp +++ b/src/Functions/translate.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; - extern const int ARGUMENT_OUT_OF_BOUND; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; } From fa59133463bbdb44ed860c6e607829d7cad9b166 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 13 Jul 2022 11:51:11 +0000 Subject: [PATCH 553/627] Do not spam log in MergeJoinAlgorithm --- src/Processors/Transforms/MergeJoinTransform.cpp | 2 +- src/Processors/Transforms/MergeJoinTransform.h | 15 +-------------- 2 files changed, 2 insertions(+), 15 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 690f751209f..c7b7afab541 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -855,7 +855,7 @@ MergeJoinTransform::MergeJoinTransform( void MergeJoinTransform::onFinish() { - algorithm.logElapsed(total_stopwatch.elapsedSeconds(), true); + algorithm.logElapsed(total_stopwatch.elapsedSeconds()); } } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 0098e470e6f..9f60eafb455 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -233,19 +233,14 @@ public: virtual void consume(Input & input, size_t source_num) override; virtual Status merge() override; - void logElapsed(double seconds, bool force) + void logElapsed(double seconds) { - /// Do not log more frequently than once per ten seconds - if (seconds - stat.last_log_seconds < 10 && !force) - return; - LOG_TRACE(log, "Finished pocessing in {} seconds" ", left: {} blocks, {} rows; right: {} blocks, {} rows" ", max blocks loaded to memory: {}", seconds, stat.num_blocks[0], stat.num_rows[0], stat.num_blocks[1], stat.num_rows[1], stat.max_blocks_loaded); - stat.last_log_seconds = seconds; } private: @@ -277,8 +272,6 @@ private: size_t num_rows[2] = {0, 0}; size_t max_blocks_loaded = 0; - - double last_log_seconds = 0; }; Statistic stat; @@ -303,12 +296,6 @@ public: protected: void onFinish() override; - void work() override - { - algorithm.logElapsed(total_stopwatch.elapsedSeconds(), true); - Base::work(); - } - Poco::Logger * log; }; From 549a85fee922f3f62cfde582dd530e4967bb6f84 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 13 Jul 2022 11:53:46 +0000 Subject: [PATCH 554/627] Throw logical error on child idx mismatch in tryAddNewFilterStep --- src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index e9aa7ae5e12..959402bf582 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -55,7 +55,8 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con // std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; if (child_idx >= child->getInputStreams().size() || child_idx >= child_node->children.size()) - return 0; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Child index {} is out of range (streams: {}, children: {})", + child_idx, child->getInputStreams().size(), child_node->children.size()); const auto & all_inputs = child->getInputStreams()[child_idx].header.getColumnsWithTypeAndName(); From 37f0d96e36f03e04d5e7a3b3b65709e7cae023b0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jul 2022 12:04:42 +0000 Subject: [PATCH 555/627] Add test for #39132 --- .../0_stateless/02357_query_cancellation_race.reference | 0 .../queries/0_stateless/02357_query_cancellation_race.sh | 9 +++++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02357_query_cancellation_race.reference create mode 100755 tests/queries/0_stateless/02357_query_cancellation_race.sh diff --git a/tests/queries/0_stateless/02357_query_cancellation_race.reference b/tests/queries/0_stateless/02357_query_cancellation_race.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02357_query_cancellation_race.sh b/tests/queries/0_stateless/02357_query_cancellation_race.sh new file mode 100755 index 00000000000..585e57947d0 --- /dev/null +++ b/tests/queries/0_stateless/02357_query_cancellation_race.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: race + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "create table tab (x UInt64, y String) engine = MergeTree order by x" +for i in $(seq 1 100); do timeout -s 2 0.05 $CLICKHOUSE_CLIENT --interactive_delay 1000 -q "insert into tab select number, toString(number) from system.numbers" || true; done From 4124dc9ac412aa20d7eba343d8418e451885024d Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 13 Jul 2022 12:06:29 +0000 Subject: [PATCH 556/627] Rewrite tryPushDownFilter for join with lambda --- .../Optimizations/filterPushDown.cpp | 85 +++++++++++-------- 1 file changed, 49 insertions(+), 36 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 959402bf582..680d158ecaf 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -1,3 +1,10 @@ +#include + +#include +#include + +#include + #include #include #include @@ -11,13 +18,10 @@ #include #include #include + #include #include #include -#include -#include - -#include namespace DB::ErrorCodes { @@ -216,43 +220,52 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * join = typeid_cast(child.get())) { - const auto & table_join = join->getJoin()->getTableJoin(); - std::vector kinds = {ASTTableJoin::Kind::Left}; - - /// For not full sorting merge join push down is for left table only, because left and right streams are not independent. - /// Only inner and left(/right) join are supported. Other types may generate default values for left table keys. - /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (join->allowPushDownToRight()) - kinds.emplace_back(ASTTableJoin::Kind::Right); - - for (const auto kind : kinds) + auto join_push_down = [&](ASTTableJoin::Kind kind) -> size_t { - if (table_join.kind() == ASTTableJoin::Kind::Inner || table_join.kind() == kind) + const auto & table_join = join->getJoin()->getTableJoin(); + + /// Only inner and left(/right) join are supported. Other types may generate default values for left table keys. + /// So, if we push down a condition like `key != 0`, not all rows may be filtered. + if (table_join.kind() != ASTTableJoin::Kind::Inner && table_join.kind() != kind) + return 0; + + bool is_left = kind == ASTTableJoin::Kind::Left; + const auto & input_header = is_left ? join->getInputStreams().front().header : join->getInputStreams().back().header; + const auto & res_header = join->getOutputStream().header; + Names allowed_keys; + const auto & source_columns = input_header.getNames(); + for (const auto & name : source_columns) { - const auto & streams = join->getInputStreams(); - const auto & input_header = kind == ASTTableJoin::Kind::Left ? streams.front().header : streams.back().header; - const auto & res_header = join->getOutputStream().header; - Names allowed_keys; - const auto & source_columns = input_header.getNames(); - for (const auto & name : source_columns) - { - /// Skip key if it is renamed. - /// I don't know if it is possible. Just in case. - if (!input_header.has(name) || !res_header.has(name)) - continue; + /// Skip key if it is renamed. + /// I don't know if it is possible. Just in case. + if (!input_header.has(name) || !res_header.has(name)) + continue; - /// Skip if type is changed. Push down expression expect equal types. - if (!input_header.getByName(name).type->equals(*res_header.getByName(name).type)) - continue; + /// Skip if type is changed. Push down expression expect equal types. + if (!input_header.getByName(name).type->equals(*res_header.getByName(name).type)) + continue; - allowed_keys.push_back(name); - } - - const bool can_remove_filter - = std::find(source_columns.begin(), source_columns.end(), filter->getFilterColumnName()) == source_columns.end(); - if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_keys, can_remove_filter, kind == ASTTableJoin::Kind::Left ? 0 : 1)) - return updated_steps; + allowed_keys.push_back(name); } + + const bool can_remove_filter + = std::find(source_columns.begin(), source_columns.end(), filter->getFilterColumnName()) == source_columns.end(); + size_t updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_keys, can_remove_filter, is_left ? 0 : 1); + if (updated_steps > 0) + { + LOG_DEBUG(&Poco::Logger::get("tryPushDownFilter"), "Pushed down filter to {} side of join", kind); + } + return updated_steps; + }; + + if (size_t updated_steps = join_push_down(ASTTableJoin::Kind::Left)) + return updated_steps; + + /// For full sorting merge join we push down both to the left and right tables, because left and right streams are not independent. + if (join->allowPushDownToRight()) + { + if (size_t updated_steps = join_push_down(ASTTableJoin::Kind::Right)) + return updated_steps; } } From 8122c1b07fd95736e8d6dbef71f78a3c9e525b32 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Jul 2022 11:44:53 +0300 Subject: [PATCH 557/627] Optimize accesses to system.stack_trace - filter by thread_id - filter by thread_name - filter by requested columns (query_id/trace, thread_name) Signed-off-by: Azat Khuzhin --- .../System/StorageSystemStackTrace.cpp | 236 +++++++++++++----- src/Storages/System/StorageSystemStackTrace.h | 20 +- .../01051_system_stack_trace.reference | 14 ++ .../0_stateless/01051_system_stack_trace.sql | 10 +- 4 files changed, 205 insertions(+), 75 deletions(-) diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index 5d37aa5b08a..4eef69d8634 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -5,10 +5,14 @@ #include #include +#include #include #include +#include +#include +#include #include #include #include @@ -16,8 +20,11 @@ #include #include #include -#include +#include #include +#include +#include +#include namespace DB @@ -147,13 +154,82 @@ namespace throw Exception("Logical error: read wrong number of bytes from pipe", ErrorCodes::LOGICAL_ERROR); } } + + ColumnPtr getFilteredThreadIds(ASTPtr query, ContextPtr context) + { + MutableColumnPtr all_thread_ids = ColumnUInt64::create(); + + std::filesystem::directory_iterator end; + + /// There is no better way to enumerate threads in a process other than looking into procfs. + for (std::filesystem::directory_iterator it("/proc/self/task"); it != end; ++it) + { + pid_t tid = parse(it->path().filename()); + all_thread_ids->insert(tid); + } + + Block block { ColumnWithTypeAndName(std::move(all_thread_ids), std::make_shared(), "thread_id") }; + VirtualColumnUtils::filterBlockWithQuery(query, block, context); + return block.getByPosition(0).column; + } + + using ThreadIdToName = std::unordered_map>; + ThreadIdToName getFilteredThreadNames(ASTPtr query, ContextPtr context, const PaddedPODArray & thread_ids) + { + ThreadIdToName tid_to_name; + MutableColumnPtr all_thread_names = ColumnString::create(); + + for (UInt64 tid : thread_ids) + { + std::filesystem::path thread_name_path = fmt::format("/proc/self/task/{}/comm", tid); + String thread_name; + if (std::filesystem::exists(thread_name_path)) + { + constexpr size_t comm_buf_size = 32; /// More than enough for thread name + ReadBufferFromFile comm(thread_name_path.string(), comm_buf_size); + readEscapedStringUntilEOL(thread_name, comm); + comm.close(); + } + + tid_to_name[tid] = thread_name; + all_thread_names->insert(thread_name); + } + + Block block { ColumnWithTypeAndName(std::move(all_thread_names), std::make_shared(), "thread_name") }; + VirtualColumnUtils::filterBlockWithQuery(query, block, context); + ColumnPtr thread_names = std::move(block.getByPosition(0).column); + + std::unordered_set filtered_thread_names; + for (size_t i = 0; i != thread_names->size(); ++i) + { + const auto & thread_name = thread_names->getDataAt(i); + filtered_thread_names.emplace(thread_name); + } + + for (const auto & [tid, name] : tid_to_name) + { + if (!filtered_thread_names.contains(name)) + tid_to_name.erase(tid); + } + + return tid_to_name; + } } StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_) - : IStorageSystemOneBlock(table_id_) + : IStorage(table_id_) , log(&Poco::Logger::get("StorageSystemStackTrace")) { + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription({ + { "thread_name", std::make_shared() }, + { "thread_id", std::make_shared() }, + { "query_id", std::make_shared() }, + { "trace", std::make_shared(std::make_shared()) }, + }, { /* aliases */ })); + setInMemoryMetadata(storage_metadata); + notification_pipe.open(); /// Setup signal handler. @@ -173,23 +249,40 @@ StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_) } -NamesAndTypesList StorageSystemStackTrace::getNamesAndTypes() +Pipe StorageSystemStackTrace::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) { - return - { - { "thread_name", std::make_shared() }, - { "thread_id", std::make_shared() }, - { "query_id", std::make_shared() }, - { "trace", std::make_shared(std::make_shared()) } - }; -} + storage_snapshot->check(column_names); - -void StorageSystemStackTrace::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const -{ /// It shouldn't be possible to do concurrent reads from this table. std::lock_guard lock(mutex); + /// Create a mask of what columns are needed in the result. + + NameSet names_set(column_names.begin(), column_names.end()); + + Block sample_block = storage_snapshot->metadata->getSampleBlock(); + + std::vector columns_mask(sample_block.columns()); + for (size_t i = 0, size = columns_mask.size(); i < size; ++i) + { + if (names_set.contains(sample_block.getByPosition(i).name)) + { + columns_mask[i] = 1; + } + } + + bool send_signal = names_set.contains("trace") || names_set.contains("query_id"); + bool read_thread_names = names_set.contains("thread_name"); + + MutableColumns res_columns = sample_block.cloneEmptyColumns(); + /// Send a signal to every thread and wait for result. /// We must wait for every thread one by one sequentially, /// because there is a limit on number of queued signals in OS and otherwise signals may get lost. @@ -197,71 +290,80 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, ContextPtr, /// Obviously, results for different threads may be out of sync. - /// There is no better way to enumerate threads in a process other than looking into procfs. + ColumnPtr thread_ids = getFilteredThreadIds(query_info.query, context); + const auto & thread_ids_data = assert_cast(*thread_ids).getData(); - std::filesystem::directory_iterator end; - for (std::filesystem::directory_iterator it("/proc/self/task"); it != end; ++it) + ThreadIdToName thread_names; + if (read_thread_names) + thread_names = getFilteredThreadNames(query_info.query, context, thread_ids_data); + + for (UInt64 tid : thread_ids_data) { - pid_t tid = parse(it->path().filename()); - - sigval sig_value{}; - sig_value.sival_int = sequence_num.load(std::memory_order_acquire); - if (0 != ::sigqueue(tid, sig, sig_value)) - { - /// The thread may has been already finished. - if (ESRCH == errno) - continue; - - throwFromErrno("Cannot send signal with sigqueue", ErrorCodes::CANNOT_SIGQUEUE); - } - - std::filesystem::path thread_name_path = it->path(); - thread_name_path.append("comm"); + size_t res_index = 0; String thread_name; - if (std::filesystem::exists(thread_name_path)) + if (auto it = thread_names.find(tid); it != thread_names.end()) + thread_name = it->second; + + if (!send_signal) { - constexpr size_t comm_buf_size = 32; /// More than enough for thread name - ReadBufferFromFile comm(thread_name_path.string(), comm_buf_size); - readEscapedStringUntilEOL(thread_name, comm); - comm.close(); - } - - /// Just in case we will wait for pipe with timeout. In case signal didn't get processed. - - if (wait(100) && sig_value.sival_int == data_ready_num.load(std::memory_order_acquire)) - { - size_t stack_trace_size = stack_trace.getSize(); - size_t stack_trace_offset = stack_trace.getOffset(); - - Array arr; - arr.reserve(stack_trace_size - stack_trace_offset); - for (size_t i = stack_trace_offset; i < stack_trace_size; ++i) - arr.emplace_back(reinterpret_cast(stack_trace.getFramePointers()[i])); - - res_columns[0]->insert(thread_name); - res_columns[1]->insert(tid); - res_columns[2]->insertData(query_id_data, query_id_size); - res_columns[3]->insert(arr); + res_columns[res_index++]->insert(thread_name); + res_columns[res_index++]->insert(tid); + res_columns[res_index++]->insertDefault(); + res_columns[res_index++]->insertDefault(); } else { - LOG_DEBUG(log, "Cannot obtain a stack trace for thread {}", tid); + sigval sig_value{}; - /// Cannot obtain a stack trace. But create a record in result nevertheless. + sig_value.sival_int = sequence_num.load(std::memory_order_acquire); + if (0 != ::sigqueue(tid, sig, sig_value)) + { + /// The thread may has been already finished. + if (ESRCH == errno) + continue; - res_columns[0]->insert(thread_name); - res_columns[1]->insert(tid); - res_columns[2]->insertDefault(); - res_columns[3]->insertDefault(); + throwFromErrno("Cannot send signal with sigqueue", ErrorCodes::CANNOT_SIGQUEUE); + } + + /// Just in case we will wait for pipe with timeout. In case signal didn't get processed. + if (send_signal && wait(100) && sig_value.sival_int == data_ready_num.load(std::memory_order_acquire)) + { + size_t stack_trace_size = stack_trace.getSize(); + size_t stack_trace_offset = stack_trace.getOffset(); + + Array arr; + arr.reserve(stack_trace_size - stack_trace_offset); + for (size_t i = stack_trace_offset; i < stack_trace_size; ++i) + arr.emplace_back(reinterpret_cast(stack_trace.getFramePointers()[i])); + + res_columns[res_index++]->insert(thread_name); + res_columns[res_index++]->insert(tid); + res_columns[res_index++]->insertData(query_id_data, query_id_size); + res_columns[res_index++]->insert(arr); + } + else + { + LOG_DEBUG(log, "Cannot obtain a stack trace for thread {}", tid); + + res_columns[res_index++]->insert(thread_name); + res_columns[res_index++]->insert(tid); + res_columns[res_index++]->insertDefault(); + res_columns[res_index++]->insertDefault(); + } + + /// Signed integer overflow is undefined behavior in both C and C++. However, according to + /// C++ standard, Atomic signed integer arithmetic is defined to use two's complement; there + /// are no undefined results. See https://en.cppreference.com/w/cpp/atomic/atomic and + /// http://eel.is/c++draft/atomics.types.generic#atomics.types.int-8 + ++sequence_num; } - - /// Signed integer overflow is undefined behavior in both C and C++. However, according to - /// C++ standard, Atomic signed integer arithmetic is defined to use two's complement; there - /// are no undefined results. See https://en.cppreference.com/w/cpp/atomic/atomic and - /// http://eel.is/c++draft/atomics.types.generic#atomics.types.int-8 - ++sequence_num; } + + UInt64 num_rows = res_columns.at(0)->size(); + Chunk chunk(std::move(res_columns), num_rows); + + return Pipe(std::make_shared(sample_block, std::move(chunk))); } } diff --git a/src/Storages/System/StorageSystemStackTrace.h b/src/Storages/System/StorageSystemStackTrace.h index c039ae53170..dd613882e49 100644 --- a/src/Storages/System/StorageSystemStackTrace.h +++ b/src/Storages/System/StorageSystemStackTrace.h @@ -3,7 +3,7 @@ #ifdef OS_LINUX /// Because of 'sigqueue' functions and RT signals. #include -#include +#include namespace Poco { @@ -19,20 +19,26 @@ class Context; /// Allows to introspect stack trace of all server threads. /// It acts like an embedded debugger. /// More than one instance of this table cannot be used. -class StorageSystemStackTrace final : public IStorageSystemOneBlock +class StorageSystemStackTrace final : public IStorage { public: explicit StorageSystemStackTrace(const StorageID & table_id_); String getName() const override { return "SystemStackTrace"; } - static NamesAndTypesList getNamesAndTypes(); + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + bool isSystemStorage() const override { return true; } protected: - using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; - mutable std::mutex mutex; - Poco::Logger * log; }; diff --git a/tests/queries/0_stateless/01051_system_stack_trace.reference b/tests/queries/0_stateless/01051_system_stack_trace.reference index d00491fd7e5..ab01c749f41 100644 --- a/tests/queries/0_stateless/01051_system_stack_trace.reference +++ b/tests/queries/0_stateless/01051_system_stack_trace.reference @@ -1 +1,15 @@ +-- { echo } +SELECT count() > 0 FROM system.stack_trace WHERE query_id != ''; +1 +-- opimization for not reading /proc/self/task/{}/comm and avoid sending signal +SELECT countIf(thread_id > 0) > 0 FROM system.stack_trace; +1 +-- optimization for trace +SELECT length(trace) > 0 FROM system.stack_trace LIMIT 1; +1 +-- optimization for query_id +SELECT length(query_id) > 0 FROM system.stack_trace WHERE query_id != '' LIMIT 1; +1 +-- optimization for thread_name +SELECT length(thread_name) > 0 FROM system.stack_trace WHERE thread_name != '' LIMIT 1; 1 diff --git a/tests/queries/0_stateless/01051_system_stack_trace.sql b/tests/queries/0_stateless/01051_system_stack_trace.sql index e495e2198ea..c712a1367c0 100644 --- a/tests/queries/0_stateless/01051_system_stack_trace.sql +++ b/tests/queries/0_stateless/01051_system_stack_trace.sql @@ -1,4 +1,12 @@ -- Tags: race --- at least this query should be present +-- { echo } SELECT count() > 0 FROM system.stack_trace WHERE query_id != ''; +-- opimization for not reading /proc/self/task/{}/comm and avoid sending signal +SELECT countIf(thread_id > 0) > 0 FROM system.stack_trace; +-- optimization for trace +SELECT length(trace) > 0 FROM system.stack_trace LIMIT 1; +-- optimization for query_id +SELECT length(query_id) > 0 FROM system.stack_trace WHERE query_id != '' LIMIT 1; +-- optimization for thread_name +SELECT length(thread_name) > 0 FROM system.stack_trace WHERE thread_name != '' LIMIT 1; From 991493e6fc227789ad922f9bd1ebd7fec0a6cd2e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 13 Jul 2022 15:22:36 +0300 Subject: [PATCH 558/627] Update run.sh --- docker/test/stress/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index fcf99b34064..cb80c8ebdc9 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -353,6 +353,7 @@ else # Error messages (we should ignore some errors) # FIXME https://github.com/ClickHouse/ClickHouse/issues/38643 ("Unknown index: idx.") + # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64") echo "Check for Error messages in server log:" zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ -e "Code: 236. DB::Exception: Cancelled mutating parts" \ @@ -376,6 +377,7 @@ else -e "found in queue and some source parts for it was lost" \ -e "is lost forever." \ -e "Unknown index: idx." \ + -e "Cannot parse string 'Hello' as UInt64" \ /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv From 83c3670f2aafaeaa0a3462a6ac7f35ecaede478f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jul 2022 12:23:01 +0000 Subject: [PATCH 559/627] Fix style --- tests/queries/0_stateless/02357_query_cancellation_race.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02357_query_cancellation_race.sh b/tests/queries/0_stateless/02357_query_cancellation_race.sh index 585e57947d0..6b20e050ce3 100755 --- a/tests/queries/0_stateless/02357_query_cancellation_race.sh +++ b/tests/queries/0_stateless/02357_query_cancellation_race.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "create table tab (x UInt64, y String) engine = MergeTree order by x" -for i in $(seq 1 100); do timeout -s 2 0.05 $CLICKHOUSE_CLIENT --interactive_delay 1000 -q "insert into tab select number, toString(number) from system.numbers" || true; done +for _ in $(seq 1 100); do timeout -s 2 0.05 $CLICKHOUSE_CLIENT --interactive_delay 1000 -q "insert into tab select number, toString(number) from system.numbers" || true; done From 3473b8007789e3a990c9a18e69f7f4365cbb0cae Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Jul 2022 10:03:48 +0300 Subject: [PATCH 560/627] ThreadPool: add some clarification comments Signed-off-by: Azat Khuzhin --- src/Common/ThreadPool.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 4bd1cf391ef..ba14dc78952 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -165,7 +165,9 @@ public: : state(std::make_shared()) , thread_id(std::make_shared()) { - /// NOTE: If this will throw an exception, the destructor won't be called. + /// NOTE: + /// - If this will throw an exception, the destructor won't be called + /// - this pointer cannot be passed in the lambda, since after detach() it will not be valid GlobalThreadPool::instance().scheduleOrThrow([ thread_id = thread_id, state = state, From 25eb82f1208ca804917e54ceeafb313e52e6cc4f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Jul 2022 10:01:31 +0300 Subject: [PATCH 561/627] ThreadPool: do not use joinable() internally joinable() should be used only outside, since internally it is enough to know `state` to know that something is wrong. Signed-off-by: Azat Khuzhin --- src/Common/ThreadPool.h | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index ba14dc78952..9aa353971cf 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -198,7 +198,7 @@ public: ThreadFromGlobalPool & operator=(ThreadFromGlobalPool && rhs) noexcept { - if (joinable()) + if (initialized()) abort(); state = std::move(rhs.state); thread_id = std::move(rhs.thread_id); @@ -207,13 +207,13 @@ public: ~ThreadFromGlobalPool() { - if (joinable()) + if (initialized()) abort(); } void join() { - if (!joinable()) + if (!initialized()) abort(); state->wait(); @@ -222,7 +222,7 @@ public: void detach() { - if (!joinable()) + if (!initialized()) abort(); state.reset(); } @@ -241,6 +241,14 @@ private: /// The state used in this object and inside the thread job. std::shared_ptr state; std::shared_ptr thread_id; + + /// Internally initialized() should be used over joinable(), + /// since it is enough to know that the thread is initialized, + /// and ignore that fact that thread cannot join itself. + bool initialized() const + { + return static_cast(state); + } }; From 7ffa15b94b6363c63d3956083e3ff63d357d6988 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Jul 2022 11:03:19 +0300 Subject: [PATCH 562/627] ThreadPool: fix thread_id assignment As found by @KochetovNicolai before this patch, lambda in ThreadFromGlobalPool() ctor assigns value only to a copy of the thread_id value, and so check in joinable() had been working incorrectly, fix this by changing the value not the shared_ptr itself. Also it is not safe to assign thread_id w/o atomics, since this can be racy, so wrap id with std::atomic<> Fixes: #28431 Signed-off-by: Azat Khuzhin --- src/Common/ThreadPool.h | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 9aa353971cf..15e6dc9a2b5 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -163,7 +163,7 @@ public: template explicit ThreadFromGlobalPool(Function && func, Args &&... args) : state(std::make_shared()) - , thread_id(std::make_shared()) + , thread_id(std::make_shared()) { /// NOTE: /// - If this will throw an exception, the destructor won't be called @@ -177,7 +177,7 @@ public: auto event = std::move(state); SCOPE_EXIT(event->set()); - thread_id = std::make_shared(std::this_thread::get_id()); + thread_id->id = std::this_thread::get_id(); /// This moves are needed to destroy function and arguments before exit. /// It will guarantee that after ThreadFromGlobalPool::join all captured params are destroyed. @@ -232,7 +232,7 @@ public: if (!state) return false; /// Thread cannot join itself. - if (*thread_id == std::this_thread::get_id()) + if (thread_id->id == std::this_thread::get_id()) return false; return true; } @@ -240,7 +240,14 @@ public: private: /// The state used in this object and inside the thread job. std::shared_ptr state; - std::shared_ptr thread_id; + + struct ThreadIdHolder + { + /// Should be atomic() because of possible concurrent access between + /// assignment and joinable() check. + std::atomic id; + }; + std::shared_ptr thread_id; /// Internally initialized() should be used over joinable(), /// since it is enough to know that the thread is initialized, From e9dcc7d05a0511e18a81353ed15aa280eaaa0aef Mon Sep 17 00:00:00 2001 From: HeenaBansal2009 Date: Wed, 13 Jul 2022 05:49:03 -0700 Subject: [PATCH 563/627] try to fix flaky test as suggested in review comments --- .../02297_regex_parsing_file_names.sh | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/02297_regex_parsing_file_names.sh b/tests/queries/0_stateless/02297_regex_parsing_file_names.sh index 605fde2d0ba..12ccb54235b 100755 --- a/tests/queries/0_stateless/02297_regex_parsing_file_names.sh +++ b/tests/queries/0_stateless/02297_regex_parsing_file_names.sh @@ -13,26 +13,28 @@ CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/ -rm -rf ${CLICKHOUSE_USER_FILES_PATH}/file_{0..10}.json +rm -rf ${CLICKHOUSE_USER_FILES_PATH}/file_{0..10}.csv -echo '{"obj": "aaa", "id": 1, "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_0.json -echo '{"id": 2, "obj": "bbb", "s": "bar"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_1.json -echo '{"id": 3, "obj": "ccc", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_2.json -echo '{"id": 4, "obj": "ddd", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_3.json -echo '{"id": 5, "obj": "eee", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_4.json -echo '{"id": 6, "obj": "fff", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_5.json -echo '{"id": 7, "obj": "ggg", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_6.json -echo '{"id": 8, "obj": "hhh", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_7.json -echo '{"id": 9, "obj": "iii", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_8.json -echo '{"id": 10, "obj":"jjj", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_9.json -echo '{"id": 11, "obj": "kkk", "s": "foo"}' > ${CLICKHOUSE_USER_FILES_PATH}/file_10.json +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_0.csv +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_1.csv +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_2.csv +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_3.csv +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_4.csv +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_5.csv +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_6.csv +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_7.csv +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_8.csv +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_9.csv +echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_10.csv + +# echo '' > ${CLICKHOUSE_USER_FILES_PATH}/file_10.csv ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_regex;" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_regex (id UInt64, obj String, s String) ENGINE = MergeTree() order by id;" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_regex (id UInt64) ENGINE = MergeTree() order by id;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO t_regex SELECT * FROM file('file_{0..10}.json','JSONEachRow');" +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_regex SELECT * FROM file('file_{0..10}.csv','CSV');" ${CLICKHOUSE_CLIENT} -q "SELECT count() from t_regex;" -rm -rf ${CLICKHOUSE_USER_FILES_PATH}/file_{0..10}.json; +rm -rf ${CLICKHOUSE_USER_FILES_PATH}/file_{0..10}.csv; ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_regex;" From 68789895a46e492f899d9d997786511919b1e931 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Jul 2022 15:49:13 +0300 Subject: [PATCH 564/627] ThreadPool: incapsulate Poco::Event and std::thread::id into State struct Signed-off-by: Azat Khuzhin --- src/Common/ThreadPool.h | 26 +++++++++++--------------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 15e6dc9a2b5..eb3f631b92a 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -162,22 +162,19 @@ public: template explicit ThreadFromGlobalPool(Function && func, Args &&... args) - : state(std::make_shared()) - , thread_id(std::make_shared()) + : state(std::make_shared()) { /// NOTE: /// - If this will throw an exception, the destructor won't be called /// - this pointer cannot be passed in the lambda, since after detach() it will not be valid GlobalThreadPool::instance().scheduleOrThrow([ - thread_id = thread_id, state = state, func = std::forward(func), args = std::make_tuple(std::forward(args)...)]() mutable /// mutable is needed to destroy capture { - auto event = std::move(state); - SCOPE_EXIT(event->set()); + SCOPE_EXIT(state->event.set()); - thread_id->id = std::this_thread::get_id(); + state->thread_id = std::this_thread::get_id(); /// This moves are needed to destroy function and arguments before exit. /// It will guarantee that after ThreadFromGlobalPool::join all captured params are destroyed. @@ -201,7 +198,6 @@ public: if (initialized()) abort(); state = std::move(rhs.state); - thread_id = std::move(rhs.thread_id); return *this; } @@ -216,7 +212,7 @@ public: if (!initialized()) abort(); - state->wait(); + state->event.wait(); state.reset(); } @@ -232,22 +228,22 @@ public: if (!state) return false; /// Thread cannot join itself. - if (thread_id->id == std::this_thread::get_id()) + if (state->thread_id == std::this_thread::get_id()) return false; return true; } private: - /// The state used in this object and inside the thread job. - std::shared_ptr state; - - struct ThreadIdHolder + struct State { /// Should be atomic() because of possible concurrent access between /// assignment and joinable() check. - std::atomic id; + std::atomic thread_id; + + /// The state used in this object and inside the thread job. + Poco::Event event; }; - std::shared_ptr thread_id; + std::shared_ptr state; /// Internally initialized() should be used over joinable(), /// since it is enough to know that the thread is initialized, From 87197f8f607aceaa46d8e4527969885eafa73e66 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Jul 2022 13:49:40 +0000 Subject: [PATCH 565/627] Enable -Wdeprecated-dynamic-exception-spec --- base/base/JSON.cpp | 7 +++++++ base/base/JSON.h | 8 +++++++- cmake/warnings.cmake | 1 - src/Common/mysqlxx/mysqlxx/Exception.h | 20 ++++++++++---------- 4 files changed, 24 insertions(+), 12 deletions(-) diff --git a/base/base/JSON.cpp b/base/base/JSON.cpp index 54074f20f16..a01063ee426 100644 --- a/base/base/JSON.cpp +++ b/base/base/JSON.cpp @@ -12,7 +12,14 @@ #define JSON_MAX_DEPTH 100 +#ifdef __clang__ +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wdeprecated-dynamic-exception-spec" +#endif POCO_IMPLEMENT_EXCEPTION(JSONException, Poco::Exception, "JSONException") // NOLINT(cert-err60-cpp, modernize-use-noexcept, hicpp-use-noexcept) +#ifdef __clang__ +# pragma clang diagnostic pop +#endif /// Прочитать беззнаковое целое в простом формате из не-0-terminated строки. diff --git a/base/base/JSON.h b/base/base/JSON.h index 5965b2b17a9..3226a0d09e8 100644 --- a/base/base/JSON.h +++ b/base/base/JSON.h @@ -38,8 +38,14 @@ */ +#ifdef __clang__ +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wdeprecated-dynamic-exception-spec" +#endif POCO_DECLARE_EXCEPTION(Foundation_API, JSONException, Poco::Exception) - +#ifdef __clang__ +# pragma clang diagnostic pop +#endif class JSON { diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 6b08f1fda05..b8fc4229ad9 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -29,7 +29,6 @@ if (COMPILER_CLANG) no_warning(c99-extensions) no_warning(conversion) no_warning(ctad-maybe-unsupported) # clang 9+, linux-only - no_warning(deprecated-dynamic-exception-spec) no_warning(disabled-macro-expansion) no_warning(documentation-unknown-command) no_warning(double-promotion) diff --git a/src/Common/mysqlxx/mysqlxx/Exception.h b/src/Common/mysqlxx/mysqlxx/Exception.h index f022335a9dd..7886368e747 100644 --- a/src/Common/mysqlxx/mysqlxx/Exception.h +++ b/src/Common/mysqlxx/mysqlxx/Exception.h @@ -12,8 +12,8 @@ struct Exception : public Poco::Exception { explicit Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {} int errnum() const { return code(); } - const char * name() const throw() override { return "mysqlxx::Exception"; } - const char * className() const throw() override { return "mysqlxx::Exception"; } + const char * name() const noexcept override { return "mysqlxx::Exception"; } + const char * className() const noexcept override { return "mysqlxx::Exception"; } }; @@ -21,8 +21,8 @@ struct Exception : public Poco::Exception struct ConnectionFailed : public Exception { explicit ConnectionFailed(const std::string & msg, int code = 0) : Exception(msg, code) {} - const char * name() const throw() override { return "mysqlxx::ConnectionFailed"; } - const char * className() const throw() override { return "mysqlxx::ConnectionFailed"; } + const char * name() const noexcept override { return "mysqlxx::ConnectionFailed"; } + const char * className() const noexcept override { return "mysqlxx::ConnectionFailed"; } }; @@ -30,8 +30,8 @@ struct ConnectionFailed : public Exception struct ConnectionLost : public Exception { explicit ConnectionLost(const std::string & msg, int code = 0) : Exception(msg, code) {} - const char * name() const throw() override { return "mysqlxx::ConnectionLost"; } - const char * className() const throw() override { return "mysqlxx::ConnectionLost"; } + const char * name() const noexcept override { return "mysqlxx::ConnectionLost"; } + const char * className() const noexcept override { return "mysqlxx::ConnectionLost"; } }; @@ -39,8 +39,8 @@ struct ConnectionLost : public Exception struct BadQuery : public Exception { explicit BadQuery(const std::string & msg, int code = 0) : Exception(msg, code) {} - const char * name() const throw() override { return "mysqlxx::BadQuery"; } - const char * className() const throw() override { return "mysqlxx::BadQuery"; } + const char * name() const noexcept override { return "mysqlxx::BadQuery"; } + const char * className() const noexcept override { return "mysqlxx::BadQuery"; } }; @@ -48,8 +48,8 @@ struct BadQuery : public Exception struct CannotParseValue : public Exception { explicit CannotParseValue(const std::string & msg, int code = 0) : Exception(msg, code) {} - const char * name() const throw() override { return "mysqlxx::CannotParseValue"; } - const char * className() const throw() override { return "mysqlxx::CannotParseValue"; } + const char * name() const noexcept override { return "mysqlxx::CannotParseValue"; } + const char * className() const noexcept override { return "mysqlxx::CannotParseValue"; } }; From 3fd367ea580c51a293ce659779b023433010158c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 13 Jul 2022 10:08:24 -0400 Subject: [PATCH 566/627] check metadata_snapshot is not empty --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index e75fdcd6e3f..6460ed536e9 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1851,7 +1851,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { ExpressionActionsChain chain(context); - Names additional_required_columns_after_prewhere = metadata_snapshot->getColumnsRequiredForSortingKey(); + Names additional_required_columns_after_prewhere = metadata_snapshot ? metadata_snapshot->getColumnsRequiredForSortingKey() : Names{}; if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { From 3720a3843c30e77e9b1e787aa30e32a3e5b8b29c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Jul 2022 17:03:39 +0300 Subject: [PATCH 567/627] Add merge_algorithm to system.part_log Signed-off-by: Azat Khuzhin --- src/Interpreters/PartLog.cpp | 38 ++++++++++++++++--- src/Interpreters/PartLog.h | 12 ++++++ src/Storages/MergeTree/MergeTreeData.cpp | 5 +++ .../02362_part_log_merge_algorithm.reference | 5 +++ .../02362_part_log_merge_algorithm.sql | 26 +++++++++++++ 5 files changed, 80 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02362_part_log_merge_algorithm.reference create mode 100644 tests/queries/0_stateless/02362_part_log_merge_algorithm.sql diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index f79be6a67e0..c3152f31808 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -25,17 +25,32 @@ PartLogElement::MergeReasonType PartLogElement::getMergeReasonType(MergeType mer { switch (merge_type) { - case MergeType::Regular: - return REGULAR_MERGE; - case MergeType::TTLDelete: - return TTL_DELETE_MERGE; - case MergeType::TTLRecompress: - return TTL_RECOMPRESS_MERGE; + case MergeType::Regular: + return REGULAR_MERGE; + case MergeType::TTLDelete: + return TTL_DELETE_MERGE; + case MergeType::TTLRecompress: + return TTL_RECOMPRESS_MERGE; } throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); } +PartLogElement::PartMergeAlgorithm PartLogElement::getMergeAlgorithm(MergeAlgorithm merge_algorithm_) +{ + switch (merge_algorithm_) + { + case MergeAlgorithm::Undecided: + return UNDECIDED; + case MergeAlgorithm::Horizontal: + return HORIZONTAL; + case MergeAlgorithm::Vertical: + return VERTICAL; + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeAlgorithm {}", static_cast(merge_algorithm_)); +} + NamesAndTypesList PartLogElement::getNamesAndTypes() { auto event_type_datatype = std::make_shared( @@ -60,12 +75,22 @@ NamesAndTypesList PartLogElement::getNamesAndTypes() } ); + auto merge_algorithm_datatype = std::make_shared( + DataTypeEnum8::Values + { + {"Undecided", static_cast(UNDECIDED)}, + {"Horizontal", static_cast(HORIZONTAL)}, + {"Vertical", static_cast(VERTICAL)}, + } + ); + ColumnsWithTypeAndName columns_with_type_and_name; return { {"query_id", std::make_shared()}, {"event_type", std::move(event_type_datatype)}, {"merge_reason", std::move(merge_reason_datatype)}, + {"merge_algorithm", std::move(merge_algorithm_datatype)}, {"event_date", std::make_shared()}, {"event_time", std::make_shared()}, @@ -104,6 +129,7 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(query_id); columns[i++]->insert(event_type); columns[i++]->insert(merge_reason); + columns[i++]->insert(merge_algorithm); columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType()); columns[i++]->insert(event_time); columns[i++]->insert(event_time_microseconds); diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 16a7e37ee9d..2ce0dfd76de 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -22,6 +23,14 @@ struct PartLogElement MOVE_PART = 6, }; + /// Copy of MergeAlgorithm since values are written to disk. + enum PartMergeAlgorithm + { + UNDECIDED = 0, + VERTICAL = 1, + HORIZONTAL = 2, + }; + enum MergeReasonType { /// merge_reason is relevant only for event_type = 'MERGE_PARTS', in other cases it is NOT_A_MERGE @@ -38,6 +47,7 @@ struct PartLogElement Type event_type = NEW_PART; MergeReasonType merge_reason = NOT_A_MERGE; + PartMergeAlgorithm merge_algorithm = UNDECIDED; time_t event_time = 0; Decimal64 event_time_microseconds = 0; @@ -72,6 +82,8 @@ struct PartLogElement static std::string name() { return "PartLog"; } static MergeReasonType getMergeReasonType(MergeType merge_type); + static PartMergeAlgorithm getMergeAlgorithm(MergeAlgorithm merge_algorithm_); + static NamesAndTypesList getNamesAndTypes(); static NamesAndAliases getNamesAndAliases() { return {}; } void appendToBlock(MutableColumns & columns) const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5900ea0fdb7..64aaa40bd4c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6229,8 +6229,13 @@ try part_log_elem.event_type = type; if (part_log_elem.event_type == PartLogElement::MERGE_PARTS) + { if (merge_entry) + { part_log_elem.merge_reason = PartLogElement::getMergeReasonType((*merge_entry)->merge_type); + part_log_elem.merge_algorithm = PartLogElement::getMergeAlgorithm((*merge_entry)->merge_algorithm); + } + } part_log_elem.error = static_cast(execution_status.code); part_log_elem.exception = execution_status.message; diff --git a/tests/queries/0_stateless/02362_part_log_merge_algorithm.reference b/tests/queries/0_stateless/02362_part_log_merge_algorithm.reference new file mode 100644 index 00000000000..91a959d4255 --- /dev/null +++ b/tests/queries/0_stateless/02362_part_log_merge_algorithm.reference @@ -0,0 +1,5 @@ +data_horizontal all_1_1_0 NewPart Undecided +data_horizontal all_1_1_1 MergeParts Horizontal +data_vertical all_1_1_0 NewPart Undecided +data_vertical all_2_2_0 NewPart Undecided +data_vertical all_1_2_1 MergeParts Vertical diff --git a/tests/queries/0_stateless/02362_part_log_merge_algorithm.sql b/tests/queries/0_stateless/02362_part_log_merge_algorithm.sql new file mode 100644 index 00000000000..6446b46c393 --- /dev/null +++ b/tests/queries/0_stateless/02362_part_log_merge_algorithm.sql @@ -0,0 +1,26 @@ +CREATE TABLE data_horizontal ( + key Int +) +Engine=MergeTree() +ORDER BY key; + +INSERT INTO data_horizontal VALUES (1); +OPTIMIZE TABLE data_horizontal FINAL; +SYSTEM FLUSH LOGS; +SELECT table, part_name, event_type, merge_algorithm FROM system.part_log WHERE event_date >= yesterday() AND database = currentDatabase() AND table = 'data_horizontal' ORDER BY event_time_microseconds; + +CREATE TABLE data_vertical +( + key UInt64, + value String +) +ENGINE = MergeTree() +ORDER BY key +SETTINGS index_granularity_bytes = 0, enable_mixed_granularity_parts = 0, min_bytes_for_wide_part = 0, +vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1; + +INSERT INTO data_vertical VALUES (1, '1'); +INSERT INTO data_vertical VALUES (2, '2'); +OPTIMIZE TABLE data_vertical FINAL; +SYSTEM FLUSH LOGS; +SELECT table, part_name, event_type, merge_algorithm FROM system.part_log WHERE event_date >= yesterday() AND database = currentDatabase() AND table = 'data_vertical' ORDER BY event_time_microseconds; From 765503cd61046cb188ed8bc39350863d9f5a21f4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Jul 2022 17:55:24 +0200 Subject: [PATCH 568/627] Update test.py --- tests/integration/test_storage_mongodb/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 1fc3e255a5b..2755dfaa02a 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -263,6 +263,9 @@ def test_simple_insert_select(started_cluster): simple_mongo_table = db["simple_table"] node = started_cluster.instances["node"] + node.query( + "DROP TABLE IF EXISTS simple_mongo_table" + ) node.query( "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" ) From 14aa62f66137e881f083ca55c6b6cd27a6c44a79 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 13 Jul 2022 16:14:51 +0000 Subject: [PATCH 569/627] Fix test output --- tests/queries/0_stateless/01961_roaring_memory_tracking.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01961_roaring_memory_tracking.sql b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql index 64c31472e89..9e14bb9e138 100644 --- a/tests/queries/0_stateless/01961_roaring_memory_tracking.sql +++ b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql @@ -1,4 +1,4 @@ -- Tags: no-replicated-database SET max_memory_usage = '100M'; -SELECT cityHash64(rand() % 1000) as n, groupBitmapState(number) FROM numbers_mt(2000000000) GROUP BY n; -- { serverError 241 } +SELECT cityHash64(rand() % 1000) as n, groupBitmapState(number) FROM numbers_mt(2000000000) GROUP BY n FORMAT Null; -- { serverError 241 } From 189de392fdfb2d3161106cd03d1a49d96ae0c5e1 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Wed, 13 Jul 2022 19:20:32 +0300 Subject: [PATCH 570/627] Update mergetree.md Functions hasAny and hasAll also can use a Bloom filter. --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 3ac79a5e718..e1e1b0b4268 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -335,7 +335,7 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 Поддерживаемые типы данных: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`. - Фильтром могут пользоваться функции: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions), [notIn](../../../sql-reference/functions/in-functions), [has](../../../sql-reference/functions/array-functions#hasarr-elem). + Фильтром могут пользоваться функции: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions), [notIn](../../../sql-reference/functions/in-functions), [has](../../../sql-reference/functions/array-functions#hasarr-elem), [hasAny](../../../sql-reference/functions/array-functions#hasany), [hasAll](../../../sql-reference/functions/array-functions#hasall). **Примеры** From 60c6a5b23c39520c000ce9f793135176d0bd644a Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 13 Jul 2022 18:36:08 +0200 Subject: [PATCH 571/627] fix s3 read/write throttling instant --- src/IO/ReadBufferFromS3.cpp | 6 ++---- src/IO/WriteBufferFromS3.cpp | 9 ++------- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 1e49346c8a2..265ce3012e0 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -165,6 +165,8 @@ bool ReadBufferFromS3::nextImpl() ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, working_buffer.size()); offset += working_buffer.size(); + if (read_settings.throttler) + read_settings.throttler->add(working_buffer.size()); return true; } @@ -300,10 +302,6 @@ std::unique_ptr ReadBufferFromS3::initialize() if (outcome.IsSuccess()) { read_result = outcome.GetResultWithOwnership(); - - if (read_settings.throttler) - read_settings.throttler->add(read_result.GetContentLength()); - size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; return std::make_unique(read_result.GetBody(), buffer_size); } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 7aca544a1a9..27210d50e44 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -124,8 +124,9 @@ void WriteBufferFromS3::nextImpl() } ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Bytes, offset()); - last_part_size += offset(); + if (write_settings.throttler) + write_settings.throttler->add(offset()); /// Data size exceeds singlepart upload threshold, need to use multipart upload. if (multipart_upload_id.empty() && last_part_size > s3_settings.max_single_part_upload_size) @@ -334,8 +335,6 @@ void WriteBufferFromS3::fillUploadRequest(Aws::S3::Model::UploadPartRequest & re void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) { auto outcome = client_ptr->UploadPart(task.req); - if (write_settings.throttler) - write_settings.throttler->add(bytes); if (outcome.IsSuccess()) { @@ -465,11 +464,7 @@ void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req) void WriteBufferFromS3::processPutRequest(PutObjectTask & task) { - size_t bytes = task.req.GetContentLength(); auto outcome = client_ptr->PutObject(task.req); - if (write_settings.throttler) - write_settings.throttler->add(bytes); - bool with_pool = static_cast(schedule); if (outcome.IsSuccess()) LOG_TRACE(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}, WithPool: {}", bucket, key, task.req.GetContentLength(), with_pool); From f81650e94a6f7f549d48994edb7b9be7c566d5b4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Jul 2022 19:03:48 +0200 Subject: [PATCH 572/627] Fix extremely rare condition in hardlinks creation --- .../FakeMetadataStorageFromDisk.cpp | 79 ----------- .../FakeMetadataStorageFromDisk.h | 8 +- .../MetadataStorageFromDisk.cpp | 41 ++---- .../ObjectStorages/MetadataStorageFromDisk.h | 2 +- ...taStorageFromDiskTransactionOperations.cpp | 96 ++++++++++++-- ...dataStorageFromDiskTransactionOperations.h | 125 +++++++++++++++--- 6 files changed, 204 insertions(+), 147 deletions(-) diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp index a6b2f6b5f02..76e6aebff1c 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp @@ -114,85 +114,6 @@ const IMetadataStorage & FakeMetadataStorageFromDiskTransaction::getStorageForNo return metadata_storage; } -void FakeMetadataStorageFromDiskTransaction::addOperation(MetadataOperationPtr && operation) -{ - if (state != MetadataFromDiskTransactionState::PREPARING) - throw Exception( - ErrorCodes::FS_METADATA_ERROR, - "Cannot add operations to transaction in {} state, it should be in {} state", - toString(state), toString(MetadataFromDiskTransactionState::PREPARING)); - - operations.emplace_back(std::move(operation)); -} - -void FakeMetadataStorageFromDiskTransaction::commit() -{ - if (state != MetadataFromDiskTransactionState::PREPARING) - throw Exception( - ErrorCodes::FS_METADATA_ERROR, - "Cannot commit transaction in {} state, it should be in {} state", - toString(state), toString(MetadataFromDiskTransactionState::PREPARING)); - - { - std::unique_lock lock(metadata_storage.metadata_mutex); - for (size_t i = 0; i < operations.size(); ++i) - { - try - { - operations[i]->execute(); - } - catch (Exception & ex) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - ex.addMessage(fmt::format("While committing metadata operation #{}", i)); - state = MetadataFromDiskTransactionState::FAILED; - rollback(i); - throw; - } - } - } - - /// Do it in "best effort" mode - for (size_t i = 0; i < operations.size(); ++i) - { - try - { - operations[i]->finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to finalize operation #{}", i)); - } - } - - state = MetadataFromDiskTransactionState::COMMITTED; -} - -void FakeMetadataStorageFromDiskTransaction::rollback(size_t until_pos) -{ - /// Otherwise everything is alright - if (state == MetadataFromDiskTransactionState::FAILED) - { - for (int64_t i = until_pos; i >= 0; --i) - { - try - { - operations[i]->undo(); - } - catch (Exception & ex) - { - state = MetadataFromDiskTransactionState::PARTIALLY_ROLLED_BACK; - ex.addMessage(fmt::format("While rolling back operation #{}", i)); - throw; - } - } - } - else - { - /// Nothing to do, transaction committed or not even started to commit - } -} - void FakeMetadataStorageFromDiskTransaction::writeStringToFile(const std::string & path, const std::string & data) { auto wb = disk->writeFile(path); diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h index 3fc223ea75b..b6426df1e9a 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h @@ -65,12 +65,6 @@ private: const FakeMetadataStorageFromDisk & metadata_storage; std::vector operations; - MetadataFromDiskTransactionState state{MetadataFromDiskTransactionState::PREPARING}; - - void addOperation(MetadataOperationPtr && operation); - - void rollback(size_t until_pos); - public: FakeMetadataStorageFromDiskTransaction( const FakeMetadataStorageFromDisk & metadata_storage_, DiskPtr disk_) @@ -82,7 +76,7 @@ public: const IMetadataStorage & getStorageForNonTransactionalReads() const final; - void commit() final; + void commit() final {} void writeStringToFile(const std::string & path, const std::string & data) override; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp index a664433a3d0..489772647d1 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp @@ -87,6 +87,14 @@ DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const return metadata; } +DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::unique_lock &) const +{ + auto metadata = std::make_unique(disk->getPath(), object_storage_root_path, path); + auto str = readFileToString(path); + metadata->deserializeFromString(str); + return metadata; +} + DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadata(const std::string & path) const { std::shared_lock lock(metadata_mutex); @@ -112,13 +120,7 @@ std::unordered_map MetadataStorageFromDisk::getSerializedMetadat void MetadataStorageFromDiskTransaction::createHardLink(const std::string & path_from, const std::string & path_to) { - auto metadata = metadata_storage.readMetadata(path_from); - - metadata->incrementRefCount(); - - writeStringToFile(path_from, metadata->serializeToString()); - - addOperation(std::make_unique(path_from, path_to, *metadata_storage.getDisk())); + addOperation(std::make_unique(path_from, path_to, *metadata_storage.disk, metadata_storage)); } MetadataTransactionPtr MetadataStorageFromDisk::createTransaction() const @@ -177,12 +179,12 @@ void MetadataStorageFromDiskTransaction::commit() toString(state), toString(MetadataFromDiskTransactionState::PREPARING)); { - std::lock_guard lock(metadata_storage.metadata_mutex); + std::unique_lock lock(metadata_storage.metadata_mutex); for (size_t i = 0; i < operations.size(); ++i) { try { - operations[i]->execute(); + operations[i]->execute(lock); } catch (Exception & ex) { @@ -316,29 +318,12 @@ void MetadataStorageFromDiskTransaction::createMetadataFile(const std::string & void MetadataStorageFromDiskTransaction::addBlobToMetadata(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) { - DiskObjectStorageMetadataPtr metadata; - if (metadata_storage.exists(path)) - { - metadata = metadata_storage.readMetadata(path); - metadata->addObject(blob_name, size_in_bytes); - writeStringToFile(path, metadata->serializeToString()); - } - else - { - createMetadataFile(path, blob_name, size_in_bytes); - } + addOperation(std::make_unique(path, blob_name, metadata_storage.object_storage_root_path, size_in_bytes, *metadata_storage.disk, metadata_storage)); } void MetadataStorageFromDiskTransaction::unlinkMetadata(const std::string & path) { - auto metadata = metadata_storage.readMetadata(path); - uint32_t ref_count = metadata->getRefCount(); - if (ref_count != 0) - { - metadata->decrementRefCount(); - writeStringToFile(path, metadata->serializeToString()); - } - unlinkFile(path); + addOperation(std::make_unique(path, *metadata_storage.disk, metadata_storage)); } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h index e84b8fe36eb..104e9d54bff 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h @@ -55,9 +55,9 @@ public: std::string getObjectStorageRootPath() const override { return object_storage_root_path; } -private: DiskObjectStorageMetadataPtr readMetadata(const std::string & path) const; + DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::unique_lock & lock) const; DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::shared_lock & lock) const; }; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp index dce4ae2f1f7..7c0a60509ab 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include #include @@ -24,7 +26,7 @@ SetLastModifiedOperation::SetLastModifiedOperation(const std::string & path_, Po { } -void SetLastModifiedOperation::execute() +void SetLastModifiedOperation::execute(std::unique_lock &) { old_timestamp = disk.getLastModified(path); disk.setLastModified(path, new_timestamp); @@ -41,7 +43,7 @@ UnlinkFileOperation::UnlinkFileOperation(const std::string & path_, IDisk & disk { } -void UnlinkFileOperation::execute() +void UnlinkFileOperation::execute(std::unique_lock &) { auto buf = disk.readFile(path); readStringUntilEOF(prev_data, *buf); @@ -61,7 +63,7 @@ CreateDirectoryOperation::CreateDirectoryOperation(const std::string & path_, ID { } -void CreateDirectoryOperation::execute() +void CreateDirectoryOperation::execute(std::unique_lock &) { disk.createDirectory(path); } @@ -77,7 +79,7 @@ CreateDirectoryRecursiveOperation::CreateDirectoryRecursiveOperation(const std:: { } -void CreateDirectoryRecursiveOperation::execute() +void CreateDirectoryRecursiveOperation::execute(std::unique_lock &) { namespace fs = std::filesystem; fs::path p(path); @@ -104,7 +106,7 @@ RemoveDirectoryOperation::RemoveDirectoryOperation(const std::string & path_, ID { } -void RemoveDirectoryOperation::execute() +void RemoveDirectoryOperation::execute(std::unique_lock &) { disk.removeDirectory(path); } @@ -121,7 +123,7 @@ RemoveRecursiveOperation::RemoveRecursiveOperation(const std::string & path_, ID { } -void RemoveRecursiveOperation:: execute() +void RemoveRecursiveOperation::execute(std::unique_lock &) { if (disk.isFile(path)) disk.moveFile(path, temp_path); @@ -146,20 +148,31 @@ void RemoveRecursiveOperation::finalize() disk.removeRecursive(path); } -CreateHardlinkOperation::CreateHardlinkOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_) +CreateHardlinkOperation::CreateHardlinkOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_, const MetadataStorageFromDisk & metadata_storage_) : path_from(path_from_) , path_to(path_to_) , disk(disk_) + , metadata_storage(metadata_storage_) { } -void CreateHardlinkOperation::execute() +void CreateHardlinkOperation::execute(std::unique_lock & lock) { + auto metadata = metadata_storage.readMetadataUnlocked(path_from, lock); + + metadata->incrementRefCount(); + + write_operation = std::make_unique(path_from, disk, metadata->serializeToString()); + + write_operation->execute(lock); + disk.createHardLink(path_from, path_to); } void CreateHardlinkOperation::undo() { + if (write_operation) + write_operation->undo(); disk.removeFile(path_to); } @@ -170,7 +183,7 @@ MoveFileOperation::MoveFileOperation(const std::string & path_from_, const std:: { } -void MoveFileOperation::execute() +void MoveFileOperation::execute(std::unique_lock &) { disk.moveFile(path_from, path_to); } @@ -187,7 +200,7 @@ MoveDirectoryOperation::MoveDirectoryOperation(const std::string & path_from_, c { } -void MoveDirectoryOperation::execute() +void MoveDirectoryOperation::execute(std::unique_lock &) { disk.moveDirectory(path_from, path_to); } @@ -197,7 +210,6 @@ void MoveDirectoryOperation::undo() disk.moveDirectory(path_to, path_from); } - ReplaceFileOperation::ReplaceFileOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_) : path_from(path_from_) , path_to(path_to_) @@ -206,7 +218,7 @@ ReplaceFileOperation::ReplaceFileOperation(const std::string & path_from_, const { } -void ReplaceFileOperation::execute() +void ReplaceFileOperation::execute(std::unique_lock &) { if (disk.exists(path_to)) disk.moveFile(path_to, temp_path_to); @@ -232,7 +244,7 @@ WriteFileOperation::WriteFileOperation(const std::string & path_, IDisk & disk_, { } -void WriteFileOperation::execute() +void WriteFileOperation::execute(std::unique_lock &) { if (disk.exists(path)) { @@ -258,4 +270,62 @@ void WriteFileOperation::undo() } } +void AddBlobOperation::execute(std::unique_lock & metadata_lock) +{ + DiskObjectStorageMetadataPtr metadata; + if (metadata_storage.exists(path)) + metadata = metadata_storage.readMetadataUnlocked(path, metadata_lock); + else + metadata = std::make_unique(disk.getPath(), root_path, path); + + metadata->addObject(blob_name, size_in_bytes); + + write_operation = std::make_unique(path, disk, metadata->serializeToString()); + + write_operation->execute(metadata_lock); +} + +void AddBlobOperation::undo() +{ + if (write_operation) + write_operation->undo(); +} + +void UnlinkMetadataFileOperation::execute(std::unique_lock & metadata_lock) +{ + auto metadata = metadata_storage.readMetadataUnlocked(path, metadata_lock); + uint32_t ref_count = metadata->getRefCount(); + if (ref_count != 0) + { + metadata->decrementRefCount(); + write_operation = std::make_unique(path, disk, metadata->serializeToString()); + write_operation->execute(metadata_lock); + } + unlink_operation = std::make_unique(path, disk); + unlink_operation->execute(metadata_lock); +} + +void UnlinkMetadataFileOperation::undo() +{ + if (write_operation) + write_operation->undo(); + + if (unlink_operation) + unlink_operation->undo(); +} + +void SetReadonlyFileOperation::execute(std::unique_lock & metadata_lock) +{ + auto metadata = metadata_storage.readMetadataUnlocked(path, metadata_lock); + metadata->setReadOnly(); + write_operation = std::make_unique(path, disk, metadata->serializeToString()); + write_operation->execute(metadata_lock); +} + +void SetReadonlyFileOperation::undo() +{ + if (write_operation) + write_operation->undo(); +} + } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h index f998771a68f..5f8e772ebc7 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h @@ -4,6 +4,7 @@ namespace DB { +class MetadataStorageFromDisk; class IDisk; /** @@ -12,7 +13,7 @@ class IDisk; struct IMetadataOperation { - virtual void execute() = 0; + virtual void execute(std::unique_lock & metadata_lock) = 0; virtual void undo() = 0; virtual void finalize() {} virtual ~IMetadataOperation() = default; @@ -25,7 +26,7 @@ struct SetLastModifiedOperation final : public IMetadataOperation { SetLastModifiedOperation(const std::string & path_, Poco::Timestamp new_timestamp_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -41,7 +42,7 @@ struct UnlinkFileOperation final : public IMetadataOperation { UnlinkFileOperation(const std::string & path_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -56,7 +57,7 @@ struct CreateDirectoryOperation final : public IMetadataOperation { CreateDirectoryOperation(const std::string & path_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -70,7 +71,7 @@ struct CreateDirectoryRecursiveOperation final : public IMetadataOperation { CreateDirectoryRecursiveOperation(const std::string & path_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -85,7 +86,7 @@ struct RemoveDirectoryOperation final : public IMetadataOperation { RemoveDirectoryOperation(const std::string & path_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -98,7 +99,7 @@ struct RemoveRecursiveOperation final : public IMetadataOperation { RemoveRecursiveOperation(const std::string & path_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -110,12 +111,30 @@ private: std::string temp_path; }; +struct WriteFileOperation final : public IMetadataOperation +{ + WriteFileOperation(const std::string & path_, IDisk & disk_, const std::string & data_); + + void execute(std::unique_lock & metadata_lock) override; + + void undo() override; +private: + std::string path; + IDisk & disk; + std::string data; + bool existed = false; + std::string prev_data; +}; struct CreateHardlinkOperation final : public IMetadataOperation { - CreateHardlinkOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); + CreateHardlinkOperation( + const std::string & path_from_, + const std::string & path_to_, + IDisk & disk_, + const MetadataStorageFromDisk & metadata_storage_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -123,6 +142,8 @@ private: std::string path_from; std::string path_to; IDisk & disk; + std::unique_ptr write_operation; + const MetadataStorageFromDisk & metadata_storage; }; @@ -130,7 +151,7 @@ struct MoveFileOperation final : public IMetadataOperation { MoveFileOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -145,7 +166,7 @@ struct MoveDirectoryOperation final : public IMetadataOperation { MoveDirectoryOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -160,7 +181,7 @@ struct ReplaceFileOperation final : public IMetadataOperation { ReplaceFileOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -173,20 +194,86 @@ private: std::string temp_path_to; }; - -struct WriteFileOperation final : public IMetadataOperation +struct AddBlobOperation final : public IMetadataOperation { - WriteFileOperation(const std::string & path_, IDisk & disk_, const std::string & data_); + AddBlobOperation( + const std::string & path_, + const std::string & blob_name_, + const std::string & root_path_, + uint64_t size_in_bytes_, + IDisk & disk_, + const MetadataStorageFromDisk & metadata_storage_) + : path(path_) + , blob_name(blob_name_) + , root_path(root_path_) + , size_in_bytes(size_in_bytes_) + , disk(disk_) + , metadata_storage(metadata_storage_) + {} - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; + +private: + std::string path; + std::string blob_name; + std::string root_path; + uint64_t size_in_bytes; + IDisk & disk; + const MetadataStorageFromDisk & metadata_storage; + + std::unique_ptr write_operation; +}; + + +struct UnlinkMetadataFileOperation final : public IMetadataOperation +{ + UnlinkMetadataFileOperation( + const std::string & path_, + IDisk & disk_, + const MetadataStorageFromDisk & metadata_storage_) + : path(path_) + , disk(disk_) + , metadata_storage(metadata_storage_) + { + } + + void execute(std::unique_lock & metadata_lock) override; + + void undo() override; + private: std::string path; IDisk & disk; - std::string data; - bool existed = false; - std::string prev_data; + const MetadataStorageFromDisk & metadata_storage; + + std::unique_ptr write_operation; + std::unique_ptr unlink_operation; +}; + +struct SetReadonlyFileOperation final : public IMetadataOperation +{ + SetReadonlyFileOperation( + const std::string & path_, + IDisk & disk_, + const MetadataStorageFromDisk & metadata_storage_) + : path(path_) + , disk(disk_) + , metadata_storage(metadata_storage_) + { + } + + void execute(std::unique_lock & metadata_lock) override; + + void undo() override; + +private: + std::string path; + IDisk & disk; + const MetadataStorageFromDisk & metadata_storage; + + std::unique_ptr write_operation; }; } From af4a5a5f3ab0777e93e45dab07dcd3e480392abc Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 13 Jul 2022 19:44:29 +0200 Subject: [PATCH 573/627] fix conflicts --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 1 - src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 9 --------- src/IO/ReadBufferFromAzureBlobStorage.cpp | 1 - src/IO/ReadBufferFromAzureBlobStorage.h | 1 - src/Storages/Hive/HiveFile.cpp | 8 -------- 5 files changed, 20 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 8f70ab876ff..14614871185 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -90,7 +90,6 @@ SeekableReadBufferPtr ReadBufferFromAzureBlobStorageGather::createImplementation settings, max_single_read_retries, max_single_download_retries, - settings, /* use_external_buffer */true, read_until_position); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index da5051f8f47..55c466d45f6 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -193,14 +193,8 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files"); bool cache_on_write = cache -<<<<<<< HEAD - && fs::path(path).extension() != ".tmp" - && disk_write_settings.enable_filesystem_cache_on_write_operations - && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations; -======= && write_settings.enable_filesystem_cache_on_write_operations && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations; ->>>>>>> master auto settings_ptr = s3_settings.get(); auto s3_buffer = std::make_unique( @@ -211,10 +205,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN attributes, buf_size, threadPoolCallbackRunner(getThreadPoolWriter()), -<<<<<<< HEAD disk_write_settings, -======= ->>>>>>> master cache_on_write ? cache : nullptr); diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp index f624de56dc2..32e70305bea 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -27,7 +27,6 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( const ReadSettings & read_settings_, size_t max_single_read_retries_, size_t max_single_download_retries_, - const ReadSettings & read_settings_, bool use_external_buffer_, size_t read_until_position_) : ReadBufferFromFileBase(read_settings_.remote_fs_buffer_size, nullptr, 0) diff --git a/src/IO/ReadBufferFromAzureBlobStorage.h b/src/IO/ReadBufferFromAzureBlobStorage.h index 136f1573fed..5396fcf9719 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/IO/ReadBufferFromAzureBlobStorage.h @@ -23,7 +23,6 @@ public: const ReadSettings & read_settings_, size_t max_single_read_retries_, size_t max_single_download_retries_, - const ReadSettings & read_settings_, bool use_external_buffer_ = false, size_t read_until_position_ = 0); diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 23abc074da4..09c3aff4455 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -267,15 +267,7 @@ bool HiveParquetFile::useSplitMinMaxIndex() const void HiveParquetFile::prepareReader() { -<<<<<<< HEAD - in = std::make_unique( - namenode_url, - path, - getContext()->getGlobalContext()->getConfigRef(), - ReadSettings{.throttler = getContext()->getRemoteReadThrottler()}); -======= in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); ->>>>>>> master auto format_settings = getFormatSettings(getContext()); std::atomic is_stopped{0}; THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES), arrow::default_memory_pool(), &reader)); From 15ab3bc99fb594dc891d93fb8e9086cb9fa7aa60 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 13 Jul 2022 19:48:57 +0200 Subject: [PATCH 574/627] use `context->getWriteSettings()` --- src/Interpreters/Context.cpp | 2 ++ src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/StorageS3.cpp | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index bf011456e89..70a0b201180 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3476,6 +3476,8 @@ WriteSettings Context::getWriteSettings() const res.enable_filesystem_cache_on_write_operations = settings.enable_filesystem_cache_on_write_operations; + res.throttler = getRemoteWriteThrottler(); + return res; } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index d708891d1f6..57e893e9683 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -418,7 +418,7 @@ public: uri, context->getGlobalContext()->getConfigRef(), context->getSettingsRef().hdfs_replication, - WriteSettings{.throttler = context->getRemoteWriteThrottler()}), + context->getWriteSettings()), compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index fffd383955f..130bc75a65c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -588,7 +588,7 @@ public: std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, threadPoolCallbackRunner(IOThreadPool::get()), - WriteSettings{.throttler = context->getRemoteWriteThrottler()}), + context->getWriteSettings()), compression_method, 3); writer From b7f46d954ecc968ee52908007486e1a25d40fcad Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 13 Jul 2022 17:46:09 +0000 Subject: [PATCH 575/627] Cleanup: related to #38719 (SortingStep: deduce way to sort based on input stream sort) --- .../Merges/Algorithms/MergingSortedAlgorithm.cpp | 6 +++--- .../Merges/Algorithms/MergingSortedAlgorithm.h | 4 ++-- src/Processors/Merges/MergingSortedTransform.cpp | 10 +++++++--- src/Processors/Merges/MergingSortedTransform.h | 2 +- src/Processors/Transforms/FinishSortingTransform.cpp | 5 ++--- src/Processors/Transforms/PartialSortingTransform.cpp | 7 +++---- src/Processors/Transforms/PartialSortingTransform.h | 6 +++--- src/Processors/Transforms/SortingTransform.h | 6 +++--- 8 files changed, 24 insertions(+), 22 deletions(-) diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp index 25719166acd..77db1e06d06 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp @@ -10,7 +10,7 @@ namespace DB MergingSortedAlgorithm::MergingSortedAlgorithm( Block header_, size_t num_inputs, - SortDescription description_, + const SortDescription & description_, size_t max_block_size, SortingQueueStrategy sorting_queue_strategy_, UInt64 limit_, @@ -18,7 +18,7 @@ MergingSortedAlgorithm::MergingSortedAlgorithm( bool use_average_block_sizes) : header(std::move(header_)) , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) - , description(std::move(description_)) + , description(description_) , limit(limit_) , out_row_sources_buf(out_row_sources_buf_) , current_inputs(num_inputs) @@ -29,7 +29,7 @@ MergingSortedAlgorithm::MergingSortedAlgorithm( sort_description_types.reserve(description.size()); /// Replace column names in description to positions. - for (auto & column_description : description) + for (const auto & column_description : description) { has_collation |= column_description.collator != nullptr; sort_description_types.emplace_back(header.getByName(column_description.column_name).type); diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h index cf32e5fd4dd..2537c48b128 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h @@ -16,7 +16,7 @@ public: MergingSortedAlgorithm( Block header_, size_t num_inputs, - SortDescription description_, + const SortDescription & description_, size_t max_block_size, SortingQueueStrategy sorting_queue_strategy_, UInt64 limit_ = 0, @@ -38,7 +38,7 @@ private: /// Settings SortDescription description; - UInt64 limit; + const UInt64 limit; bool has_collation = false; /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 4cb74ffc71e..9ee91ef6db7 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -10,7 +10,7 @@ namespace DB MergingSortedTransform::MergingSortedTransform( const Block & header, size_t num_inputs, - SortDescription description_, + const SortDescription & description_, size_t max_block_size, SortingQueueStrategy sorting_queue_strategy, UInt64 limit_, @@ -19,10 +19,14 @@ MergingSortedTransform::MergingSortedTransform( bool use_average_block_sizes, bool have_all_inputs_) : IMergingTransform( - num_inputs, header, header, have_all_inputs_, limit_, + num_inputs, + header, + header, + have_all_inputs_, + limit_, header, num_inputs, - std::move(description_), + description_, max_block_size, sorting_queue_strategy, limit_, diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index 16e3e2791ee..50586177c6d 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -14,7 +14,7 @@ public: MergingSortedTransform( const Block & header, size_t num_inputs, - SortDescription description, + const SortDescription & description, size_t max_block_size, SortingQueueStrategy sorting_queue_strategy, UInt64 limit_ = 0, diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 3842e034de7..86b5c4c9a00 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -36,9 +36,8 @@ FinishSortingTransform::FinishSortingTransform( /// The target description is modified in SortingTransform constructor. /// To avoid doing the same actions with description_sorted just copy it from prefix of target description. - size_t prefix_size = description_sorted_.size(); - for (size_t i = 0; i < prefix_size; ++i) - description_with_positions.emplace_back(description[i], header_without_constants.getPositionByName(description[i].column_name)); + for (const auto & column_sort_desc : description_sorted_) + description_with_positions.emplace_back(column_sort_desc, header_without_constants.getPositionByName(column_sort_desc.column_name)); } void FinishSortingTransform::consume(Chunk chunk) diff --git a/src/Processors/Transforms/PartialSortingTransform.cpp b/src/Processors/Transforms/PartialSortingTransform.cpp index b0f866cb3fd..3a2cecd3b26 100644 --- a/src/Processors/Transforms/PartialSortingTransform.cpp +++ b/src/Processors/Transforms/PartialSortingTransform.cpp @@ -85,7 +85,7 @@ bool compareWithThreshold(const ColumnRawPtrs & raw_block_columns, size_t min_bl } PartialSortingTransform::PartialSortingTransform( - const Block & header_, SortDescription & description_, UInt64 limit_) + const Block & header_, const SortDescription & description_, UInt64 limit_) : ISimpleTransform(header_, header_, false) , description(description_) , limit(limit_) @@ -93,9 +93,8 @@ PartialSortingTransform::PartialSortingTransform( // Sorting by no columns doesn't make sense. assert(!description_.empty()); - size_t description_size = description.size(); - for (size_t i = 0; i < description_size; ++i) - description_with_positions.emplace_back(description[i], header_.getPositionByName(description[i].column_name)); + for (const auto & column_sort_desc : description) + description_with_positions.emplace_back(column_sort_desc, header_.getPositionByName(column_sort_desc.column_name)); } void PartialSortingTransform::transform(Chunk & chunk) diff --git a/src/Processors/Transforms/PartialSortingTransform.h b/src/Processors/Transforms/PartialSortingTransform.h index 78ce80bdeeb..6dab4497fc7 100644 --- a/src/Processors/Transforms/PartialSortingTransform.h +++ b/src/Processors/Transforms/PartialSortingTransform.h @@ -15,7 +15,7 @@ public: /// limit - if not 0, then you can sort each block not completely, but only `limit` first rows by order. PartialSortingTransform( const Block & header_, - SortDescription & description_, + const SortDescription & description_, UInt64 limit_ = 0); String getName() const override { return "PartialSortingTransform"; } @@ -26,9 +26,9 @@ protected: void transform(Chunk & chunk) override; private: - SortDescription description; + const SortDescription description; SortDescriptionWithPositions description_with_positions; - UInt64 limit; + const UInt64 limit; RowsBeforeLimitCounterPtr read_rows; Columns sort_description_threshold_columns; diff --git a/src/Processors/Transforms/SortingTransform.h b/src/Processors/Transforms/SortingTransform.h index a607e52550d..d9a30699f92 100644 --- a/src/Processors/Transforms/SortingTransform.h +++ b/src/Processors/Transforms/SortingTransform.h @@ -73,8 +73,8 @@ public: ~SortingTransform() override; protected: - Status prepare() override final; - void work() override final; + Status prepare() final; + void work() final; virtual void consume(Chunk chunk) = 0; virtual void generate() = 0; @@ -82,7 +82,7 @@ protected: SortDescription description; size_t max_merged_block_size; - UInt64 limit; + const UInt64 limit; /// Before operation, will remove constant columns from blocks. And after, place constant columns back. /// (to avoid excessive virtual function calls and because constants cannot be serialized in Native format for temporary files) From 7ce5757d90a967c61e1226c89700f5aeb1b2545a Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 13 Jul 2022 15:07:20 -0400 Subject: [PATCH 576/627] add migration example --- .../sql-reference/table-functions/remote.md | 89 ++++++++++++++++--- 1 file changed, 75 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/table-functions/remote.md b/docs/en/sql-reference/table-functions/remote.md index f9e6860b081..5948fa6f5e1 100644 --- a/docs/en/sql-reference/table-functions/remote.md +++ b/docs/en/sql-reference/table-functions/remote.md @@ -5,11 +5,11 @@ sidebar_label: remote # remote, remoteSecure -Allows to access remote servers without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. `remoteSecure` - same as `remote` but with a secured connection. +Allows accessing remote servers, including migration of data, without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. `remoteSecure` - same as `remote` but with a secured connection. Both functions can be used in `SELECT` and `INSERT` queries. -**Syntax** +## Syntax ``` sql remote('addresses_expr', db, table[, 'user'[, 'password'], sharding_key]) @@ -18,7 +18,7 @@ remoteSecure('addresses_expr', db, table[, 'user'[, 'password'], sharding_key]) remoteSecure('addresses_expr', db.table[, 'user'[, 'password'], sharding_key]) ``` -**Parameters** +## Parameters - `addresses_expr` — An expression that generates addresses of remote servers. This may be just one server address. The server address is `host:port`, or just `host`. @@ -36,28 +36,31 @@ remoteSecure('addresses_expr', db.table[, 'user'[, 'password'], sharding_key]) - `password` — User password. If the password is not specified, an empty password is used. Type: [String](../../sql-reference/data-types/string.md). - `sharding_key` — Sharding key to support distributing data across nodes. For example: `insert into remote('127.0.0.1:9000,127.0.0.2', db, table, 'default', rand())`. Type: [UInt32](../../sql-reference/data-types/int-uint.md). -**Returned value** +## Returned value The dataset from remote servers. -**Usage** +## Usage -Using the `remote` table function is less optimal than creating a `Distributed` table because in this case the server connection is re-established for every request. Also, if hostnames are set, the names are resolved, and errors are not counted when working with various replicas. When processing a large number of queries, always create the `Distributed` table ahead of time, and do not use the `remote` table function. +Unless you are migrating data from one system to another, using the `remote` table function is less optimal than creating a `Distributed` table because in this case the server connection is re-established for every request. Also, if hostnames are set, the names are resolved, and errors are not counted when working with various replicas. When processing a large number of queries, always create the `Distributed` table ahead of time, and do not use the `remote` table function. The `remote` table function can be useful in the following cases: -- Accessing a specific server for data comparison, debugging, and testing. -- Queries between various ClickHouse clusters for research purposes. -- Infrequent distributed requests that are made manually. -- Distributed requests where the set of servers is re-defined each time. +- Migrating data from one system to another +- Accessing a specific server for data comparison, debugging, and testing. +- Queries between various ClickHouse clusters for research purposes. +- Infrequent distributed requests that are made manually. +- Distributed requests where the set of servers is re-defined each time. -**Adresses** +### Adresses ``` text example01-01-1 +example01-01-1:9440 example01-01-1:9000 localhost 127.0.0.1 +[::]:9440 [::]:9000 [2a02:6b8:0:1111::11]:9000 ``` @@ -68,15 +71,15 @@ Multiple addresses can be comma-separated. In this case, ClickHouse will use dis example01-01-1,example01-02-1 ``` -**Examples** +## Examples -Selecting data from a remote server: +### Selecting data from a remote server: ``` sql SELECT * FROM remote('127.0.0.1', db.remote_engine_table) LIMIT 3; ``` -Inserting data from a remote server into a table: +### Inserting data from a remote server into a table: ``` sql CREATE TABLE remote_table (name String, value UInt32) ENGINE=Memory; @@ -84,6 +87,64 @@ INSERT INTO FUNCTION remote('127.0.0.1', currentDatabase(), 'remote_table') VALU SELECT * FROM remote_table; ``` +### Migration of tables from one system to another: +This example uses one table from a sample dataset. The database is `imdb`, and the table is `actors`. + +#### On the source ClickHouse system (the system that currently hosts the data) +- Verify the source database and table name (`imdb.actors`) +```sql +show databases +``` + +```sql +show tables in imdb +``` + +- Get the CREATE TABLE statement from the source: +``` +select create_table_query +from system.tables +where database = 'imdb' and table = 'actors' +``` + +Response +```sql +CREATE TABLE imdb.actors (`id` UInt32, + `first_name` String, + `last_name` String, + `gender` FixedString(1)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') + ORDER BY (id, first_name, last_name, gender) + SETTINGS index_granularity = 8192 +``` + +#### On the destination ClickHouse system: + +- Create the destination database: +```sql +CREATE DATABASE imdb +``` + +- Using the CREATE TABLE statement from the source, create the destination: +```sql +CREATE TABLE imdb.actors (`id` UInt32, + `first_name` String, + `last_name` String, + `gender` FixedString(1)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') + ORDER BY (id, first_name, last_name, gender) + SETTINGS index_granularity = 8192 +``` + +#### Back on the source deployment: + +Insert into the new database and table created on the remote system. You will need the host, port, username, password, destination database, and destination table. +```sql +INSERT INTO FUNCTION +remoteSecure('remote.clickhouse.cloud:9440', 'imdb.actors', 'USER', 'PASSWORD', rand()) +SELECT * from imdb.actors +``` + ## Globs in Addresses {globs-in-addresses} Patterns in curly brackets `{ }` are used to generate a set of shards and to specify replicas. If there are multiple pairs of curly brackets, then the direct product of the corresponding sets is generated. From b4321904bf3dfa45e800d6ad0db68b444fbbab87 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Jul 2022 22:59:14 +0200 Subject: [PATCH 577/627] Update test.py --- tests/integration/test_storage_mongodb/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 2755dfaa02a..74b2b15fda0 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -263,9 +263,7 @@ def test_simple_insert_select(started_cluster): simple_mongo_table = db["simple_table"] node = started_cluster.instances["node"] - node.query( - "DROP TABLE IF EXISTS simple_mongo_table" - ) + node.query("DROP TABLE IF EXISTS simple_mongo_table") node.query( "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" ) From ba863b11a4f7a1daa3d397d1cea2517ddde29eef Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jul 2022 21:06:15 +0000 Subject: [PATCH 578/627] A little fix. Added test. --- src/Interpreters/ExpressionAnalyzer.cpp | 44 +++++++++++-------- .../02354_read_in_order_prewhere.reference | 10 +++++ .../02354_read_in_order_prewhere.sql | 30 +++++++++++++ 3 files changed, 66 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/02354_read_in_order_prewhere.reference create mode 100644 tests/queries/0_stateless/02354_read_in_order_prewhere.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 6460ed536e9..7847f0714ca 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1850,8 +1850,25 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( }; { + bool join_allow_read_in_order = true; + if (hasJoin()) + { + /// You may find it strange but we support read_in_order for HashJoin and do not support for MergeJoin. + join_has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows(); + join_allow_read_in_order = typeid_cast(join.get()) && !join_has_delayed_stream; + } + + optimize_read_in_order = + settings.optimize_read_in_order + && storage + && query.orderBy() + && !query_analyzer.hasAggregation() + && !query_analyzer.hasWindow() + && !query.final() + && join_allow_read_in_order; + ExpressionActionsChain chain(context); - Names additional_required_columns_after_prewhere = metadata_snapshot ? metadata_snapshot->getColumnsRequiredForSortingKey() : Names{}; + Names additional_required_columns_after_prewhere; if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { @@ -1867,6 +1884,14 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( columns_for_final.begin(), columns_for_final.end()); } + if (storage && optimize_read_in_order) + { + std::cerr << "!!!!!!!!!!!!!!\n"; + Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); + additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), + columns_for_sorting_key.begin(), columns_for_sorting_key.end()); + } + if (storage && filter_info_) { filter_info = filter_info_; @@ -1951,23 +1976,6 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( } } - bool join_allow_read_in_order = true; - if (hasJoin()) - { - /// You may find it strange but we support read_in_order for HashJoin and do not support for MergeJoin. - join_has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows(); - join_allow_read_in_order = typeid_cast(join.get()) && !join_has_delayed_stream; - } - - optimize_read_in_order = - settings.optimize_read_in_order - && storage - && query.orderBy() - && !query_analyzer.hasAggregation() - && !query_analyzer.hasWindow() - && !query.final() - && join_allow_read_in_order; - /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. query_analyzer.appendSelect(chain, only_types || (need_aggregate ? !second_stage : !first_stage)); diff --git a/tests/queries/0_stateless/02354_read_in_order_prewhere.reference b/tests/queries/0_stateless/02354_read_in_order_prewhere.reference new file mode 100644 index 00000000000..7d5543bf9cc --- /dev/null +++ b/tests/queries/0_stateless/02354_read_in_order_prewhere.reference @@ -0,0 +1,10 @@ +1 +1 +1 +2001 +2001 +1 +1 +1 +2001 +2001 diff --git a/tests/queries/0_stateless/02354_read_in_order_prewhere.sql b/tests/queries/0_stateless/02354_read_in_order_prewhere.sql new file mode 100644 index 00000000000..c5abd5945f3 --- /dev/null +++ b/tests/queries/0_stateless/02354_read_in_order_prewhere.sql @@ -0,0 +1,30 @@ +drop table if exists order; + +CREATE TABLE order +( + ID Int64, + Type Int64, + Num UInt64, + t DateTime +) +ENGINE = MergeTree() +PARTITION BY toYYYYMMDD(t) +ORDER BY (ID, Type, Num); + +system stop merges order; + +insert into order select number%2000, 1, number, (1656700561 - intDiv(intHash32(number), 1000)) from numbers(100000); +insert into order select number%2000, 1, number, (1656700561 - intDiv(intHash32(number), 1000)) from numbers(100000); +insert into order select number%2000, 1, number, (1656700561 - intDiv(intHash32(number), 1000)) from numbers(100000); + +SELECT Num +FROM order +WHERE Type = 1 AND ID = 1 +ORDER BY Num ASC limit 5; + +SELECT Num +FROM order +PREWHERE Type = 1 +WHERE ID = 1 +ORDER BY Num ASC limit 5; + From dc20b850781020bb94572d3c3762abd379ab3bae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jul 2022 21:08:23 +0000 Subject: [PATCH 579/627] Remove debug output. --- src/Interpreters/ExpressionAnalyzer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b439583bd0a..e2c106ff796 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1894,7 +1894,6 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && optimize_read_in_order) { - std::cerr << "!!!!!!!!!!!!!!\n"; Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), columns_for_sorting_key.begin(), columns_for_sorting_key.end()); From add169278b65406c71af3f85b4a88fb702e70564 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 13 Jul 2022 19:27:33 -0400 Subject: [PATCH 580/627] fix indent and link --- .../sql-reference/table-functions/remote.md | 70 +++++++++---------- 1 file changed, 35 insertions(+), 35 deletions(-) diff --git a/docs/en/sql-reference/table-functions/remote.md b/docs/en/sql-reference/table-functions/remote.md index 5948fa6f5e1..61018a3d5a7 100644 --- a/docs/en/sql-reference/table-functions/remote.md +++ b/docs/en/sql-reference/table-functions/remote.md @@ -91,50 +91,50 @@ SELECT * FROM remote_table; This example uses one table from a sample dataset. The database is `imdb`, and the table is `actors`. #### On the source ClickHouse system (the system that currently hosts the data) -- Verify the source database and table name (`imdb.actors`) -```sql -show databases -``` +- Verify the source database and table name (`imdb.actors`) + ```sql + show databases + ``` -```sql -show tables in imdb -``` + ```sql + show tables in imdb + ``` - Get the CREATE TABLE statement from the source: -``` -select create_table_query -from system.tables -where database = 'imdb' and table = 'actors' -``` + ``` + select create_table_query + from system.tables + where database = 'imdb' and table = 'actors' + ``` -Response -```sql -CREATE TABLE imdb.actors (`id` UInt32, - `first_name` String, - `last_name` String, - `gender` FixedString(1)) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') - ORDER BY (id, first_name, last_name, gender) - SETTINGS index_granularity = 8192 -``` + Response + ```sql + CREATE TABLE imdb.actors (`id` UInt32, + `first_name` String, + `last_name` String, + `gender` FixedString(1)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') + ORDER BY (id, first_name, last_name, gender) + SETTINGS index_granularity = 8192 + ``` #### On the destination ClickHouse system: - Create the destination database: -```sql -CREATE DATABASE imdb -``` + ```sql + CREATE DATABASE imdb + ``` - Using the CREATE TABLE statement from the source, create the destination: -```sql -CREATE TABLE imdb.actors (`id` UInt32, - `first_name` String, - `last_name` String, - `gender` FixedString(1)) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') - ORDER BY (id, first_name, last_name, gender) - SETTINGS index_granularity = 8192 -``` + ```sql + CREATE TABLE imdb.actors (`id` UInt32, + `first_name` String, + `last_name` String, + `gender` FixedString(1)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') + ORDER BY (id, first_name, last_name, gender) + SETTINGS index_granularity = 8192 + ``` #### Back on the source deployment: @@ -145,7 +145,7 @@ remoteSecure('remote.clickhouse.cloud:9440', 'imdb.actors', 'USER', 'PASSWORD', SELECT * from imdb.actors ``` -## Globs in Addresses {globs-in-addresses} +## Globs in Addresses {#globs-in-addresses} Patterns in curly brackets `{ }` are used to generate a set of shards and to specify replicas. If there are multiple pairs of curly brackets, then the direct product of the corresponding sets is generated. The following pattern types are supported. From d108efa2f625a5c76bac0f15a2998cfa4eb12868 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 14 Jul 2022 01:36:45 -0400 Subject: [PATCH 581/627] merge_algorithm is added --- docs/en/operations/system-tables/part_log.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/operations/system-tables/part_log.md b/docs/en/operations/system-tables/part_log.md index 298667f895b..e1134f6baf6 100644 --- a/docs/en/operations/system-tables/part_log.md +++ b/docs/en/operations/system-tables/part_log.md @@ -19,6 +19,10 @@ The `system.part_log` table contains the following columns: - `REGULAR_MERGE` — Some regular merge. - `TTL_DELETE_MERGE` — Cleaning up expired data. - `TTL_RECOMPRESS_MERGE` — Recompressing data part with the. +- `merge_algorithm` ([Enum8](../../sql-reference/data-types/enum.md)) — Merge algorithm for the event with type `MERGE_PARTS`. Can have one of the following values: + - `UNDECIDED` + - `HORIZONTAL` + - `VERTICAL` - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time. - `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds precision. @@ -52,6 +56,7 @@ Row 1: query_id: 983ad9c7-28d5-4ae1-844e-603116b7de31 event_type: NewPart merge_reason: NotAMerge +merge_algorithm: Undecided event_date: 2021-02-02 event_time: 2021-02-02 11:14:28 event_time_microseconds: 2021-02-02 11:14:28.861919 From 887a34869dfa7e691e4b081a66e41854a56029eb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 14 Jul 2022 01:44:19 -0400 Subject: [PATCH 582/627] merge_reason and merge_algorithm are added --- docs/ru/operations/system-tables/part_log.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/ru/operations/system-tables/part_log.md b/docs/ru/operations/system-tables/part_log.md index 78e9a7c0fbe..73ded8c7355 100644 --- a/docs/ru/operations/system-tables/part_log.md +++ b/docs/ru/operations/system-tables/part_log.md @@ -14,6 +14,15 @@ - `REMOVE_PART` — удаление или отсоединение из таблицы с помощью [DETACH PARTITION](../../sql-reference/statements/alter/partition.md#alter_detach-partition). - `MUTATE_PART` — изменение куска. - `MOVE_PART` — перемещение куска между дисками. +- `merge_reason` ([Enum8](../../sql-reference/data-types/enum.md)) — Причина события с типом `MERGE_PARTS`. Может принимать одно из следующих значений: + - `NOT_A_MERGE` — событие имеет тип иной, чем `MERGE_PARTS`. + - `REGULAR_MERGE` — обычное слияние. + - `TTL_DELETE_MERGE` — очистка истекших данных. + - `TTL_RECOMPRESS_MERGE` — переупаковка куска. +- `merge_algorithm` ([Enum8](../../sql-reference/data-types/enum.md)) — Алгоритм слияния для события с типом `MERGE_PARTS`. Может принимать одно из следующих значений: + - `UNDECIDED` + - `HORIZONTAL` + - `VERTICAL` - `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата события. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время события. - `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — время события с точностью до микросекунд. @@ -46,6 +55,8 @@ Row 1: ────── query_id: 983ad9c7-28d5-4ae1-844e-603116b7de31 event_type: NewPart +merge_reason: NotAMerge +merge_algorithm: Undecided event_date: 2021-02-02 event_time: 2021-02-02 11:14:28 event_time_microseconds: 2021-02-02 11:14:28.861919 From 3e77b80b882ee34b502d8ec0caa0405da2a58124 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Jul 2022 11:38:46 +0300 Subject: [PATCH 583/627] Fix fetch of in-memory part with allow_remote_fs_zero_copy_replication CI founds the following error during trying to fetch in-memory part [1]: 2022.07.13 08:06:54.231033 [ 141093 ] {} InterserverIOHTTPHandler: Code: 107. DB::ErrnoException: Cannot open file /var/lib/clickhouse/disks/s3/store/886/88635b40-e4e3-4fe6-a0a0-1b6755463358/all_0_0_0/data.bin, errno: 2, strerror: No such file or directory. (FILE_DOESNT_EXIST), Stack trace (when copying this message, always include the lines below):

stacktrace 2022.07.13 08:06:54.101825 [ 189342 ] {8dbd11b3-f38a-4d5d-9ded-148987adb71d} executeQuery: (from [::1]:54570) (comment: 01643_replicated_merge_tree_fsync_smoke.sql) select 'memory in_memory_parts_insert_sync'; (stage: Complete) 2022.07.13 08:06:54.131309 [ 691 ] {} test_26u6kx.rep_fsync_r2 (39c3823c-22e5-4c05-9dec-cdffd8872c40): Fetching part all_0_0_0 from /clickhouse/tables/test_26u6kx/rep_fsync/replicas/r1 2022.07.13 08:06:54.231033 [ 141093 ] {} InterserverIOHTTPHandler: Code: 107. DB::ErrnoException: Cannot open file /var/lib/clickhouse/disks/s3/store/886/88635b40-e4e3-4fe6-a0a0-1b6755463358/all_0_0_0/data.bin, errno: 2, strerror: No such file or directory. (FILE_DOESNT_EXIST), Stack trace (when copying this message, always include the lines below): 0. DB::Exception::Exception() @ 0xba0191a in /usr/bin/clickhouse 1. DB::throwFromErrnoWithPath() @ 0xba029ca in /usr/bin/clickhouse 2. DB::OpenedFile::open() const @ 0x156e7fb0 in /usr/bin/clickhouse 3. DB::OpenedFile::getFD() const @ 0x156e8003 in /usr/bin/clickhouse 4. DB::ReadBufferFromFilePReadWithDescriptorsCache::ReadBufferFromFilePReadWithDescriptorsCache() @ 0x156e5f23 in /usr/bin/clickhouse 5. ? @ 0x156e53f0 in /usr/bin/clickhouse 6. DB::createReadBufferFromFileBase() @ 0x156e52b5 in /usr/bin/clickhouse 7. DB::DiskLocal::readFile() const @ 0x15e45ea8 in /usr/bin/clickhouse 8. DB::MetadataStorageFromDisk::readFileToString() const @ 0x15e6ab8b in /usr/bin/clickhouse 9. DB::MetadataStorageFromDisk::readMetadataUnlocked() const @ 0x15e6cdeb in /usr/bin/clickhouse 10. DB::MetadataStorageFromDisk::getSerializedMetadata() const @ 0x15e6cfc4 in /usr/bin/clickhouse 11. DB::DiskObjectStorage::getSerializedMetadata() const @ 0x15e19e2e in /usr/bin/clickhouse 12. DB::DiskDecorator::getSerializedMetadata() const @ 0x15e54ed1 in /usr/bin/clickhouse 13. DB::DiskDecorator::getSerializedMetadata() const @ 0x15e54ed1 in /usr/bin/clickhouse 14. DB::DataPartsExchange::Service::sendPartFromDiskRemoteMeta() @ 0x1700bb9e in /usr/bin/clickhouse 15. DB::DataPartsExchange::Service::processQuery(DB::HTMLForm const&, DB::ReadBuffer&, DB::WriteBuffer&, DB::HTTPServerResponse&) @ 0x1700a649 in /usr/bin/clickhouse 16. DB::InterserverIOHTTPHandler::processQuery(DB::HTTPServerRequest&, DB::HTTPServerResponse&, DB::InterserverIOHTTPHandler::Output&) @ 0x17433c53 in /usr/bin/clickhouse 17. DB::InterserverIOHTTPHandler::handleRequest(DB::HTTPServerRequest&, DB::HTTPServerResponse&) @ 0x174344f1 in /usr/bin/clickhouse 18. DB::HTTPServerConnection::run() @ 0x1768714d in /usr/bin/clickhouse 19. Poco::Net::TCPServerConnection::start() @ 0x1a398093 in /usr/bin/clickhouse 20. Poco::Net::TCPServerDispatcher::run() @ 0x1a399411 in /usr/bin/clickhouse 21. Poco::PooledThread::run() @ 0x1a54b7bb in /usr/bin/clickhouse 22. Poco::ThreadImpl::runnableEntry(void*) @ 0x1a548ec0 in /usr/bin/clickhouse 23. ? @ 0x7fdf1c204609 in ? 24. clone @ 0x7fdf1c129133 in ? (version 22.7.1.1781 (official build))
[1]: https://s3.amazonaws.com/clickhouse-test-reports/0/8b6e31cc615ca52c80724b6e5097777cb9514f07/stateless_tests__release__s3_storage__actions_.html Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/DataPartsExchange.cpp | 2 ++ .../0_stateless/01643_replicated_merge_tree_fsync_smoke.sql | 3 +-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 0e655166abf..9e18dbc6281 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -174,6 +174,8 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write std::sregex_token_iterator()); if (data_settings->allow_remote_fs_zero_copy_replication && + /// In memory data part does not have metadata yet. + !isInMemoryPart(part) && client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_ZERO_COPY) { auto disk_type = part->data_part_storage->getDiskType(); diff --git a/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql b/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql index ee0617e42a3..dadd7eaba6c 100644 --- a/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql +++ b/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql @@ -1,6 +1,5 @@ --- Tags: no-parallel, no-s3-storage +-- Tags: no-parallel -- no-parallel -- for flaky check and to avoid "Removing leftovers from table" (for other tables) --- no-s3-storage -- hangs now, need investigation -- Temporarily skip warning 'table was created by another server at the same moment, will retry' set send_logs_level='error'; From 531e3b13ce5d960a94d9a015dcd07a929bfe98bd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Jul 2022 09:06:47 +0300 Subject: [PATCH 584/627] Optimize accesses to system.stack_trace (filter by name before sending signal) Because of failed rebase/push this part of the patch was lost, so resending separatelly (sigh). Follow-up for: #39177 (cc @yakov-olkhovskiy) Signed-off-by: Azat Khuzhin --- src/Storages/System/StorageSystemStackTrace.cpp | 17 ++++++++++++----- .../01051_system_stack_trace.reference | 3 +++ .../0_stateless/01051_system_stack_trace.sql | 2 ++ 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index 4eef69d8634..cdd04964f55 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -206,10 +206,12 @@ namespace filtered_thread_names.emplace(thread_name); } - for (const auto & [tid, name] : tid_to_name) + for (auto it = tid_to_name.begin(); it != tid_to_name.end();) { - if (!filtered_thread_names.contains(name)) - tid_to_name.erase(tid); + if (!filtered_thread_names.contains(it->second)) + it = tid_to_name.erase(it); + else + ++it; } return tid_to_name; @@ -302,8 +304,13 @@ Pipe StorageSystemStackTrace::read( size_t res_index = 0; String thread_name; - if (auto it = thread_names.find(tid); it != thread_names.end()) - thread_name = it->second; + if (read_thread_names) + { + if (auto it = thread_names.find(tid); it != thread_names.end()) + thread_name = it->second; + else + continue; /// was filtered out by "thread_name" condition + } if (!send_signal) { diff --git a/tests/queries/0_stateless/01051_system_stack_trace.reference b/tests/queries/0_stateless/01051_system_stack_trace.reference index ab01c749f41..b82bda76142 100644 --- a/tests/queries/0_stateless/01051_system_stack_trace.reference +++ b/tests/queries/0_stateless/01051_system_stack_trace.reference @@ -13,3 +13,6 @@ SELECT length(query_id) > 0 FROM system.stack_trace WHERE query_id != '' LIMIT 1 -- optimization for thread_name SELECT length(thread_name) > 0 FROM system.stack_trace WHERE thread_name != '' LIMIT 1; 1 +-- enough rows (optimizations works "correctly") +SELECT count() > 100 FROM system.stack_trace; +1 diff --git a/tests/queries/0_stateless/01051_system_stack_trace.sql b/tests/queries/0_stateless/01051_system_stack_trace.sql index c712a1367c0..d018d01fa22 100644 --- a/tests/queries/0_stateless/01051_system_stack_trace.sql +++ b/tests/queries/0_stateless/01051_system_stack_trace.sql @@ -10,3 +10,5 @@ SELECT length(trace) > 0 FROM system.stack_trace LIMIT 1; SELECT length(query_id) > 0 FROM system.stack_trace WHERE query_id != '' LIMIT 1; -- optimization for thread_name SELECT length(thread_name) > 0 FROM system.stack_trace WHERE thread_name != '' LIMIT 1; +-- enough rows (optimizations works "correctly") +SELECT count() > 100 FROM system.stack_trace; From 35d3a0dd1fe6783a73c94060231fb971e5752fa7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 14 Jul 2022 11:08:26 +0200 Subject: [PATCH 585/627] Don't report system.errors when the disk is not rotational --- src/Common/filesystemHelpers.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 00764269269..6f6afcac657 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -87,7 +87,10 @@ BlockDeviceType getBlockDeviceType([[maybe_unused]] const String & device_id) #if defined(OS_LINUX) try { - ReadBufferFromFile in("/sys/dev/block/" + device_id + "/queue/rotational"); + const String path = "/sys/dev/block/" + device_id + "/queue/rotational"; + if (!std::filesystem::exists(path)) + return BlockDeviceType::UNKNOWN; + ReadBufferFromFile in(path); int rotational; readText(rotational, in); return rotational ? BlockDeviceType::ROT : BlockDeviceType::NONROT; From da4724e7f5701c3fb0478ff1b479b5a772f27c54 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 09:13:08 +0000 Subject: [PATCH 586/627] Enable -Wreturn-std-move-in-c++11 --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index b8fc4229ad9..60b50ae8b38 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -40,7 +40,6 @@ if (COMPILER_CLANG) no_warning(nested-anon-types) no_warning(packed) no_warning(padded) - no_warning(return-std-move-in-c++11) # clang 7+ no_warning(shift-sign-overflow) no_warning(sign-conversion) no_warning(switch-enum) From 439b78615612770afc714ad137d4db59c4b20b78 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 09:54:00 +0000 Subject: [PATCH 587/627] Enable -Wunused-command-line-argument --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 60b50ae8b38..10ccdcc5305 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -23,7 +23,6 @@ if (COMPILER_CLANG) no_warning(vla-extension) no_warning(zero-length-array) no_warning(c11-extensions) - no_warning(unused-command-line-argument) no_warning(c++98-compat-pedantic) no_warning(c++98-compat) no_warning(c99-extensions) From 72d45bdff3d7bc097586ea7271f9c08659c7c420 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 10:17:32 +0000 Subject: [PATCH 588/627] Enable -Wvla-extension --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 10ccdcc5305..9e59b96084c 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -20,7 +20,6 @@ if (COMPILER_CLANG) # We want to get everything out of the compiler for code quality. add_warning(everything) add_warning(pedantic) - no_warning(vla-extension) no_warning(zero-length-array) no_warning(c11-extensions) no_warning(c++98-compat-pedantic) From 3d734a0640891dd153f5366aefa16c7db8f2cbb3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 11:42:25 +0000 Subject: [PATCH 589/627] Enable -Wnested-anon-types --- cmake/warnings.cmake | 1 - src/Common/TaskStatsInfoGetter.cpp | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 9e59b96084c..a4e9adfba12 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -35,7 +35,6 @@ if (COMPILER_CLANG) no_warning(global-constructors) no_warning(missing-prototypes) no_warning(missing-variable-declarations) - no_warning(nested-anon-types) no_warning(packed) no_warning(padded) no_warning(shift-sign-overflow) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index 36e8a0fce00..304ccc84765 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -21,6 +21,7 @@ #if defined(__clang__) #pragma clang diagnostic ignored "-Wgnu-anonymous-struct" + #pragma clang diagnostic ignored "-Wnested-anon-types" #endif /// Basic idea is motivated by "iotop" tool. From e7bc973ffb81143572179452e9d98f503b63dcde Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 12:29:42 +0000 Subject: [PATCH 590/627] Enable -Wshift-sign-overflow --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index a4e9adfba12..ea279870b1d 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -37,7 +37,6 @@ if (COMPILER_CLANG) no_warning(missing-variable-declarations) no_warning(packed) no_warning(padded) - no_warning(shift-sign-overflow) no_warning(sign-conversion) no_warning(switch-enum) no_warning(undefined-func-template) From 97aa9a73710d8d361e3b91a86e12e613fd60a44a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 12:29:54 +0000 Subject: [PATCH 591/627] Enable -Wpacked --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index ea279870b1d..7da73b0dd08 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -35,7 +35,6 @@ if (COMPILER_CLANG) no_warning(global-constructors) no_warning(missing-prototypes) no_warning(missing-variable-declarations) - no_warning(packed) no_warning(padded) no_warning(sign-conversion) no_warning(switch-enum) From 32e48f661ad1b6fa26183fc1050982f9effb6794 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 14 Jul 2022 12:51:56 +0000 Subject: [PATCH 592/627] Fixed query_log result_bytes metric that showed inseted rows instead of bytes --- src/Interpreters/executeQuery.cpp | 2 +- .../0_stateless/02356_insert_query_log_metrics.reference | 1 + tests/queries/0_stateless/02356_insert_query_log_metrics.sql | 4 ++++ 3 files changed, 6 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02356_insert_query_log_metrics.reference create mode 100644 tests/queries/0_stateless/02356_insert_query_log_metrics.sql diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 85c4ea261a0..ae915aab867 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -876,7 +876,7 @@ static std::tuple executeQueryImpl( { auto progress_out = process_list_elem->getProgressOut(); elem.result_rows = progress_out.written_rows; - elem.result_bytes = progress_out.written_rows; + elem.result_bytes = progress_out.written_bytes; } if (elem.read_rows != 0) diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.reference b/tests/queries/0_stateless/02356_insert_query_log_metrics.reference new file mode 100644 index 00000000000..0d707252cc7 --- /dev/null +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.reference @@ -0,0 +1 @@ +1,1,1,1 diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql new file mode 100644 index 00000000000..3825f37183b --- /dev/null +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql @@ -0,0 +1,4 @@ +CREATE TABLE 02356_destination (a Int64, b String) ENGINE = Memory; + +INSERT INTO 02356_destination (a, b) SELECT * FROM generateRandom('a Int64, b String') LIMIT 100 SETTINGS max_threads=1, max_block_size=100; +SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' FORMAT CSV; \ No newline at end of file From 9955b097d405346d86c33a781fba70b4635b3c33 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 14 Jul 2022 09:00:54 -0400 Subject: [PATCH 593/627] spelling --- docs/en/sql-reference/table-functions/null.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/null.md b/docs/en/sql-reference/table-functions/null.md index ae84705cb66..57a885afc54 100644 --- a/docs/en/sql-reference/table-functions/null.md +++ b/docs/en/sql-reference/table-functions/null.md @@ -5,7 +5,7 @@ sidebar_label: null function # null -Creates a temporary table of the specified structure with the [Null](../../engines/table-engines/special/null.md) table engine. According to the `Null`-engine properties, the table data is ignored and the table itself is immediately droped right after the query execution. The function is used for the convenience of test writing and demonstrations. +Creates a temporary table of the specified structure with the [Null](../../engines/table-engines/special/null.md) table engine. According to the `Null`-engine properties, the table data is ignored and the table itself is immediately dropped right after the query execution. The function is used for the convenience of test writing and demonstrations. **Syntax** @@ -40,4 +40,3 @@ See also: - [Null table engine](../../engines/table-engines/special/null.md) -[Original article](https://clickhouse.com/docs/en/sql-reference/table-functions/null/) From 0804ff9284fcbc74fa9cd6ea6408ade7379e7db6 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Thu, 14 Jul 2022 07:12:59 -0700 Subject: [PATCH 594/627] Update as suggested --- contrib/thrift-cmake/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 3454b1b87e3..832e104e8a3 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -92,5 +92,5 @@ include_directories("${CMAKE_CURRENT_BINARY_DIR}") add_library(_thrift ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) add_library(ch_contrib::thrift ALIAS _thrift) -target_include_directories(_thrift SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src" ${CMAKE_CURRENT_BINARY_DIR} "${ClickHouse_SOURCE_DIR}/contrib/boringssl/include") -target_link_libraries (_thrift PUBLIC boost::headers_only) +target_include_directories(_thrift SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src" ${CMAKE_CURRENT_BINARY_DIR}) +target_link_libraries (_thrift PUBLIC OpenSSL::SSL) From b20407fab9e5e4e7dd3e1298a01191d6afe1718b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Jul 2022 16:46:01 +0300 Subject: [PATCH 595/627] Fix NOEXCEPT_SCOPE (before it calls std::terminate and looses the exception) Current implementation of NOEXCEPT_SCOPE will not work, you cannot rethrow exception outside the catch block, this will simply terminate (via std::terminate) the program. In other words NOEXCEPT_SCOPE macro will simply call std::terminate on exception and will lost original exception. But if NOEXCEPT_SCOPE will accept the code that should be runned w/o exceptions, then it can catch exception and log it, rewrite it in this way. Signed-off-by: Azat Khuzhin --- src/Common/noexcept_scope.h | 46 +++---- src/Interpreters/DatabaseCatalog.cpp | 9 +- src/Interpreters/MergeTreeTransaction.cpp | 11 +- src/Interpreters/ThreadStatusExt.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 15 +-- .../MergeTree/MergeTreeBackgroundExecutor.cpp | 61 +++++---- src/Storages/MergeTree/MergeTreeData.cpp | 118 +++++++++--------- 7 files changed, 132 insertions(+), 130 deletions(-) diff --git a/src/Common/noexcept_scope.h b/src/Common/noexcept_scope.h index 56fb44ff0bf..bdd7a98925a 100644 --- a/src/Common/noexcept_scope.h +++ b/src/Common/noexcept_scope.h @@ -1,36 +1,28 @@ #pragma once -#include #include #include - -#define NOEXCEPT_SCOPE_IMPL_CONCAT(n, expected) \ - LockMemoryExceptionInThread lock_memory_tracker##n(VariableContext::Global); \ - SCOPE_EXIT( \ - { \ - const auto uncaught = std::uncaught_exceptions(); \ - assert((expected) == uncaught || (expected) + 1 == uncaught); \ - if ((expected) < uncaught) \ - { \ - tryLogCurrentException("NOEXCEPT_SCOPE"); \ - abort(); \ - } \ - } \ - ) - -#define NOEXCEPT_SCOPE_IMPL(n, expected) NOEXCEPT_SCOPE_IMPL_CONCAT(n, expected) - -#define NOEXCEPT_SCOPE_CONCAT(n) \ - const auto num_curr_exceptions##n = std::uncaught_exceptions(); \ - NOEXCEPT_SCOPE_IMPL(n, num_curr_exceptions##n) - -#define NOEXCEPT_SCOPE_FWD(n) NOEXCEPT_SCOPE_CONCAT(n) - - /// It can be used in critical places to exit on unexpected exceptions. /// SIGABRT is usually better that broken in-memory state with unpredictable consequences. /// It also temporarily disables exception from memory tracker in current thread. /// Strict version does not take into account nested exception (i.e. it aborts even when we're in catch block). -#define NOEXCEPT_SCOPE_STRICT NOEXCEPT_SCOPE_IMPL(__LINE__, 0) -#define NOEXCEPT_SCOPE NOEXCEPT_SCOPE_FWD(__LINE__) +#define NOEXCEPT_SCOPE_IMPL(...) do { \ + LockMemoryExceptionInThread \ + noexcept_lock_memory_tracker(VariableContext::Global); \ + try \ + { \ + __VA_ARGS__; \ + } \ + catch (...) \ + { \ + DB::tryLogCurrentException(__PRETTY_FUNCTION__); \ + std::terminate(); \ + } \ +} while (0) /* to allow leading semi-colon */ + +#define NOEXCEPT_SCOPE_STRICT(...) \ + if (std::uncaught_exceptions()) std::terminate(); \ + NOEXCEPT_SCOPE_IMPL(__VA_ARGS__) + +#define NOEXCEPT_SCOPE(...) NOEXCEPT_SCOPE_IMPL(__VA_ARGS__) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index bae2aed2cd5..aefba3aaa2d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -403,10 +403,11 @@ void DatabaseCatalog::attachDatabase(const String & database_name, const Databas std::lock_guard lock{databases_mutex}; assertDatabaseDoesntExistUnlocked(database_name); databases.emplace(database_name, database); - NOEXCEPT_SCOPE; - UUID db_uuid = database->getUUID(); - if (db_uuid != UUIDHelpers::Nil) - addUUIDMapping(db_uuid, database, nullptr); + NOEXCEPT_SCOPE({ + UUID db_uuid = database->getUUID(); + if (db_uuid != UUIDHelpers::Nil) + addUUIDMapping(db_uuid, database, nullptr); + }); } diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 432116feaf5..2a57d637333 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -149,11 +149,12 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP checkIsNotCancelled(); part_to_remove->version.lockRemovalTID(tid, context); - NOEXCEPT_SCOPE; - storages.insert(storage); - if (maybe_lock) - table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock)); - removing_parts.push_back(part_to_remove); + NOEXCEPT_SCOPE({ + storages.insert(storage); + if (maybe_lock) + table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock)); + removing_parts.push_back(part_to_remove); + }); } part_to_remove->appendRemovalTIDToVersionMetadata(); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 53d7fd0457a..9ad3dc7c4a9 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -343,7 +343,7 @@ void ThreadStatus::finalizeQueryProfiler() void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) { - NOEXCEPT_SCOPE; + LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery) { diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index a08f940a748..ec7969952f5 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -140,8 +140,7 @@ void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_ite } futures.clear(); - NOEXCEPT_SCOPE_STRICT; - { + NOEXCEPT_SCOPE_STRICT({ std::lock_guard lock{mutex}; for (const auto & entry : loaded) { @@ -151,7 +150,8 @@ void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_ite tid_to_csn.emplace(entry.first, entry.second); } last_loaded_entry = last_entry; - } + }); + { std::lock_guard lock{running_list_mutex}; latest_snapshot = loaded.back().second.csn; @@ -445,10 +445,11 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn, bool /// Do not allow exceptions between commit point and the and of transaction finalization /// (otherwise it may stuck in COMMITTING state holding snapshot). - NOEXCEPT_SCOPE_STRICT; - /// FIXME Transactions: Sequential node numbers in ZooKeeper are Int32, but 31 bit is not enough for production use - /// (overflow is possible in a several weeks/months of active usage) - allocated_csn = deserializeCSN(csn_path_created.substr(zookeeper_path_log.size() + 1)); + NOEXCEPT_SCOPE_STRICT({ + /// FIXME Transactions: Sequential node numbers in ZooKeeper are Int32, but 31 bit is not enough for production use + /// (overflow is possible in a several weeks/months of active usage) + allocated_csn = deserializeCSN(csn_path_created.substr(zookeeper_path_log.size() + 1)); + }); } return finalizeCommittedTransaction(txn.get(), allocated_csn, state_guard); diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index c3f83771338..1f7ecb39a1d 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -138,18 +138,20 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) } catch (const Exception & e) { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. - LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); - else - tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. + LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); + else + tryLogCurrentException(__PRETTY_FUNCTION__); + }); } catch (...) { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + tryLogCurrentException(__PRETTY_FUNCTION__); + }); } if (need_execute_again) @@ -162,9 +164,10 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) /// This is significant to order the destructors. { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - item->task.reset(); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + item->task.reset(); + }); } item->is_done.set(); item = nullptr; @@ -197,18 +200,20 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) } catch (const Exception & e) { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. - LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); - else - tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. + LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); + else + tryLogCurrentException(__PRETTY_FUNCTION__); + }); } catch (...) { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + tryLogCurrentException(__PRETTY_FUNCTION__); + }); } @@ -218,9 +223,10 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) /// The thread that shutdowns storage will scan queues in order to find some tasks to wait for, but will find nothing. /// So, the destructor of a task and the destructor of a storage will be executed concurrently. { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - item->task.reset(); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + item->task.reset(); + }); } item->is_done.set(); @@ -256,9 +262,10 @@ void MergeTreeBackgroundExecutor::threadFunction() } catch (...) { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + tryLogCurrentException(__PRETTY_FUNCTION__); + }); } } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5900ea0fdb7..b823899d3fe 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4942,78 +4942,78 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: } } - NOEXCEPT_SCOPE; + NOEXCEPT_SCOPE({ + auto current_time = time(nullptr); - auto current_time = time(nullptr); + size_t add_bytes = 0; + size_t add_rows = 0; + size_t add_parts = 0; - size_t add_bytes = 0; - size_t add_rows = 0; - size_t add_parts = 0; + size_t reduce_bytes = 0; + size_t reduce_rows = 0; + size_t reduce_parts = 0; - size_t reduce_bytes = 0; - size_t reduce_rows = 0; - size_t reduce_parts = 0; - - for (const DataPartPtr & part : precommitted_parts) - { - auto part_in_memory = asInMemoryPart(part); - if (part_in_memory && settings->in_memory_parts_enable_wal) + for (const DataPartPtr & part : precommitted_parts) { - if (!wal) - wal = data.getWriteAheadLog(); - - wal->addPart(part_in_memory); - } - - DataPartPtr covering_part; - DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, *owing_parts_lock); - if (covering_part) - { - LOG_WARNING(data.log, "Tried to commit obsolete part {} covered by {}", part->name, covering_part->getNameWithState()); - - part->remove_time.store(0, std::memory_order_relaxed); /// The part will be removed without waiting for old_parts_lifetime seconds. - data.modifyPartState(part, DataPartState::Outdated); - } - else - { - if (!txn) - MergeTreeTransaction::addNewPartAndRemoveCovered(data.shared_from_this(), part, covered_parts, NO_TRANSACTION_RAW); - - total_covered_parts.insert(total_covered_parts.end(), covered_parts.begin(), covered_parts.end()); - for (const auto & covered_part : covered_parts) + auto part_in_memory = asInMemoryPart(part); + if (part_in_memory && settings->in_memory_parts_enable_wal) { - covered_part->remove_time.store(current_time, std::memory_order_relaxed); + if (!wal) + wal = data.getWriteAheadLog(); - reduce_bytes += covered_part->getBytesOnDisk(); - reduce_rows += covered_part->rows_count; - - data.modifyPartState(covered_part, DataPartState::Outdated); - data.removePartContributionToColumnAndSecondaryIndexSizes(covered_part); + wal->addPart(part_in_memory); } - reduce_parts += covered_parts.size(); + DataPartPtr covering_part; + DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, *owing_parts_lock); + if (covering_part) + { + LOG_WARNING(data.log, "Tried to commit obsolete part {} covered by {}", part->name, covering_part->getNameWithState()); - add_bytes += part->getBytesOnDisk(); - add_rows += part->rows_count; - ++add_parts; + part->remove_time.store(0, std::memory_order_relaxed); /// The part will be removed without waiting for old_parts_lifetime seconds. + data.modifyPartState(part, DataPartState::Outdated); + } + else + { + if (!txn) + MergeTreeTransaction::addNewPartAndRemoveCovered(data.shared_from_this(), part, covered_parts, NO_TRANSACTION_RAW); - data.modifyPartState(part, DataPartState::Active); - data.addPartContributionToColumnAndSecondaryIndexSizes(part); + total_covered_parts.insert(total_covered_parts.end(), covered_parts.begin(), covered_parts.end()); + for (const auto & covered_part : covered_parts) + { + covered_part->remove_time.store(current_time, std::memory_order_relaxed); + + reduce_bytes += covered_part->getBytesOnDisk(); + reduce_rows += covered_part->rows_count; + + data.modifyPartState(covered_part, DataPartState::Outdated); + data.removePartContributionToColumnAndSecondaryIndexSizes(covered_part); + } + + reduce_parts += covered_parts.size(); + + add_bytes += part->getBytesOnDisk(); + add_rows += part->rows_count; + ++add_parts; + + data.modifyPartState(part, DataPartState::Active); + data.addPartContributionToColumnAndSecondaryIndexSizes(part); + } } - } - if (reduce_parts == 0) - { - for (const auto & part : precommitted_parts) - data.updateObjectColumns(part, parts_lock); - } - else - data.resetObjectColumnsFromActiveParts(parts_lock); + if (reduce_parts == 0) + { + for (const auto & part : precommitted_parts) + data.updateObjectColumns(part, parts_lock); + } + else + data.resetObjectColumnsFromActiveParts(parts_lock); - ssize_t diff_bytes = add_bytes - reduce_bytes; - ssize_t diff_rows = add_rows - reduce_rows; - ssize_t diff_parts = add_parts - reduce_parts; - data.increaseDataVolume(diff_bytes, diff_rows, diff_parts); + ssize_t diff_bytes = add_bytes - reduce_bytes; + ssize_t diff_rows = add_rows - reduce_rows; + ssize_t diff_parts = add_parts - reduce_parts; + data.increaseDataVolume(diff_bytes, diff_rows, diff_parts); + }); } clear(); From f8e23c1deb8786729bfd1d9c7358f8264fea97ed Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 14 Jul 2022 17:29:08 +0300 Subject: [PATCH 596/627] Update run.sh --- docker/test/stress/run.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index cb80c8ebdc9..e34195cdd32 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -354,6 +354,8 @@ else # Error messages (we should ignore some errors) # FIXME https://github.com/ClickHouse/ClickHouse/issues/38643 ("Unknown index: idx.") # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64") + # FIXME Not sure if it's expected, but some tests from BC check may not be finished yet when we restarting server. + # Let's just ignore all errors from queries ("} TCPHandler: Code:", "} executeQuery: Code:") echo "Check for Error messages in server log:" zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ -e "Code: 236. DB::Exception: Cancelled mutating parts" \ @@ -378,6 +380,8 @@ else -e "is lost forever." \ -e "Unknown index: idx." \ -e "Cannot parse string 'Hello' as UInt64" \ + -e "} TCPHandler: Code:" \ + -e "} executeQuery: Code:" \ /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv From 22abb97b91d0a767478f5ee973e93edb5704bd17 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 12:30:14 +0000 Subject: [PATCH 597/627] Enable -Wc11-extensions --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 7da73b0dd08..0e55f6d32c3 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -21,7 +21,6 @@ if (COMPILER_CLANG) add_warning(everything) add_warning(pedantic) no_warning(zero-length-array) - no_warning(c11-extensions) no_warning(c++98-compat-pedantic) no_warning(c++98-compat) no_warning(c99-extensions) From 62149111d3ccde6cbf293e9cee19fecb3a28bb41 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 14:36:31 +0000 Subject: [PATCH 598/627] Fix build --- programs/copier/ZooKeeperStaff.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/copier/ZooKeeperStaff.h b/programs/copier/ZooKeeperStaff.h index 66036ae2f27..a9e04578607 100644 --- a/programs/copier/ZooKeeperStaff.h +++ b/programs/copier/ZooKeeperStaff.h @@ -20,7 +20,7 @@ public: bool operator<=(const WrappingUInt32 & other) const { - const UInt32 HALF = 1 << 31; + const UInt32 HALF = static_cast(1) << 31; return (value <= other.value && other.value - value < HALF) || (value > other.value && value - other.value > HALF); } From b2317ff710b65eb23bb6e7c87f3c67a0176185f8 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 14 Jul 2022 17:33:22 +0200 Subject: [PATCH 599/627] renaming to be consistent --- src/Disks/ObjectStorages/IObjectStorage.cpp | 8 ++++---- src/Disks/ObjectStorages/IObjectStorage.h | 4 ++-- src/IO/ReadBufferFromAzureBlobStorage.cpp | 4 ++-- src/IO/ReadBufferFromS3.cpp | 4 ++-- src/IO/ReadSettings.h | 2 +- src/IO/WriteBufferFromAzureBlobStorage.cpp | 4 ++-- src/IO/WriteBufferFromS3.cpp | 4 ++-- src/IO/WriteSettings.h | 2 +- src/Interpreters/Context.cpp | 4 ++-- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 4 ++-- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 4 ++-- 11 files changed, 22 insertions(+), 22 deletions(-) diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 70159b33b87..f9c5c139b95 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -50,15 +50,15 @@ std::string IObjectStorage::getCacheBasePath() const void IObjectStorage::applyRemoteThrottlingSettings(ContextPtr context) { std::unique_lock lock{throttlers_mutex}; - read_throttler = context->getRemoteReadThrottler(); - write_throttler = context->getRemoteWriteThrottler(); + remote_read_throttler = context->getRemoteReadThrottler(); + remote_write_throttler = context->getRemoteWriteThrottler(); } ReadSettings IObjectStorage::patchSettings(const ReadSettings & read_settings) const { std::unique_lock lock{throttlers_mutex}; ReadSettings settings{read_settings}; - settings.throttler = read_throttler; + settings.remote_throttler = remote_read_throttler; return settings; } @@ -66,7 +66,7 @@ WriteSettings IObjectStorage::patchSettings(const WriteSettings & write_settings { std::unique_lock lock{throttlers_mutex}; WriteSettings settings{write_settings}; - settings.throttler = write_throttler; + settings.remote_throttler = remote_write_throttler; return settings; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index da341a04a9b..06398b11aec 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -178,8 +178,8 @@ protected: private: mutable std::mutex throttlers_mutex; - ThrottlerPtr read_throttler; - ThrottlerPtr write_throttler; + ThrottlerPtr remote_read_throttler; + ThrottlerPtr remote_write_throttler; }; using ObjectStoragePtr = std::shared_ptr; diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp index 32e70305bea..3e6581cd786 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -77,8 +77,8 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() try { bytes_read = data_stream->ReadToCount(reinterpret_cast(data_ptr), to_read_bytes); - if (read_settings.throttler) - read_settings.throttler->add(bytes_read); + if (read_settings.remote_throttler) + read_settings.remote_throttler->add(bytes_read); break; } catch (const Azure::Storage::StorageException & e) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index ee21db62507..7fb432eab22 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -165,8 +165,8 @@ bool ReadBufferFromS3::nextImpl() ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, working_buffer.size()); offset += working_buffer.size(); - if (read_settings.throttler) - read_settings.throttler->add(working_buffer.size()); + if (read_settings.remote_throttler) + read_settings.remote_throttler->add(working_buffer.size()); return true; } diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 97e72bdb147..2a2691e3c06 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -91,7 +91,7 @@ struct ReadSettings FileCachePtr remote_fs_cache; /// Bandwidth throttler to use during reading - ThrottlerPtr throttler; + ThrottlerPtr remote_throttler; size_t http_max_tries = 1; size_t http_retry_initial_backoff_ms = 100; diff --git a/src/IO/WriteBufferFromAzureBlobStorage.cpp b/src/IO/WriteBufferFromAzureBlobStorage.cpp index 0b7b6323b33..bc7b505cd91 100644 --- a/src/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/IO/WriteBufferFromAzureBlobStorage.cpp @@ -88,8 +88,8 @@ void WriteBufferFromAzureBlobStorage::nextImpl() block_blob_client.CommitBlockList(block_ids); - if (write_settings.throttler) - write_settings.throttler->add(read); + if (write_settings.remote_throttler) + write_settings.remote_throttler->add(read); } } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 27210d50e44..51f0c0d0743 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -125,8 +125,8 @@ void WriteBufferFromS3::nextImpl() ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Bytes, offset()); last_part_size += offset(); - if (write_settings.throttler) - write_settings.throttler->add(offset()); + if (write_settings.remote_throttler) + write_settings.remote_throttler->add(offset()); /// Data size exceeds singlepart upload threshold, need to use multipart upload. if (multipart_upload_id.empty() && last_part_size > s3_settings.max_single_part_upload_size) diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 547ed0eb91b..7530b27794a 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -11,7 +11,7 @@ struct WriteSettings bool enable_filesystem_cache_on_write_operations = false; /// Bandwidth throttler to use during writing - ThrottlerPtr throttler; + ThrottlerPtr remote_throttler; }; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 70a0b201180..bce294d1221 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3458,7 +3458,7 @@ ReadSettings Context::getReadSettings() const res.mmap_threshold = settings.min_bytes_to_use_mmap_io; res.priority = settings.read_priority; - res.throttler = getRemoteReadThrottler(); + res.remote_throttler = getRemoteReadThrottler(); res.http_max_tries = settings.http_max_tries; res.http_retry_initial_backoff_ms = settings.http_retry_initial_backoff_ms; @@ -3476,7 +3476,7 @@ WriteSettings Context::getWriteSettings() const res.enable_filesystem_cache_on_write_operations = settings.enable_filesystem_cache_on_write_operations; - res.throttler = getRemoteWriteThrottler(); + res.remote_throttler = getRemoteWriteThrottler(); return res; } diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 8f22c4c2821..fab810a1e49 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -100,8 +100,8 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemoryadd(bytes_read); + if (read_settings.remote_throttler) + read_settings.remote_throttler->add(bytes_read); return true; } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 80acda52ffa..f8079d95f3c 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -58,8 +58,8 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl int write(const char * start, size_t size) const { int bytes_written = hdfsWrite(fs.get(), fout, start, size); - if (write_settings.throttler) - write_settings.throttler->add(bytes_written); + if (write_settings.remote_throttler) + write_settings.remote_throttler->add(bytes_written); if (bytes_written < 0) throw Exception("Fail to write HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), From 10625e7c5422ccbc3656e0290b714b061625e01e Mon Sep 17 00:00:00 2001 From: Xoel Lopez Barata Date: Thu, 14 Jul 2022 17:41:48 +0200 Subject: [PATCH 600/627] Fix typo Replace "indicies" by "indices" --- docs/en/sql-reference/functions/string-search-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 305e0c8c5cd..86408d8fc93 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -374,7 +374,7 @@ The same as `multiMatchAny`, but returns any index that matches the haystack. ## multiMatchAllIndices(haystack, \[pattern1, pattern2, …, patternn\]) -The same as `multiMatchAny`, but returns the array of all indicies that match the haystack in any order. +The same as `multiMatchAny`, but returns the array of all indices that match the haystack in any order. ## multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) From 9291d330804730e0426798822f99f51471b21bd1 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Jul 2022 16:11:35 +0000 Subject: [PATCH 601/627] Pass const std::string_view & by value, not by reference --- src/Access/AccessControl.cpp | 4 +- src/Access/AccessRights.cpp | 70 +++++++-------- src/Access/AccessRights.h | 66 +++++++------- src/Access/Authentication.cpp | 8 +- src/Access/Common/AccessFlags.cpp | 14 +-- src/Access/Common/AccessFlags.h | 2 +- src/Access/Common/AccessRightsElement.cpp | 14 +-- src/Access/Common/AccessRightsElement.h | 12 +-- src/Access/Common/AccessType.cpp | 2 +- src/Access/Common/AuthenticationData.cpp | 4 +- src/Access/Common/AuthenticationData.h | 8 +- src/Access/ContextAccess.cpp | 46 +++++----- src/Access/ContextAccess.h | 42 ++++----- src/Access/SettingsConstraints.cpp | 20 ++--- src/Access/SettingsConstraints.h | 20 ++--- src/Common/JSONParsers/DummyJSONParser.h | 4 +- src/Common/JSONParsers/RapidJSONParser.h | 4 +- src/Common/JSONParsers/SimdJSONParser.h | 4 +- src/Common/OpenSSLHelpers.cpp | 4 +- src/Common/OpenSSLHelpers.h | 4 +- src/Common/SettingsChanges.cpp | 10 +-- src/Common/SettingsChanges.h | 10 +-- src/Common/StringUtils/StringUtils.h | 2 +- src/Compression/CompressionCodecEncrypted.cpp | 4 +- src/Core/BaseSettings.cpp | 6 +- src/Core/BaseSettings.h | 88 +++++++++---------- src/Core/Field.cpp | 2 +- src/Core/Field.h | 8 +- src/Core/SettingsFields.cpp | 2 +- src/Core/SettingsFields.h | 12 +-- src/DataTypes/NestedUtils.cpp | 2 +- src/DataTypes/NestedUtils.h | 2 +- src/Formats/ProtobufSerializer.cpp | 88 +++++++++---------- src/Formats/ProtobufWriter.cpp | 2 +- src/Formats/ProtobufWriter.h | 2 +- src/Functions/FunctionsJSON.cpp | 30 +++---- src/IO/ReadHelpers.h | 4 +- src/IO/WriteHelpers.h | 16 ++-- src/Interpreters/Context.cpp | 12 +-- src/Interpreters/Context.h | 12 +-- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/DatabaseCatalog.h | 2 +- .../TranslateQualifiedNamesVisitor.cpp | 2 +- .../TranslateQualifiedNamesVisitor.h | 2 +- .../UserDefinedSQLObjectsLoader.cpp | 2 +- .../UserDefinedSQLObjectsLoader.h | 2 +- .../Access/ParserCreateRowPolicyQuery.cpp | 4 +- src/Storages/MergeTree/KeyCondition.cpp | 2 +- src/Storages/MergeTree/KeyCondition.h | 2 +- .../System/StorageSystemPrivileges.cpp | 2 +- 50 files changed, 344 insertions(+), 344 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 7152820b5bc..abd481f0bb6 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -101,7 +101,7 @@ public: registered_prefixes = prefixes_; } - bool isSettingNameAllowed(const std::string_view & setting_name) const + bool isSettingNameAllowed(std::string_view setting_name) const { if (Settings::hasBuiltin(setting_name)) return true; @@ -116,7 +116,7 @@ public: return false; } - void checkSettingNameIsAllowed(const std::string_view & setting_name) const + void checkSettingNameIsAllowed(std::string_view setting_name) const { if (isSettingNameAllowed(setting_name)) return; diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 7c3139dbb0f..b6fed3ac912 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -252,7 +252,7 @@ public: } template - void grant(const AccessFlags & flags_, const std::string_view & name, const Args &... subnames) + void grant(const AccessFlags & flags_, std::string_view name, const Args &... subnames) { auto & child = getChild(name); child.grant(flags_, subnames...); @@ -279,7 +279,7 @@ public: } template - void revoke(const AccessFlags & flags_, const std::string_view & name, const Args &... subnames) + void revoke(const AccessFlags & flags_, std::string_view name, const Args &... subnames) { auto & child = getChild(name); @@ -306,7 +306,7 @@ public: } template - bool isGranted(const AccessFlags & flags_, const std::string_view & name, const Args &... subnames) const + bool isGranted(const AccessFlags & flags_, std::string_view name, const Args &... subnames) const { AccessFlags flags_to_check = flags_ - min_flags_with_children; if (!flags_to_check) @@ -415,7 +415,7 @@ private: AccessFlags getAllGrantableFlags() const { return ::DB::getAllGrantableFlags(level); } AccessFlags getChildAllGrantableFlags() const { return ::DB::getAllGrantableFlags(static_cast(level + 1)); } - Node * tryGetChild(const std::string_view & name) const + Node * tryGetChild(std::string_view name) const { if (!children) return nullptr; @@ -425,7 +425,7 @@ private: return &it->second; } - Node & getChild(const std::string_view & name) + Node & getChild(std::string_view name) { auto * child = tryGetChild(name); if (child) @@ -819,20 +819,20 @@ void AccessRights::grantImpl(const AccessRightsElements & elements) } void AccessRights::grant(const AccessFlags & flags) { grantImpl(flags); } -void AccessRights::grant(const AccessFlags & flags, const std::string_view & database) { grantImpl(flags, database); } -void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) { grantImpl(flags, database, table); } -void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) { grantImpl(flags, database, table, column); } -void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) { grantImpl(flags, database, table, columns); } -void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) { grantImpl(flags, database, table, columns); } +void AccessRights::grant(const AccessFlags & flags, std::string_view database) { grantImpl(flags, database); } +void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table) { grantImpl(flags, database, table); } +void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) { grantImpl(flags, database, table, column); } +void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { grantImpl(flags, database, table, columns); } +void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grant(const AccessRightsElement & element) { grantImpl(element); } void AccessRights::grant(const AccessRightsElements & elements) { grantImpl(elements); } void AccessRights::grantWithGrantOption(const AccessFlags & flags) { grantImpl(flags); } -void AccessRights::grantWithGrantOption(const AccessFlags & flags, const std::string_view & database) { grantImpl(flags, database); } -void AccessRights::grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) { grantImpl(flags, database, table); } -void AccessRights::grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) { grantImpl(flags, database, table, column); } -void AccessRights::grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) { grantImpl(flags, database, table, columns); } -void AccessRights::grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) { grantImpl(flags, database, table, columns); } +void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database) { grantImpl(flags, database); } +void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) { grantImpl(flags, database, table); } +void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) { grantImpl(flags, database, table, column); } +void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { grantImpl(flags, database, table, columns); } +void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grantWithGrantOption(const AccessRightsElement & element) { grantImpl(element); } void AccessRights::grantWithGrantOption(const AccessRightsElements & elements) { grantImpl(elements); } @@ -892,20 +892,20 @@ void AccessRights::revokeImpl(const AccessRightsElements & elements) } void AccessRights::revoke(const AccessFlags & flags) { revokeImpl(flags); } -void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database) { revokeImpl(flags, database); } -void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) { revokeImpl(flags, database, table); } -void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) { revokeImpl(flags, database, table, column); } -void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) { revokeImpl(flags, database, table, columns); } -void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) { revokeImpl(flags, database, table, columns); } +void AccessRights::revoke(const AccessFlags & flags, std::string_view database) { revokeImpl(flags, database); } +void AccessRights::revoke(const AccessFlags & flags, std::string_view database, std::string_view table) { revokeImpl(flags, database, table); } +void AccessRights::revoke(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) { revokeImpl(flags, database, table, column); } +void AccessRights::revoke(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { revokeImpl(flags, database, table, columns); } +void AccessRights::revoke(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { revokeImpl(flags, database, table, columns); } void AccessRights::revoke(const AccessRightsElement & element) { revokeImpl(element); } void AccessRights::revoke(const AccessRightsElements & elements) { revokeImpl(elements); } void AccessRights::revokeGrantOption(const AccessFlags & flags) { revokeImpl(flags); } -void AccessRights::revokeGrantOption(const AccessFlags & flags, const std::string_view & database) { revokeImpl(flags, database); } -void AccessRights::revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) { revokeImpl(flags, database, table); } -void AccessRights::revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) { revokeImpl(flags, database, table, column); } -void AccessRights::revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) { revokeImpl(flags, database, table, columns); } -void AccessRights::revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) { revokeImpl(flags, database, table, columns); } +void AccessRights::revokeGrantOption(const AccessFlags & flags, std::string_view database) { revokeImpl(flags, database); } +void AccessRights::revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) { revokeImpl(flags, database, table); } +void AccessRights::revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) { revokeImpl(flags, database, table, column); } +void AccessRights::revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { revokeImpl(flags, database, table, columns); } +void AccessRights::revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { revokeImpl(flags, database, table, columns); } void AccessRights::revokeGrantOption(const AccessRightsElement & element) { revokeImpl(element); } void AccessRights::revokeGrantOption(const AccessRightsElements & elements) { revokeImpl(elements); } @@ -984,20 +984,20 @@ bool AccessRights::isGrantedImpl(const AccessRightsElements & elements) const } bool AccessRights::isGranted(const AccessFlags & flags) const { return isGrantedImpl(flags); } -bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database) const { return isGrantedImpl(flags, database); } -bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return isGrantedImpl(flags, database, table); } -bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return isGrantedImpl(flags, database, table, column); } -bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return isGrantedImpl(flags, database, table, columns); } -bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return isGrantedImpl(flags, database, table, columns); } +bool AccessRights::isGranted(const AccessFlags & flags, std::string_view database) const { return isGrantedImpl(flags, database); } +bool AccessRights::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const { return isGrantedImpl(flags, database, table); } +bool AccessRights::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return isGrantedImpl(flags, database, table, column); } +bool AccessRights::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return isGrantedImpl(flags, database, table, columns); } +bool AccessRights::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return isGrantedImpl(flags, database, table, columns); } bool AccessRights::isGranted(const AccessRightsElement & element) const { return isGrantedImpl(element); } bool AccessRights::isGranted(const AccessRightsElements & elements) const { return isGrantedImpl(elements); } bool AccessRights::hasGrantOption(const AccessFlags & flags) const { return isGrantedImpl(flags); } -bool AccessRights::hasGrantOption(const AccessFlags & flags, const std::string_view & database) const { return isGrantedImpl(flags, database); } -bool AccessRights::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return isGrantedImpl(flags, database, table); } -bool AccessRights::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return isGrantedImpl(flags, database, table, column); } -bool AccessRights::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return isGrantedImpl(flags, database, table, columns); } -bool AccessRights::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return isGrantedImpl(flags, database, table, columns); } +bool AccessRights::hasGrantOption(const AccessFlags & flags, std::string_view database) const { return isGrantedImpl(flags, database); } +bool AccessRights::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const { return isGrantedImpl(flags, database, table); } +bool AccessRights::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return isGrantedImpl(flags, database, table, column); } +bool AccessRights::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return isGrantedImpl(flags, database, table, columns); } +bool AccessRights::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return isGrantedImpl(flags, database, table, columns); } bool AccessRights::hasGrantOption(const AccessRightsElement & element) const { return isGrantedImpl(element); } bool AccessRights::hasGrantOption(const AccessRightsElements & elements) const { return isGrantedImpl(elements); } diff --git a/src/Access/AccessRights.h b/src/Access/AccessRights.h index b7499d69f70..80e37561c2b 100644 --- a/src/Access/AccessRights.h +++ b/src/Access/AccessRights.h @@ -39,59 +39,59 @@ public: /// Grants access on a specified database/table/column. /// Does nothing if the specified access has been already granted. void grant(const AccessFlags & flags); - void grant(const AccessFlags & flags, const std::string_view & database); - void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table); - void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns); + void grant(const AccessFlags & flags, std::string_view database); + void grant(const AccessFlags & flags, std::string_view database, std::string_view table); + void grant(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column); + void grant(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns); + void grant(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns); void grant(const AccessRightsElement & element); void grant(const AccessRightsElements & elements); void grantWithGrantOption(const AccessFlags & flags); - void grantWithGrantOption(const AccessFlags & flags, const std::string_view & database); - void grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table); - void grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns); + void grantWithGrantOption(const AccessFlags & flags, std::string_view database); + void grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table); + void grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column); + void grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns); + void grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns); void grantWithGrantOption(const AccessRightsElement & element); void grantWithGrantOption(const AccessRightsElements & elements); /// Revokes a specified access granted earlier on a specified database/table/column. /// For example, revoke(AccessType::ALL) revokes all grants at all, just like clear(); void revoke(const AccessFlags & flags); - void revoke(const AccessFlags & flags, const std::string_view & database); - void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table); - void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns); + void revoke(const AccessFlags & flags, std::string_view database); + void revoke(const AccessFlags & flags, std::string_view database, std::string_view table); + void revoke(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column); + void revoke(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns); + void revoke(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns); void revoke(const AccessRightsElement & element); void revoke(const AccessRightsElements & elements); void revokeGrantOption(const AccessFlags & flags); - void revokeGrantOption(const AccessFlags & flags, const std::string_view & database); - void revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table); - void revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns); + void revokeGrantOption(const AccessFlags & flags, std::string_view database); + void revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table); + void revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column); + void revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns); + void revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns); void revokeGrantOption(const AccessRightsElement & element); void revokeGrantOption(const AccessRightsElements & elements); /// Whether a specified access granted. bool isGranted(const AccessFlags & flags) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool isGranted(const AccessFlags & flags, std::string_view database) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; bool isGranted(const AccessRightsElement & element) const; bool isGranted(const AccessRightsElements & elements) const; bool hasGrantOption(const AccessFlags & flags) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; bool hasGrantOption(const AccessRightsElement & element) const; bool hasGrantOption(const AccessRightsElements & elements) const; @@ -107,9 +107,9 @@ public: const AccessFlags & flags, const AccessFlags & min_flags_with_children, const AccessFlags & max_flags_with_children, - const std::string_view & database, - const std::string_view & table, - const std::string_view & column)>; + std::string_view database, + std::string_view table, + std::string_view column)>; void modifyFlags(const ModifyFlagsFunction & function); void modifyFlagsWithGrantOption(const ModifyFlagsFunction & function); diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 4f304cf5952..c6bbd421c77 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -26,17 +26,17 @@ namespace return (Util::stringToDigest(password) == password_plaintext); } - bool checkPasswordDoubleSHA1(const std::string_view & password, const Digest & password_double_sha1) + bool checkPasswordDoubleSHA1(std::string_view password, const Digest & password_double_sha1) { return (Util::encodeDoubleSHA1(password) == password_double_sha1); } - bool checkPasswordSHA256(const std::string_view & password, const Digest & password_sha256, const String & salt) + bool checkPasswordSHA256(std::string_view password, const Digest & password_sha256, const String & salt) { return Util::encodeSHA256(String(password).append(salt)) == password_sha256; } - bool checkPasswordDoubleSHA1MySQL(const std::string_view & scramble, const std::string_view & scrambled_password, const Digest & password_double_sha1) + bool checkPasswordDoubleSHA1MySQL(std::string_view scramble, std::string_view scrambled_password, const Digest & password_double_sha1) { /// scrambled_password = SHA1(password) XOR SHA1(scramble SHA1(SHA1(password))) @@ -61,7 +61,7 @@ namespace return calculated_password_double_sha1 == password_double_sha1; } - bool checkPasswordPlainTextMySQL(const std::string_view & scramble, const std::string_view & scrambled_password, const Digest & password_plaintext) + bool checkPasswordPlainTextMySQL(std::string_view scramble, std::string_view scrambled_password, const Digest & password_plaintext) { return checkPasswordDoubleSHA1MySQL(scramble, scrambled_password, Util::encodeDoubleSHA1(password_plaintext)); } diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp index 82e1cbfb26b..305ae3f7cf5 100644 --- a/src/Access/Common/AccessFlags.cpp +++ b/src/Access/Common/AccessFlags.cpp @@ -35,7 +35,7 @@ namespace return access_type_to_flags_mapping[static_cast(type)]; } - Flags keywordToFlags(const std::string_view & keyword) const + Flags keywordToFlags(std::string_view keyword) const { auto it = keyword_to_flags_map.find(keyword); if (it == keyword_to_flags_map.end()) @@ -142,14 +142,14 @@ namespace } }; - static String replaceUnderscoreWithSpace(const std::string_view & str) + static String replaceUnderscoreWithSpace(std::string_view str) { String res{str}; boost::replace_all(res, "_", " "); return res; } - static Strings splitAliases(const std::string_view & str) + static Strings splitAliases(std::string_view str) { Strings aliases; boost::split(aliases, str, boost::is_any_of(",")); @@ -160,10 +160,10 @@ namespace static void makeNode( AccessType access_type, - const std::string_view & name, - const std::string_view & aliases, + std::string_view name, + std::string_view aliases, NodeType node_type, - const std::string_view & parent_group_name, + std::string_view parent_group_name, std::unordered_map & nodes, std::unordered_map & owned_nodes, size_t & next_flag) @@ -353,7 +353,7 @@ namespace AccessFlags::AccessFlags(AccessType type) : flags(Helper::instance().accessTypeToFlags(type)) {} -AccessFlags::AccessFlags(const std::string_view & keyword) : flags(Helper::instance().keywordToFlags(keyword)) {} +AccessFlags::AccessFlags(std::string_view keyword) : flags(Helper::instance().keywordToFlags(keyword)) {} AccessFlags::AccessFlags(const std::vector & keywords) : flags(Helper::instance().keywordsToFlags(keywords)) {} AccessFlags::AccessFlags(const Strings & keywords) : flags(Helper::instance().keywordsToFlags(keywords)) {} String AccessFlags::toString() const { return Helper::instance().flagsToString(flags); } diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index 51bf3cd19b0..5124f4ef332 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -21,7 +21,7 @@ public: AccessFlags() = default; /// Constructs from a string like "SELECT". - AccessFlags(const std::string_view & keyword); /// NOLINT + AccessFlags(std::string_view keyword); /// NOLINT /// Constructs from a list of strings like "SELECT, UPDATE, INSERT". AccessFlags(const std::vector & keywords); /// NOLINT diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 9913fc02f4a..69a2354f25d 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -81,7 +81,7 @@ namespace } bool need_comma = false; - for (const std::string_view & keyword : keywords) + for (std::string_view keyword : keywords) { if (need_comma) result.append(", "); @@ -145,18 +145,18 @@ namespace } -AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_) +AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, std::string_view database_) : access_flags(access_flags_), database(database_), any_database(false) { } -AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_) +AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, std::string_view database_, std::string_view table_) : access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false) { } AccessRightsElement::AccessRightsElement( - AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const std::string_view & column_) + AccessFlags access_flags_, std::string_view database_, std::string_view table_, std::string_view column_) : access_flags(access_flags_) , database(database_) , table(table_) @@ -169,8 +169,8 @@ AccessRightsElement::AccessRightsElement( AccessRightsElement::AccessRightsElement( AccessFlags access_flags_, - const std::string_view & database_, - const std::string_view & table_, + std::string_view database_, + std::string_view table_, const std::vector & columns_) : access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false), any_column(false) { @@ -180,7 +180,7 @@ AccessRightsElement::AccessRightsElement( } AccessRightsElement::AccessRightsElement( - AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const Strings & columns_) + AccessFlags access_flags_, std::string_view database_, std::string_view table_, const Strings & columns_) : access_flags(access_flags_) , database(database_) , table(table_) diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index e5bf76d0017..5f65b6bcd12 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -28,19 +28,19 @@ struct AccessRightsElement explicit AccessRightsElement(AccessFlags access_flags_) : access_flags(access_flags_) {} - AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_); - AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_); + AccessRightsElement(AccessFlags access_flags_, std::string_view database_); + AccessRightsElement(AccessFlags access_flags_, std::string_view database_, std::string_view table_); AccessRightsElement( - AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const std::string_view & column_); + AccessFlags access_flags_, std::string_view database_, std::string_view table_, std::string_view column_); AccessRightsElement( AccessFlags access_flags_, - const std::string_view & database_, - const std::string_view & table_, + std::string_view database_, + std::string_view table_, const std::vector & columns_); AccessRightsElement( - AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const Strings & columns_); + AccessFlags access_flags_, std::string_view database_, std::string_view table_, const Strings & columns_); bool empty() const { return !access_flags || (!any_column && columns.empty()); } diff --git a/src/Access/Common/AccessType.cpp b/src/Access/Common/AccessType.cpp index d44d70d78b2..4df1e1bc77f 100644 --- a/src/Access/Common/AccessType.cpp +++ b/src/Access/Common/AccessType.cpp @@ -35,7 +35,7 @@ namespace #undef ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING } - void addToMapping(AccessType type, const std::string_view & str) + void addToMapping(AccessType type, std::string_view str) { String str2{str}; boost::replace_all(str2, "_", " "); diff --git a/src/Access/Common/AuthenticationData.cpp b/src/Access/Common/AuthenticationData.cpp index db0a5d54a63..f3d3bb5b758 100644 --- a/src/Access/Common/AuthenticationData.cpp +++ b/src/Access/Common/AuthenticationData.cpp @@ -71,7 +71,7 @@ const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType ty } -AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(const std::string_view & text [[maybe_unused]]) +AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(std::string_view text [[maybe_unused]]) { #if USE_SSL Digest hash; @@ -86,7 +86,7 @@ AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(const std::str } -AuthenticationData::Digest AuthenticationData::Util::encodeSHA1(const std::string_view & text) +AuthenticationData::Digest AuthenticationData::Util::encodeSHA1(std::string_view text) { Poco::SHA1Engine engine; engine.update(text.data(), text.size()); diff --git a/src/Access/Common/AuthenticationData.h b/src/Access/Common/AuthenticationData.h index 2837e0f10a1..ced9fcd4b6d 100644 --- a/src/Access/Common/AuthenticationData.h +++ b/src/Access/Common/AuthenticationData.h @@ -96,11 +96,11 @@ public: struct Util { - static Digest stringToDigest(const std::string_view & text) { return Digest(text.data(), text.data() + text.size()); } - static Digest encodeSHA256(const std::string_view & text); - static Digest encodeSHA1(const std::string_view & text); + static Digest stringToDigest(std::string_view text) { return Digest(text.data(), text.data() + text.size()); } + static Digest encodeSHA256(std::string_view text); + static Digest encodeSHA1(std::string_view text); static Digest encodeSHA1(const Digest & text) { return encodeSHA1(std::string_view{reinterpret_cast(text.data()), text.size()}); } - static Digest encodeDoubleSHA1(const std::string_view & text) { return encodeSHA1(encodeSHA1(text)); } + static Digest encodeDoubleSHA1(std::string_view text) { return encodeSHA1(encodeSHA1(text)); } static Digest encodeDoubleSHA1(const Digest & text) { return encodeSHA1(encodeSHA1(text)); } }; diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 995a46d07ca..92a5179d861 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -46,7 +46,7 @@ namespace AccessRights addImplicitAccessRights(const AccessRights & access) { - auto modifier = [&](const AccessFlags & flags, const AccessFlags & min_flags_with_children, const AccessFlags & max_flags_with_children, const std::string_view & database, const std::string_view & table, const std::string_view & column) -> AccessFlags + auto modifier = [&](const AccessFlags & flags, const AccessFlags & min_flags_with_children, const AccessFlags & max_flags_with_children, std::string_view database, std::string_view table, std::string_view column) -> AccessFlags { size_t level = !database.empty() + !table.empty() + !column.empty(); AccessFlags res = flags; @@ -141,7 +141,7 @@ namespace std::string_view getDatabase() { return {}; } template - std::string_view getDatabase(const std::string_view & arg1, const OtherArgs &...) { return arg1; } + std::string_view getDatabase(std::string_view arg1, const OtherArgs &...) { return arg1; } } @@ -519,7 +519,7 @@ bool ContextAccess::checkAccessImpl(const AccessFlags & flags) const } template -bool ContextAccess::checkAccessImpl(const AccessFlags & flags, const std::string_view & database, const Args &... args) const +bool ContextAccess::checkAccessImpl(const AccessFlags & flags, std::string_view database, const Args &... args) const { return checkAccessImplHelper(flags, database.empty() ? params.current_database : database, args...); } @@ -564,38 +564,38 @@ bool ContextAccess::checkAccessImpl(const AccessRightsElements & elements) const } bool ContextAccess::isGranted(const AccessFlags & flags) const { return checkAccessImpl(flags); } -bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(flags, database); } -bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(flags, database, table); } -bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(flags, database, table, column); } -bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } -bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } +bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(flags, database); } +bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(flags, database, table); } +bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(flags, database, table, column); } +bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } +bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } bool ContextAccess::isGranted(const AccessRightsElement & element) const { return checkAccessImpl(element); } bool ContextAccess::isGranted(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } bool ContextAccess::hasGrantOption(const AccessFlags & flags) const { return checkAccessImpl(flags); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(flags, database); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(flags, database, table); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(flags, database, table, column); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } +bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(flags, database); } +bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(flags, database, table); } +bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(flags, database, table, column); } +bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } +bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } bool ContextAccess::hasGrantOption(const AccessRightsElement & element) const { return checkAccessImpl(element); } bool ContextAccess::hasGrantOption(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } void ContextAccess::checkAccess(const AccessFlags & flags) const { checkAccessImpl(flags); } -void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database) const { checkAccessImpl(flags, database); } -void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { checkAccessImpl(flags, database, table); } -void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl(flags, database, table, column); } -void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { checkAccessImpl(flags, database, table, columns); } -void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl(flags, database, table, columns); } +void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database) const { checkAccessImpl(flags, database); } +void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const { checkAccessImpl(flags, database, table); } +void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { checkAccessImpl(flags, database, table, column); } +void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { checkAccessImpl(flags, database, table, columns); } +void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { checkAccessImpl(flags, database, table, columns); } void ContextAccess::checkAccess(const AccessRightsElement & element) const { checkAccessImpl(element); } void ContextAccess::checkAccess(const AccessRightsElements & elements) const { checkAccessImpl(elements); } void ContextAccess::checkGrantOption(const AccessFlags & flags) const { checkAccessImpl(flags); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database) const { checkAccessImpl(flags, database); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { checkAccessImpl(flags, database, table); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl(flags, database, table, column); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { checkAccessImpl(flags, database, table, columns); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl(flags, database, table, columns); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database) const { checkAccessImpl(flags, database); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const { checkAccessImpl(flags, database, table); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { checkAccessImpl(flags, database, table, column); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { checkAccessImpl(flags, database, table, columns); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { checkAccessImpl(flags, database, table, columns); } void ContextAccess::checkGrantOption(const AccessRightsElement & element) const { checkAccessImpl(element); } void ContextAccess::checkGrantOption(const AccessRightsElements & elements) const { checkAccessImpl(elements); } diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 729574898aa..fa3523977e7 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -101,40 +101,40 @@ public: /// Checks if a specified access is granted, and throws an exception if not. /// Empty database means the current database. void checkAccess(const AccessFlags & flags) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + void checkAccess(const AccessFlags & flags, std::string_view database) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; void checkAccess(const AccessRightsElement & element) const; void checkAccess(const AccessRightsElements & elements) const; void checkGrantOption(const AccessFlags & flags) const; - void checkGrantOption(const AccessFlags & flags, const std::string_view & database) const; - void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + void checkGrantOption(const AccessFlags & flags, std::string_view database) const; + void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const; + void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; void checkGrantOption(const AccessRightsElement & element) const; void checkGrantOption(const AccessRightsElements & elements) const; /// Checks if a specified access is granted, and returns false if not. /// Empty database means the current database. bool isGranted(const AccessFlags & flags) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool isGranted(const AccessFlags & flags, std::string_view database) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; bool isGranted(const AccessRightsElement & element) const; bool isGranted(const AccessRightsElements & elements) const; bool hasGrantOption(const AccessFlags & flags) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; bool hasGrantOption(const AccessRightsElement & element) const; bool hasGrantOption(const AccessRightsElements & elements) const; @@ -180,7 +180,7 @@ private: bool checkAccessImpl(const AccessFlags & flags) const; template - bool checkAccessImpl(const AccessFlags & flags, const std::string_view & database, const Args &... args) const; + bool checkAccessImpl(const AccessFlags & flags, std::string_view database, const Args &... args) const; template bool checkAccessImpl(const AccessRightsElement & element) const; diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 6084138f306..34f2e10dc83 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -36,12 +36,12 @@ void SettingsConstraints::clear() } -void SettingsConstraints::setMinValue(const std::string_view & setting_name, const Field & min_value) +void SettingsConstraints::setMinValue(std::string_view setting_name, const Field & min_value) { getConstraintRef(setting_name).min_value = Settings::castValueUtil(setting_name, min_value); } -Field SettingsConstraints::getMinValue(const std::string_view & setting_name) const +Field SettingsConstraints::getMinValue(std::string_view setting_name) const { const auto * ptr = tryGetConstraint(setting_name); if (ptr) @@ -51,12 +51,12 @@ Field SettingsConstraints::getMinValue(const std::string_view & setting_name) co } -void SettingsConstraints::setMaxValue(const std::string_view & setting_name, const Field & max_value) +void SettingsConstraints::setMaxValue(std::string_view setting_name, const Field & max_value) { getConstraintRef(setting_name).max_value = Settings::castValueUtil(setting_name, max_value); } -Field SettingsConstraints::getMaxValue(const std::string_view & setting_name) const +Field SettingsConstraints::getMaxValue(std::string_view setting_name) const { const auto * ptr = tryGetConstraint(setting_name); if (ptr) @@ -66,12 +66,12 @@ Field SettingsConstraints::getMaxValue(const std::string_view & setting_name) co } -void SettingsConstraints::setReadOnly(const std::string_view & setting_name, bool read_only) +void SettingsConstraints::setReadOnly(std::string_view setting_name, bool read_only) { getConstraintRef(setting_name).read_only = read_only; } -bool SettingsConstraints::isReadOnly(const std::string_view & setting_name) const +bool SettingsConstraints::isReadOnly(std::string_view setting_name) const { const auto * ptr = tryGetConstraint(setting_name); if (ptr) @@ -81,7 +81,7 @@ bool SettingsConstraints::isReadOnly(const std::string_view & setting_name) cons } -void SettingsConstraints::set(const std::string_view & setting_name, const Field & min_value, const Field & max_value, bool read_only) +void SettingsConstraints::set(std::string_view setting_name, const Field & min_value, const Field & max_value, bool read_only) { auto & ref = getConstraintRef(setting_name); ref.min_value = Settings::castValueUtil(setting_name, min_value); @@ -89,7 +89,7 @@ void SettingsConstraints::set(const std::string_view & setting_name, const Field ref.read_only = read_only; } -void SettingsConstraints::get(const std::string_view & setting_name, Field & min_value, Field & max_value, bool & read_only) const +void SettingsConstraints::get(std::string_view setting_name, Field & min_value, Field & max_value, bool & read_only) const { const auto * ptr = tryGetConstraint(setting_name); if (ptr) @@ -318,7 +318,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh } -SettingsConstraints::Constraint & SettingsConstraints::getConstraintRef(const std::string_view & setting_name) +SettingsConstraints::Constraint & SettingsConstraints::getConstraintRef(std::string_view setting_name) { auto it = constraints.find(setting_name); if (it == constraints.end()) @@ -331,7 +331,7 @@ SettingsConstraints::Constraint & SettingsConstraints::getConstraintRef(const st return it->second; } -const SettingsConstraints::Constraint * SettingsConstraints::tryGetConstraint(const std::string_view & setting_name) const +const SettingsConstraints::Constraint * SettingsConstraints::tryGetConstraint(std::string_view setting_name) const { auto it = constraints.find(setting_name); if (it == constraints.end()) diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index f7bca1eafb3..645a690e051 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -61,17 +61,17 @@ public: void clear(); bool empty() const { return constraints.empty(); } - void setMinValue(const std::string_view & setting_name, const Field & min_value); - Field getMinValue(const std::string_view & setting_name) const; + void setMinValue(std::string_view setting_name, const Field & min_value); + Field getMinValue(std::string_view setting_name) const; - void setMaxValue(const std::string_view & setting_name, const Field & max_value); - Field getMaxValue(const std::string_view & setting_name) const; + void setMaxValue(std::string_view setting_name, const Field & max_value); + Field getMaxValue(std::string_view setting_name) const; - void setReadOnly(const std::string_view & setting_name, bool read_only); - bool isReadOnly(const std::string_view & setting_name) const; + void setReadOnly(std::string_view setting_name, bool read_only); + bool isReadOnly(std::string_view setting_name) const; - void set(const std::string_view & setting_name, const Field & min_value, const Field & max_value, bool read_only); - void get(const std::string_view & setting_name, Field & min_value, Field & max_value, bool & read_only) const; + void set(std::string_view setting_name, const Field & min_value, const Field & max_value, bool read_only); + void get(std::string_view setting_name, Field & min_value, Field & max_value, bool & read_only) const; void merge(const SettingsConstraints & other); @@ -105,8 +105,8 @@ private: }; bool checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const; - Constraint & getConstraintRef(const std::string_view & setting_name); - const Constraint * tryGetConstraint(const std::string_view & setting_name) const; + Constraint & getConstraintRef(std::string_view setting_name); + const Constraint * tryGetConstraint(std::string_view setting_name) const; std::unordered_map constraints; const AccessControl * access_control = nullptr; diff --git a/src/Common/JSONParsers/DummyJSONParser.h b/src/Common/JSONParsers/DummyJSONParser.h index 77b958d1429..3cedd59decd 100644 --- a/src/Common/JSONParsers/DummyJSONParser.h +++ b/src/Common/JSONParsers/DummyJSONParser.h @@ -84,7 +84,7 @@ struct DummyJSONParser static Iterator begin() { return {}; } static Iterator end() { return {}; } static size_t size() { return 0; } - bool find(const std::string_view &, Element &) const { return false; } /// NOLINT + bool find(std::string_view, Element &) const { return false; } /// NOLINT #if 0 /// Optional: Provides access to an object's element by index. @@ -93,7 +93,7 @@ struct DummyJSONParser }; /// Parses a JSON document, returns the reference to its root element if succeeded. - bool parse(const std::string_view &, Element &) { throw Exception{"Functions JSON* are not supported", ErrorCodes::NOT_IMPLEMENTED}; } /// NOLINT + bool parse(std::string_view, Element &) { throw Exception{"Functions JSON* are not supported", ErrorCodes::NOT_IMPLEMENTED}; } /// NOLINT #if 0 /// Optional: Allocates memory to parse JSON documents faster. diff --git a/src/Common/JSONParsers/RapidJSONParser.h b/src/Common/JSONParsers/RapidJSONParser.h index 2d8514868e5..77e8f6b2a74 100644 --- a/src/Common/JSONParsers/RapidJSONParser.h +++ b/src/Common/JSONParsers/RapidJSONParser.h @@ -98,7 +98,7 @@ struct RapidJSONParser ALWAYS_INLINE Iterator end() const { return ptr->MemberEnd(); } ALWAYS_INLINE size_t size() const { return ptr->MemberCount(); } - bool find(const std::string_view & key, Element & result) const + bool find(std::string_view key, Element & result) const { auto it = ptr->FindMember(rapidjson::StringRef(key.data(), key.length())); if (it == ptr->MemberEnd()) @@ -122,7 +122,7 @@ struct RapidJSONParser }; /// Parses a JSON document, returns the reference to its root element if succeeded. - bool parse(const std::string_view & json, Element & result) + bool parse(std::string_view json, Element & result) { rapidjson::MemoryStream ms(json.data(), json.size()); rapidjson::EncodedInputStream, rapidjson::MemoryStream> is(ms); diff --git a/src/Common/JSONParsers/SimdJSONParser.h b/src/Common/JSONParsers/SimdJSONParser.h index 3abeb85fb56..f3bbfe4dfde 100644 --- a/src/Common/JSONParsers/SimdJSONParser.h +++ b/src/Common/JSONParsers/SimdJSONParser.h @@ -105,7 +105,7 @@ struct SimdJSONParser ALWAYS_INLINE Iterator end() const { return object.end(); } ALWAYS_INLINE size_t size() const { return object.size(); } - bool find(const std::string_view & key, Element & result) const + bool find(std::string_view key, Element & result) const { auto x = object.at_key(key); if (x.error()) @@ -131,7 +131,7 @@ struct SimdJSONParser }; /// Parses a JSON document, returns the reference to its root element if succeeded. - bool parse(const std::string_view & json, Element & result) + bool parse(std::string_view json, Element & result) { auto document = parser.parse(json.data(), json.size()); if (document.error()) diff --git a/src/Common/OpenSSLHelpers.cpp b/src/Common/OpenSSLHelpers.cpp index d73e08e79a9..4e7848afc85 100644 --- a/src/Common/OpenSSLHelpers.cpp +++ b/src/Common/OpenSSLHelpers.cpp @@ -10,7 +10,7 @@ namespace DB { #pragma GCC diagnostic warning "-Wold-style-cast" -std::string encodeSHA256(const std::string_view & text) +std::string encodeSHA256(std::string_view text) { return encodeSHA256(text.data(), text.size()); } @@ -21,7 +21,7 @@ std::string encodeSHA256(const void * text, size_t size) encodeSHA256(text, size, reinterpret_cast(out.data())); return out; } -void encodeSHA256(const std::string_view & text, unsigned char * out) +void encodeSHA256(std::string_view text, unsigned char * out) { encodeSHA256(text.data(), text.size(), out); } diff --git a/src/Common/OpenSSLHelpers.h b/src/Common/OpenSSLHelpers.h index f0dbbc10b4c..41f092f0109 100644 --- a/src/Common/OpenSSLHelpers.h +++ b/src/Common/OpenSSLHelpers.h @@ -10,10 +10,10 @@ namespace DB { /// Encodes `text` and returns it. -std::string encodeSHA256(const std::string_view & text); +std::string encodeSHA256(std::string_view text); std::string encodeSHA256(const void * text, size_t size); /// `out` must be at least 32 bytes long. -void encodeSHA256(const std::string_view & text, unsigned char * out); +void encodeSHA256(std::string_view text, unsigned char * out); void encodeSHA256(const void * text, size_t size, unsigned char * out); /// Returns concatenation of error strings for all errors that OpenSSL has recorded, emptying the error queue. diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index 370b465eba3..9fb4f361e09 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -4,7 +4,7 @@ namespace DB { namespace { - SettingChange * find(SettingsChanges & changes, const std::string_view & name) + SettingChange * find(SettingsChanges & changes, std::string_view name) { auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); if (it == changes.end()) @@ -12,7 +12,7 @@ namespace return &*it; } - const SettingChange * find(const SettingsChanges & changes, const std::string_view & name) + const SettingChange * find(const SettingsChanges & changes, std::string_view name) { auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); if (it == changes.end()) @@ -21,7 +21,7 @@ namespace } } -bool SettingsChanges::tryGet(const std::string_view & name, Field & out_value) const +bool SettingsChanges::tryGet(std::string_view name, Field & out_value) const { const auto * change = find(*this, name); if (!change) @@ -30,7 +30,7 @@ bool SettingsChanges::tryGet(const std::string_view & name, Field & out_value) c return true; } -const Field * SettingsChanges::tryGet(const std::string_view & name) const +const Field * SettingsChanges::tryGet(std::string_view name) const { const auto * change = find(*this, name); if (!change) @@ -38,7 +38,7 @@ const Field * SettingsChanges::tryGet(const std::string_view & name) const return &change->value; } -Field * SettingsChanges::tryGet(const std::string_view & name) +Field * SettingsChanges::tryGet(std::string_view name) { auto * change = find(*this, name); if (!change) diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 5f6a390d0d2..67cb69f77bf 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -14,8 +14,8 @@ struct SettingChange Field value; SettingChange() = default; - SettingChange(const std::string_view & name_, const Field & value_) : name(name_), value(value_) {} - SettingChange(const std::string_view & name_, Field && value_) : name(name_), value(std::move(value_)) {} + SettingChange(std::string_view name_, const Field & value_) : name(name_), value(value_) {} + SettingChange(std::string_view name_, Field && value_) : name(name_), value(std::move(value_)) {} friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } @@ -27,9 +27,9 @@ class SettingsChanges : public std::vector public: using std::vector::vector; - bool tryGet(const std::string_view & name, Field & out_value) const; - const Field * tryGet(const std::string_view & name) const; - Field * tryGet(const std::string_view & name); + bool tryGet(std::string_view name, Field & out_value) const; + const Field * tryGet(std::string_view name) const; + Field * tryGet(std::string_view name); }; } diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils/StringUtils.h index 21df0f5ae8b..b5a081ab693 100644 --- a/src/Common/StringUtils/StringUtils.h +++ b/src/Common/StringUtils/StringUtils.h @@ -147,7 +147,7 @@ inline bool isPunctuationASCII(char c) } -inline bool isValidIdentifier(const std::string_view & str) +inline bool isValidIdentifier(std::string_view str) { return !str.empty() && isValidIdentifierBegin(str[0]) diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index f7e597a0519..bf36fa114fb 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -131,7 +131,7 @@ std::string lastErrorString() /// This function get key and nonce and encrypt text with their help. /// If something went wrong (can't init context or can't encrypt data) it throws exception. /// It returns length of encrypted text. -size_t encrypt(const std::string_view & plaintext, char * ciphertext_and_tag, EncryptionMethod method, const String & key, const String & nonce) +size_t encrypt(std::string_view plaintext, char * ciphertext_and_tag, EncryptionMethod method, const String & key, const String & nonce) { /// Init context for encryption, using key. EVP_AEAD_CTX encrypt_ctx; @@ -160,7 +160,7 @@ size_t encrypt(const std::string_view & plaintext, char * ciphertext_and_tag, En /// This function get key and nonce and encrypt text with their help. /// If something went wrong (can't init context or can't encrypt data) it throws exception. /// It returns length of encrypted text. -size_t decrypt(const std::string_view & ciphertext, char * plaintext, EncryptionMethod method, const String & key, const String & nonce) +size_t decrypt(std::string_view ciphertext, char * plaintext, EncryptionMethod method, const String & key, const String & nonce) { /// Init context for decryption with given key. EVP_AEAD_CTX decrypt_ctx; diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index d4b2d1551b6..f03a59c1342 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -void BaseSettingsHelpers::writeString(const std::string_view & str, WriteBuffer & out) +void BaseSettingsHelpers::writeString(std::string_view str, WriteBuffer & out) { writeStringBinary(str, out); } @@ -39,13 +39,13 @@ BaseSettingsHelpers::Flags BaseSettingsHelpers::readFlags(ReadBuffer & in) } -void BaseSettingsHelpers::throwSettingNotFound(const std::string_view & name) +void BaseSettingsHelpers::throwSettingNotFound(std::string_view name) { throw Exception("Unknown setting " + String{name}, ErrorCodes::UNKNOWN_SETTING); } -void BaseSettingsHelpers::warningSettingNotFound(const std::string_view & name) +void BaseSettingsHelpers::warningSettingNotFound(std::string_view name) { static auto * log = &Poco::Logger::get("Settings"); LOG_WARNING(log, "Unknown setting {}, skipping", name); diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index a4ddc6571ed..3638a036098 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -45,16 +45,16 @@ class BaseSettings : public TTraits::Data public: using Traits = TTraits; - void set(const std::string_view & name, const Field & value); - Field get(const std::string_view & name) const; + void set(std::string_view name, const Field & value); + Field get(std::string_view name) const; - void setString(const std::string_view & name, const String & value); - String getString(const std::string_view & name) const; + void setString(std::string_view name, const String & value); + String getString(std::string_view name) const; - bool tryGet(const std::string_view & name, Field & value) const; - bool tryGetString(const std::string_view & name, String & value) const; + bool tryGet(std::string_view name, Field & value) const; + bool tryGetString(std::string_view name, String & value) const; - bool isChanged(const std::string_view & name) const; + bool isChanged(std::string_view name) const; SettingsChanges changes() const; void applyChange(const SettingChange & change); void applyChanges(const SettingsChanges & changes); @@ -63,22 +63,22 @@ public: /// Resets all the settings to their default values. void resetToDefault(); - bool has(const std::string_view & name) const { return hasBuiltin(name) || hasCustom(name); } - static bool hasBuiltin(const std::string_view & name); - bool hasCustom(const std::string_view & name) const; + bool has(std::string_view name) const { return hasBuiltin(name) || hasCustom(name); } + static bool hasBuiltin(std::string_view name); + bool hasCustom(std::string_view name) const; - const char * getTypeName(const std::string_view & name) const; - const char * getDescription(const std::string_view & name) const; + const char * getTypeName(std::string_view name) const; + const char * getDescription(std::string_view name) const; /// Checks if it's possible to assign a field to a specified value and throws an exception if not. /// This function doesn't change the fields, it performs check only. - static void checkCanSet(const std::string_view & name, const Field & value); - static void checkCanSetString(const std::string_view & name, const String & str); + static void checkCanSet(std::string_view name, const Field & value); + static void checkCanSetString(std::string_view name, const String & str); /// Conversions without changing the fields. - static Field castValueUtil(const std::string_view & name, const Field & value); - static String valueToStringUtil(const std::string_view & name, const Field & value); - static Field stringToValueUtil(const std::string_view & name, const String & str); + static Field castValueUtil(std::string_view name, const Field & value); + static String valueToStringUtil(std::string_view name, const Field & value); + static Field stringToValueUtil(std::string_view name, const String & str); void write(WriteBuffer & out, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT) const; void read(ReadBuffer & in, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT); @@ -164,19 +164,19 @@ public: Iterator end() const { return allChanged().end(); } private: - SettingFieldCustom & getCustomSetting(const std::string_view & name); - const SettingFieldCustom & getCustomSetting(const std::string_view & name) const; - const SettingFieldCustom * tryGetCustomSetting(const std::string_view & name) const; + SettingFieldCustom & getCustomSetting(std::string_view name); + const SettingFieldCustom & getCustomSetting(std::string_view name) const; + const SettingFieldCustom * tryGetCustomSetting(std::string_view name) const; std::conditional_t custom_settings_map; }; struct BaseSettingsHelpers { - [[noreturn]] static void throwSettingNotFound(const std::string_view & name); - static void warningSettingNotFound(const std::string_view & name); + [[noreturn]] static void throwSettingNotFound(std::string_view name); + static void warningSettingNotFound(std::string_view name); - static void writeString(const std::string_view & str, WriteBuffer & out); + static void writeString(std::string_view str, WriteBuffer & out); static String readString(ReadBuffer & in); enum Flags : UInt64 @@ -190,7 +190,7 @@ struct BaseSettingsHelpers }; template -void BaseSettings::set(const std::string_view & name, const Field & value) +void BaseSettings::set(std::string_view name, const Field & value) { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -200,7 +200,7 @@ void BaseSettings::set(const std::string_view & name, const Field & val } template -Field BaseSettings::get(const std::string_view & name) const +Field BaseSettings::get(std::string_view name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -210,7 +210,7 @@ Field BaseSettings::get(const std::string_view & name) const } template -void BaseSettings::setString(const std::string_view & name, const String & value) +void BaseSettings::setString(std::string_view name, const String & value) { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -220,7 +220,7 @@ void BaseSettings::setString(const std::string_view & name, const Strin } template -String BaseSettings::getString(const std::string_view & name) const +String BaseSettings::getString(std::string_view name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -230,7 +230,7 @@ String BaseSettings::getString(const std::string_view & name) const } template -bool BaseSettings::tryGet(const std::string_view & name, Field & value) const +bool BaseSettings::tryGet(std::string_view name, Field & value) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -247,7 +247,7 @@ bool BaseSettings::tryGet(const std::string_view & name, Field & value) } template -bool BaseSettings::tryGetString(const std::string_view & name, String & value) const +bool BaseSettings::tryGetString(std::string_view name, String & value) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -264,7 +264,7 @@ bool BaseSettings::tryGetString(const std::string_view & name, String & } template -bool BaseSettings::isChanged(const std::string_view & name) const +bool BaseSettings::isChanged(std::string_view name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -316,20 +316,20 @@ void BaseSettings::resetToDefault() } template -bool BaseSettings::hasBuiltin(const std::string_view & name) +bool BaseSettings::hasBuiltin(std::string_view name) { const auto & accessor = Traits::Accessor::instance(); return (accessor.find(name) != static_cast(-1)); } template -bool BaseSettings::hasCustom(const std::string_view & name) const +bool BaseSettings::hasCustom(std::string_view name) const { return tryGetCustomSetting(name); } template -const char * BaseSettings::getTypeName(const std::string_view & name) const +const char * BaseSettings::getTypeName(std::string_view name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -341,7 +341,7 @@ const char * BaseSettings::getTypeName(const std::string_view & name) c } template -const char * BaseSettings::getDescription(const std::string_view & name) const +const char * BaseSettings::getDescription(std::string_view name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -353,19 +353,19 @@ const char * BaseSettings::getDescription(const std::string_view & name } template -void BaseSettings::checkCanSet(const std::string_view & name, const Field & value) +void BaseSettings::checkCanSet(std::string_view name, const Field & value) { castValueUtil(name, value); } template -void BaseSettings::checkCanSetString(const std::string_view & name, const String & str) +void BaseSettings::checkCanSetString(std::string_view name, const String & str) { stringToValueUtil(name, str); } template -Field BaseSettings::castValueUtil(const std::string_view & name, const Field & value) +Field BaseSettings::castValueUtil(std::string_view name, const Field & value) { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -377,7 +377,7 @@ Field BaseSettings::castValueUtil(const std::string_view & name, const } template -String BaseSettings::valueToStringUtil(const std::string_view & name, const Field & value) +String BaseSettings::valueToStringUtil(std::string_view name, const Field & value) { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -389,7 +389,7 @@ String BaseSettings::valueToStringUtil(const std::string_view & name, c } template -Field BaseSettings::stringToValueUtil(const std::string_view & name, const String & str) +Field BaseSettings::stringToValueUtil(std::string_view name, const String & str) { try { @@ -521,7 +521,7 @@ bool operator!=(const BaseSettings & left, const BaseSettings } template -SettingFieldCustom & BaseSettings::getCustomSetting(const std::string_view & name) +SettingFieldCustom & BaseSettings::getCustomSetting(std::string_view name) { if constexpr (Traits::allow_custom_settings) { @@ -537,7 +537,7 @@ SettingFieldCustom & BaseSettings::getCustomSetting(const std::string_v } template -const SettingFieldCustom & BaseSettings::getCustomSetting(const std::string_view & name) const +const SettingFieldCustom & BaseSettings::getCustomSetting(std::string_view name) const { if constexpr (Traits::allow_custom_settings) { @@ -549,7 +549,7 @@ const SettingFieldCustom & BaseSettings::getCustomSetting(const std::st } template -const SettingFieldCustom * BaseSettings::tryGetCustomSetting(const std::string_view & name) const +const SettingFieldCustom * BaseSettings::tryGetCustomSetting(std::string_view name) const { if constexpr (Traits::allow_custom_settings) { @@ -780,7 +780,7 @@ bool BaseSettings::SettingFieldRef::isObsolete() const public: \ static const Accessor & instance(); \ size_t size() const { return field_infos.size(); } \ - size_t find(const std::string_view & name) const; \ + size_t find(std::string_view name) const; \ const String & getName(size_t index) const { return field_infos[index].name; } \ const char * getTypeName(size_t index) const { return field_infos[index].type; } \ const char * getDescription(size_t index) const { return field_infos[index].description; } \ @@ -851,7 +851,7 @@ bool BaseSettings::SettingFieldRef::isObsolete() const \ SETTINGS_TRAITS_NAME::Accessor::Accessor() {} \ \ - size_t SETTINGS_TRAITS_NAME::Accessor::find(const std::string_view & name) const \ + size_t SETTINGS_TRAITS_NAME::Accessor::find(std::string_view name) const \ { \ auto it = name_to_index_map.find(name); \ if (it != name_to_index_map.end()) \ diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 3a4b66e6266..acdfca7a7b2 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -286,7 +286,7 @@ String Field::dump() const return applyVisitor(FieldVisitorDump(), *this); } -Field Field::restoreFromDump(const std::string_view & dump_) +Field Field::restoreFromDump(std::string_view dump_) { auto show_error = [&dump_] { diff --git a/src/Core/Field.h b/src/Core/Field.h index 08274876914..f60b7e4902e 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -346,7 +346,7 @@ public: } /// Create a string inplace. - Field(const std::string_view & str) { create(str.data(), str.size()); } /// NOLINT + Field(std::string_view str) { create(str.data(), str.size()); } /// NOLINT Field(const String & str) { create(std::string_view{str}); } /// NOLINT Field(String && str) { create(std::move(str)); } /// NOLINT Field(const char * str) { create(std::string_view{str}); } /// NOLINT @@ -403,7 +403,7 @@ public: return *this; } - Field & operator= (const std::string_view & str); + Field & operator= (std::string_view str); Field & operator= (const String & str) { return *this = std::string_view{str}; } Field & operator= (String && str); Field & operator= (const char * str) { return *this = std::string_view{str}; } @@ -631,7 +631,7 @@ public: } String dump() const; - static Field restoreFromDump(const std::string_view & dump_); + static Field restoreFromDump(std::string_view dump_); private: std::aligned_union_t()) {} - SettingFieldString & operator =(const std::string_view & str) { value = str; changed = true; return *this; } + SettingFieldString & operator =(std::string_view str) { value = str; changed = true; return *this; } SettingFieldString & operator =(const String & str) { *this = std::string_view{str}; return *this; } SettingFieldString & operator =(String && str) { value = std::move(str); changed = true; return *this; } SettingFieldString & operator =(const char * str) { *this = std::string_view{str}; return *this; } @@ -256,7 +256,7 @@ struct SettingFieldEnum struct SettingFieldEnumHelpers { - static void writeBinary(const std::string_view & str, WriteBuffer & out); + static void writeBinary(std::string_view str, WriteBuffer & out); static String readBinary(ReadBuffer & in); }; @@ -286,7 +286,7 @@ void SettingFieldEnum::readBinary(ReadBuffer & in) { \ using EnumType = ENUM_TYPE; \ static const String & toString(EnumType value); \ - static EnumType fromString(const std::string_view & str); \ + static EnumType fromString(std::string_view str); \ }; \ \ using SettingField##NEW_NAME = SettingFieldEnum; @@ -310,7 +310,7 @@ void SettingFieldEnum::readBinary(ReadBuffer & in) ERROR_CODE_FOR_UNEXPECTED_NAME); \ } \ \ - typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(const std::string_view & str) \ + typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(std::string_view str) \ { \ static const std::unordered_map map = [] { \ std::unordered_map res; \ @@ -430,7 +430,7 @@ void SettingFieldMultiEnum::readBinary(ReadBuffer & in) using EnumType = ENUM_TYPE; \ static size_t getEnumSize(); \ static const String & toString(EnumType value); \ - static EnumType fromString(const std::string_view & str); \ + static EnumType fromString(std::string_view str); \ }; \ \ using SettingField##NEW_NAME = SettingFieldMultiEnum; diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 5dae2b7b413..b28b70f676a 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -54,7 +54,7 @@ std::pair splitName(const std::string & name, bool rev return {name.substr(0, idx), name.substr(idx + 1)}; } -std::pair splitName(const std::string_view & name, bool reverse) +std::pair splitName(std::string_view name, bool reverse) { auto idx = (reverse ? name.find_last_of('.') : name.find_first_of('.')); if (idx == std::string::npos || idx == 0 || idx + 1 == name.size()) diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index e7cda541f47..38da382254c 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -13,7 +13,7 @@ namespace Nested /// Splits name of compound identifier by first/last dot (depending on 'reverse' parameter). std::pair splitName(const std::string & name, bool reverse = false); - std::pair splitName(const std::string_view & name, bool reverse = false); + std::pair splitName(std::string_view name, bool reverse = false); /// Returns the prefix of the name to the first '.'. Or the name is unchanged if there is no dot. std::string extractTableName(const std::string & nested_name); diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 9006c9276d4..b9af9d61da0 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -77,18 +77,18 @@ namespace return convertChar(c1) == convertChar(c2); } - static bool equals(const std::string_view & s1, const std::string_view & s2) + static bool equals(std::string_view s1, std::string_view s2) { return (s1.length() == s2.length()) && std::equal(s1.begin(), s1.end(), s2.begin(), [](char c1, char c2) { return convertChar(c1) == convertChar(c2); }); } - static bool less(const std::string_view & s1, const std::string_view & s2) + static bool less(std::string_view s1, std::string_view s2) { return std::lexicographical_compare(s1.begin(), s1.end(), s2.begin(), s2.end(), [](char c1, char c2) { return convertChar(c1) < convertChar(c2); }); } - static bool startsWith(const std::string_view & s1, const std::string_view & s2) + static bool startsWith(std::string_view s1, std::string_view s2) { return (s1.length() >= s2.length()) && equals(s1.substr(0, s2.length()), s2); } @@ -195,7 +195,7 @@ namespace { protected: ProtobufSerializerSingleValue( - const std::string_view & column_name_, + std::string_view column_name_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : column_name(column_name_) @@ -264,7 +264,7 @@ namespace return reader->readFixed(); } - void writeStr(const std::string_view & str) + void writeStr(std::string_view str) { if (!str.empty() || !skip_zero_or_empty) writer->writeString(field_tag, str); @@ -274,7 +274,7 @@ namespace void readStrAndAppend(PaddedPODArray & str) { reader->readStringAndAppend(str); } template - DestType parseFromStr(const std::string_view & str) const + DestType parseFromStr(std::string_view str) const { try { @@ -307,7 +307,7 @@ namespace return result; } - [[noreturn]] void incompatibleColumnType(const std::string_view & column_type) const + [[noreturn]] void incompatibleColumnType(std::string_view column_type) const { throw Exception( ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD, @@ -318,7 +318,7 @@ namespace field_descriptor.type_name()); } - [[noreturn]] void cannotConvertValue(const std::string_view & src_value, const std::string_view & src_type_name, const std::string_view & dest_type_name) const + [[noreturn]] void cannotConvertValue(std::string_view src_value, std::string_view src_type_name, std::string_view dest_type_name) const { throw Exception( "Could not convert value '" + String{src_value} + "' from type " + String{src_type_name} + " to type " @@ -351,7 +351,7 @@ namespace public: using ColumnType = ColumnVector; - ProtobufSerializerNumber(const std::string_view & column_name_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) + ProtobufSerializerNumber(std::string_view column_name_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) { setFunctions(); @@ -590,7 +590,7 @@ namespace using ColumnType = std::conditional_t; ProtobufSerializerString( - const std::string_view & column_name_, + std::string_view column_name_, const std::shared_ptr & fixed_string_data_type_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -604,7 +604,7 @@ namespace } ProtobufSerializerString( - const std::string_view & column_name_, + std::string_view column_name_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) @@ -727,7 +727,7 @@ namespace { case FieldTypeId::TYPE_INT32: { - write_function = [this](const std::string_view & str) { writeInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int32()); }; break; @@ -735,7 +735,7 @@ namespace case FieldTypeId::TYPE_SINT32: { - write_function = [this](const std::string_view & str) { writeSInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeSInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readSInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int32()); }; break; @@ -743,7 +743,7 @@ namespace case FieldTypeId::TYPE_UINT32: { - write_function = [this](const std::string_view & str) { writeUInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeUInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readUInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_uint32()); }; break; @@ -751,7 +751,7 @@ namespace case FieldTypeId::TYPE_INT64: { - write_function = [this](const std::string_view & str) { writeInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int64()); }; break; @@ -759,7 +759,7 @@ namespace case FieldTypeId::TYPE_SINT64: { - write_function = [this](const std::string_view & str) { writeSInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeSInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readSInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int64()); }; break; @@ -767,7 +767,7 @@ namespace case FieldTypeId::TYPE_UINT64: { - write_function = [this](const std::string_view & str) { writeUInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeUInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readUInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_uint64()); }; break; @@ -775,7 +775,7 @@ namespace case FieldTypeId::TYPE_FIXED32: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_uint32()); }; break; @@ -783,7 +783,7 @@ namespace case FieldTypeId::TYPE_SFIXED32: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int32()); }; break; @@ -791,7 +791,7 @@ namespace case FieldTypeId::TYPE_FIXED64: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_uint64()); }; break; @@ -799,7 +799,7 @@ namespace case FieldTypeId::TYPE_SFIXED64: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int64()); }; break; @@ -807,7 +807,7 @@ namespace case FieldTypeId::TYPE_FLOAT: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_float()); }; break; @@ -815,7 +815,7 @@ namespace case FieldTypeId::TYPE_DOUBLE: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_double()); }; break; @@ -823,7 +823,7 @@ namespace case FieldTypeId::TYPE_BOOL: { - write_function = [this](const std::string_view & str) + write_function = [this](std::string_view str) { if (str == "true") writeUInt(1); @@ -855,7 +855,7 @@ namespace case FieldTypeId::TYPE_STRING: case FieldTypeId::TYPE_BYTES: { - write_function = [this](const std::string_view & str) { writeStr(str); }; + write_function = [this](std::string_view str) { writeStr(str); }; read_function = [this](PaddedPODArray & str) { readStrAndAppend(str); }; default_function = [this]() -> String { return field_descriptor.default_value_string(); }; break; @@ -863,7 +863,7 @@ namespace case FieldTypeId::TYPE_ENUM: { - write_function = [this](const std::string_view & str) { writeInt(stringToProtobufEnumValue(str)); }; + write_function = [this](std::string_view str) { writeInt(stringToProtobufEnumValue(str)); }; read_function = [this](PaddedPODArray & str) { protobufEnumValueToStringAppend(readInt(), str); }; default_function = [this]() -> String { return field_descriptor.default_value_enum()->name(); }; break; @@ -908,7 +908,7 @@ namespace } } - int stringToProtobufEnumValue(const std::string_view & str) const + int stringToProtobufEnumValue(std::string_view str) const { auto it = string_to_protobuf_enum_value_map.find(str); if (it == string_to_protobuf_enum_value_map.end()) @@ -932,7 +932,7 @@ namespace const std::shared_ptr fixed_string_data_type; const size_t n = 0; - std::function write_function; + std::function write_function; std::function &)> read_function; std::function default_function; std::unordered_map string_to_protobuf_enum_value_map; @@ -953,7 +953,7 @@ namespace using BaseClass = ProtobufSerializerNumber; ProtobufSerializerEnum( - const std::string_view & column_name_, + std::string_view column_name_, const std::shared_ptr & enum_data_type_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -1067,7 +1067,7 @@ namespace protobuf_enum_value_to_enum_data_type_value_map.emplace(protobuf_enum_value, enum_data_type_value); }; - auto iless = [](const std::string_view & s1, const std::string_view & s2) { return ColumnNameWithProtobufFieldNameComparator::less(s1, s2); }; + auto iless = [](std::string_view s1, std::string_view s2) { return ColumnNameWithProtobufFieldNameComparator::less(s1, s2); }; boost::container::flat_map string_to_protobuf_enum_value_map; typename decltype(string_to_protobuf_enum_value_map)::sequence_type string_to_protobuf_enum_value_seq; for (int i : collections::range(enum_descriptor.value_count())) @@ -1133,9 +1133,9 @@ namespace Int64 readInt() { return ProtobufSerializerSingleValue::readInt(); } void writeInt(Int64 value) { ProtobufSerializerSingleValue::writeInt(value); } - void writeStr(const std::string_view & str) { ProtobufSerializerSingleValue::writeStr(str); } + void writeStr(std::string_view str) { ProtobufSerializerSingleValue::writeStr(str); } void readStr(String & str) { ProtobufSerializerSingleValue::readStr(str); } - [[noreturn]] void cannotConvertValue(const std::string_view & src_value, const std::string_view & src_type_name, const std::string_view & dest_type_name) const { ProtobufSerializerSingleValue::cannotConvertValue(src_value, src_type_name, dest_type_name); } + [[noreturn]] void cannotConvertValue(std::string_view src_value, std::string_view src_type_name, std::string_view dest_type_name) const { ProtobufSerializerSingleValue::cannotConvertValue(src_value, src_type_name, dest_type_name); } const std::shared_ptr enum_data_type; std::unordered_map enum_data_type_value_to_protobuf_enum_value_map; @@ -1152,7 +1152,7 @@ namespace using ColumnType = ColumnDecimal; ProtobufSerializerDecimal( - const std::string_view & column_name_, + std::string_view column_name_, const DataTypeDecimalBase & decimal_data_type_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -1412,7 +1412,7 @@ namespace { public: ProtobufSerializerDate( - const std::string_view & column_name_, + std::string_view column_name_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : ProtobufSerializerNumber(column_name_, field_descriptor_, reader_or_writer_) @@ -1490,7 +1490,7 @@ namespace { public: ProtobufSerializerDateTime( - const std::string_view & column_name_, + std::string_view column_name_, const DataTypeDateTime & type, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -1574,7 +1574,7 @@ namespace { public: ProtobufSerializerUUID( - const std::string_view & column_name_, + std::string_view column_name_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) @@ -1654,7 +1654,7 @@ namespace { public: ProtobufSerializerAggregateFunction( - const std::string_view & column_name_, + std::string_view column_name_, const std::shared_ptr & aggregate_function_data_type_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -2061,7 +2061,7 @@ namespace { public: ProtobufSerializerTupleAsArray( - const std::string_view & column_name_, + std::string_view column_name_, const std::shared_ptr & tuple_data_type_, const FieldDescriptor & field_descriptor_, std::vector> element_serializers_) @@ -2833,7 +2833,7 @@ namespace return field_names; } - static bool columnNameEqualsToFieldName(const std::string_view & column_name, const FieldDescriptor & field_descriptor) + static bool columnNameEqualsToFieldName(std::string_view column_name, const FieldDescriptor & field_descriptor) { std::string_view suffix; return columnNameStartsWithFieldName(column_name, field_descriptor, suffix) && suffix.empty(); @@ -2844,7 +2844,7 @@ namespace /// which doesn't match to the field's name. /// The function requires that rest part of the column's name to be started with a dot '.' or underline '_', /// but doesn't include those '.' or '_' characters into `suffix`. - static bool columnNameStartsWithFieldName(const std::string_view & column_name, const FieldDescriptor & field_descriptor, std::string_view & suffix) + static bool columnNameStartsWithFieldName(std::string_view column_name, const FieldDescriptor & field_descriptor, std::string_view & suffix) { size_t matching_length = 0; const MessageDescriptor & containing_type = *field_descriptor.containing_type(); @@ -2887,7 +2887,7 @@ namespace /// for that case suffixes are also returned. /// This is only the first filter, buildMessageSerializerImpl() does other checks after calling this function. static bool findFieldsByColumnName( - const std::string_view & column_name, + std::string_view column_name, const MessageDescriptor & message_descriptor, std::vector> & out_field_descriptors_with_suffixes, bool google_wrappers_special_treatment) @@ -3030,7 +3030,7 @@ namespace used_column_indices_sorted.reserve(num_columns); size_t sequential_column_index = 0; - auto add_field_serializer = [&](const std::string_view & column_name_, + auto add_field_serializer = [&](std::string_view column_name_, std::vector && column_indices_, const FieldDescriptor & field_descriptor_, std::unique_ptr field_serializer_) @@ -3243,7 +3243,7 @@ namespace /// Builds a serializer for one-to-one match: /// one column is serialized as one field in the protobuf message. std::unique_ptr buildFieldSerializer( - const std::string_view & column_name, + std::string_view column_name, const DataTypePtr & data_type, const FieldDescriptor & field_descriptor, bool allow_repeat, @@ -3395,7 +3395,7 @@ namespace } } - [[noreturn]] static void throwFieldNotRepeated(const FieldDescriptor & field_descriptor, const std::string_view & column_name) + [[noreturn]] static void throwFieldNotRepeated(const FieldDescriptor & field_descriptor, std::string_view column_name) { if (!field_descriptor.is_repeated()) throw Exception( diff --git a/src/Formats/ProtobufWriter.cpp b/src/Formats/ProtobufWriter.cpp index ece4f78b1c8..da680fae601 100644 --- a/src/Formats/ProtobufWriter.cpp +++ b/src/Formats/ProtobufWriter.cpp @@ -196,7 +196,7 @@ template void ProtobufWriter::writeFixed(int field_number, UInt64 value) template void ProtobufWriter::writeFixed(int field_number, Float32 value); template void ProtobufWriter::writeFixed(int field_number, Float64 value); -void ProtobufWriter::writeString(int field_number, const std::string_view & str) +void ProtobufWriter::writeString(int field_number, std::string_view str) { size_t length = str.length(); size_t old_size = buffer.size(); diff --git a/src/Formats/ProtobufWriter.h b/src/Formats/ProtobufWriter.h index 1dcc8f4ef7c..3ede956e910 100644 --- a/src/Formats/ProtobufWriter.h +++ b/src/Formats/ProtobufWriter.h @@ -30,7 +30,7 @@ public: void writeSInt(int field_number, Int64 value); template void writeFixed(int field_number, T value); - void writeString(int field_number, const std::string_view & str); + void writeString(int field_number, std::string_view str); void startRepeatedPack(); void endRepeatedPack(int field_number, bool skip_if_empty); diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index fa573ac829a..2f10a3c773b 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -304,7 +304,7 @@ private: /// Performs moves of types MoveType::Key and MoveType::ConstKey. template - static bool moveToElementByKey(typename JSONParser::Element & element, const std::string_view & key) + static bool moveToElementByKey(typename JSONParser::Element & element, std::string_view key) { if (!element.isObject()) return false; @@ -504,7 +504,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view) { ColumnVector & col_vec = assert_cast &>(dest); col_vec.insertValue(1); @@ -532,7 +532,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &) { return 0; } - static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view) { /// This function is called only if JSON is valid. /// If JSON isn't valid then `FunctionJSON::Executor::run()` adds default value (=zero) to `dest` without calling this function. @@ -556,7 +556,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { size_t size; if (element.isArray()) @@ -586,7 +586,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view & last_key) + static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view last_key) { if (last_key.empty()) return false; @@ -620,7 +620,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { UInt8 type; if (element.isInt64()) @@ -662,7 +662,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { NumberType value; @@ -737,7 +737,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (!element.isBool()) return false; @@ -764,7 +764,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (element.isNull()) return false; @@ -1164,7 +1164,7 @@ public: extract_tree = JSONExtractTree::build(function_name, result_type); } - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { return extract_tree->insertResultToColumn(dest, element); } @@ -1207,7 +1207,7 @@ public: extract_tree = JSONExtractTree::build(function_name, value_type); } - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (!element.isObject()) return false; @@ -1251,7 +1251,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { ColumnString & col_str = assert_cast(dest); auto & chars = col_str.getChars(); @@ -1355,7 +1355,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (!element.isArray()) return false; @@ -1387,7 +1387,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (!element.isObject()) return false; @@ -1423,7 +1423,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (!element.isObject()) return false; diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 4cd07dddf25..57283a396d9 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -836,7 +836,7 @@ template inline T parse(const char * data, size_t size); template -inline T parseFromString(const std::string_view & str) +inline T parseFromString(std::string_view str) { return parse(str.data(), str.size()); } @@ -1338,7 +1338,7 @@ inline T parseWithSizeSuffix(const char * data, size_t size) } template -inline T parseWithSizeSuffix(const std::string_view & s) +inline T parseWithSizeSuffix(std::string_view s) { return parseWithSizeSuffix(s.data(), s.size()); } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 5eab75f14b1..c3bbaac097d 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -113,7 +113,7 @@ inline void writeStringBinary(const char * s, WriteBuffer & buf) writeStringBinary(StringRef{s}, buf); } -inline void writeStringBinary(const std::string_view & s, WriteBuffer & buf) +inline void writeStringBinary(std::string_view s, WriteBuffer & buf) { writeStringBinary(StringRef{s}, buf); } @@ -365,7 +365,7 @@ inline void writeJSONString(const StringRef & s, WriteBuffer & buf, const Format writeJSONString(s.data, s.data + s.size, buf, settings); } -inline void writeJSONString(const std::string_view & s, WriteBuffer & buf, const FormatSettings & settings) +inline void writeJSONString(std::string_view s, WriteBuffer & buf, const FormatSettings & settings) { writeJSONString(StringRef{s}, buf, settings); } @@ -440,7 +440,7 @@ inline void writeEscapedString(const StringRef & ref, WriteBuffer & buf) writeEscapedString(ref.data, ref.size, buf); } -inline void writeEscapedString(const std::string_view & ref, WriteBuffer & buf) +inline void writeEscapedString(std::string_view ref, WriteBuffer & buf) { writeEscapedString(ref.data(), ref.size(), buf); } @@ -478,7 +478,7 @@ inline void writeQuotedString(const StringRef & ref, WriteBuffer & buf) writeAnyQuotedString<'\''>(ref, buf); } -inline void writeQuotedString(const std::string_view & ref, WriteBuffer & buf) +inline void writeQuotedString(std::string_view ref, WriteBuffer & buf) { writeAnyQuotedString<'\''>(ref.data(), ref.data() + ref.size(), buf); } @@ -493,7 +493,7 @@ inline void writeDoubleQuotedString(const StringRef & s, WriteBuffer & buf) writeAnyQuotedString<'"'>(s, buf); } -inline void writeDoubleQuotedString(const std::string_view & s, WriteBuffer & buf) +inline void writeDoubleQuotedString(std::string_view s, WriteBuffer & buf) { writeAnyQuotedString<'"'>(s.data(), s.data() + s.size(), buf); } @@ -891,7 +891,7 @@ inline void writeBinary(const T & x, WriteBuffer & buf) { writePODBinary(x, buf) inline void writeBinary(const String & x, WriteBuffer & buf) { writeStringBinary(x, buf); } inline void writeBinary(const StringRef & x, WriteBuffer & buf) { writeStringBinary(x, buf); } -inline void writeBinary(const std::string_view & x, WriteBuffer & buf) { writeStringBinary(x, buf); } +inline void writeBinary(std::string_view x, WriteBuffer & buf) { writeStringBinary(x, buf); } inline void writeBinary(const Decimal32 & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const Decimal64 & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const Decimal128 & x, WriteBuffer & buf) { writePODBinary(x, buf); } @@ -1015,7 +1015,7 @@ inline void writeQuoted(const T & x, WriteBuffer & buf) { writeText(x, buf); } inline void writeQuoted(const String & x, WriteBuffer & buf) { writeQuotedString(x, buf); } -inline void writeQuoted(const std::string_view & x, WriteBuffer & buf) { writeQuotedString(x, buf); } +inline void writeQuoted(std::string_view x, WriteBuffer & buf) { writeQuotedString(x, buf); } inline void writeQuoted(const StringRef & x, WriteBuffer & buf) { writeQuotedString(x, buf); } @@ -1048,7 +1048,7 @@ inline void writeDoubleQuoted(const T & x, WriteBuffer & buf) { writeText(x, buf inline void writeDoubleQuoted(const String & x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } -inline void writeDoubleQuoted(const std::string_view & x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } +inline void writeDoubleQuoted(std::string_view x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } inline void writeDoubleQuoted(const StringRef & x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b83b6420548..3cee7db6434 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -854,13 +854,13 @@ void Context::checkAccessImpl(const Args &... args) const } void Context::checkAccess(const AccessFlags & flags) const { return checkAccessImpl(flags); } -void Context::checkAccess(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(flags, database); } -void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(flags, database, table); } -void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(flags, database, table, column); } -void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } -void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(flags, database); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(flags, database, table); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(flags, database, table, column); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName()); } -void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::string_view & column) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); } +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, std::string_view column) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); } void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); } void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); } void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 37c6b4c9caa..a4f69602a61 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -474,13 +474,13 @@ public: /// Checks access rights. /// Empty database means the current database. void checkAccess(const AccessFlags & flags) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + void checkAccess(const AccessFlags & flags, std::string_view database) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; void checkAccess(const AccessFlags & flags, const StorageID & table_id) const; - void checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::string_view & column) const; + void checkAccess(const AccessFlags & flags, const StorageID & table_id, std::string_view column) const; void checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector & columns) const; void checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const; void checkAccess(const AccessRightsElement & element) const; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index bae2aed2cd5..587d58d9ad3 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -234,7 +234,7 @@ void DatabaseCatalog::shutdownImpl() view_dependencies.clear(); } -bool DatabaseCatalog::isPredefinedDatabase(const std::string_view & database_name) +bool DatabaseCatalog::isPredefinedDatabase(std::string_view database_name) { return database_name == TEMPORARY_DATABASE || database_name == SYSTEM_DATABASE || database_name == INFORMATION_SCHEMA || database_name == INFORMATION_SCHEMA_UPPERCASE; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 133cf0c5126..d82ad56eadd 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -131,7 +131,7 @@ public: static constexpr const char * INFORMATION_SCHEMA_UPPERCASE = "INFORMATION_SCHEMA"; /// Returns true if a passed name is one of the predefined databases' names. - static bool isPredefinedDatabase(const std::string_view & database_name); + static bool isPredefinedDatabase(std::string_view database_name); static DatabaseCatalog & init(ContextMutablePtr global_context_); static DatabaseCatalog & instance(); diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index b58b90b6d47..070fac7ccbd 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes extern const int UNSUPPORTED_JOIN_KEYS; extern const int LOGICAL_ERROR; } -bool TranslateQualifiedNamesMatcher::Data::matchColumnName(const std::string_view & name, const String & column_name, DataTypePtr column_type) +bool TranslateQualifiedNamesMatcher::Data::matchColumnName(std::string_view name, const String & column_name, DataTypePtr column_type) { if (name.size() < column_name.size()) return false; diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.h b/src/Interpreters/TranslateQualifiedNamesVisitor.h index e0c2f6b6bc0..73e45fc7ea0 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -39,7 +39,7 @@ public: bool hasTable() const { return !tables.empty(); } bool processAsterisks() const { return hasTable() && has_columns; } bool unknownColumn(size_t table_pos, const ASTIdentifier & identifier) const; - static bool matchColumnName(const std::string_view & name, const String & column_name, DataTypePtr column_type); + static bool matchColumnName(std::string_view name, const String & column_name, DataTypePtr column_type); }; static void visit(ASTPtr & ast, Data & data); diff --git a/src/Interpreters/UserDefinedSQLObjectsLoader.cpp b/src/Interpreters/UserDefinedSQLObjectsLoader.cpp index 75b91f3a817..c6f50fc4a0a 100644 --- a/src/Interpreters/UserDefinedSQLObjectsLoader.cpp +++ b/src/Interpreters/UserDefinedSQLObjectsLoader.cpp @@ -43,7 +43,7 @@ UserDefinedSQLObjectsLoader::UserDefinedSQLObjectsLoader() : log(&Poco::Logger::get("UserDefinedSQLObjectsLoader")) {} -void UserDefinedSQLObjectsLoader::loadUserDefinedObject(ContextPtr context, UserDefinedSQLObjectType object_type, const std::string_view & name, const String & path) +void UserDefinedSQLObjectsLoader::loadUserDefinedObject(ContextPtr context, UserDefinedSQLObjectType object_type, std::string_view name, const String & path) { auto name_ref = StringRef(name.data(), name.size()); LOG_DEBUG(log, "Loading user defined object {} from file {}", backQuote(name_ref), path); diff --git a/src/Interpreters/UserDefinedSQLObjectsLoader.h b/src/Interpreters/UserDefinedSQLObjectsLoader.h index 2e747f67a8d..9dfba1181c1 100644 --- a/src/Interpreters/UserDefinedSQLObjectsLoader.h +++ b/src/Interpreters/UserDefinedSQLObjectsLoader.h @@ -29,7 +29,7 @@ public: private: - void loadUserDefinedObject(ContextPtr context, UserDefinedSQLObjectType object_type, const std::string_view & object_name, const String & file_path); + void loadUserDefinedObject(ContextPtr context, UserDefinedSQLObjectType object_type, std::string_view object_name, const String & file_path); Poco::Logger * log; bool enable_persistence = true; }; diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index 83156c6a8e1..2c25fc14e7d 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -75,7 +75,7 @@ namespace { for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - const std::string_view & command = RowPolicyFilterTypeInfo::get(filter_type).command; + std::string_view command = RowPolicyFilterTypeInfo::get(filter_type).command; commands.emplace(command); } } @@ -96,7 +96,7 @@ namespace for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - const std::string_view & command = RowPolicyFilterTypeInfo::get(filter_type).command; + std::string_view command = RowPolicyFilterTypeInfo::get(filter_type).command; if (ParserKeyword{command.data()}.ignore(pos, expected)) { res_commands.emplace(command); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 40f23fe5294..daf31698aad 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -2050,7 +2050,7 @@ bool KeyCondition::mayBeTrueInRange( } String KeyCondition::RPNElement::toString() const { return toString("column " + std::to_string(key_column), false); } -String KeyCondition::RPNElement::toString(const std::string_view & column_name, bool print_constants) const +String KeyCondition::RPNElement::toString(std::string_view column_name, bool print_constants) const { auto print_wrapped_column = [this, &column_name, print_constants](WriteBuffer & buf) { diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 9a8719afa19..af85a90dd62 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -320,7 +320,7 @@ private: : function(function_), range(range_), key_column(key_column_) {} String toString() const; - String toString(const std::string_view & column_name, bool print_constants) const; + String toString(std::string_view column_name, bool print_constants) const; Function function = FUNCTION_UNKNOWN; diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index 8cf1accfe34..70163979f72 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -85,7 +85,7 @@ void StorageSystemPrivileges::fillData(MutableColumns & res_columns, ContextPtr, auto & column_parent_group = assert_cast(assert_cast(*res_columns[column_index]).getNestedColumn()).getData(); auto & column_parent_group_null_map = assert_cast(*res_columns[column_index++]).getNullMapData(); - auto add_row = [&](AccessType access_type, const std::string_view & aliases, Level max_level, AccessType parent_group) + auto add_row = [&](AccessType access_type, std::string_view aliases, Level max_level, AccessType parent_group) { column_access_type.push_back(static_cast(access_type)); From efb418a7e78d10b6aa6c1cd85fd25545ff6e679f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 14 Jul 2022 16:30:59 +0000 Subject: [PATCH 602/627] fix --- tests/queries/0_stateless/02356_insert_query_log_metrics.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql index 3825f37183b..347073f868f 100644 --- a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql @@ -1,4 +1,4 @@ CREATE TABLE 02356_destination (a Int64, b String) ENGINE = Memory; INSERT INTO 02356_destination (a, b) SELECT * FROM generateRandom('a Int64, b String') LIMIT 100 SETTINGS max_threads=1, max_block_size=100; -SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' FORMAT CSV; \ No newline at end of file +SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' order by event_time desc limit 1 FORMAT CSV; \ No newline at end of file From 3fd327b23a39d97ef4dcb08843f653cc25e8b944 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 14 Jul 2022 19:25:40 +0200 Subject: [PATCH 603/627] Use std::filesystem::path::operator/ --- src/Common/filesystemHelpers.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 6f6afcac657..1e8e53bf1ea 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -87,7 +87,7 @@ BlockDeviceType getBlockDeviceType([[maybe_unused]] const String & device_id) #if defined(OS_LINUX) try { - const String path = "/sys/dev/block/" + device_id + "/queue/rotational"; + const auto path{std::filesystem::path("/sys/dev/block/") / device_id / "queue/rotational"}; if (!std::filesystem::exists(path)) return BlockDeviceType::UNKNOWN; ReadBufferFromFile in(path); @@ -112,7 +112,8 @@ UInt64 getBlockDeviceReadAheadBytes([[maybe_unused]] const String & device_id) #if defined(OS_LINUX) try { - ReadBufferFromFile in("/sys/dev/block/" + device_id + "/queue/read_ahead_kb"); + const auto path{std::filesystem::path("/sys/dev/block/") / device_id / "queue/read_ahead_kb"}; + ReadBufferFromFile in(path); int read_ahead_kb; readText(read_ahead_kb, in); return read_ahead_kb * 1024; From 3c9d4c537a56f5cd026684997c15dbb7d1520675 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jul 2022 18:36:50 +0000 Subject: [PATCH 604/627] Another try. --- src/Interpreters/ExpressionAnalyzer.cpp | 77 ++++++++++++++----------- src/Interpreters/ExpressionAnalyzer.h | 2 +- 2 files changed, 43 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index e2c106ff796..23258c60099 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1253,7 +1253,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin( } ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( - ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns) + ExpressionActionsChain & chain, bool only_types) { const auto * select_query = getSelectQuery(); if (!select_query->prewhere()) @@ -1290,14 +1290,6 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( NameSet required_source_columns(required_columns.begin(), required_columns.end()); required_source_columns.insert(first_action_names.begin(), first_action_names.end()); - /// Add required columns to required output in order not to remove them after prewhere execution. - /// TODO: add sampling and final execution to common chain. - for (const auto & column : additional_required_columns) - { - if (required_source_columns.contains(column)) - step.addRequiredOutput(column); - } - auto names = step.actions()->getNames(); NameSet name_set(names.begin(), names.end()); @@ -1844,12 +1836,28 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( const Settings & settings = context->getSettingsRef(); const ConstStoragePtr & storage = query_analyzer.storage(); + Names additional_required_columns_after_prewhere; ssize_t prewhere_step_num = -1; ssize_t where_step_num = -1; ssize_t having_step_num = -1; auto finalize_chain = [&](ExpressionActionsChain & chain) { + if (prewhere_step_num >= 0) + { + ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num); + + auto required_columns = prewhere_info->prewhere_actions->getRequiredColumnsNames(); + NameSet required_source_columns(required_columns.begin(), required_columns.end()); + /// Add required columns to required output in order not to remove them after prewhere execution. + /// TODO: add sampling and final execution to common chain. + for (const auto & column : additional_required_columns_after_prewhere) + { + if (required_source_columns.contains(column)) + step.addRequiredOutput(column); + } + } + chain.finalize(); finalize(chain, prewhere_step_num, where_step_num, having_step_num, query); @@ -1858,25 +1866,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( }; { - bool join_allow_read_in_order = true; - if (hasJoin()) - { - /// You may find it strange but we support read_in_order for HashJoin and do not support for MergeJoin. - join_has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows(); - join_allow_read_in_order = typeid_cast(join.get()) && !join_has_delayed_stream; - } - - optimize_read_in_order = - settings.optimize_read_in_order - && storage - && query.orderBy() - && !query_analyzer.hasAggregation() - && !query_analyzer.hasWindow() - && !query.final() - && join_allow_read_in_order; - ExpressionActionsChain chain(context); - Names additional_required_columns_after_prewhere; if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { @@ -1892,20 +1882,13 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( columns_for_final.begin(), columns_for_final.end()); } - if (storage && optimize_read_in_order) - { - Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); - additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), - columns_for_sorting_key.begin(), columns_for_sorting_key.end()); - } - if (storage && filter_info_) { filter_info = filter_info_; filter_info->do_remove_column = true; } - if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) + if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage)) { /// Prewhere is always the first one. prewhere_step_num = 0; @@ -1983,6 +1966,30 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( } } + bool join_allow_read_in_order = true; + if (hasJoin()) + { + /// You may find it strange but we support read_in_order for HashJoin and do not support for MergeJoin. + join_has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows(); + join_allow_read_in_order = typeid_cast(join.get()) && !join_has_delayed_stream; + } + + optimize_read_in_order = + settings.optimize_read_in_order + && storage + && query.orderBy() + && !query_analyzer.hasAggregation() + && !query_analyzer.hasWindow() + && !query.final() + && join_allow_read_in_order; + + if (storage && optimize_read_in_order) + { + Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); + additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), + columns_for_sorting_key.begin(), columns_for_sorting_key.end()); + } + /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. query_analyzer.appendSelect(chain, only_types || (need_aggregate ? !second_stage : !first_stage)); diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index aae45482a97..019cda8b924 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -403,7 +403,7 @@ private: /// remove_filter is set in ExpressionActionsChain::finalize(); /// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier). - ActionsDAGPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns); + ActionsDAGPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types); bool appendWhere(ExpressionActionsChain & chain, bool only_types); bool appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order, ManyExpressionActions &); void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types); From 79e1d39211a5242d2f59d37ced4be55812c325ec Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Jul 2022 18:45:37 +0000 Subject: [PATCH 605/627] do not color logs on client if redirected to file --- src/Client/ClientBase.cpp | 6 +++- src/Client/ClientBase.h | 1 + .../02360_send_logs_level_colors.reference | 3 ++ .../02360_send_logs_level_colors.sh | 31 +++++++++++++++++++ 4 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02360_send_logs_level_colors.reference create mode 100644 tests/queries/0_stateless/02360_send_logs_level_colors.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 0e243f97aaf..b435b483b71 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -601,6 +601,7 @@ void ClientBase::initLogsOutputStream() { WriteBuffer * wb = out_logs_buf.get(); + bool color_logs = false; if (!out_logs_buf) { if (server_logs_file.empty()) @@ -608,11 +609,13 @@ void ClientBase::initLogsOutputStream() /// Use stderr by default out_logs_buf = std::make_unique(STDERR_FILENO); wb = out_logs_buf.get(); + color_logs = stderr_is_a_tty; } else if (server_logs_file == "-") { /// Use stdout if --server_logs_file=- specified wb = &std_out; + color_logs = stdout_is_a_tty; } else { @@ -622,7 +625,7 @@ void ClientBase::initLogsOutputStream() } } - logs_out_stream = std::make_unique(*wb, stdout_is_a_tty); + logs_out_stream = std::make_unique(*wb, color_logs); } } @@ -2155,6 +2158,7 @@ void ClientBase::init(int argc, char ** argv) stdin_is_a_tty = isatty(STDIN_FILENO); stdout_is_a_tty = isatty(STDOUT_FILENO); + stderr_is_a_tty = isatty(STDERR_FILENO); terminal_width = getTerminalWidth(); Arguments common_arguments{""}; /// 0th argument is ignored. diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index ec2267a3be6..b012680fc3c 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -173,6 +173,7 @@ protected: bool stdin_is_a_tty = false; /// stdin is a terminal. bool stdout_is_a_tty = false; /// stdout is a terminal. + bool stderr_is_a_tty = false; /// stderr is a terminal. uint64_t terminal_width = 0; ServerConnectionPtr connection; diff --git a/tests/queries/0_stateless/02360_send_logs_level_colors.reference b/tests/queries/0_stateless/02360_send_logs_level_colors.reference new file mode 100644 index 00000000000..fe2824243c4 --- /dev/null +++ b/tests/queries/0_stateless/02360_send_logs_level_colors.reference @@ -0,0 +1,3 @@ +ASCII text +ASCII text +ASCII text diff --git a/tests/queries/0_stateless/02360_send_logs_level_colors.sh b/tests/queries/0_stateless/02360_send_logs_level_colors.sh new file mode 100644 index 00000000000..4e5ce057702 --- /dev/null +++ b/tests/queries/0_stateless/02360_send_logs_level_colors.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +file_name="${CLICKHOUSE_TMP}/res_${CLICKHOUSE_DATABASE}.log" +CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') + +# Run query via expect to make isatty() return true +function run() +{ + command=$1 + expect << EOF +log_user 0 +set timeout 3 +match_max 100000 + +spawn bash -c "$command" +expect 1 +EOF + + file "$file_name" | grep -o "ASCII text" + file "$file_name" | grep -o "with escape sequences" +} + +run "$CLICKHOUSE_CLIENT -q 'SELECT 1' 2>$file_name" +run "$CLICKHOUSE_CLIENT -q 'SELECT 1' --server_logs_file=$file_name" +run "$CLICKHOUSE_CLIENT -q 'SELECT 1' --server_logs_file=- >$file_name" + +rm -f "$file_name" From 1bc943264c24f9ec92fa95ff8c787d6377362003 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Jul 2022 21:38:17 +0200 Subject: [PATCH 606/627] Add concurrent backups test --- .../configs/storage_conf.xml | 35 ++++++++++++++ .../test_concurrent_backups_s3/test.py | 48 +++++++++++++++++++ 2 files changed, 83 insertions(+) create mode 100644 tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml create mode 100644 tests/integration/test_concurrent_backups_s3/test.py diff --git a/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml b/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml new file mode 100644 index 00000000000..ef55f3a62d7 --- /dev/null +++ b/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml @@ -0,0 +1,35 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + 33554432 + + + local + / + + + + + +
+ s3 +
+
+
+
+
+ + + 0 + + + hdd + /backups/ + + +
diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py new file mode 100644 index 00000000000..591715aff17 --- /dev/null +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -0,0 +1,48 @@ +#!/usr/bin/env python3 +import pytest +import re +import os.path +from multiprocessing.dummy import Pool +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, TSV +import time + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/storage_conf.xml"], + with_minio=True, +) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_concurrent_backups(start_cluster): + node.query("DROP TABLE IF EXISTS s3_test NO DELAY") + columns = [f"column_{i} UInt64" for i in range(1000)] + columns_str = ', '.join(columns) + node.query( + f"CREATE TABLE s3_test ({columns_str}) Engine=MergeTree() ORDER BY tuple() SETTINGS storage_policy='s3';" + ) + node.query( + f"INSERT INTO s3_test SELECT * FROM generateRandom('{columns_str}') LIMIT 10000" + ) + + def create_backup(i): + backup_name = f"Disk('hdd', '/backups/{i}')" + node.query(f"BACKUP TABLE s3_test TO {backup_name} ASYNC") + + p = Pool(20) + + p.map(create_backup, range(40)) + + for _ in range(20): + print(node.query("SELECT * FROM system.backups FORMAT Vertical")) + time.sleep(0.1) + + assert node.query("SELECT count() FROM s3_test where not ignore(*)") == "10000\n" From 435d98738d4edeb693faf5d54a2cd20f32012eb9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 14 Jul 2022 22:41:01 +0300 Subject: [PATCH 607/627] Update ReplicatedMergeTreeQueue.cpp --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index f6c80baba05..7967726edca 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1102,9 +1102,8 @@ bool ReplicatedMergeTreeQueue::isCoveredByFuturePartsImpl(const LogEntry & entry if (future_part.isDisjoint(result_part)) continue; - /// Parts are not disjoint, so new_part_name either contains or covers future_part. - if (!(future_part.contains(result_part) || result_part.contains(future_part))) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got unexpected non-disjoint parts: {} and {}", future_part_elem.first, new_part_name); + /// Parts are not disjoint. They can be even intersecting and it's not a problem, + /// because we may have two queue entries producing intersecting parts if there's DROP_RANGE between them (so virtual_parts are ok). /// We cannot execute `entry` (or upgrade its actual_part_name to `new_part_name`) /// while any covered or covering parts are processed. From efed13475819f570add3e82e1c0fa40d17a8b975 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Jul 2022 19:41:32 +0000 Subject: [PATCH 608/627] fix test --- docker/test/fasttest/Dockerfile | 1 + docker/test/stateless/Dockerfile | 1 + tests/queries/0_stateless/02360_send_logs_level_colors.sh | 0 3 files changed, 2 insertions(+) mode change 100644 => 100755 tests/queries/0_stateless/02360_send_logs_level_colors.sh diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 03a79b45a10..aa209f197df 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -71,6 +71,7 @@ RUN apt-get update \ software-properties-common \ tzdata \ unixodbc \ + file \ --yes --no-install-recommends RUN pip3 install numpy scipy pandas Jinja2 diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 96c06845812..9141e89d744 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -34,6 +34,7 @@ RUN apt-get update -y \ unixodbc \ wget \ zstd \ + file \ && apt-get clean diff --git a/tests/queries/0_stateless/02360_send_logs_level_colors.sh b/tests/queries/0_stateless/02360_send_logs_level_colors.sh old mode 100644 new mode 100755 From d2b725761717ccd09b243ed3a42ffb35de398fa7 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Fri, 15 Jul 2022 00:12:38 +0200 Subject: [PATCH 609/627] Update tests/queries/0_stateless/02356_insert_query_log_metrics.sql Co-authored-by: Alexander Tokmakov --- tests/queries/0_stateless/02356_insert_query_log_metrics.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql index 347073f868f..08ae22dee95 100644 --- a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql @@ -1,4 +1,5 @@ CREATE TABLE 02356_destination (a Int64, b String) ENGINE = Memory; INSERT INTO 02356_destination (a, b) SELECT * FROM generateRandom('a Int64, b String') LIMIT 100 SETTINGS max_threads=1, max_block_size=100; +SYSTEM FLUSH LOGS; SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' order by event_time desc limit 1 FORMAT CSV; \ No newline at end of file From 4d96ff9b9768a6009d339359635493cb8e222354 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 14 Jul 2022 22:25:10 +0000 Subject: [PATCH 610/627] style --- tests/queries/0_stateless/02356_insert_query_log_metrics.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql index 08ae22dee95..bb37c8b4eb1 100644 --- a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql @@ -2,4 +2,4 @@ CREATE TABLE 02356_destination (a Int64, b String) ENGINE = Memory; INSERT INTO 02356_destination (a, b) SELECT * FROM generateRandom('a Int64, b String') LIMIT 100 SETTINGS max_threads=1, max_block_size=100; SYSTEM FLUSH LOGS; -SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' order by event_time desc limit 1 FORMAT CSV; \ No newline at end of file +SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' and has(databases, currentDatabase()) FORMAT CSV; \ No newline at end of file From c7da3d61bf7be2867ce45679d91852846b501b3d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 00:32:20 +0200 Subject: [PATCH 611/627] More stressful test --- tests/integration/test_concurrent_backups_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index 591715aff17..ed8edbc77d4 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -37,7 +37,7 @@ def test_concurrent_backups(start_cluster): backup_name = f"Disk('hdd', '/backups/{i}')" node.query(f"BACKUP TABLE s3_test TO {backup_name} ASYNC") - p = Pool(20) + p = Pool(40) p.map(create_backup, range(40)) From 687961cf4242fc28bf0957054716441f4701af70 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 00:38:58 +0200 Subject: [PATCH 612/627] Fix style --- src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp | 1 - .../MetadataStorageFromDiskTransactionOperations.cpp | 1 - tests/integration/test_concurrent_backups_s3/test.py | 4 +++- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp index 76e6aebff1c..d1413bd4e88 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp @@ -11,7 +11,6 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; - extern const int FS_METADATA_ERROR; } FakeMetadataStorageFromDisk::FakeMetadataStorageFromDisk( diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp index 7c0a60509ab..72da240cf8a 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index ed8edbc77d4..b1f41f04b99 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -14,6 +14,7 @@ node = cluster.add_instance( with_minio=True, ) + @pytest.fixture(scope="module") def start_cluster(): try: @@ -22,10 +23,11 @@ def start_cluster(): finally: cluster.shutdown() + def test_concurrent_backups(start_cluster): node.query("DROP TABLE IF EXISTS s3_test NO DELAY") columns = [f"column_{i} UInt64" for i in range(1000)] - columns_str = ', '.join(columns) + columns_str = ", ".join(columns) node.query( f"CREATE TABLE s3_test ({columns_str}) Engine=MergeTree() ORDER BY tuple() SETTINGS storage_policy='s3';" ) From d4a24a5f8ef601598de84ecc9d66b049cf84b44c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 00:50:03 +0200 Subject: [PATCH 613/627] Better test --- tests/integration/test_concurrent_backups_s3/test.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index b1f41f04b99..c262bc9e331 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -43,8 +43,13 @@ def test_concurrent_backups(start_cluster): p.map(create_backup, range(40)) - for _ in range(20): - print(node.query("SELECT * FROM system.backups FORMAT Vertical")) + for _ in range(100): + result = node.query( + "SELECT count() FROM system.backups WHERE status != 'BACKUP_COMPLETE' and status != 'FAILED_TO_BACKUP'" + ).strip() + if result == "0": + break + time.sleep(0.1) assert node.query("SELECT count() FROM s3_test where not ignore(*)") == "10000\n" From f3650d8769e9d4daf354df46ece226328fa289ea Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Thu, 14 Jul 2022 18:42:41 -0700 Subject: [PATCH 614/627] Add back accidentally removed line --- contrib/thrift-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 832e104e8a3..d6aa6b9e5f2 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -93,4 +93,4 @@ include_directories("${CMAKE_CURRENT_BINARY_DIR}") add_library(_thrift ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) add_library(ch_contrib::thrift ALIAS _thrift) target_include_directories(_thrift SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src" ${CMAKE_CURRENT_BINARY_DIR}) -target_link_libraries (_thrift PUBLIC OpenSSL::SSL) +target_link_libraries (_thrift PUBLIC OpenSSL::SSL boost::headers_only) From 1f85358625614c7329d5dafbb53000505214f672 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 15 Jul 2022 08:25:24 +0000 Subject: [PATCH 615/627] Add X86 prefix to x86 performance tests reduces ambiguity with the recently introduced ARM performance tests --- .github/workflows/master.yml | 16 +++++++-------- .github/workflows/pull_request.yml | 32 +++++++++++++++--------------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 66ba8547894..b76ff39f0c8 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -2902,7 +2902,7 @@ jobs: ############################################################################################# #################################### PERFORMANCE TESTS ###################################### ############################################################################################# - PerformanceComparison0: + PerformanceComparisonX86-0: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -2940,7 +2940,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison1: + PerformanceComparisonX86-1: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -2978,7 +2978,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison2: + PerformanceComparisonX86-2: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3016,7 +3016,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison3: + PerformanceComparisonX86-3: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3096,10 +3096,10 @@ jobs: - IntegrationTestsTsan1 - IntegrationTestsTsan2 - IntegrationTestsTsan3 - - PerformanceComparison0 - - PerformanceComparison1 - - PerformanceComparison2 - - PerformanceComparison3 + - PerformanceComparisonX86-0 + - PerformanceComparisonX86-1 + - PerformanceComparisonX86-2 + - PerformanceComparisonX86-3 - CompatibilityCheck - ASTFuzzerTestDebug - ASTFuzzerTestAsan diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 9cd8fd6f49d..05fd337e57c 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3118,7 +3118,7 @@ jobs: ############################################################################################# #################################### PERFORMANCE TESTS ###################################### ############################################################################################# - PerformanceComparison0: + PerformanceComparisonX86-0: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3156,7 +3156,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison1: + PerformanceComparisonX86-1: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3194,7 +3194,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison2: + PerformanceComparisonX86-2: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3232,7 +3232,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison3: + PerformanceComparisonX86-3: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3270,7 +3270,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch0: + PerformanceComparisonAarch-0: needs: [BuilderDebAarch64] runs-on: [self-hosted, func-tester-aarch64] steps: @@ -3308,7 +3308,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch1: + PerformanceComparisonAarch-1: needs: [BuilderDebAarch64] runs-on: [self-hosted, func-tester-aarch64] steps: @@ -3346,7 +3346,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch2: + PerformanceComparisonAarch-2: needs: [BuilderDebAarch64] runs-on: [self-hosted, func-tester-aarch64] steps: @@ -3384,7 +3384,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch3: + PerformanceComparisonAarch-3: needs: [BuilderDebAarch64] runs-on: [self-hosted, func-tester-aarch64] steps: @@ -3481,14 +3481,14 @@ jobs: - IntegrationTestsTsan1 - IntegrationTestsTsan2 - IntegrationTestsTsan3 - - PerformanceComparison0 - - PerformanceComparison1 - - PerformanceComparison2 - - PerformanceComparison3 - - PerformanceComparisonAarch0 - - PerformanceComparisonAarch1 - - PerformanceComparisonAarch2 - - PerformanceComparisonAarch3 + - PerformanceComparisonX86-0 + - PerformanceComparisonX86-1 + - PerformanceComparisonX86-2 + - PerformanceComparisonX86-3 + - PerformanceComparisonAarch-0 + - PerformanceComparisonAarch-1 + - PerformanceComparisonAarch-2 + - PerformanceComparisonAarch-3 - UnitTestsAsan - UnitTestsTsan - UnitTestsMsan From 1276bfdffd807fc58bb7dbaf296d3ca8e519c044 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 15 Jul 2022 08:53:58 +0000 Subject: [PATCH 616/627] Enable -Wsign-conversion --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 0e55f6d32c3..e9a41dda01d 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -35,7 +35,6 @@ if (COMPILER_CLANG) no_warning(missing-prototypes) no_warning(missing-variable-declarations) no_warning(padded) - no_warning(sign-conversion) no_warning(switch-enum) no_warning(undefined-func-template) no_warning(unused-template) From 429aab2f12ca3994965e6c30ad687283c4a5900d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 15 Jul 2022 11:55:13 +0200 Subject: [PATCH 617/627] Fix making a query scope for async backups. --- src/Backups/BackupsWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index bf90d58d009..ba9a5b6ca71 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -107,7 +107,7 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c { if (async) { - query_scope.emplace(context_in_use); + query_scope.emplace(mutable_context); setThreadName("BackupWorker"); } From c133f27724819ba93dd6c5d31b6a899ccd3df6bd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 15 Jul 2022 09:56:56 +0000 Subject: [PATCH 618/627] style again --- tests/queries/0_stateless/02356_insert_query_log_metrics.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql index bb37c8b4eb1..dabb898093e 100644 --- a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql @@ -2,4 +2,4 @@ CREATE TABLE 02356_destination (a Int64, b String) ENGINE = Memory; INSERT INTO 02356_destination (a, b) SELECT * FROM generateRandom('a Int64, b String') LIMIT 100 SETTINGS max_threads=1, max_block_size=100; SYSTEM FLUSH LOGS; -SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' and has(databases, currentDatabase()) FORMAT CSV; \ No newline at end of file +SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' and current_database = currentDatabase() FORMAT CSV; \ No newline at end of file From 77487e996c4be698cd09630f287067fd923edb59 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 15 Jul 2022 12:23:28 +0200 Subject: [PATCH 619/627] fix stacktraces in gdb in bc check --- docker/test/stress/run.sh | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index cb80c8ebdc9..085c0c045b7 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -322,7 +322,11 @@ else clickhouse-client --query="SELECT 'Server version: ', version()" # Install new package before running stress test because we should use new clickhouse-client and new clickhouse-test + # But we should leave old binary in /usr/bin/ for gdb (so it will print sane stacktarces) + mv /usr/bin/clickhouse previous_release_package_folder/ install_packages package_folder + mv /usr/bin/clickhouse package_folder/ + mv previous_release_package_folder/clickhouse /usr/bin/ mkdir tmp_stress_output @@ -337,6 +341,7 @@ else mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.stress.log # Start new server + mv package_folder/clickhouse /usr/bin/ configure start 500 clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK'" >> /test_output/test_results.tsv \ From 8d07a1427f35c3e8182e6b56dd378c20efbbb7dc Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 12:28:56 +0200 Subject: [PATCH 620/627] Update storage_conf.xml --- .../test_concurrent_backups_s3/configs/storage_conf.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml b/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml index ef55f3a62d7..9124d583f8c 100644 --- a/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml +++ b/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml @@ -29,7 +29,6 @@ hdd - /backups/ From c4cbefb5211c471c9fb664ff8dd10b9f066676ff Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 15 Jul 2022 13:44:37 +0300 Subject: [PATCH 621/627] Update test.py --- tests/integration/test_s3_zero_copy_replication/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 39be0d564df..7b7fb9d21ad 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -150,6 +150,7 @@ def test_s3_zero_copy_replication(cluster, policy): node2.query("DROP TABLE IF EXISTS s3_test NO DELAY") +@pytest.mark.skip(reason="Test is flaky (and never was stable)") def test_s3_zero_copy_on_hybrid_storage(cluster): node1 = cluster.instances["node1"] node2 = cluster.instances["node2"] From 74fc53e5c22b9d5e40dc2c63a32d3ada41b07cef Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 15 Jul 2022 10:58:09 +0000 Subject: [PATCH 622/627] Enable -Wc99-extension --- cmake/warnings.cmake | 1 - src/Parsers/ExpressionListParsers.h | 9 +++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index e9a41dda01d..994f14c6149 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -23,7 +23,6 @@ if (COMPILER_CLANG) no_warning(zero-length-array) no_warning(c++98-compat-pedantic) no_warning(c++98-compat) - no_warning(c99-extensions) no_warning(conversion) no_warning(ctad-maybe-unsupported) # clang 9+, linux-only no_warning(disabled-macro-expansion) diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 2b127dc2607..05c7ec946ee 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -9,6 +9,11 @@ #include #include +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wc99-extensions" +#endif + namespace DB { @@ -564,3 +569,7 @@ protected: }; } + +#ifdef __clang__ +#pragma clang diagnostic pop +#endif From cb6bc63af16a90f8407636811a46b01e1cfac59d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 14:43:27 +0200 Subject: [PATCH 623/627] Better test --- tests/integration/test_concurrent_backups_s3/test.py | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index c262bc9e331..7bf5ea9a5dc 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -4,7 +4,7 @@ import re import os.path from multiprocessing.dummy import Pool from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry, TSV +from helpers.test_tools import assert_eq_with_retry import time cluster = ClickHouseCluster(__file__) @@ -43,13 +43,5 @@ def test_concurrent_backups(start_cluster): p.map(create_backup, range(40)) - for _ in range(100): - result = node.query( - "SELECT count() FROM system.backups WHERE status != 'BACKUP_COMPLETE' and status != 'FAILED_TO_BACKUP'" - ).strip() - if result == "0": - break - - time.sleep(0.1) - + assert_eq_with_retry(node, "SELECT count() FROM system.backups WHERE status != 'BACKUP_COMPLETE' and status != 'FAILED_TO_BACKUP'", "0", retry_count=100) assert node.query("SELECT count() FROM s3_test where not ignore(*)") == "10000\n" From 6c5de5a867743a1838411bd5b7d2327a21e19653 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 15:34:44 +0200 Subject: [PATCH 624/627] FIX BLACK --- tests/integration/test_concurrent_backups_s3/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index 7bf5ea9a5dc..608144843d9 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -43,5 +43,10 @@ def test_concurrent_backups(start_cluster): p.map(create_backup, range(40)) - assert_eq_with_retry(node, "SELECT count() FROM system.backups WHERE status != 'BACKUP_COMPLETE' and status != 'FAILED_TO_BACKUP'", "0", retry_count=100) + assert_eq_with_retry( + node, + "SELECT count() FROM system.backups WHERE status != 'BACKUP_COMPLETE' and status != 'FAILED_TO_BACKUP'", + "0", + retry_count=100, + ) assert node.query("SELECT count() FROM s3_test where not ignore(*)") == "10000\n" From ec15be27989296a29bd2315422a79b049f1545a2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 19:00:50 +0200 Subject: [PATCH 625/627] Add missed file --- tests/integration/test_concurrent_backups_s3/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_concurrent_backups_s3/__init__.py diff --git a/tests/integration/test_concurrent_backups_s3/__init__.py b/tests/integration/test_concurrent_backups_s3/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From b770f9903d120c691de5840360e8fb5e1b41f65b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 15 Jul 2022 20:03:00 +0300 Subject: [PATCH 626/627] Update run.sh --- docker/test/stress/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index cc76456fac2..06bf05a1727 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -361,6 +361,7 @@ else # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64") # FIXME Not sure if it's expected, but some tests from BC check may not be finished yet when we restarting server. # Let's just ignore all errors from queries ("} TCPHandler: Code:", "} executeQuery: Code:") + # FIXME https://github.com/ClickHouse/ClickHouse/issues/39197 ("Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'") echo "Check for Error messages in server log:" zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ -e "Code: 236. DB::Exception: Cancelled mutating parts" \ @@ -387,6 +388,7 @@ else -e "Cannot parse string 'Hello' as UInt64" \ -e "} TCPHandler: Code:" \ -e "} executeQuery: Code:" \ + -e "Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'" \ /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv From 1ef9910453a45dd12891a9a8f4838ec95506eff9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 16 Jul 2022 10:50:22 +0300 Subject: [PATCH 627/627] Fix builder image for releases w/o diagnostics tool Signed-off-by: Azat Khuzhin --- docker/packager/binary/build.sh | 26 +++++++++++++++----------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 270c93c105c..87f98df2ad8 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -29,17 +29,21 @@ env if [ -n "$MAKE_DEB" ]; then rm -rf /build/packages/root - if [ -z "$SANITIZER" ]; then - # We need to check if clickhouse-diagnostics is fine and build it - ( - cd /build/programs/diagnostics - make test-no-docker - GOARCH="${DEB_ARCH}" CGO_ENABLED=0 make VERSION="$VERSION_STRING" build - mv clickhouse-diagnostics .. - ) - else - echo -e "#!/bin/sh\necho 'Not implemented for this type of package'" > /build/programs/clickhouse-diagnostics - chmod +x /build/programs/clickhouse-diagnostics + # NOTE: this is for backward compatibility with previous releases, + # that does not diagnostics tool (only script). + if [ -d /build/programs/diagnostics ]; then + if [ -z "$SANITIZER" ]; then + # We need to check if clickhouse-diagnostics is fine and build it + ( + cd /build/programs/diagnostics + make test-no-docker + GOARCH="${DEB_ARCH}" CGO_ENABLED=0 make VERSION="$VERSION_STRING" build + mv clickhouse-diagnostics .. + ) + else + echo -e "#!/bin/sh\necho 'Not implemented for this type of package'" > /build/programs/clickhouse-diagnostics + chmod +x /build/programs/clickhouse-diagnostics + fi fi fi